From ff43309e39fb1db2bf2ae5a3d7d0972440880dca Mon Sep 17 00:00:00 2001 From: Robert Hastings Date: Tue, 3 May 2016 13:40:56 -0700 Subject: [PATCH 01/58] Addresses network flood from KafkaSpout to kafka server. * Allows minBytes in fetch request to be configured from KafkaConfig.fetchMinBytes. * Defaults new configuration KafkaConfig.fetchMinBytes to 1. * Defaults fetchMaxWait to 100ms instead of 10000ms. Discovered 30 megabits of traffic flowing between a set of KafkaSpouts and our kafka servers even though no Kafka messages were moving. Using the wireshark kafka dissector, we were able to see that each FetchRequest had maxWait set to 10000 and minBytes set to 0. When binBytes is set to 0 the kafka server responds immediately when there are no messages. In turn the KafkaSpout polls without any delay causing a constant stream of FetchRequest/ FetchResponse messages. Using a non-KafkaSpout client had a similar traffic pattern with two key differences 1) minBytes was 1 2) maxWait was 100 With these FetchRequest parameters and no messages flowing, the kafka server delays the FetchResponse by 100 ms. This reduces the network traffic from megabits to the low kilobits. It also reduced the CPU utilization of our kafka server from 140% to 2%. Hopefully the risk of this change is low because the old behavior can be restored using the API by setting KafkaConfig.fetchMaxWait to 10000 KafkaConfig.fetchMinBytes to 0 Conflicts: external/storm-kafka/src/jvm/storm/kafka/KafkaConfig.java external/storm-kafka/src/jvm/storm/kafka/KafkaUtils.java --- .../src/jvm/org/apache/storm/kafka/KafkaConfig.java | 3 ++- .../storm-kafka/src/jvm/org/apache/storm/kafka/KafkaUtils.java | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaConfig.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaConfig.java index e1e1d242cae..77c77487aba 100644 --- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaConfig.java +++ b/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaConfig.java @@ -31,7 +31,8 @@ public class KafkaConfig implements Serializable { public int fetchSizeBytes = 1024 * 1024; public int socketTimeoutMs = 10000; - public int fetchMaxWait = 10000; + public int fetchMaxWait = 100; + public int fetchMinBytes = 1; public int bufferSizeBytes = 1024 * 1024; public MultiScheme scheme = new RawMultiScheme(); public boolean ignoreZkOffsets = false; 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 a2be825a326..47afbadb0dc 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 @@ -185,7 +185,7 @@ public static ByteBufferMessageSet fetchMessages(KafkaConfig config, SimpleConsu int partitionId = partition.partition; FetchRequestBuilder builder = new FetchRequestBuilder(); FetchRequest fetchRequest = builder.addFetch(topic, partitionId, offset, config.fetchSizeBytes). - clientId(config.clientId).maxWait(config.fetchMaxWait).build(); + clientId(config.clientId).maxWait(config.fetchMaxWait).minBytes(config.fetchMinBytes).build(); FetchResponse fetchResponse; try { fetchResponse = consumer.fetch(fetchRequest); From b8e7e4d33a4bf98ee77901a76e554919d0ab2d61 Mon Sep 17 00:00:00 2001 From: David Luu Date: Sun, 8 May 2016 15:39:22 -0700 Subject: [PATCH 02/58] backport change from master: add convenience methods for checking tuple type in node.js library --- .../src/main/resources/resources/storm.js | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/storm-multilang/javascript/src/main/resources/resources/storm.js b/storm-multilang/javascript/src/main/resources/resources/storm.js index dc6efc13863..c8462ba24fd 100755 --- a/storm-multilang/javascript/src/main/resources/resources/storm.js +++ b/storm-multilang/javascript/src/main/resources/resources/storm.js @@ -199,7 +199,7 @@ Storm.prototype.emitDirect = function(commandDetails) { /** * Initialize storm component according to the configuration received. - * @param conf configuration object accrding to storm protocol. + * @param conf configuration object according to storm protocol. * @param context context object according to storm protocol. * @param done callback. Call this method when finished initializing. */ @@ -221,10 +221,18 @@ function Tuple(id, component, stream, task, values) { this.values = values; } +Tuple.prototype.isTickTuple = function(){ + return this.task === -1 && this.stream === "__tick"; +} + +Tuple.prototype.isHeartbeatTuple = function(){ + return this.task === -1 && this.stream === "__heartbeat"; +} + /** * Base class for storm bolt. * To create a bolt implement 'process' method. - * You may also implement initialize method to + * You may also implement initialize method too */ function BasicBolt() { Storm.call(this); @@ -262,7 +270,7 @@ BasicBolt.prototype.handleNewCommand = function(command) { var self = this; var tup = new Tuple(command["id"], command["comp"], command["stream"], command["task"], command["tuple"]); - if (tup.task === -1 && tup.stream === "__heartbeat") { + if (tup.isHeartbeatTuple()) { self.sync(); return; } @@ -293,7 +301,6 @@ BasicBolt.prototype.fail = function(tup, err) { this.sendMsgToParent({"command": "fail", "id": tup.id}); } - /** * Base class for storm spout. * To create a spout implement the following methods: nextTuple, ack and fail (nextTuple - mandatory, ack and fail @@ -370,4 +377,4 @@ Spout.prototype.__emit = function(commandDetails) { } module.exports.BasicBolt = BasicBolt; -module.exports.Spout = Spout; \ No newline at end of file +module.exports.Spout = Spout; From 37a0f291ec308f3e2087a67204301806ea17e80b Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Thu, 12 May 2016 12:56:25 -0700 Subject: [PATCH 03/58] Added STORM-1773 to CHANGELOG. --- CHANGELOG.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 5ab8528baab..17d1e4f4ebb 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,5 @@ +## 1.0.2 + * STORM-1773: Utils.javaDeserialize() doesn't work with primitive types ## 1.0.1 * STORM-1741: remove unconditional setting of JAVA_HOME from storm-env.sh * STORM-1739: update the minor JAVA version dependency in 0.10.0 and above From e8db7eb1d33d59f5a5c579a7426c3e5244341f45 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Fri, 13 May 2016 17:01:50 +0900 Subject: [PATCH 04/58] bump next version to 1.1.0 for this branch --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 3048f67d979..d015bb2e9c8 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ org.apache.storm storm - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT pom Storm Distributed and fault-tolerant realtime computation From 68beefd2bf40215678a7b5d248ac01a84c741be1 Mon Sep 17 00:00:00 2001 From: Xin Wang Date: Sat, 14 May 2016 19:52:50 +0800 Subject: [PATCH 05/58] STORM-1835: add lock info in thread dump --- storm-core/src/jvm/org/apache/storm/utils/Utils.java | 4 ++++ 1 file changed, 4 insertions(+) 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 1a76b8a287a..150bd88e466 100644 --- a/storm-core/src/jvm/org/apache/storm/utils/Utils.java +++ b/storm-core/src/jvm/org/apache/storm/utils/Utils.java @@ -1208,6 +1208,10 @@ public static String threadDump() { dump.append('"'); dump.append(threadInfo.getThreadName()); dump.append("\" "); + dump.append("\n lock: "); + dump.append(threadInfo.getLockName()); + dump.append(" owner: "); + dump.append(threadInfo.getLockOwnerName()); final Thread.State state = threadInfo.getThreadState(); dump.append("\n java.lang.Thread.State: "); dump.append(state); From 98c077c6f6b8e30cf9e60a9bbb07e3742aa23c0e Mon Sep 17 00:00:00 2001 From: Sanket Date: Wed, 30 Mar 2016 11:36:27 -0500 Subject: [PATCH 06/58] added acl validation config --- conf/defaults.yaml | 2 ++ storm-core/src/jvm/org/apache/storm/Config.java | 11 +++++++++-- .../apache/storm/blobstore/BlobStoreAclHandler.java | 10 ++++++++++ 3 files changed, 21 insertions(+), 2 deletions(-) diff --git a/conf/defaults.yaml b/conf/defaults.yaml index 3f6a1c9e72e..9e731d02396 100644 --- a/conf/defaults.yaml +++ b/conf/defaults.yaml @@ -133,6 +133,8 @@ nimbus.blobstore.expiration.secs: 600 storm.blobstore.inputstream.buffer.size.bytes: 65536 client.blobstore.class: "org.apache.storm.blobstore.NimbusBlobStore" storm.blobstore.replication.factor: 3 +# For secure mode we would want to change this config to true +storm.blobstore.acl.validation.enabled: false ### supervisor.* configs are for node supervisors # Define the amount of workers that can be run on this machine. Each worker is assigned a port to use for communication diff --git a/storm-core/src/jvm/org/apache/storm/Config.java b/storm-core/src/jvm/org/apache/storm/Config.java index 507614b0d75..21aa010c423 100644 --- a/storm-core/src/jvm/org/apache/storm/Config.java +++ b/storm-core/src/jvm/org/apache/storm/Config.java @@ -941,7 +941,7 @@ public class Config extends HashMap { /** * The maximum number of threads that should be used by the Pacemaker. - * When Pacemaker gets loaded it will spawn new threads, up to + * When Pacemaker gets loaded it will spawn new threads, up to * this many total, to handle the load. */ @isNumber @@ -965,7 +965,7 @@ public class Config extends HashMap { */ @CustomValidator(validatorClass=PacemakerAuthTypeValidator.class) public static final String PACEMAKER_AUTH_METHOD = "pacemaker.auth.method"; - + /** * List of DRPC servers so that the DRPCSpout knows who to talk to. */ @@ -1260,6 +1260,13 @@ public class Config extends HashMap { @isInteger public static final String STORM_BLOBSTORE_REPLICATION_FACTOR = "storm.blobstore.replication.factor"; + /** + * For secure mode we would want to turn on this config + * By default this is turned off assuming the default is insecure + */ + @isBoolean + public static final String STORM_BLOBSTORE_ACL_VALIDATION_ENABLED = "storm.blobstore.acl.validation.enabled"; + /** * What blobstore implementation nimbus should use. */ diff --git a/storm-core/src/jvm/org/apache/storm/blobstore/BlobStoreAclHandler.java b/storm-core/src/jvm/org/apache/storm/blobstore/BlobStoreAclHandler.java index 8194e91273a..5b3866dd6cd 100644 --- a/storm-core/src/jvm/org/apache/storm/blobstore/BlobStoreAclHandler.java +++ b/storm-core/src/jvm/org/apache/storm/blobstore/BlobStoreAclHandler.java @@ -54,6 +54,7 @@ public class BlobStoreAclHandler { public static final List DEFAULT = new ArrayList(); private Set _supervisors; private Set _admins; + private boolean doAclValidation; public BlobStoreAclHandler(Map conf) { _ptol = AuthUtils.GetPrincipalToLocalPlugin(conf); @@ -65,6 +66,9 @@ public BlobStoreAclHandler(Map conf) { if (conf.containsKey(Config.NIMBUS_ADMINS)) { _admins.addAll((List)conf.get(Config.NIMBUS_ADMINS)); } + if (conf.containsKey(Config.STORM_BLOBSTORE_ACL_VALIDATION_ENABLED)) { + doAclValidation = (boolean)conf.get(Config.STORM_BLOBSTORE_ACL_VALIDATION_ENABLED); + } } private static AccessControlType parseACLType(String type) { @@ -245,6 +249,9 @@ public void validateUserCanReadMeta(List acl, Subject who, String * @throws AuthorizationException */ public void hasAnyPermissions(List acl, int mask, Subject who, String key) throws AuthorizationException { + if (!doAclValidation) { + return; + } Set user = constructUserFromPrincipals(who); LOG.debug("user {}", user); if (checkForValidUsers(who, mask)) { @@ -275,6 +282,9 @@ public void hasAnyPermissions(List acl, int mask, Subject who, St * @throws AuthorizationException */ public void hasPermissions(List acl, int mask, Subject who, String key) throws AuthorizationException { + if (!doAclValidation) { + return; + } Set user = constructUserFromPrincipals(who); LOG.debug("user {}", user); if (checkForValidUsers(who, mask)) { From 823a62153be6b12903a6da443b3c8be27a2f4937 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Mon, 16 May 2016 14:48:33 +0900 Subject: [PATCH 07/58] add STORM-1661 to CHANGELOG.md --- CHANGELOG.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 17d1e4f4ebb..a85e78671bf 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,7 @@ ## 1.0.2 * STORM-1773: Utils.javaDeserialize() doesn't work with primitive types + * STORM-1661: Introduce a config to turn off blobstore acl validation in insecure mode + ## 1.0.1 * STORM-1741: remove unconditional setting of JAVA_HOME from storm-env.sh * STORM-1739: update the minor JAVA version dependency in 0.10.0 and above From a6de2898561af0f8f79d1330a42dffe0c5adc9a3 Mon Sep 17 00:00:00 2001 From: darionyaphet Date: Fri, 6 May 2016 20:00:00 +0800 Subject: [PATCH 08/58] STORM-1720 support GEO in storm-redis --- external/storm-redis/pom.xml | 2 +- .../apache/storm/redis/bolt/RedisLookupBolt.java | 4 ++++ .../apache/storm/redis/bolt/RedisStoreBolt.java | 15 +++++++++++++-- .../common/container/JedisClusterContainer.java | 7 ++++++- .../common/mapper/RedisDataTypeDescription.java | 7 ++++--- 5 files changed, 28 insertions(+), 7 deletions(-) diff --git a/external/storm-redis/pom.xml b/external/storm-redis/pom.xml index 6cd5443aa67..77ba5bff106 100644 --- a/external/storm-redis/pom.xml +++ b/external/storm-redis/pom.xml @@ -41,7 +41,7 @@ - 2.7.0 + 2.8.1 diff --git a/external/storm-redis/src/main/java/org/apache/storm/redis/bolt/RedisLookupBolt.java b/external/storm-redis/src/main/java/org/apache/storm/redis/bolt/RedisLookupBolt.java index 4d6dc4ec71a..968ade0daf4 100644 --- a/external/storm-redis/src/main/java/org/apache/storm/redis/bolt/RedisLookupBolt.java +++ b/external/storm-redis/src/main/java/org/apache/storm/redis/bolt/RedisLookupBolt.java @@ -105,6 +105,10 @@ public void execute(Tuple input) { lookupValue = jedisCommand.pfcount(key); break; + case GEO: + lookupValue = jedisCommand.geopos(additionalKey, key); + break; + default: throw new IllegalArgumentException("Cannot process such data type: " + dataType); } diff --git a/external/storm-redis/src/main/java/org/apache/storm/redis/bolt/RedisStoreBolt.java b/external/storm-redis/src/main/java/org/apache/storm/redis/bolt/RedisStoreBolt.java index c774dc09316..00ff2186dad 100644 --- a/external/storm-redis/src/main/java/org/apache/storm/redis/bolt/RedisStoreBolt.java +++ b/external/storm-redis/src/main/java/org/apache/storm/redis/bolt/RedisStoreBolt.java @@ -17,12 +17,12 @@ */ package org.apache.storm.redis.bolt; -import org.apache.storm.topology.OutputFieldsDeclarer; -import org.apache.storm.tuple.Tuple; import org.apache.storm.redis.common.config.JedisClusterConfig; import org.apache.storm.redis.common.config.JedisPoolConfig; import org.apache.storm.redis.common.mapper.RedisDataTypeDescription; import org.apache.storm.redis.common.mapper.RedisStoreMapper; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.tuple.Tuple; import redis.clients.jedis.JedisCommands; /** @@ -100,6 +100,17 @@ public void execute(Tuple input) { jedisCommand.pfadd(key, value); break; + case GEO: + String[] array = value.split(":"); + if (array.length != 2) { + throw new IllegalArgumentException("value structure should be longitude:latitude"); + } + + double longitude = Double.valueOf(array[0]); + double latitude = Double.valueOf(array[1]); + jedisCommand.geoadd(additionalKey, longitude, latitude, key); + break; + default: throw new IllegalArgumentException("Cannot process such data type: " + dataType); } diff --git a/external/storm-redis/src/main/java/org/apache/storm/redis/common/container/JedisClusterContainer.java b/external/storm-redis/src/main/java/org/apache/storm/redis/common/container/JedisClusterContainer.java index ea294a6f883..a588d1c05bc 100644 --- a/external/storm-redis/src/main/java/org/apache/storm/redis/common/container/JedisClusterContainer.java +++ b/external/storm-redis/src/main/java/org/apache/storm/redis/common/container/JedisClusterContainer.java @@ -21,6 +21,7 @@ import redis.clients.jedis.JedisCommands; import java.io.Closeable; +import java.io.IOException; /** * Container for managing JedisCluster. @@ -60,6 +61,10 @@ public void returnInstance(JedisCommands jedisCommands) { */ @Override public void close() { - this.jedisCluster.close(); + try { + this.jedisCluster.close(); + } catch (IOException e) { + e.printStackTrace(); + } } } diff --git a/external/storm-redis/src/main/java/org/apache/storm/redis/common/mapper/RedisDataTypeDescription.java b/external/storm-redis/src/main/java/org/apache/storm/redis/common/mapper/RedisDataTypeDescription.java index ede18687923..7e5c7851885 100644 --- a/external/storm-redis/src/main/java/org/apache/storm/redis/common/mapper/RedisDataTypeDescription.java +++ b/external/storm-redis/src/main/java/org/apache/storm/redis/common/mapper/RedisDataTypeDescription.java @@ -23,7 +23,7 @@ * RedisDataTypeDescription defines data type and additional key if needed for lookup / store tuples. */ public class RedisDataTypeDescription implements Serializable { - public enum RedisDataType { STRING, HASH, LIST, SET, SORTED_SET, HYPER_LOG_LOG } + public enum RedisDataType { STRING, HASH, LIST, SET, SORTED_SET, HYPER_LOG_LOG, GEO } private RedisDataType dataType; private String additionalKey; @@ -45,9 +45,10 @@ public RedisDataTypeDescription(RedisDataType dataType, String additionalKey) { this.dataType = dataType; this.additionalKey = additionalKey; - if (dataType == RedisDataType.HASH || dataType == RedisDataType.SORTED_SET) { + if (dataType == RedisDataType.HASH || + dataType == RedisDataType.SORTED_SET || dataType == RedisDataType.GEO) { if (additionalKey == null) { - throw new IllegalArgumentException("Hash and Sorted Set should have additional key"); + throw new IllegalArgumentException("Hash, Sorted Set and GEO should have additional key"); } } } From c4f0cebf550b3a05b2f22f4e7c55c48b6634db87 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Mon, 16 May 2016 15:02:31 +0900 Subject: [PATCH 09/58] add STORM-1720 to CHANGELOG.md --- CHANGELOG.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index a85e78671bf..89a3b0af202 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,6 @@ +## 1.1.0 + * STORM-1720: Support GEO in storm-redis + ## 1.0.2 * STORM-1773: Utils.javaDeserialize() doesn't work with primitive types * STORM-1661: Introduce a config to turn off blobstore acl validation in insecure mode From 6a502485a2adb5d7bb8f4388d053482b0de3fbb0 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Mon, 16 May 2016 15:55:22 +0900 Subject: [PATCH 10/58] add STORM-1678 to CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 89a3b0af202..9257d00aacf 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,6 +2,7 @@ * STORM-1720: Support GEO in storm-redis ## 1.0.2 + * STORM-1678: abstract batch processing to common api `BatchHelper` * STORM-1773: Utils.javaDeserialize() doesn't work with primitive types * STORM-1661: Introduce a config to turn off blobstore acl validation in insecure mode From c7ee99a9e747ad539676cdda0c7ab435f7fb4eaa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stig=20Rohde=20D=C3=B8ssing?= Date: Fri, 15 Apr 2016 11:26:19 +0200 Subject: [PATCH 11/58] STORM-1713: Replace NotImplementedException with UnsupportedOperationException --- .../jvm/org/apache/storm/kafka/StringMultiSchemeWithTopic.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/StringMultiSchemeWithTopic.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/StringMultiSchemeWithTopic.java index d92a8794091..e1973183bab 100644 --- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/StringMultiSchemeWithTopic.java +++ b/external/storm-kafka/src/jvm/org/apache/storm/kafka/StringMultiSchemeWithTopic.java @@ -20,7 +20,6 @@ import org.apache.storm.spout.MultiScheme; import org.apache.storm.tuple.Fields; import org.apache.storm.tuple.Values; -import sun.reflect.generics.reflectiveObjects.NotImplementedException; import java.nio.ByteBuffer; import java.util.Collections; @@ -34,7 +33,7 @@ public class StringMultiSchemeWithTopic @Override public Iterable> deserialize(ByteBuffer bytes) { - throw new NotImplementedException(); + throw new UnsupportedOperationException(); } public Iterable> deserializeWithTopic(String topic, ByteBuffer bytes) { From c4f9255e034038eaf88b1e39aeecfcb40065c07e Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Mon, 16 May 2016 16:35:58 +0900 Subject: [PATCH 12/58] add STORM-1713 to CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 9257d00aacf..16d67d6328b 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,6 +2,7 @@ * STORM-1720: Support GEO in storm-redis ## 1.0.2 + * STORM-1713: Replace NotImplementedException with UnsupportedOperationException * STORM-1678: abstract batch processing to common api `BatchHelper` * STORM-1773: Utils.javaDeserialize() doesn't work with primitive types * STORM-1661: Introduce a config to turn off blobstore acl validation in insecure mode From c76f94c95489f037ea717969b5d4336b23e3b455 Mon Sep 17 00:00:00 2001 From: Xin Wang Date: Tue, 19 Apr 2016 10:09:11 +0800 Subject: [PATCH 13/58] StormSubmitter link fix --- bin/storm.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/storm.py b/bin/storm.py index 5f550ff1345..bc44f56c636 100755 --- a/bin/storm.py +++ b/bin/storm.py @@ -228,7 +228,7 @@ def jar(jarfile, klass, *args): 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 - (http://storm.apache.org/apidocs/org/apache/storm/StormSubmitter.html) + (http://storm.apache.org/releases/current/javadocs/org/apache/storm/StormSubmitter.html) will upload the jar at topology-jar-path when the topology is submitted. """ transform_class = confvalue("client.jartransformer.class", [CLUSTER_CONF_DIR]) From ee8f144f574ca8457f4a0b62f32b4739d14d5bb0 Mon Sep 17 00:00:00 2001 From: darionyaphet Date: Wed, 20 Apr 2016 20:57:09 +0800 Subject: [PATCH 14/58] using Jedis Protocol.DEFAULT_HOST to replace DEFAULT_HOST --- .../org/apache/storm/redis/common/config/JedisPoolConfig.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/external/storm-redis/src/main/java/org/apache/storm/redis/common/config/JedisPoolConfig.java b/external/storm-redis/src/main/java/org/apache/storm/redis/common/config/JedisPoolConfig.java index d555a2a9537..149bdad078f 100644 --- a/external/storm-redis/src/main/java/org/apache/storm/redis/common/config/JedisPoolConfig.java +++ b/external/storm-redis/src/main/java/org/apache/storm/redis/common/config/JedisPoolConfig.java @@ -25,7 +25,6 @@ * Configuration for JedisPool. */ public class JedisPoolConfig implements Serializable { - public static final String DEFAULT_HOST = "127.0.0.1"; private String host; private int port; @@ -99,7 +98,7 @@ public String getPassword() { * Builder for initializing JedisPoolConfig. */ public static class Builder { - private String host = DEFAULT_HOST; + private String host = Protocol.DEFAULT_HOST; private int port = Protocol.DEFAULT_PORT; private int timeout = Protocol.DEFAULT_TIMEOUT; private int database = Protocol.DEFAULT_DATABASE; From 6ac45df99b5f382f5881cd3ab88249a8cfd64198 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Mon, 16 May 2016 16:58:23 +0900 Subject: [PATCH 15/58] add STORM-1715 to CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 16d67d6328b..35c9b4c81aa 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,6 +2,7 @@ * STORM-1720: Support GEO in storm-redis ## 1.0.2 + * STORM-1715: using Jedis Protocol.DEFAULT_HOST to replace DEFAULT_HOST * STORM-1713: Replace NotImplementedException with UnsupportedOperationException * STORM-1678: abstract batch processing to common api `BatchHelper` * STORM-1773: Utils.javaDeserialize() doesn't work with primitive types From 93637a3e6308ef01ca3c2c1f05a8e9725340fae2 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Mon, 16 May 2016 17:21:30 +0900 Subject: [PATCH 16/58] add STORM-1750 to CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 35c9b4c81aa..9abb0829555 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,6 +2,7 @@ * STORM-1720: Support GEO in storm-redis ## 1.0.2 + * STORM-1750: Ensure worker dies when report-error-and-die is called. * STORM-1715: using Jedis Protocol.DEFAULT_HOST to replace DEFAULT_HOST * STORM-1713: Replace NotImplementedException with UnsupportedOperationException * STORM-1678: abstract batch processing to common api `BatchHelper` From e7b7a01b560e4edf118a62189ebce067f3192b57 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stig=20Rohde=20D=C3=B8ssing?= Date: Mon, 2 May 2016 15:07:07 +0200 Subject: [PATCH 17/58] STORM-1756: Explicitly null KafkaServer reference in KafkaTestBroker to prevent out of memory on large test classes --- .../src/test/org/apache/storm/kafka/KafkaTestBroker.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/external/storm-kafka/src/test/org/apache/storm/kafka/KafkaTestBroker.java b/external/storm-kafka/src/test/org/apache/storm/kafka/KafkaTestBroker.java index e2fb60f5a03..90c56e6c55e 100644 --- a/external/storm-kafka/src/test/org/apache/storm/kafka/KafkaTestBroker.java +++ b/external/storm-kafka/src/test/org/apache/storm/kafka/KafkaTestBroker.java @@ -78,7 +78,12 @@ public int getPort() { return port; } public void shutdown() { - kafka.shutdown(); + if (kafka != null) { + kafka.shutdown(); + kafka.awaitShutdown(); + } + //Ensure kafka server is eligible for garbage collection immediately + kafka = null; if (zookeeper.getState().equals(CuratorFrameworkState.STARTED)) { zookeeper.close(); } From f9780140f5620d824a03f8f26b3fe693f23229b4 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Mon, 16 May 2016 18:01:40 +0900 Subject: [PATCH 18/58] add STORM-1756 to CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 9abb0829555..cb7843880ac 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,6 +2,7 @@ * STORM-1720: Support GEO in storm-redis ## 1.0.2 + * STORM-1756: Explicitly null KafkaServer reference in KafkaTestBroker to prevent out of memory on large test classes. * STORM-1750: Ensure worker dies when report-error-and-die is called. * STORM-1715: using Jedis Protocol.DEFAULT_HOST to replace DEFAULT_HOST * STORM-1713: Replace NotImplementedException with UnsupportedOperationException From 928bfe53df5cb59775b3d2f8bf2b47669ca95038 Mon Sep 17 00:00:00 2001 From: Hugo Louro Date: Tue, 3 May 2016 16:32:45 -0700 Subject: [PATCH 19/58] STORM-1761: Storm-Solr Example Throws ArrayIndexOutOfBoundsException in Remote Cluster Mode --- .../java/org/apache/storm/solr/topology/SolrTopology.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/external/storm-solr/src/test/java/org/apache/storm/solr/topology/SolrTopology.java b/external/storm-solr/src/test/java/org/apache/storm/solr/topology/SolrTopology.java index e0f4dc6906c..92d90a32110 100644 --- a/external/storm-solr/src/test/java/org/apache/storm/solr/topology/SolrTopology.java +++ b/external/storm-solr/src/test/java/org/apache/storm/solr/topology/SolrTopology.java @@ -18,12 +18,12 @@ package org.apache.storm.solr.topology; +import org.apache.solr.client.solrj.SolrClient; +import org.apache.solr.client.solrj.impl.CloudSolrClient; import org.apache.storm.Config; import org.apache.storm.LocalCluster; import org.apache.storm.StormSubmitter; import org.apache.storm.generated.StormTopology; -import org.apache.solr.client.solrj.SolrClient; -import org.apache.solr.client.solrj.impl.CloudSolrClient; import org.apache.storm.solr.config.SolrCommitStrategy; import org.apache.storm.solr.config.SolrConfig; @@ -39,7 +39,7 @@ public void run(String[] args) throws Exception { if (args.length == 0) { submitTopologyLocalCluster(topology, config); } else { - submitTopologyRemoteCluster(args[1], topology, config); + submitTopologyRemoteCluster(args[0], topology, config); } } From 673326f2660bea6c6d672f9a58afd33852bc860d Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Mon, 16 May 2016 18:08:34 +0900 Subject: [PATCH 20/58] add STORM-1761 to CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index cb7843880ac..b5c1ee38342 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,6 +2,7 @@ * STORM-1720: Support GEO in storm-redis ## 1.0.2 + * STORM-1761: Storm-Solr Example Throws ArrayIndexOutOfBoundsException in Remote Cluster Mode * STORM-1756: Explicitly null KafkaServer reference in KafkaTestBroker to prevent out of memory on large test classes. * STORM-1750: Ensure worker dies when report-error-and-die is called. * STORM-1715: using Jedis Protocol.DEFAULT_HOST to replace DEFAULT_HOST From 3280fcf763e548679973156dadaf26189ba98b03 Mon Sep 17 00:00:00 2001 From: Kyle Nusbaum Date: Mon, 2 May 2016 14:29:06 -0500 Subject: [PATCH 21/58] Test Pacemaker Fix --- .../storm/pacemaker/PacemakerServer.java | 9 ++++++-- .../storm/pacemaker/codec/ThriftDecoder.java | 21 +++++++------------ 2 files changed, 15 insertions(+), 15 deletions(-) diff --git a/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerServer.java b/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerServer.java index 46ba3641d6e..fa73d966ba7 100644 --- a/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerServer.java +++ b/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerServer.java @@ -138,8 +138,13 @@ public void received(Object mesg, String remote, Channel channel) throws Interru LOG.debug("received message. Passing to handler. {} : {} : {}", handler.toString(), m.toString(), channel.toString()); HBMessage response = handler.handleMessage(m, authenticated); - LOG.debug("Got Response from handler: {}", response.toString()); - channel.write(response); + if(response != null) { + LOG.debug("Got Response from handler: {}", response); + channel.write(response); + } + else { + LOG.info("Got null response from handler handling message: {}", m); + } } public void closeChannel(Channel c) { diff --git a/storm-core/src/jvm/org/apache/storm/pacemaker/codec/ThriftDecoder.java b/storm-core/src/jvm/org/apache/storm/pacemaker/codec/ThriftDecoder.java index 25edefe522b..7f21cf800c6 100644 --- a/storm-core/src/jvm/org/apache/storm/pacemaker/codec/ThriftDecoder.java +++ b/storm-core/src/jvm/org/apache/storm/pacemaker/codec/ThriftDecoder.java @@ -29,18 +29,20 @@ public class ThriftDecoder extends FrameDecoder { + private static final int INTEGER_SIZE = 4; + @Override protected Object decode(ChannelHandlerContext ctx, Channel channel, ChannelBuffer buf) throws Exception { long available = buf.readableBytes(); - if(available < 2) { + if(available < INTEGER_SIZE) { return null; } buf.markReaderIndex(); int thriftLen = buf.readInt(); - available -= 4; + available -= INTEGER_SIZE; if(available < thriftLen) { // We haven't received the entire object yet, return and wait for more bytes. @@ -48,18 +50,11 @@ protected Object decode(ChannelHandlerContext ctx, Channel channel, ChannelBuffe return null; } - buf.discardReadBytes(); + + byte serialized[] = new byte[thriftLen]; + buf.readBytes(serialized, 0, thriftLen); + HBMessage m = (HBMessage)Utils.thriftDeserialize(HBMessage.class, serialized); - HBMessage m; - if(buf.hasArray()) { - m = Utils.thriftDeserialize(HBMessage.class, buf.array(), 0, thriftLen); - buf.readerIndex(buf.readerIndex() + thriftLen); - } - else { - byte serialized[] = new byte[thriftLen]; - buf.readBytes(serialized, 0, thriftLen); - m = Utils.thriftDeserialize(HBMessage.class, serialized); - } if(m.get_type() == HBServerMessageType.CONTROL_MESSAGE) { ControlMessage cm = ControlMessage.read(m.get_data().get_message_blob()); From c886c8d7190815332556bff9c5494ee2f07c3ea6 Mon Sep 17 00:00:00 2001 From: Kyle Nusbaum Date: Wed, 4 May 2016 11:00:51 -0500 Subject: [PATCH 22/58] Bit of cleanup --- .../src/jvm/org/apache/storm/pacemaker/codec/ThriftDecoder.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/storm-core/src/jvm/org/apache/storm/pacemaker/codec/ThriftDecoder.java b/storm-core/src/jvm/org/apache/storm/pacemaker/codec/ThriftDecoder.java index 7f21cf800c6..ce9c8990125 100644 --- a/storm-core/src/jvm/org/apache/storm/pacemaker/codec/ThriftDecoder.java +++ b/storm-core/src/jvm/org/apache/storm/pacemaker/codec/ThriftDecoder.java @@ -50,12 +50,10 @@ protected Object decode(ChannelHandlerContext ctx, Channel channel, ChannelBuffe return null; } - byte serialized[] = new byte[thriftLen]; buf.readBytes(serialized, 0, thriftLen); HBMessage m = (HBMessage)Utils.thriftDeserialize(HBMessage.class, serialized); - if(m.get_type() == HBServerMessageType.CONTROL_MESSAGE) { ControlMessage cm = ControlMessage.read(m.get_data().get_message_blob()); return cm; From 5975ccfd54611e86b931c4c04e0baab09a40bc84 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Mon, 16 May 2016 18:22:24 +0900 Subject: [PATCH 23/58] add STORM-1764 to CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index b5c1ee38342..f8b9d319a5c 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,6 +2,7 @@ * STORM-1720: Support GEO in storm-redis ## 1.0.2 + * STORM-1764: Pacemaker is throwing some stack traces * STORM-1761: Storm-Solr Example Throws ArrayIndexOutOfBoundsException in Remote Cluster Mode * STORM-1756: Explicitly null KafkaServer reference in KafkaTestBroker to prevent out of memory on large test classes. * STORM-1750: Ensure worker dies when report-error-and-die is called. From bbb06d4b7df820fa563abc3b968b2e0578281c7d Mon Sep 17 00:00:00 2001 From: Abhishek Agarwal Date: Thu, 5 May 2016 23:41:40 +0530 Subject: [PATCH 24/58] STORM-1749: Fix storm-starter github links --- docs/Clojure-DSL.md | 4 ++-- docs/Common-patterns.md | 4 ++-- docs/Distributed-RPC.md | 2 +- docs/Transactional-topologies.md | 8 ++++---- docs/Trident-state.md | 4 ++-- docs/Tutorial.md | 2 +- examples/storm-starter/README.markdown | 12 ++++++------ 7 files changed, 18 insertions(+), 18 deletions(-) diff --git a/docs/Clojure-DSL.md b/docs/Clojure-DSL.md index 816a5e35448..e71243c5be9 100644 --- a/docs/Clojure-DSL.md +++ b/docs/Clojure-DSL.md @@ -17,7 +17,7 @@ This page outlines all the pieces of the Clojure DSL, including: 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]({{page.git-blob-base}}/examples/storm-starter/src/clj/storm/starter/clj/word_count.clj): +Let's take a look at an example topology definition [from the storm-starter project]({{page.git-blob-base}}/examples/storm-starter/src/clj/org/apache/storm/starter/clj/word_count.clj): ```clojure (topology @@ -203,7 +203,7 @@ The signature for `defspout` looks like the following: 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]({{page.git-blob-base}}/examples/storm-starter/src/clj/storm/starter/clj/word_count.clj): +Here's an example `defspout` implementation from [storm-starter]({{page.git-blob-base}}/examples/storm-starter/src/clj/org/apache/storm/starter/clj/word_count.clj): ```clojure (defspout sentence-spout ["sentence"] diff --git a/docs/Common-patterns.md b/docs/Common-patterns.md index 9f5ffe7fac5..b98fe90ce24 100644 --- a/docs/Common-patterns.md +++ b/docs/Common-patterns.md @@ -70,7 +70,7 @@ 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]({{page.git-blob-base}}/examples/storm-starter/src/jvm/storm/starter/RollingTopWords.java). +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]({{page.git-blob-base}}/examples/storm-starter/src/jvm/org/apache/storm/starter/RollingTopWords.java). If however you have a known skew in the data being processed it can be advantageous to use partialKeyGrouping instead of fieldsGrouping. This will distribute the load for each key between two downstream bolts instead of a single one. @@ -83,7 +83,7 @@ builder.setBolt("merge", new MergeRanksObjects()) .globalGrouping("rank"); ``` -The topology needs an extra layer of processing to aggregate the partial counts from the upstream bolts but this only processes aggregated values now so the bolt it is not subject to the load caused by the skewed data. You can see an example of this pattern in storm-starter [here]({{page.git-blob-base}}/examples/storm-starter/src/jvm/storm/starter/SkewedRollingTopWords.java). +The topology needs an extra layer of processing to aggregate the partial counts from the upstream bolts but this only processes aggregated values now so the bolt it is not subject to the load caused by the skewed data. You can see an example of this pattern in storm-starter [here]({{page.git-blob-base}}/examples/storm-starter/src/jvm/org/apache/storm/starter/SkewedRollingTopWords.java). ### TimeCacheMap for efficiently keeping a cache of things that have been recently updated diff --git a/docs/Distributed-RPC.md b/docs/Distributed-RPC.md index 2ad63e57e96..b20419a080d 100644 --- a/docs/Distributed-RPC.md +++ b/docs/Distributed-RPC.md @@ -118,7 +118,7 @@ The reach of a URL is the number of unique people exposed to a URL on Twitter. T 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]({{page.git-blob-base}}/examples/storm-starter/src/jvm/storm/starter/ReachTopology.java). Here's how you define the reach topology: +A sample reach topology is defined in storm-starter [here]({{page.git-blob-base}}/examples/storm-starter/src/jvm/org/apache/storm/starter/ReachTopology.java). Here's how you define the reach topology: ```java LinearDRPCTopologyBuilder builder = new LinearDRPCTopologyBuilder("reach"); diff --git a/docs/Transactional-topologies.md b/docs/Transactional-topologies.md index a91d6c2fe59..db5509f6c08 100644 --- a/docs/Transactional-topologies.md +++ b/docs/Transactional-topologies.md @@ -81,7 +81,7 @@ Finally, another thing to note is that transactional topologies require a source ## The basics through example -You build transactional topologies by using [TransactionalTopologyBuilder](javadocs/org/apache/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]({{page.git-blob-base}}/examples/storm-starter/src/jvm/storm/starter/TransactionalGlobalCount.java) in storm-starter. +You build transactional topologies by using [TransactionalTopologyBuilder](javadocs/org/apache/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]({{page.git-blob-base}}/examples/storm-starter/src/jvm/org/apache/storm/starter/TransactionalGlobalCount.java) in storm-starter. ```java MemoryTransactionalSpout spout = new MemoryTransactionalSpout(DATA, new Fields("word"), PARTITION_TAKE_PER_BATCH); @@ -201,7 +201,7 @@ First, notice that this bolt implements the `ICommitter` interface. This tells S 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]({{page.git-blob-base}}/examples/storm-starter/src/jvm/storm/starter/TransactionalWords.java) class. +A more involved transactional topology example that updates multiple databases idempotently can be found in storm-starter in the [TransactionalWords]({{page.git-blob-base}}/examples/storm-starter/src/jvm/org/apache/storm/starter/TransactionalWords.java) class. ## Transactional Topology API @@ -255,7 +255,7 @@ The details of implementing a `TransactionalSpout` are in [the Javadoc](javadocs #### 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]({{page.git-tree-base}}/external/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/org/apache/storm/transactional/partitioned/IPartitionedTransactionalSpout.html) for more details. +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]({{page.git-tree-base}}/external/storm-kafka/src/jvm/org/apache/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/org/apache/storm/transactional/partitioned/IPartitionedTransactionalSpout.html) for more details. ### Configuration @@ -325,7 +325,7 @@ In this scenario, tuples 41-50 are skipped. By failing all subsequent transactio 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/org/apache/storm/transactional/partitioned/IOpaquePartitionedTransactionalSpout.html) is an interface for implementing opaque partitioned transactional spouts, of which [OpaqueTransactionalKafkaSpout]({{page.git-tree-base}}/external/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. +A non-idempotent transactional spout is more concisely referred to as an "OpaqueTransactionalSpout" (opaque is the opposite of idempotent). [IOpaquePartitionedTransactionalSpout](javadocs/org/apache/storm/transactional/partitioned/IOpaquePartitionedTransactionalSpout.html) is an interface for implementing opaque partitioned transactional spouts, of which [OpaqueTransactionalKafkaSpout]({{page.git-tree-base}}/external/storm-kafka/src/jvm/org/apache/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 diff --git a/docs/Trident-state.md b/docs/Trident-state.md index 4ebb60ad2c2..bb5b1ee70b0 100644 --- a/docs/Trident-state.md +++ b/docs/Trident-state.md @@ -28,7 +28,7 @@ Remember, Trident processes tuples as small batches with each batch being given 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]({{page.git-tree-base}}/external/storm-kafka/src/jvm/storm/kafka/trident/TransactionalTridentKafkaSpout.java) for Kafka. +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]({{page.git-tree-base}}/external/storm-kafka/src/jvm/org/apache/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. @@ -72,7 +72,7 @@ As described before, an opaque transactional spout cannot guarantee that the bat 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]({{page.git-tree-base}}/external/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. +[OpaqueTridentKafkaSpout]({{page.git-tree-base}}/external/storm-kafka/src/jvm/org/apache/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. diff --git a/docs/Tutorial.md b/docs/Tutorial.md index 95e4283f490..5dad8340de1 100644 --- a/docs/Tutorial.md +++ b/docs/Tutorial.md @@ -245,7 +245,7 @@ A stream grouping tells a topology how to send tuples between two components. Re 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/apache/storm/blob/{{page.version}}/examples/storm-starter). This [WordCountTopology]({{page.git-blob-base}}/examples/storm-starter/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: +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/apache/storm/blob/{{page.version}}/examples/storm-starter). This [WordCountTopology]({{page.git-blob-base}}/examples/storm-starter/src/jvm/org/apache/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(); diff --git a/examples/storm-starter/README.markdown b/examples/storm-starter/README.markdown index 48d9d274b40..d68ca6bbc23 100644 --- a/examples/storm-starter/README.markdown +++ b/examples/storm-starter/README.markdown @@ -35,13 +35,13 @@ code. storm-starter contains a variety of examples of using Storm. If this is your first time working with Storm, check out these topologies first: -1. [ExclamationTopology](src/jvm/storm/starter/ExclamationTopology.java): Basic topology written in all Java -2. [WordCountTopology](src/jvm/storm/starter/WordCountTopology.java): Basic topology that makes use of multilang by +1. [ExclamationTopology](src/jvm/org/apache/storm/starter/ExclamationTopology.java): Basic topology written in all Java +2. [WordCountTopology](src/jvm/org/apache/storm/starter/WordCountTopology.java): Basic topology that makes use of multilang by implementing one bolt in Python -3. [ReachTopology](src/jvm/storm/starter/ReachTopology.java): Example of complex DRPC on top of Storm +3. [ReachTopology](src/jvm/org/apache/storm/starter/ReachTopology.java): Example of complex DRPC on top of Storm After you have familiarized yourself with these topologies, take a look at the other topopologies in -[src/jvm/storm/starter/](src/jvm/storm/starter/) such as [RollingTopWords](src/jvm/storm/starter/RollingTopWords.java) +[src/jvm/org/apache/storm/starter/](src/jvm/org/apache/storm/starter/) such as [RollingTopWords](src/jvm/org/apache/storm/starter/RollingTopWords.java) for more advanced implementations. If you want to learn more about how Storm works, please head over to the @@ -99,9 +99,9 @@ With submitting you can run topologies which use multilang, for example, `WordCo _Submitting a topology in local vs. remote mode:_ It depends on the actual code of a topology how you can or even must tell Storm whether to run the topology locally (in an in-memory LocalCluster instance of Storm) or remotely (in a "real" Storm cluster). In the case of -[RollingTopWords](src/jvm/storm/starter/RollingTopWords.java), for instance, this can be done by passing command line +[RollingTopWords](src/jvm/org/apache/storm/starter/RollingTopWords.java), for instance, this can be done by passing command line arguments. -Topologies other than `RollingTopWords` -- such as [ExclamationTopology](src/jvm/storm/starter/ExclamationTopology.java) +Topologies other than `RollingTopWords` -- such as [ExclamationTopology](src/jvm/org/apache/storm/starter/ExclamationTopology.java) -- may behave differently, e.g. by always submitting to a remote cluster (i.e. hardcoded in a way that you, as a user, cannot change without modifying the topology code), or by requiring a customized configuration file that the topology code will parse prior submitting the topology to Storm. Similarly, further options such as the name of the topology may From ca2447217589cdc1c63c97cd076950eecb0edab7 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Mon, 16 May 2016 18:37:05 +0900 Subject: [PATCH 25/58] add STORM-1749 to CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index f8b9d319a5c..2db48ae9445 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,6 +2,7 @@ * STORM-1720: Support GEO in storm-redis ## 1.0.2 + * STORM-1749: Fix storm-starter github links * STORM-1764: Pacemaker is throwing some stack traces * STORM-1761: Storm-Solr Example Throws ArrayIndexOutOfBoundsException in Remote Cluster Mode * STORM-1756: Explicitly null KafkaServer reference in KafkaTestBroker to prevent out of memory on large test classes. From 80d508b5efe0dd1da2b94817da4eb5a6dd0cb841 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Mon, 16 May 2016 18:49:29 +0900 Subject: [PATCH 26/58] add STORM-1835 to CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 2db48ae9445..8459b595ea9 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,6 +2,7 @@ * STORM-1720: Support GEO in storm-redis ## 1.0.2 + * STORM-1835: add lock info in thread dump * STORM-1749: Fix storm-starter github links * STORM-1764: Pacemaker is throwing some stack traces * STORM-1761: Storm-Solr Example Throws ArrayIndexOutOfBoundsException in Remote Cluster Mode From 64c88d3aa53a45c2683ce171d5073a9dcb9b8ddc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stig=20Rohde=20D=C3=B8ssing?= Date: Wed, 27 Apr 2016 12:03:43 +0200 Subject: [PATCH 27/58] STORM-1735: Nimbus should log that replication succeeded when min replicas was reached exactly --- storm-core/src/clj/org/apache/storm/daemon/nimbus.clj | 6 +++--- 1 file changed, 3 insertions(+), 3 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 07868244bb4..03664881008 100644 --- a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj +++ b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj @@ -513,9 +513,9 @@ (reset! current-replication-count-jar (get-blob-replication-count (master-stormjar-key storm-id) nimbus))) (reset! current-replication-count-code (get-blob-replication-count (master-stormcode-key storm-id) nimbus)) (reset! current-replication-count-conf (get-blob-replication-count (master-stormconf-key storm-id) nimbus)))) - (if (and (< min-replication-count @current-replication-count-conf) - (< min-replication-count @current-replication-count-code) - (< min-replication-count @current-replication-count-jar)) + (if (and (<= min-replication-count @current-replication-count-conf) + (<= min-replication-count @current-replication-count-code) + (<= min-replication-count @current-replication-count-jar)) (log-message "desired replication count " min-replication-count " achieved, " "current-replication-count for conf key = " @current-replication-count-conf ", " "current-replication-count for code key = " @current-replication-count-code ", " From 780ec4168d9a920548c1de6b9e93c11e77fd209d Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Mon, 16 May 2016 19:09:22 +0900 Subject: [PATCH 28/58] add STORM-1735 to CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 8459b595ea9..c752d89c8c5 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,6 +2,7 @@ * STORM-1720: Support GEO in storm-redis ## 1.0.2 + * STORM-1735: Nimbus should log that replication succeeded when min replicas was reached exactly * STORM-1835: add lock info in thread dump * STORM-1749: Fix storm-starter github links * STORM-1764: Pacemaker is throwing some stack traces From 3b316ea4d2c40b5e0d6d72b56f18f50113cbe391 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Mon, 16 May 2016 20:37:07 -0700 Subject: [PATCH 29/58] Merge branch 'Sorrow17-patch-1' of https://github.com/Sorrow17/storm --- storm-core/src/clj/org/apache/storm/daemon/worker.clj | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 b8bc423e5c7..689f89fc786 100644 --- a/storm-core/src/clj/org/apache/storm/daemon/worker.clj +++ b/storm-core/src/clj/org/apache/storm/daemon/worker.clj @@ -75,7 +75,7 @@ (try (.worker-heartbeat! (:storm-cluster-state worker) (:storm-id worker) (:assignment-id worker) (:port worker) zk-hb) (catch Exception exc - (log-error exc "Worker failed to write heatbeats to ZK or Pacemaker...will retry"))))) + (log-error exc "Worker failed to write heartbeats to ZK or Pacemaker...will retry"))))) (defn do-heartbeat [worker] (let [conf (:conf worker) From 0a1c6db79f4a4b9dd46a6245c28c71fdc619c0b8 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Mon, 16 May 2016 22:12:30 -0700 Subject: [PATCH 30/58] Merge branch 'STORM-1709' of https://github.com/arunmahadevan/storm into STORM-1709 --- .../org/apache/storm/sql/StormSqlImpl.java | 25 +- .../storm/sql/compiler/ExprCompiler.java | 2 +- .../standalone/BuiltinAggregateFunctions.java | 178 ++++++++++++++ .../backends/standalone/PlanCompiler.java | 3 +- .../backends/standalone/RelNodeCompiler.java | 227 +++++++++++++++++- .../org/apache/storm/sql/TestStormSql.java | 151 ++++++++++++ .../sql/runtime/AbstractChannelHandler.java | 5 + .../storm/sql/runtime/ChannelContext.java | 1 + .../storm/sql/runtime/ChannelHandler.java | 2 + .../apache/storm/sql/runtime/Channels.java | 15 ++ .../test/org/apache/storm/sql/TestUtils.java | 36 +++ 11 files changed, 640 insertions(+), 5 deletions(-) create mode 100644 external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/BuiltinAggregateFunctions.java diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlImpl.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlImpl.java index bf5a921c360..dbe0ddde254 100644 --- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlImpl.java +++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlImpl.java @@ -19,6 +19,8 @@ import org.apache.calcite.jdbc.CalciteSchema; import org.apache.calcite.prepare.CalciteCatalogReader; +import org.apache.calcite.schema.Function; +import org.apache.calcite.schema.impl.AggregateFunctionImpl; import org.apache.calcite.schema.impl.ScalarFunctionImpl; import org.apache.calcite.sql.SqlOperatorTable; import org.apache.calcite.sql.fun.SqlStdOperatorTable; @@ -51,6 +53,7 @@ import java.io.ByteArrayOutputStream; import java.io.FileOutputStream; import java.io.IOException; +import java.lang.reflect.Method; import java.nio.file.Files; import java.nio.file.Path; import java.util.ArrayList; @@ -168,11 +171,29 @@ private void handleCreateFunction(SqlCreateFunction sqlCreateFunction) throws Cl if(sqlCreateFunction.jarName() != null) { throw new UnsupportedOperationException("UDF 'USING JAR' not implemented"); } - schema.add(sqlCreateFunction.functionName().toUpperCase(), - ScalarFunctionImpl.create(Class.forName(sqlCreateFunction.className()), "evaluate")); + Method method; + Function function; + if ((method=findMethod(sqlCreateFunction.className(), "evaluate")) != null) { + function = ScalarFunctionImpl.create(method); + } else if (findMethod(sqlCreateFunction.className(), "add") != null) { + function = AggregateFunctionImpl.create(Class.forName(sqlCreateFunction.className())); + } else { + throw new RuntimeException("Invalid scalar or aggregate function"); + } + schema.add(sqlCreateFunction.functionName().toUpperCase(), function); hasUdf = true; } + private Method findMethod(String clazzName, String methodName) throws ClassNotFoundException { + Class clazz = Class.forName(clazzName); + for (Method method : clazz.getMethods()) { + if (method.getName().equals(methodName)) { + return method; + } + } + return null; + } + private void handleCreateTableForTrident( SqlCreateTable n, Map dataSources) { List fields = updateSchema(n); 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 df0c27f7100..0977acc9b15 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 @@ -494,7 +494,7 @@ private static String foldNullExpr(String notNullExpr, String } } - private static String printMethodCall(Method method, List args) { + public static String printMethodCall(Method method, List args) { return printMethodCall(method.getDeclaringClass(), method.getName(), Modifier.isStatic(method.getModifiers()), args); } diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/BuiltinAggregateFunctions.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/BuiltinAggregateFunctions.java new file mode 100644 index 00000000000..780affc474f --- /dev/null +++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/BuiltinAggregateFunctions.java @@ -0,0 +1,178 @@ +/** + * 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.sql.compiler.backends.standalone; + +import com.google.common.collect.ImmutableList; +import org.apache.storm.tuple.Values; + +import java.lang.reflect.Type; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Built-in implementations for some of the standard aggregation operations. + * Aggregations can be implemented as a class with the following methods viz. init, add and result. + * The class could contain only static methods, only non-static methods or be generic. + */ +public class BuiltinAggregateFunctions { + // binds the type information and the class implementing the aggregation + public static class TypeClass { + public static class GenericType { + } + + public final Type ty; + public final Class clazz; + + private TypeClass(Type ty, Class clazz) { + this.ty = ty; + this.clazz = clazz; + } + + static TypeClass of(Type ty, Class clazz) { + return new TypeClass(ty, clazz); + } + } + + static final Map> TABLE = new HashMap<>(); + + public static class IntSum { + public static Integer init() { + return 0; + } + + public static Integer add(Integer accumulator, Integer val) { + return accumulator + val; + } + + public static Integer result(Integer accumulator) { + return accumulator; + } + } + + public static class DoubleSum { + public static Double init() { + return 0.0; + } + + public static Double add(Double accumulator, Double val) { + return accumulator + val; + } + + public static Double result(Double accumulator) { + return accumulator; + } + } + + public static class Max> { + public T init() { + return null; + } + + public T add(T accumulator, T val) { + return (accumulator == null || accumulator.compareTo(val) < 0) ? val : accumulator; + } + + public T result(T accumulator) { + return accumulator; + } + } + + public static class Min> { + public T init() { + return null; + } + + public T add(T accumulator, T val) { + return (accumulator == null || accumulator.compareTo(val) > 0) ? val : accumulator; + } + + public T result(T accumulator) { + return accumulator; + } + } + + public static class IntAvg { + private int count; + + public Integer init() { + return 0; + } + + public Integer add(Integer accumulator, Integer val) { + ++count; + return accumulator + val; + } + + public Integer result(Integer accumulator) { + Integer result = accumulator / count; + count = 0; + return result; + } + } + + public static class DoubleAvg { + private int count; + + public Double init() { + return 0.0; + } + + public Double add(Double accumulator, Double val) { + ++count; + return accumulator + val; + } + + public Double result(Double accumulator) { + Double result = accumulator / count; + count = 0; + return result; + } + } + + public static class Count { + public static Long init() { + return 0L; + } + + public static Long add(Long accumulator, Values vals) { + for (Object val : vals) { + if (val == null) { + return accumulator; + } + } + return accumulator + 1; + } + + public static Long result(Long accumulator) { + return accumulator; + } + } + + static { + TABLE.put("SUM", ImmutableList.of( + TypeClass.of(double.class, DoubleSum.class), + TypeClass.of(int.class, IntSum.class))); + TABLE.put("AVG", ImmutableList.of( + TypeClass.of(double.class, DoubleAvg.class), + TypeClass.of(int.class, IntAvg.class))); + TABLE.put("COUNT", ImmutableList.of(TypeClass.of(long.class, Count.class))); + TABLE.put("MAX", ImmutableList.of(TypeClass.of(TypeClass.GenericType.class, Max.class))); + TABLE.put("MIN", ImmutableList.of(TypeClass.of(TypeClass.GenericType.class, Min.class))); + } +} diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PlanCompiler.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PlanCompiler.java index eb7f4ea39db..3a92ae5a912 100644 --- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PlanCompiler.java +++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PlanCompiler.java @@ -38,7 +38,8 @@ public class PlanCompiler { private static final String PACKAGE_NAME = "org.apache.storm.sql.generated"; private static final String PROLOGUE = NEW_LINE_JOINER.join( "// GENERATED CODE", "package " + PACKAGE_NAME + ";", "", - "import java.util.Iterator;", "import java.util.Map;", + "import java.util.Iterator;", "import java.util.Map;", "import java.util.HashMap;", + "import java.util.List;", "import java.util.ArrayList;", "import org.apache.storm.tuple.Values;", "import org.apache.storm.sql.runtime.AbstractChannelHandler;", "import org.apache.storm.sql.runtime.Channels;", 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 845bb3ab00c..7782529516a 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 @@ -18,16 +18,29 @@ package org.apache.storm.sql.compiler.backends.standalone; import com.google.common.base.Joiner; +import com.google.common.primitives.Primitives; import org.apache.calcite.adapter.java.JavaTypeFactory; import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.Aggregate; +import org.apache.calcite.rel.core.AggregateCall; import org.apache.calcite.rel.core.Filter; import org.apache.calcite.rel.core.Project; import org.apache.calcite.rel.core.TableScan; import org.apache.calcite.rel.stream.Delta; +import org.apache.calcite.schema.AggregateFunction; +import org.apache.calcite.schema.impl.AggregateFunctionImpl; +import org.apache.calcite.sql.SqlAggFunction; +import org.apache.calcite.sql.validate.SqlUserDefinedAggFunction; import org.apache.storm.sql.compiler.ExprCompiler; import org.apache.storm.sql.compiler.PostOrderRelNodeVisitor; import java.io.PrintWriter; +import java.io.StringWriter; +import java.lang.reflect.Type; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; /** * Compile RelNodes into individual functions. @@ -44,10 +57,44 @@ class RelNodeCompiler extends PostOrderRelNodeVisitor { " public void dataReceived(ChannelContext ctx, Values _data) {", "" ); + + private static final String AGGREGATE_STAGE_PROLOGUE = NEW_LINE_JOINER.join( + " private static final ChannelHandler %1$s = ", + " new AbstractChannelHandler() {", + " private final Values EMPTY_VALUES = new Values();", + " private List prevGroupValues = null;", + " private final Map accumulators = new HashMap<>();", + " private final int[] groupIndices = new int[] {%2$s};", + " private List getGroupValues(Values _data) {", + " List res = new ArrayList<>();", + " for (int i: groupIndices) {", + " res.add(_data.get(i));", + " }", + " return res;", + " }", + "", + " @Override", + " public void flush(ChannelContext ctx) {", + " emitAggregateResults(ctx);", + " super.flush(ctx);", + " prevGroupValues = null;", + " }", + "", + " private void emitAggregateResults(ChannelContext ctx) {", + " %3$s", + " }", + "", + " @Override", + " public void dataReceived(ChannelContext ctx, Values _data) {", + "" + ); private static final String STAGE_PASSTHROUGH = NEW_LINE_JOINER.join( " private static final ChannelHandler %1$s = AbstractChannelHandler.PASS_THROUGH;", ""); + private int nameCount; + private Map aggregateCallVarNames = new HashMap<>(); + RelNodeCompiler(PrintWriter pw, JavaTypeFactory typeFactory) { this.pw = pw; this.typeFactory = typeFactory; @@ -77,7 +124,6 @@ public Void visitFilter(Filter filter) throws Exception { public Void visitProject(Project project) throws Exception { beginStage(project); ExprCompiler compiler = new ExprCompiler(pw, typeFactory); - int size = project.getChildExps().size(); String[] res = new String[size]; for (int i = 0; i < size; ++i) { @@ -101,10 +147,177 @@ public Void visitTableScan(TableScan scan) throws Exception { return null; } + @Override + public Void visitAggregate(Aggregate aggregate) throws Exception { + beginAggregateStage(aggregate); + pw.println(" List curGroupValues = _data == null ? null : getGroupValues(_data);"); + pw.println(" if (prevGroupValues != null && !prevGroupValues.equals(curGroupValues)) {"); + pw.println(" emitAggregateResults(ctx);"); + pw.println(" }"); + pw.println(" if (curGroupValues != null) {"); + for (AggregateCall call : aggregate.getAggCallList()) { + aggregate(call); + } + pw.println(" }"); + pw.println(" if (prevGroupValues != curGroupValues) {"); + pw.println(" prevGroupValues = curGroupValues;"); + pw.println(" }"); + endStage(); + return null; + } + + private String groupValueEmitStr(String var, int n) { + int count = 0; + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < n; i++) { + if (++count > 1) { + sb.append(", "); + } + sb.append(var).append(".").append("get(").append(i).append(")"); + } + return sb.toString(); + } + + private String emitAggregateStmts(Aggregate aggregate) { + List res = new ArrayList<>(); + StringWriter sw = new StringWriter(); + for (AggregateCall call : aggregate.getAggCallList()) { + res.add(aggregateResult(call, new PrintWriter(sw))); + } + return NEW_LINE_JOINER.join(sw.toString(), + String.format(" ctx.emit(new Values(%s, %s));", + groupValueEmitStr("prevGroupValues", aggregate.getGroupSet().cardinality()), + Joiner.on(", ").join(res)), + " accumulators.clear();" + ); + } + + private String aggregateResult(AggregateCall call, PrintWriter pw) { + SqlAggFunction aggFunction = call.getAggregation(); + String aggregationName = call.getAggregation().getName(); + Type ty = typeFactory.getJavaClass(call.getType()); + String result; + if (aggFunction instanceof SqlUserDefinedAggFunction) { + AggregateFunction aggregateFunction = ((SqlUserDefinedAggFunction) aggFunction).function; + result = doAggregateResult((AggregateFunctionImpl) aggregateFunction, reserveAggVarName(call), ty, pw); + } else { + List typeClasses = BuiltinAggregateFunctions.TABLE.get(aggregationName); + if (typeClasses == null) { + throw new UnsupportedOperationException(aggregationName + " Not implemented"); + } + result = doAggregateResult(AggregateFunctionImpl.create(findMatchingClass(aggregationName, typeClasses, ty)), + reserveAggVarName(call), ty, pw); + } + return result; + } + + private String doAggregateResult(AggregateFunctionImpl aggFn, String varName, Type ty, PrintWriter pw) { + String resultName = varName + "_result"; + List args = new ArrayList<>(); + if (!aggFn.isStatic) { + String aggObjName = String.format("%s_obj", varName); + String aggObjClassName = (aggFn.initMethod.getDeclaringClass().getCanonicalName()); + boolean genericType = aggFn.initMethod.getDeclaringClass().getTypeParameters().length > 0; + if (genericType) { + pw.println(" @SuppressWarnings(\"unchecked\")"); + pw.print(String.format(" final %1$s<%3$s> %2$s = (%1$s<%3$s>) accumulators.get(\"%2$s\");", aggObjClassName, + aggObjName, Primitives.wrap((Class) ty).getCanonicalName())); + } else { + pw.print(String.format(" final %1$s %2$s = (%1$s) accumulators.get(\"%2$s\");", aggObjClassName, aggObjName)); + } + args.add(aggObjName); + } + args.add(String.format("(%s)accumulators.get(\"%s\")", ((Class) ty).getCanonicalName(), varName)); + pw.print(String.format(" final %s %s = %s;", ((Class) ty).getCanonicalName(), + resultName, ExprCompiler.printMethodCall(aggFn.resultMethod, args))); + + return resultName; + } + + private void aggregate(AggregateCall call) { + SqlAggFunction aggFunction = call.getAggregation(); + String aggregationName = call.getAggregation().getName(); + Type ty = typeFactory.getJavaClass(call.getType()); + if (call.getArgList().size() != 1) { + if (aggregationName.equals("COUNT")) { + if (call.getArgList().size() != 0) { + throw new UnsupportedOperationException("Count with nullable fields"); + } + } else { + throw new IllegalArgumentException("Aggregate call should have one argument"); + } + } + if (aggFunction instanceof SqlUserDefinedAggFunction) { + AggregateFunction aggregateFunction = ((SqlUserDefinedAggFunction) aggFunction).function; + doAggregate((AggregateFunctionImpl) aggregateFunction, reserveAggVarName(call), ty, call.getArgList()); + } else { + List typeClasses = BuiltinAggregateFunctions.TABLE.get(aggregationName); + if (typeClasses == null) { + throw new UnsupportedOperationException(aggregationName + " Not implemented"); + } + doAggregate(AggregateFunctionImpl.create(findMatchingClass(aggregationName, typeClasses, ty)), + reserveAggVarName(call), ty, call.getArgList()); + } + } + + private Class findMatchingClass(String aggregationName, List typeClasses, Type ty) { + for (BuiltinAggregateFunctions.TypeClass typeClass : typeClasses) { + if (typeClass.ty.equals(BuiltinAggregateFunctions.TypeClass.GenericType.class) || typeClass.ty.equals(ty)) { + return typeClass.clazz; + } + } + throw new UnsupportedOperationException(aggregationName + " Not implemeted for type '" + ty + "'"); + } + + private void doAggregate(AggregateFunctionImpl aggFn, String varName, Type ty, List argList) { + List args = new ArrayList<>(); + if (!aggFn.isStatic) { + String aggObjName = String.format("%s_obj", varName); + String aggObjClassName = aggFn.initMethod.getDeclaringClass().getCanonicalName(); + pw.println(String.format(" if (!accumulators.containsKey(\"%s\")) { ", aggObjName)); + pw.println(String.format(" accumulators.put(\"%s\", new %s());", aggObjName, aggObjClassName)); + pw.println(" }"); + boolean genericType = aggFn.initMethod.getDeclaringClass().getTypeParameters().length > 0; + if (genericType) { + pw.println(" @SuppressWarnings(\"unchecked\")"); + pw.println(String.format(" final %1$s<%3$s> %2$s = (%1$s<%3$s>) accumulators.get(\"%2$s\");", aggObjClassName, + aggObjName, Primitives.wrap((Class) ty).getCanonicalName())); + } else { + pw.println(String.format(" final %1$s %2$s = (%1$s) accumulators.get(\"%2$s\");", aggObjClassName, aggObjName)); + } + args.add(aggObjName); + } + args.add(String.format("%1$s == null ? %2$s : (%3$s) %1$s", + "accumulators.get(\"" + varName + "\")", + ExprCompiler.printMethodCall(aggFn.initMethod, args), + Primitives.wrap((Class) ty).getCanonicalName())); + if (argList.isEmpty()) { + args.add("EMPTY_VALUES"); + } else { + args.add(String.format("(%s) %s", ((Class) ty).getCanonicalName(), "_data.get(" + argList.get(0) + ")")); + } + pw.print(String.format(" accumulators.put(\"%s\", %s);\n", + varName, + ExprCompiler.printMethodCall(aggFn.addMethod, args))); + } + + private String reserveAggVarName(AggregateCall call) { + String varName; + if ((varName = aggregateCallVarNames.get(call)) == null) { + varName = call.getAggregation().getName() + ++nameCount; + aggregateCallVarNames.put(call, varName); + } + return varName; + } + private void beginStage(RelNode n) { pw.print(String.format(STAGE_PROLOGUE, getStageName(n))); } + private void beginAggregateStage(Aggregate n) { + pw.print(String.format(AGGREGATE_STAGE_PROLOGUE, getStageName(n), getGroupByIndices(n), emitAggregateStmts(n))); + } + private void endStage() { pw.print(" }\n };\n"); } @@ -112,4 +325,16 @@ private void endStage() { static String getStageName(RelNode n) { return n.getClass().getSimpleName().toUpperCase() + "_" + n.getId(); } + + private String getGroupByIndices(Aggregate n) { + StringBuilder res = new StringBuilder(); + int count = 0; + for (int i : n.getGroupSet()) { + if (++count > 1) { + res.append(", "); + } + res.append(i); + } + return res.toString(); + } } 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 872b83dc447..2dd0ba06fa8 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 @@ -20,6 +20,7 @@ import com.google.common.collect.ImmutableMap; import org.apache.calcite.sql.validate.SqlValidatorException; import org.apache.calcite.tools.ValidationException; +import org.apache.storm.sql.compiler.backends.standalone.BuiltinAggregateFunctions; import org.apache.storm.sql.runtime.ChannelHandler; import org.apache.storm.sql.runtime.DataSource; import org.apache.storm.sql.runtime.DataSourcesProvider; @@ -79,11 +80,32 @@ public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, } } + private static class MockGroupDataSourceProvider implements DataSourcesProvider { + @Override + public String scheme() { + return "mockgroup"; + } + + @Override + public DataSource construct( + URI uri, String inputFormatClass, String outputFormatClass, + List fields) { + return new TestUtils.MockGroupDataSource(); + } + + @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()); + DataSourcesRegistry.providerMap().put("mockgroup", new MockGroupDataSourceProvider()); } @AfterClass @@ -190,4 +212,133 @@ public void testExternalUdfUsingJar() throws Exception { ChannelHandler h = new TestUtils.CollectDataChannelHandler(values); sql.execute(stmt, h); } + + @Test + public void testGroupbyBuiltin() throws Exception { + List stmt = new ArrayList<>(); + stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'"); + stmt.add("SELECT STREAM ID, COUNT(*), SUM(SALARY), AVG(SALARY) FROM FOO GROUP BY (ID)"); + StormSql sql = StormSql.construct(); + List values = new ArrayList<>(); + ChannelHandler h = new TestUtils.CollectDataChannelHandler(values); + sql.execute(stmt, h); + Assert.assertEquals(4, values.size()); + Assert.assertEquals(3, values.get(0).get(2)); + Assert.assertEquals(12, values.get(1).get(2)); + Assert.assertEquals(21, values.get(2).get(2)); + Assert.assertEquals(9, values.get(3).get(2)); + } + + @Test + public void testGroupbyBuiltinWithFilter() throws Exception { + List stmt = new ArrayList<>(); + stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'"); + stmt.add("SELECT STREAM ID, COUNT(*), SUM(SALARY), AVG(PCT) FROM FOO WHERE ID = 1 GROUP BY (ID)"); + StormSql sql = StormSql.construct(); + List values = new ArrayList<>(); + ChannelHandler h = new TestUtils.CollectDataChannelHandler(values); + sql.execute(stmt, h); + Assert.assertEquals(1, values.size()); + Assert.assertEquals(1, values.get(0).get(0)); + Assert.assertEquals(3L, values.get(0).get(1)); + Assert.assertEquals(12, values.get(0).get(2)); + Assert.assertEquals(2.5, values.get(0).get(3)); + } + + @Test + public void testGroupbyBuiltinAndUDF() throws Exception { + List stmt = new ArrayList<>(); + stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'"); + stmt.add("CREATE FUNCTION MYCONCAT AS 'org.apache.storm.sql.TestUtils$MyConcat'"); + stmt.add("SELECT STREAM ID, SUM(SALARY), MYCONCAT(NAME) FROM FOO GROUP BY (ID)"); + StormSql sql = StormSql.construct(); + List values = new ArrayList<>(); + ChannelHandler h = new TestUtils.CollectDataChannelHandler(values); + sql.execute(stmt, h); + Assert.assertEquals(4, values.size()); + Assert.assertEquals(3, values.get(0).get(1)); + Assert.assertEquals("xxx", values.get(0).get(2)); + Assert.assertEquals(12, values.get(1).get(1)); + Assert.assertEquals("xxx", values.get(1).get(2)); + Assert.assertEquals(21, values.get(2).get(1)); + Assert.assertEquals("xxx", values.get(2).get(2)); + Assert.assertEquals(9, values.get(3).get(1)); + Assert.assertEquals("x", values.get(3).get(2)); + } + + @Test + public void testGroupbySameAggregateOnDifferentColumns() throws Exception { + List stmt = new ArrayList<>(); + stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'"); + stmt.add("SELECT STREAM ID, COUNT(*), AVG(SALARY), AVG(PCT) FROM FOO WHERE ID = 1 GROUP BY (ID)"); + StormSql sql = StormSql.construct(); + List values = new ArrayList<>(); + ChannelHandler h = new TestUtils.CollectDataChannelHandler(values); + sql.execute(stmt, h); + Assert.assertEquals(1, values.size()); + Assert.assertEquals(1, values.get(0).get(0)); + Assert.assertEquals(3L, values.get(0).get(1)); + Assert.assertEquals(4, values.get(0).get(2)); + Assert.assertEquals(2.5, values.get(0).get(3)); + } + + @Test(expected = UnsupportedOperationException.class) + public void testGroupbyBuiltinNotimplemented() throws Exception { + List stmt = new ArrayList<>(); + stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'"); + stmt.add("SELECT STREAM ID, COUNT(*), STDDEV_POP(SALARY) FROM FOO GROUP BY (ID)"); + StormSql sql = StormSql.construct(); + List values = new ArrayList<>(); + ChannelHandler h = new TestUtils.CollectDataChannelHandler(values); + sql.execute(stmt, h); + } + + @Test + public void testMinMax() throws Exception { + List stmt = new ArrayList<>(); + stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'"); + stmt.add("SELECT STREAM ID, COUNT(*), MIN(SALARY), MAX(PCT) FROM FOO GROUP BY (ID)"); + StormSql sql = StormSql.construct(); + List values = new ArrayList<>(); + ChannelHandler h = new TestUtils.CollectDataChannelHandler(values); + sql.execute(stmt, h); + Assert.assertEquals(4, values.size()); + Assert.assertEquals(0, values.get(0).get(2)); + Assert.assertEquals(3, values.get(1).get(2)); + Assert.assertEquals(6, values.get(2).get(2)); + Assert.assertEquals(9, values.get(3).get(2)); + + Assert.assertEquals(1.5, values.get(0).get(3)); + Assert.assertEquals(3.0, values.get(1).get(3)); + Assert.assertEquals(4.5, values.get(2).get(3)); + Assert.assertEquals(5.0, values.get(3).get(3)); + } + @Test + public void testFilterGroupbyHaving() throws Exception { + List stmt = new ArrayList<>(); + stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'"); + stmt.add("SELECT STREAM ID, MIN(SALARY) FROM FOO where ID > 0 GROUP BY (ID) HAVING ID > 2 AND MAX(SALARY) > 5"); + StormSql sql = StormSql.construct(); + List values = new ArrayList<>(); + ChannelHandler h = new TestUtils.CollectDataChannelHandler(values); + sql.execute(stmt, h); + Assert.assertEquals(1, values.size()); + Assert.assertEquals(3, values.get(0).get(0)); + Assert.assertEquals(9, values.get(0).get(1)); + } + + @Test + public void testGroupByMultipleFields() throws Exception { + List stmt = new ArrayList<>(); + stmt.add("CREATE EXTERNAL TABLE FOO (DEPTID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR, EMPID INT) LOCATION 'mockgroup:///foo'"); + stmt.add("SELECT STREAM DEPTID, EMPID, COUNT(*), MIN(SALARY), MAX(PCT) FROM FOO GROUP BY DEPTID, EMPID"); + StormSql sql = StormSql.construct(); + List values = new ArrayList<>(); + ChannelHandler h = new TestUtils.CollectDataChannelHandler(values); + sql.execute(stmt, h); + Assert.assertEquals(7, values.size()); + Assert.assertEquals(0, values.get(0).get(0)); + Assert.assertEquals(0, values.get(0).get(1)); + Assert.assertEquals(2L, values.get(0).get(2)); + } } diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractChannelHandler.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractChannelHandler.java index 6a8bbe5a610..019a495ddb7 100644 --- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractChannelHandler.java +++ b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractChannelHandler.java @@ -35,6 +35,11 @@ public void exceptionCaught(Throwable cause) { } + @Override + public void flush(ChannelContext ctx) { + ctx.flush(); + } + public static final AbstractChannelHandler PASS_THROUGH = new AbstractChannelHandler() { @Override public void dataReceived(ChannelContext ctx, Values data) { diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelContext.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelContext.java index c29c6b15641..6e5a120f69a 100644 --- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelContext.java +++ b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelContext.java @@ -27,4 +27,5 @@ public interface ChannelContext { */ void emit(Values data); void fireChannelInactive(); + void flush(); } diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelHandler.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelHandler.java index f30d0a392b6..2369e76a58f 100644 --- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelHandler.java +++ b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelHandler.java @@ -36,4 +36,6 @@ public interface ChannelHandler { void channelInactive(ChannelContext ctx); void exceptionCaught(Throwable cause); + + void flush(ChannelContext ctx); } diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/Channels.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/Channels.java index 47dab1154c6..8f21919353f 100644 --- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/Channels.java +++ b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/Channels.java @@ -28,6 +28,11 @@ public void emit(Values data) {} @Override public void fireChannelInactive() {} + + @Override + public void flush() { + + } }; private static class ChannelContextAdapter implements ChannelContext { @@ -49,6 +54,11 @@ public void emit(Values data) { public void fireChannelInactive() { handler.channelInactive(next); } + + @Override + public void flush() { + handler.flush(next); + } } private static class ForwardingChannelContext implements ChannelContext { @@ -67,6 +77,11 @@ public void emit(Values data) { public void fireChannelInactive() { next.fireChannelInactive(); } + + @Override + public void flush() { + next.flush(); + } } public static ChannelContext chain( 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 5091e3a540f..60e134362e1 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 @@ -47,6 +47,19 @@ public static Integer evaluate(Integer x, Integer y) { } } + public static class MyConcat { + public static String init() { + return ""; + } + public static String add(String accumulator, String val) { + return accumulator + val; + } + public static String result(String accumulator) { + return accumulator; + } + } + + public static class MockDataSource implements DataSource { private final ArrayList RECORDS = new ArrayList<>(); @@ -65,6 +78,26 @@ public void open(ChannelContext ctx) { } } + public static class MockGroupDataSource implements DataSource { + private final ArrayList RECORDS = new ArrayList<>(); + + public MockGroupDataSource() { + for (int i = 0; i < 10; ++i) { + RECORDS.add(new Values(i/3, i, (i+1)* 0.5, "x", i/2)); + } + } + + @Override + public void open(ChannelContext ctx) { + for (Values v : RECORDS) { + ctx.emit(v); + } + // force evaluation of the aggregate function on the last group + ctx.flush(); + ctx.fireChannelInactive(); + } + } + public static class MockNestedDataSource implements DataSource { private final ArrayList RECORDS = new ArrayList<>(); @@ -182,6 +215,9 @@ public void channelInactive(ChannelContext ctx) {} public void exceptionCaught(Throwable cause) { throw new RuntimeException(cause); } + + @Override + public void flush(ChannelContext ctx) {} } public static long monotonicNow() { From 16886f58044a1ce52b7593f7bd9d23fda9bc8df9 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Mon, 16 May 2016 22:16:28 -0700 Subject: [PATCH 31/58] Added STORM-1709 to CHANGELOG. --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index c752d89c8c5..4c19733ab0a 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.1.0 + * STORM-1709: Added group by support in storm sql standalone mode * STORM-1720: Support GEO in storm-redis ## 1.0.2 From 844a1fab656efd2e2ab2498ae08d6e6e1483ea49 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stig=20Rohde=20D=C3=B8ssing?= Date: Fri, 29 Apr 2016 11:05:55 +0200 Subject: [PATCH 32/58] STORM-1745: Add partition to log output in PartitionManager --- .../apache/storm/kafka/PartitionManager.java | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) 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 4db8af68adb..4fb5685ff33 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 @@ -112,7 +112,7 @@ public PartitionManager(DynamicPartitionConnections connections, String topology spoutConfig.maxOffsetBehind + " behind latest offset " + currentOffset + ", resetting to startOffsetTime=" + spoutConfig.startOffsetTime); } - LOG.info("Starting Kafka " + _consumer.host() + ":" + id.partition + " from offset " + _committedTo); + LOG.info("Starting Kafka " + _consumer.host() + " " + id + " from offset " + _committedTo); _emittedToOffset = _committedTo; _fetchAPILatencyMax = new CombinedMetric(new MaxMetric()); @@ -204,13 +204,13 @@ private void fill() { _lostMessageCount.incrBy(omitted.size()); } - LOG.warn("Removing the failed offsets that are out of range: {}", omitted); + LOG.warn("Removing the failed offsets for {} that are out of range: {}", _partition, omitted); } if (offset > _emittedToOffset) { _lostMessageCount.incrBy(offset - _emittedToOffset); _emittedToOffset = offset; - LOG.warn("{} Using new offset: {}", _partition.partition, _emittedToOffset); + LOG.warn("{} Using new offset: {}", _partition, _emittedToOffset); } return; @@ -258,12 +258,16 @@ public void ack(Long offset) { public void fail(Long offset) { if (offset < _emittedToOffset - _spoutConfig.maxOffsetBehind) { LOG.info( - "Skipping failed tuple at offset=" + offset + - " because it's more than maxOffsetBehind=" + _spoutConfig.maxOffsetBehind + - " behind _emittedToOffset=" + _emittedToOffset + "Skipping failed tuple at offset={}" + + " because it's more than maxOffsetBehind={}" + + " behind _emittedToOffset={} for {}", + offset, + _spoutConfig.maxOffsetBehind, + _emittedToOffset, + _partition ); } else { - LOG.debug("failing at offset={} with _pending.size()={} pending and _emittedToOffset={}", offset, _pending.size(), _emittedToOffset); + LOG.debug("Failing at offset={} with _pending.size()={} pending and _emittedToOffset={} for {}", offset, _pending.size(), _emittedToOffset, _partition); numberFailed++; if (numberAcked == 0 && numberFailed > _spoutConfig.maxOffsetBehind) { throw new RuntimeException("Too many tuple failures"); From 3d20fcf2944051fe9c68cea3b5ad2466ead4254e Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Tue, 17 May 2016 15:58:53 +0900 Subject: [PATCH 33/58] add STORM-1745 to CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 4c19733ab0a..6794ec3a2ef 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -3,6 +3,7 @@ * STORM-1720: Support GEO in storm-redis ## 1.0.2 + * STORM-1745: Add partition to log output in PartitionManager * STORM-1735: Nimbus should log that replication succeeded when min replicas was reached exactly * STORM-1835: add lock info in thread dump * STORM-1749: Fix storm-starter github links From 4c7117d75f80b820da110c3abee88134ca6ecaaa Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Tue, 17 May 2016 20:45:03 +0900 Subject: [PATCH 34/58] STORM-1842 Forward references in storm.thrift cause tooling issues * python doesn't support forward reference so ttypes.py has a problem regarding this issue * reorder HBPulse, HBRecords, HBNodes to resolve forward reference issue --- .../apache/storm/generated/HBMessageData.java | 6 +- storm-core/src/py/storm/ttypes.py | 422 +++++++++--------- storm-core/src/storm.thrift | 27 +- 3 files changed, 227 insertions(+), 228 deletions(-) diff --git a/storm-core/src/jvm/org/apache/storm/generated/HBMessageData.java b/storm-core/src/jvm/org/apache/storm/generated/HBMessageData.java index 77368f7132c..b100da1e1a2 100644 --- a/storm-core/src/jvm/org/apache/storm/generated/HBMessageData.java +++ b/storm-core/src/jvm/org/apache/storm/generated/HBMessageData.java @@ -139,13 +139,13 @@ public String getFieldName() { tmpMap.put(_Fields.PATH, new org.apache.thrift.meta_data.FieldMetaData("path", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); tmpMap.put(_Fields.PULSE, new org.apache.thrift.meta_data.FieldMetaData("pulse", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT , "HBPulse"))); + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, HBPulse.class))); tmpMap.put(_Fields.BOOLVAL, new org.apache.thrift.meta_data.FieldMetaData("boolval", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); tmpMap.put(_Fields.RECORDS, new org.apache.thrift.meta_data.FieldMetaData("records", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT , "HBRecords"))); + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, HBRecords.class))); tmpMap.put(_Fields.NODES, new org.apache.thrift.meta_data.FieldMetaData("nodes", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT , "HBNodes"))); + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, HBNodes.class))); tmpMap.put(_Fields.MESSAGE_BLOB, new org.apache.thrift.meta_data.FieldMetaData("message_blob", org.apache.thrift.TFieldRequirementType.OPTIONAL, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); metaDataMap = Collections.unmodifiableMap(tmpMap); diff --git a/storm-core/src/py/storm/ttypes.py b/storm-core/src/py/storm/ttypes.py index 98a7ba42689..1934fb2d1c8 100644 --- a/storm-core/src/py/storm/ttypes.py +++ b/storm-core/src/py/storm/ttypes.py @@ -10095,35 +10095,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class HBMessageData: +class HBPulse: """ Attributes: - - path - - pulse - - boolval - - records - - nodes - - message_blob + - id + - details """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'path', None, None, ), # 1 - (2, TType.STRUCT, 'pulse', (HBPulse, HBPulse.thrift_spec), None, ), # 2 - (3, TType.BOOL, 'boolval', None, None, ), # 3 - (4, TType.STRUCT, 'records', (HBRecords, HBRecords.thrift_spec), None, ), # 4 - (5, TType.STRUCT, 'nodes', (HBNodes, HBNodes.thrift_spec), None, ), # 5 - None, # 6 - (7, TType.STRING, 'message_blob', None, None, ), # 7 + (1, TType.STRING, 'id', None, None, ), # 1 + (2, TType.STRING, 'details', None, None, ), # 2 ) - def __init__(self, path=None, pulse=None, boolval=None, records=None, nodes=None, message_blob=None,): - self.path = path - self.pulse = pulse - self.boolval = boolval - self.records = records - self.nodes = nodes - self.message_blob = message_blob + def __init__(self, id=None, details=None,): + self.id = id + self.details = details def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -10136,35 +10123,12 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.path = iprot.readString().decode('utf-8') + self.id = iprot.readString().decode('utf-8') else: iprot.skip(ftype) elif fid == 2: - if ftype == TType.STRUCT: - self.pulse = HBPulse() - self.pulse.read(iprot) - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.BOOL: - self.boolval = iprot.readBool() - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.STRUCT: - self.records = HBRecords() - self.records.read(iprot) - else: - iprot.skip(ftype) - elif fid == 5: - if ftype == TType.STRUCT: - self.nodes = HBNodes() - self.nodes.read(iprot) - else: - iprot.skip(ftype) - elif fid == 7: if ftype == TType.STRING: - self.message_blob = iprot.readString() + self.details = iprot.readString() else: iprot.skip(ftype) else: @@ -10176,46 +10140,28 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('HBMessageData') - if self.path is not None: - oprot.writeFieldBegin('path', TType.STRING, 1) - oprot.writeString(self.path.encode('utf-8')) - oprot.writeFieldEnd() - if self.pulse is not None: - oprot.writeFieldBegin('pulse', TType.STRUCT, 2) - self.pulse.write(oprot) - oprot.writeFieldEnd() - if self.boolval is not None: - oprot.writeFieldBegin('boolval', TType.BOOL, 3) - oprot.writeBool(self.boolval) - oprot.writeFieldEnd() - if self.records is not None: - oprot.writeFieldBegin('records', TType.STRUCT, 4) - self.records.write(oprot) - oprot.writeFieldEnd() - if self.nodes is not None: - oprot.writeFieldBegin('nodes', TType.STRUCT, 5) - self.nodes.write(oprot) + oprot.writeStructBegin('HBPulse') + if self.id is not None: + oprot.writeFieldBegin('id', TType.STRING, 1) + oprot.writeString(self.id.encode('utf-8')) oprot.writeFieldEnd() - if self.message_blob is not None: - oprot.writeFieldBegin('message_blob', TType.STRING, 7) - oprot.writeString(self.message_blob) + if self.details is not None: + oprot.writeFieldBegin('details', TType.STRING, 2) + oprot.writeString(self.details) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): + if self.id is None: + raise TProtocol.TProtocolException(message='Required field id is unset!') return def __hash__(self): value = 17 - value = (value * 31) ^ hash(self.path) - value = (value * 31) ^ hash(self.pulse) - value = (value * 31) ^ hash(self.boolval) - value = (value * 31) ^ hash(self.records) - value = (value * 31) ^ hash(self.nodes) - value = (value * 31) ^ hash(self.message_blob) + value = (value * 31) ^ hash(self.id) + value = (value * 31) ^ hash(self.details) return value def __repr__(self): @@ -10229,25 +10175,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class HBMessage: +class HBRecords: """ Attributes: - - type - - data - - message_id + - pulses """ thrift_spec = ( None, # 0 - (1, TType.I32, 'type', None, None, ), # 1 - (2, TType.STRUCT, 'data', (HBMessageData, HBMessageData.thrift_spec), None, ), # 2 - (3, TType.I32, 'message_id', None, -1, ), # 3 + (1, TType.LIST, 'pulses', (TType.STRUCT,(HBPulse, HBPulse.thrift_spec)), None, ), # 1 ) - def __init__(self, type=None, data=None, message_id=thrift_spec[3][4],): - self.type = type - self.data = data - self.message_id = message_id + def __init__(self, pulses=None,): + self.pulses = pulses def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -10259,19 +10199,14 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.I32: - self.type = iprot.readI32() - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.data = HBMessageData() - self.data.read(iprot) - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.I32: - self.message_id = iprot.readI32() + if ftype == TType.LIST: + self.pulses = [] + (_etype648, _size645) = iprot.readListBegin() + for _i649 in xrange(_size645): + _elem650 = HBPulse() + _elem650.read(iprot) + self.pulses.append(_elem650) + iprot.readListEnd() else: iprot.skip(ftype) else: @@ -10283,18 +10218,13 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('HBMessage') - if self.type is not None: - oprot.writeFieldBegin('type', TType.I32, 1) - oprot.writeI32(self.type) - oprot.writeFieldEnd() - if self.data is not None: - oprot.writeFieldBegin('data', TType.STRUCT, 2) - self.data.write(oprot) - oprot.writeFieldEnd() - if self.message_id is not None: - oprot.writeFieldBegin('message_id', TType.I32, 3) - oprot.writeI32(self.message_id) + oprot.writeStructBegin('HBRecords') + if self.pulses is not None: + oprot.writeFieldBegin('pulses', TType.LIST, 1) + oprot.writeListBegin(TType.STRUCT, len(self.pulses)) + for iter651 in self.pulses: + iter651.write(oprot) + oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -10305,9 +10235,7 @@ def validate(self): def __hash__(self): value = 17 - value = (value * 31) ^ hash(self.type) - value = (value * 31) ^ hash(self.data) - value = (value * 31) ^ hash(self.message_id) + value = (value * 31) ^ hash(self.pulses) return value def __repr__(self): @@ -10321,19 +10249,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class HBAuthorizationException(TException): +class HBNodes: """ Attributes: - - msg + - pulseIds """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'msg', None, None, ), # 1 + (1, TType.LIST, 'pulseIds', (TType.STRING,None), None, ), # 1 ) - def __init__(self, msg=None,): - self.msg = msg + def __init__(self, pulseIds=None,): + self.pulseIds = pulseIds def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -10345,8 +10273,13 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRING: - self.msg = iprot.readString().decode('utf-8') + if ftype == TType.LIST: + self.pulseIds = [] + (_etype655, _size652) = iprot.readListBegin() + for _i656 in xrange(_size652): + _elem657 = iprot.readString().decode('utf-8') + self.pulseIds.append(_elem657) + iprot.readListEnd() else: iprot.skip(ftype) else: @@ -10358,26 +10291,24 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('HBAuthorizationException') - if self.msg is not None: - oprot.writeFieldBegin('msg', TType.STRING, 1) - oprot.writeString(self.msg.encode('utf-8')) + oprot.writeStructBegin('HBNodes') + if self.pulseIds is not None: + oprot.writeFieldBegin('pulseIds', TType.LIST, 1) + oprot.writeListBegin(TType.STRING, len(self.pulseIds)) + for iter658 in self.pulseIds: + oprot.writeString(iter658.encode('utf-8')) + oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): - if self.msg is None: - raise TProtocol.TProtocolException(message='Required field msg is unset!') return - def __str__(self): - return repr(self) - def __hash__(self): value = 17 - value = (value * 31) ^ hash(self.msg) + value = (value * 31) ^ hash(self.pulseIds) return value def __repr__(self): @@ -10391,19 +10322,35 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class HBExecutionException(TException): +class HBMessageData: """ Attributes: - - msg + - path + - pulse + - boolval + - records + - nodes + - message_blob """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'msg', None, None, ), # 1 + (1, TType.STRING, 'path', None, None, ), # 1 + (2, TType.STRUCT, 'pulse', (HBPulse, HBPulse.thrift_spec), None, ), # 2 + (3, TType.BOOL, 'boolval', None, None, ), # 3 + (4, TType.STRUCT, 'records', (HBRecords, HBRecords.thrift_spec), None, ), # 4 + (5, TType.STRUCT, 'nodes', (HBNodes, HBNodes.thrift_spec), None, ), # 5 + None, # 6 + (7, TType.STRING, 'message_blob', None, None, ), # 7 ) - def __init__(self, msg=None,): - self.msg = msg + def __init__(self, path=None, pulse=None, boolval=None, records=None, nodes=None, message_blob=None,): + self.path = path + self.pulse = pulse + self.boolval = boolval + self.records = records + self.nodes = nodes + self.message_blob = message_blob def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -10416,7 +10363,35 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.msg = iprot.readString().decode('utf-8') + self.path = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.pulse = HBPulse() + self.pulse.read(iprot) + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.BOOL: + self.boolval = iprot.readBool() + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.records = HBRecords() + self.records.read(iprot) + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.STRUCT: + self.nodes = HBNodes() + self.nodes.read(iprot) + else: + iprot.skip(ftype) + elif fid == 7: + if ftype == TType.STRING: + self.message_blob = iprot.readString() else: iprot.skip(ftype) else: @@ -10428,26 +10403,46 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('HBExecutionException') - if self.msg is not None: - oprot.writeFieldBegin('msg', TType.STRING, 1) - oprot.writeString(self.msg.encode('utf-8')) + oprot.writeStructBegin('HBMessageData') + if self.path is not None: + oprot.writeFieldBegin('path', TType.STRING, 1) + oprot.writeString(self.path.encode('utf-8')) + oprot.writeFieldEnd() + if self.pulse is not None: + oprot.writeFieldBegin('pulse', TType.STRUCT, 2) + self.pulse.write(oprot) + oprot.writeFieldEnd() + if self.boolval is not None: + oprot.writeFieldBegin('boolval', TType.BOOL, 3) + oprot.writeBool(self.boolval) + oprot.writeFieldEnd() + if self.records is not None: + oprot.writeFieldBegin('records', TType.STRUCT, 4) + self.records.write(oprot) + oprot.writeFieldEnd() + if self.nodes is not None: + oprot.writeFieldBegin('nodes', TType.STRUCT, 5) + self.nodes.write(oprot) + oprot.writeFieldEnd() + if self.message_blob is not None: + oprot.writeFieldBegin('message_blob', TType.STRING, 7) + oprot.writeString(self.message_blob) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): - if self.msg is None: - raise TProtocol.TProtocolException(message='Required field msg is unset!') return - def __str__(self): - return repr(self) - def __hash__(self): value = 17 - value = (value * 31) ^ hash(self.msg) + value = (value * 31) ^ hash(self.path) + value = (value * 31) ^ hash(self.pulse) + value = (value * 31) ^ hash(self.boolval) + value = (value * 31) ^ hash(self.records) + value = (value * 31) ^ hash(self.nodes) + value = (value * 31) ^ hash(self.message_blob) return value def __repr__(self): @@ -10461,22 +10456,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class HBPulse: +class HBMessage: """ Attributes: - - id - - details + - type + - data + - message_id """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'id', None, None, ), # 1 - (2, TType.STRING, 'details', None, None, ), # 2 + (1, TType.I32, 'type', None, None, ), # 1 + (2, TType.STRUCT, 'data', (HBMessageData, HBMessageData.thrift_spec), None, ), # 2 + (3, TType.I32, 'message_id', None, -1, ), # 3 ) - def __init__(self, id=None, details=None,): - self.id = id - self.details = details + def __init__(self, type=None, data=None, message_id=thrift_spec[3][4],): + self.type = type + self.data = data + self.message_id = message_id def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -10488,13 +10486,19 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRING: - self.id = iprot.readString().decode('utf-8') + if ftype == TType.I32: + self.type = iprot.readI32() else: iprot.skip(ftype) elif fid == 2: - if ftype == TType.STRING: - self.details = iprot.readString() + if ftype == TType.STRUCT: + self.data = HBMessageData() + self.data.read(iprot) + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.I32: + self.message_id = iprot.readI32() else: iprot.skip(ftype) else: @@ -10506,28 +10510,31 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('HBPulse') - if self.id is not None: - oprot.writeFieldBegin('id', TType.STRING, 1) - oprot.writeString(self.id.encode('utf-8')) + oprot.writeStructBegin('HBMessage') + if self.type is not None: + oprot.writeFieldBegin('type', TType.I32, 1) + oprot.writeI32(self.type) oprot.writeFieldEnd() - if self.details is not None: - oprot.writeFieldBegin('details', TType.STRING, 2) - oprot.writeString(self.details) + if self.data is not None: + oprot.writeFieldBegin('data', TType.STRUCT, 2) + self.data.write(oprot) + oprot.writeFieldEnd() + if self.message_id is not None: + oprot.writeFieldBegin('message_id', TType.I32, 3) + oprot.writeI32(self.message_id) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): - if self.id is None: - raise TProtocol.TProtocolException(message='Required field id is unset!') return def __hash__(self): value = 17 - value = (value * 31) ^ hash(self.id) - value = (value * 31) ^ hash(self.details) + value = (value * 31) ^ hash(self.type) + value = (value * 31) ^ hash(self.data) + value = (value * 31) ^ hash(self.message_id) return value def __repr__(self): @@ -10541,19 +10548,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class HBRecords: +class HBAuthorizationException(TException): """ Attributes: - - pulses + - msg """ thrift_spec = ( None, # 0 - (1, TType.LIST, 'pulses', (TType.STRUCT,(HBPulse, HBPulse.thrift_spec)), None, ), # 1 + (1, TType.STRING, 'msg', None, None, ), # 1 ) - def __init__(self, pulses=None,): - self.pulses = pulses + def __init__(self, msg=None,): + self.msg = msg def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -10565,14 +10572,8 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.LIST: - self.pulses = [] - (_etype648, _size645) = iprot.readListBegin() - for _i649 in xrange(_size645): - _elem650 = HBPulse() - _elem650.read(iprot) - self.pulses.append(_elem650) - iprot.readListEnd() + if ftype == TType.STRING: + self.msg = iprot.readString().decode('utf-8') else: iprot.skip(ftype) else: @@ -10584,24 +10585,26 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('HBRecords') - if self.pulses is not None: - oprot.writeFieldBegin('pulses', TType.LIST, 1) - oprot.writeListBegin(TType.STRUCT, len(self.pulses)) - for iter651 in self.pulses: - iter651.write(oprot) - oprot.writeListEnd() + oprot.writeStructBegin('HBAuthorizationException') + if self.msg is not None: + oprot.writeFieldBegin('msg', TType.STRING, 1) + oprot.writeString(self.msg.encode('utf-8')) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): + if self.msg is None: + raise TProtocol.TProtocolException(message='Required field msg is unset!') return + def __str__(self): + return repr(self) + def __hash__(self): value = 17 - value = (value * 31) ^ hash(self.pulses) + value = (value * 31) ^ hash(self.msg) return value def __repr__(self): @@ -10615,19 +10618,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class HBNodes: +class HBExecutionException(TException): """ Attributes: - - pulseIds + - msg """ thrift_spec = ( None, # 0 - (1, TType.LIST, 'pulseIds', (TType.STRING,None), None, ), # 1 + (1, TType.STRING, 'msg', None, None, ), # 1 ) - def __init__(self, pulseIds=None,): - self.pulseIds = pulseIds + def __init__(self, msg=None,): + self.msg = msg def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -10639,13 +10642,8 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.LIST: - self.pulseIds = [] - (_etype655, _size652) = iprot.readListBegin() - for _i656 in xrange(_size652): - _elem657 = iprot.readString().decode('utf-8') - self.pulseIds.append(_elem657) - iprot.readListEnd() + if ftype == TType.STRING: + self.msg = iprot.readString().decode('utf-8') else: iprot.skip(ftype) else: @@ -10657,24 +10655,26 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('HBNodes') - if self.pulseIds is not None: - oprot.writeFieldBegin('pulseIds', TType.LIST, 1) - oprot.writeListBegin(TType.STRING, len(self.pulseIds)) - for iter658 in self.pulseIds: - oprot.writeString(iter658.encode('utf-8')) - oprot.writeListEnd() + oprot.writeStructBegin('HBExecutionException') + if self.msg is not None: + oprot.writeFieldBegin('msg', TType.STRING, 1) + oprot.writeString(self.msg.encode('utf-8')) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): + if self.msg is None: + raise TProtocol.TProtocolException(message='Required field msg is unset!') return + def __str__(self): + return repr(self) + def __hash__(self): value = 17 - value = (value * 31) ^ hash(self.pulseIds) + value = (value * 31) ^ hash(self.msg) return value def __repr__(self): diff --git a/storm-core/src/storm.thrift b/storm-core/src/storm.thrift index 5a7169b63af..4cfb9ad5715 100644 --- a/storm-core/src/storm.thrift +++ b/storm-core/src/storm.thrift @@ -697,6 +697,19 @@ enum HBServerMessageType { NOT_AUTHORIZED } +struct HBPulse { + 1: required string id; + 2: binary details; +} + +struct HBRecords { + 1: list pulses; +} + +struct HBNodes { + 1: list pulseIds; +} + union HBMessageData { 1: string path, 2: HBPulse pulse, @@ -712,7 +725,6 @@ struct HBMessage { 3: optional i32 message_id = -1, } - exception HBAuthorizationException { 1: required string msg; } @@ -720,16 +732,3 @@ exception HBAuthorizationException { exception HBExecutionException { 1: required string msg; } - -struct HBPulse { - 1: required string id; - 2: binary details; -} - -struct HBRecords { - 1: list pulses; -} - -struct HBNodes { - 1: list pulseIds; -} From be90d7831345f00aac71a860ad5110b31b0a4848 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Wed, 18 May 2016 08:56:08 -0700 Subject: [PATCH 35/58] Merge branch 'STORM-1841' of https://github.com/arunmahadevan/storm into STORM-1841 --- docs/Windowing.md | 13 +-- .../storm/starter/SlidingWindowTopology.java | 4 +- .../storm/topology/base/BaseWindowedBolt.java | 87 +++++++++++++++++++ 3 files changed, 96 insertions(+), 8 deletions(-) diff --git a/docs/Windowing.md b/docs/Windowing.md index 44512f79d4f..9f5869f23a4 100644 --- a/docs/Windowing.md +++ b/docs/Windowing.md @@ -17,15 +17,16 @@ Tuples are grouped in windows and window slides every sliding interval. A tuple For example a time duration based sliding window with length 10 secs and sliding interval of 5 seconds. ``` -| e1 e2 | e3 e4 e5 e6 | e7 e8 e9 |... -0 5 10 15 -> time - -|<------- w1 -------->| - |------------ w2 ------->| +........| e1 e2 | e3 e4 e5 e6 | e7 e8 e9 |... +-5 0 5 10 15 -> time +|<------- w1 -->| + |<---------- w2 ----->| + |<-------------- w3 ---->| ``` The window is evaluated every 5 seconds and some of the tuples in the first window overlaps with the second one. - + +Note: The window first slides at t = 5 secs and would contain events received up to the first five secs. ## Tumbling Window diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/SlidingWindowTopology.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/SlidingWindowTopology.java index cedcec5773e..8477ad11739 100644 --- a/examples/storm-starter/src/jvm/org/apache/storm/starter/SlidingWindowTopology.java +++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/SlidingWindowTopology.java @@ -87,9 +87,9 @@ public void declareOutputFields(OutputFieldsDeclarer declarer) { public static void main(String[] args) throws Exception { TopologyBuilder builder = new TopologyBuilder(); builder.setSpout("integer", new RandomIntegerSpout(), 1); - builder.setBolt("slidingsum", new SlidingWindowSumBolt().withWindow(new Count(30), new Count(10)), 1) + builder.setBolt("slidingsum", new SlidingWindowSumBolt().withWindow(Count.of(30), Count.of(10)), 1) .shuffleGrouping("integer"); - builder.setBolt("tumblingavg", new TumblingWindowAvgBolt().withTumblingWindow(new Count(3)), 1) + builder.setBolt("tumblingavg", new TumblingWindowAvgBolt().withTumblingWindow(Count.of(3)), 1) .shuffleGrouping("slidingsum"); builder.setBolt("printer", new PrinterBolt(), 1).shuffleGrouping("tumblingavg"); Config conf = new Config(); diff --git a/storm-core/src/jvm/org/apache/storm/topology/base/BaseWindowedBolt.java b/storm-core/src/jvm/org/apache/storm/topology/base/BaseWindowedBolt.java index a1d9ebe55b8..e1725dd32d8 100644 --- a/storm-core/src/jvm/org/apache/storm/topology/base/BaseWindowedBolt.java +++ b/storm-core/src/jvm/org/apache/storm/topology/base/BaseWindowedBolt.java @@ -43,6 +43,23 @@ public static class Count { public Count(int value) { this.value = value; } + + /** + * Returns a {@link Count} of given value. + * + * @param value the count value + * @return the Count + */ + public static Count of(int value) { + return new Count(value); + } + + @Override + public String toString() { + return "Count{" + + "value=" + value + + '}'; + } } /** @@ -54,6 +71,63 @@ public static class Duration { public Duration(int value, TimeUnit timeUnit) { this.value = (int) timeUnit.toMillis(value); } + + /** + * Returns a {@link Duration} corresponding to the the given value in milli seconds. + * + * @param milliseconds the duration in milliseconds + * @return the Duration + */ + public static Duration of(int milliseconds) { + return new Duration(milliseconds, TimeUnit.MILLISECONDS); + } + + /** + * Returns a {@link Duration} corresponding to the the given value in days. + * + * @param days the number of days + * @return the Duration + */ + public static Duration days(int days) { + return new Duration(days, TimeUnit.DAYS); + } + + /** + * Returns a {@link Duration} corresponding to the the given value in hours. + * + * @param hours the number of hours + * @return the Duration + */ + public static Duration hours(int hours) { + return new Duration(hours, TimeUnit.HOURS); + } + + /** + * Returns a {@link Duration} corresponding to the the given value in minutes. + * + * @param minutes the number of minutes + * @return the Duration + */ + public static Duration minutes(int minutes) { + return new Duration(minutes, TimeUnit.MINUTES); + } + + /** + * Returns a {@link Duration} corresponding to the the given value in seconds. + * + * @param seconds the number of seconds + * @return the Duration + */ + public static Duration seconds(int seconds) { + return new Duration(seconds, TimeUnit.SECONDS); + } + + @Override + public String toString() { + return "Duration{" + + "value=" + value + + '}'; + } } protected BaseWindowedBolt() { @@ -61,21 +135,34 @@ protected BaseWindowedBolt() { } private BaseWindowedBolt withWindowLength(Count count) { + if (count.value < 0) { + throw new IllegalArgumentException("Negative window length [" + count + "]"); + } windowConfiguration.put(Config.TOPOLOGY_BOLTS_WINDOW_LENGTH_COUNT, count.value); return this; } private BaseWindowedBolt withWindowLength(Duration duration) { + if (duration.value < 0) { + throw new IllegalArgumentException("Negative window length [" + duration + "]"); + } + windowConfiguration.put(Config.TOPOLOGY_BOLTS_WINDOW_LENGTH_DURATION_MS, duration.value); return this; } private BaseWindowedBolt withSlidingInterval(Count count) { + if (count.value < 0) { + throw new IllegalArgumentException("Negative sliding interval [" + count + "]"); + } windowConfiguration.put(Config.TOPOLOGY_BOLTS_SLIDING_INTERVAL_COUNT, count.value); return this; } private BaseWindowedBolt withSlidingInterval(Duration duration) { + if (duration.value < 0) { + throw new IllegalArgumentException("Negative sliding interval [" + duration + "]"); + } windowConfiguration.put(Config.TOPOLOGY_BOLTS_SLIDING_INTERVAL_DURATION_MS, duration.value); return this; } From b1abf8221b28cf25abfe5a0f3087258ecdc5d5fb Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Wed, 18 May 2016 09:10:02 -0700 Subject: [PATCH 36/58] Added STORM-1841 to CHANGELOG --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 6794ec3a2ef..0225288bd4b 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.1.0 + * STORM-1841: Address a few minor issues in windowing and doc * STORM-1709: Added group by support in storm sql standalone mode * STORM-1720: Support GEO in storm-redis From 6e46e27a6971fd15fefcc1072edf87cfa2b7dcb9 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Wed, 18 May 2016 09:35:14 -0700 Subject: [PATCH 37/58] Merge branch 'master' of https://github.com/fbyrne/storm into STORM-1730 --- .../org/apache/storm/utils/DisruptorQueue.java | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java b/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java index 2a6feadf673..ef6f65b1080 100644 --- a/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java +++ b/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java @@ -17,6 +17,7 @@ */ package org.apache.storm.utils; +import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.lmax.disruptor.AlertException; import com.lmax.disruptor.EventFactory; import com.lmax.disruptor.EventHandler; @@ -44,6 +45,7 @@ import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ThreadFactory; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -62,11 +64,21 @@ public class DisruptorQueue implements IStatefulObject { private static final FlusherPool FLUSHER = new FlusherPool(); private static class FlusherPool { - private Timer _timer = new Timer("disruptor-flush-trigger", true); - private ThreadPoolExecutor _exec = new ThreadPoolExecutor(1, 100, 10, TimeUnit.SECONDS, new ArrayBlockingQueue(1024), new ThreadPoolExecutor.DiscardPolicy()); + private static final String THREAD_PREFIX = "disruptor-flush"; + private Timer _timer = new Timer(THREAD_PREFIX + "-trigger", true); + private ThreadPoolExecutor _exec; private HashMap> _pendingFlush = new HashMap<>(); private HashMap _tt = new HashMap<>(); + public FlusherPool() { + _exec = new ThreadPoolExecutor(1, 100, 10, TimeUnit.SECONDS, new ArrayBlockingQueue(1024), new ThreadPoolExecutor.DiscardPolicy()); + ThreadFactory threadFactory = new ThreadFactoryBuilder() + .setDaemon(true) + .setNameFormat(THREAD_PREFIX + "-task-pool") + .build(); + _exec.setThreadFactory(threadFactory); + } + public synchronized void start(Flusher flusher, final long flushInterval) { ArrayList pending = _pendingFlush.get(flushInterval); if (pending == null) { From 00685adfd9c0750c64da8478073e8ce2c4fd8817 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Wed, 18 May 2016 10:01:53 -0700 Subject: [PATCH 38/58] Added STORM-1730 to CHANGELOG. --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 0225288bd4b..bc4e3524b75 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.1.0 + * STORM-1730: LocalCluster#shutdown() does not terminate all storm threads/thread pools. * STORM-1841: Address a few minor issues in windowing and doc * STORM-1709: Added group by support in storm sql standalone mode * STORM-1720: Support GEO in storm-redis From ba5f426304f68ef4c05e374c394fe22ab3b543ce Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Thu, 19 May 2016 14:40:17 +0900 Subject: [PATCH 39/58] Bump version to 1.1.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-client/pom.xml | 2 +- external/storm-kafka/pom.xml | 2 +- external/storm-metrics/pom.xml | 2 +- external/storm-mongodb/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 +- 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 +- 34 files changed, 35 insertions(+), 35 deletions(-) diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml index 62c262fc83b..7a3a2f42f15 100644 --- a/examples/storm-starter/pom.xml +++ b/examples/storm-starter/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/flux/flux-core/pom.xml b/external/flux/flux-core/pom.xml index 3f86dbf64c1..48e31eb6203 100644 --- a/external/flux/flux-core/pom.xml +++ b/external/flux/flux-core/pom.xml @@ -21,7 +21,7 @@ org.apache.storm flux - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT ../pom.xml diff --git a/external/flux/flux-examples/pom.xml b/external/flux/flux-examples/pom.xml index 1edfaa386ed..d522be4e6d1 100644 --- a/external/flux/flux-examples/pom.xml +++ b/external/flux/flux-examples/pom.xml @@ -21,7 +21,7 @@ org.apache.storm flux - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT ../pom.xml diff --git a/external/flux/flux-wrappers/pom.xml b/external/flux/flux-wrappers/pom.xml index 6f875eab74d..56113d13061 100644 --- a/external/flux/flux-wrappers/pom.xml +++ b/external/flux/flux-wrappers/pom.xml @@ -21,7 +21,7 @@ org.apache.storm flux - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT ../pom.xml diff --git a/external/flux/pom.xml b/external/flux/pom.xml index c0c727c5ab7..9b8f9bb3fb2 100644 --- a/external/flux/pom.xml +++ b/external/flux/pom.xml @@ -26,7 +26,7 @@ storm org.apache.storm - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/sql/pom.xml b/external/sql/pom.xml index 40aa8b27af7..96bb012f77d 100644 --- a/external/sql/pom.xml +++ b/external/sql/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/sql/storm-sql-core/pom.xml b/external/sql/storm-sql-core/pom.xml index 0a5e8b2684a..4d6e3cdd1cf 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 - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT ../../../pom.xml diff --git a/external/sql/storm-sql-kafka/pom.xml b/external/sql/storm-sql-kafka/pom.xml index 6b6e364b77f..e183fe519f5 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 - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT ../../../pom.xml diff --git a/external/sql/storm-sql-runtime/pom.xml b/external/sql/storm-sql-runtime/pom.xml index 7512a90a104..dc85149ba29 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 - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT ../../../pom.xml diff --git a/external/storm-cassandra/pom.xml b/external/storm-cassandra/pom.xml index 65da01de848..0bcc2fd5314 100644 --- a/external/storm-cassandra/pom.xml +++ b/external/storm-cassandra/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/storm-elasticsearch/pom.xml b/external/storm-elasticsearch/pom.xml index 5ef24580f70..dc4d67dd716 100644 --- a/external/storm-elasticsearch/pom.xml +++ b/external/storm-elasticsearch/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/storm-eventhubs/pom.xml b/external/storm-eventhubs/pom.xml index 7e9503652cd..2c87b2186c4 100755 --- a/external/storm-eventhubs/pom.xml +++ b/external/storm-eventhubs/pom.xml @@ -21,12 +21,12 @@ storm org.apache.storm - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT ../../pom.xml storm-eventhubs - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT jar storm-eventhubs EventHubs Storm Spout diff --git a/external/storm-hbase/pom.xml b/external/storm-hbase/pom.xml index e77a8f8b76f..d8c3a40be3f 100644 --- a/external/storm-hbase/pom.xml +++ b/external/storm-hbase/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml index fccbd4c237c..214d1216282 100644 --- a/external/storm-hdfs/pom.xml +++ b/external/storm-hdfs/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/storm-hive/pom.xml b/external/storm-hive/pom.xml index 54a1ae4f587..351222b455b 100644 --- a/external/storm-hive/pom.xml +++ b/external/storm-hive/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/storm-jdbc/pom.xml b/external/storm-jdbc/pom.xml index c8657355794..75d0f524c88 100644 --- a/external/storm-jdbc/pom.xml +++ b/external/storm-jdbc/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/storm-kafka-client/pom.xml b/external/storm-kafka-client/pom.xml index a7f1c43973b..c91ddb8c9e8 100644 --- a/external/storm-kafka-client/pom.xml +++ b/external/storm-kafka-client/pom.xml @@ -22,7 +22,7 @@ storm org.apache.storm - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/storm-kafka/pom.xml b/external/storm-kafka/pom.xml index 41e0013b14f..138302337c3 100644 --- a/external/storm-kafka/pom.xml +++ b/external/storm-kafka/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/storm-metrics/pom.xml b/external/storm-metrics/pom.xml index 96b93877310..a113109fd3f 100644 --- a/external/storm-metrics/pom.xml +++ b/external/storm-metrics/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/storm-mongodb/pom.xml b/external/storm-mongodb/pom.xml index f6b72d9c2f7..f1550f5d8e9 100644 --- a/external/storm-mongodb/pom.xml +++ b/external/storm-mongodb/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/storm-mqtt/core/pom.xml b/external/storm-mqtt/core/pom.xml index b3495235b0a..0e2be784b70 100644 --- a/external/storm-mqtt/core/pom.xml +++ b/external/storm-mqtt/core/pom.xml @@ -25,7 +25,7 @@ org.apache.storm storm-mqtt-parent - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT ../pom.xml diff --git a/external/storm-mqtt/examples/pom.xml b/external/storm-mqtt/examples/pom.xml index 123edac2fa9..3b152ae37e2 100644 --- a/external/storm-mqtt/examples/pom.xml +++ b/external/storm-mqtt/examples/pom.xml @@ -26,7 +26,7 @@ org.apache.storm storm-mqtt-parent - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT ../pom.xml diff --git a/external/storm-mqtt/pom.xml b/external/storm-mqtt/pom.xml index e4b76bcbf19..5d7a5bb0149 100644 --- a/external/storm-mqtt/pom.xml +++ b/external/storm-mqtt/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/storm-redis/pom.xml b/external/storm-redis/pom.xml index 77ba5bff106..470e2d5934c 100644 --- a/external/storm-redis/pom.xml +++ b/external/storm-redis/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/external/storm-solr/pom.xml b/external/storm-solr/pom.xml index 45eff909390..d6852da85f3 100644 --- a/external/storm-solr/pom.xml +++ b/external/storm-solr/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/storm-buildtools/maven-shade-clojure-transformer/pom.xml b/storm-buildtools/maven-shade-clojure-transformer/pom.xml index 67a69af305a..1ef88861530 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 - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT ../../pom.xml diff --git a/storm-buildtools/storm-maven-plugins/pom.xml b/storm-buildtools/storm-maven-plugins/pom.xml index 09150b28d7c..4aab471c6bf 100644 --- a/storm-buildtools/storm-maven-plugins/pom.xml +++ b/storm-buildtools/storm-maven-plugins/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT ../../pom.xml org.apache.storm diff --git a/storm-core/pom.xml b/storm-core/pom.xml index 9e3b488b5ef..a33bd76e872 100644 --- a/storm-core/pom.xml +++ b/storm-core/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT .. org.apache.storm diff --git a/storm-dist/binary/pom.xml b/storm-dist/binary/pom.xml index 5e199619a2d..95bf4bd3481 100644 --- a/storm-dist/binary/pom.xml +++ b/storm-dist/binary/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT ../../pom.xml org.apache.storm diff --git a/storm-dist/source/pom.xml b/storm-dist/source/pom.xml index eb88065f4f4..47615dade86 100644 --- a/storm-dist/source/pom.xml +++ b/storm-dist/source/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT ../../pom.xml org.apache.storm diff --git a/storm-multilang/javascript/pom.xml b/storm-multilang/javascript/pom.xml index 568f2788e37..438c2e221f1 100644 --- a/storm-multilang/javascript/pom.xml +++ b/storm-multilang/javascript/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT ../../pom.xml org.apache.storm diff --git a/storm-multilang/python/pom.xml b/storm-multilang/python/pom.xml index 6b07cdf791d..dbaafb45cd8 100644 --- a/storm-multilang/python/pom.xml +++ b/storm-multilang/python/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT ../../pom.xml org.apache.storm diff --git a/storm-multilang/ruby/pom.xml b/storm-multilang/ruby/pom.xml index 5144a008cbd..39c75540c92 100644 --- a/storm-multilang/ruby/pom.xml +++ b/storm-multilang/ruby/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT ../../pom.xml org.apache.storm diff --git a/storm-rename-hack/pom.xml b/storm-rename-hack/pom.xml index 1a50309bab3..da46bb3b1bd 100644 --- a/storm-rename-hack/pom.xml +++ b/storm-rename-hack/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 1.0.2-SNAPSHOT + 1.1.0-SNAPSHOT ../pom.xml From 068d5c9c2d75f6a0031b3783333c514628ff8333 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Fri, 20 May 2016 14:09:27 -0400 Subject: [PATCH 40/58] add STORM-1842 to changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index bc4e3524b75..a315b1b5241 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.1.0 + * STORM-1842: Forward references in storm.thrift cause tooling issues * STORM-1730: LocalCluster#shutdown() does not terminate all storm threads/thread pools. * STORM-1841: Address a few minor issues in windowing and doc * STORM-1709: Added group by support in storm sql standalone mode From dd15981bfa729bd1cee5709e0a9e994add50bfd6 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Fri, 20 May 2016 14:38:32 -0400 Subject: [PATCH 41/58] add STORM-1674 to changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index a315b1b5241..e37bf56698a 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.1.0 + * STORM-1674: Idle KafkaSpout consumes more bandwidth than needed * STORM-1842: Forward references in storm.thrift cause tooling issues * STORM-1730: LocalCluster#shutdown() does not terminate all storm threads/thread pools. * STORM-1841: Address a few minor issues in windowing and doc From f069cfb08099a54448bba3f96f20c5b929ea6af3 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Fri, 20 May 2016 15:24:00 -0400 Subject: [PATCH 42/58] STORM-1674: rename KafkaConfig.fetchMinBytes to KafkaConfig.minFetchByte to retain backward compatibility with 1.0.x versions --- .../src/jvm/org/apache/storm/kafka/KafkaConfig.java | 3 ++- .../storm-kafka/src/jvm/org/apache/storm/kafka/KafkaUtils.java | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaConfig.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaConfig.java index 7c55550fab4..605e7fd62a0 100644 --- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaConfig.java +++ b/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaConfig.java @@ -34,7 +34,6 @@ public class KafkaConfig implements Serializable { public int fetchSizeBytes = 1024 * 1024; public int socketTimeoutMs = 10000; public int fetchMaxWait = FetchRequest.DefaultMaxWait(); - public int fetchMinBytes = FetchRequest.DefaultMinBytes(); public int bufferSizeBytes = 1024 * 1024; public MultiScheme scheme = new RawMultiScheme(); public boolean ignoreZkOffsets = false; @@ -42,6 +41,8 @@ public class KafkaConfig implements Serializable { public long maxOffsetBehind = Long.MAX_VALUE; public boolean useStartOffsetTimeIfOffsetOutOfRange = true; public int metricsTimeBucketSizeInSecs = 60; + public int minFetchByte = FetchRequest.DefaultMinBytes(); + public KafkaConfig(BrokerHosts hosts, String topic) { this(hosts, topic, kafka.api.OffsetRequest.DefaultClientId()); 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 47afbadb0dc..48d3ab7e410 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 @@ -185,7 +185,7 @@ public static ByteBufferMessageSet fetchMessages(KafkaConfig config, SimpleConsu int partitionId = partition.partition; FetchRequestBuilder builder = new FetchRequestBuilder(); FetchRequest fetchRequest = builder.addFetch(topic, partitionId, offset, config.fetchSizeBytes). - clientId(config.clientId).maxWait(config.fetchMaxWait).minBytes(config.fetchMinBytes).build(); + clientId(config.clientId).maxWait(config.fetchMaxWait).minBytes(config.minFetchByte).build(); FetchResponse fetchResponse; try { fetchResponse = consumer.fetch(fetchRequest); From b66c0152a6d3c5c2070e7e7f1adb25730b3d9a92 Mon Sep 17 00:00:00 2001 From: omallassi Date: Thu, 19 May 2016 10:43:06 +0200 Subject: [PATCH 43/58] State Checkpointing Documentation update update the documentation following this discussion https://mail-archives.apache.org/mod_mbox/storm-user/201605.mbox/%3CF9D9F747-8431-4A26-9028-BD74C09BCA84%40hortonworks.com%3E --- docs/State-checkpointing.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/State-checkpointing.md b/docs/State-checkpointing.md index 7c59aad5d8d..3d6212c94ce 100644 --- a/docs/State-checkpointing.md +++ b/docs/State-checkpointing.md @@ -94,6 +94,9 @@ is saved and then the checkpoint tuple is forwarded to the next component. Each streams before it saves its state so that the state represents a consistent state across the topology. Once the checkpoint spout receives ACK from all the bolts, the state commit is complete and the transaction is recorded as committed by the checkpoint spout. +The state checkpointing does not currently checkpoint the state of the spout. Yet, once the state of all bolts are checkpointed, and once the checkpoint tuples are acked, the tuples emitted by the spout are also acked. +It also implies that `topology.state.checkpoint.interval.ms` is lower than `topology.message.timeout.secs`. + The state commit works like a three phase commit protocol with a prepare and commit phase so that the state across the topology is saved in a consistent and atomic manner. From 039f5c5a940d9a268faa29b6da8855b549260412 Mon Sep 17 00:00:00 2001 From: Arun Mahadevan Date: Mon, 23 May 2016 11:32:52 +0530 Subject: [PATCH 44/58] Added STORM-1850 to CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index e37bf56698a..9c6da9d5881 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.1.0 + * STORM-1850: State Checkpointing Documentation update * STORM-1674: Idle KafkaSpout consumes more bandwidth than needed * STORM-1842: Forward references in storm.thrift cause tooling issues * STORM-1730: LocalCluster#shutdown() does not terminate all storm threads/thread pools. From e210932dde69ba6d173364a1d3a8e8e9b6b4dc3c Mon Sep 17 00:00:00 2001 From: Dan Herrera Date: Tue, 17 May 2016 14:12:57 -0700 Subject: [PATCH 45/58] [Storm-1728] TransactionalTridentKafkaSpout error --- .../spout/PartitionedTridentSpoutExecutor.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/storm-core/src/jvm/org/apache/storm/trident/spout/PartitionedTridentSpoutExecutor.java b/storm-core/src/jvm/org/apache/storm/trident/spout/PartitionedTridentSpoutExecutor.java index 8aa97283b6b..389a9298bc9 100644 --- a/storm-core/src/jvm/org/apache/storm/trident/spout/PartitionedTridentSpoutExecutor.java +++ b/storm-core/src/jvm/org/apache/storm/trident/spout/PartitionedTridentSpoutExecutor.java @@ -29,7 +29,7 @@ import org.apache.storm.trident.topology.state.TransactionalState; -public class PartitionedTridentSpoutExecutor implements ITridentSpout { +public class PartitionedTridentSpoutExecutor implements ITridentSpout { IPartitionedTridentSpout _spout; public PartitionedTridentSpoutExecutor(IPartitionedTridentSpout spout) { @@ -40,7 +40,7 @@ public IPartitionedTridentSpout getPartitioned return _spout; } - class Coordinator implements ITridentSpout.BatchCoordinator { + class Coordinator implements ITridentSpout.BatchCoordinator { private IPartitionedTridentSpout.Coordinator _coordinator; public Coordinator(Map conf, TopologyContext context) { @@ -48,7 +48,7 @@ public Coordinator(Map conf, TopologyContext context) { } @Override - public Integer initializeTransaction(long txid, Integer prevMetadata, Integer currMetadata) { + public Object initializeTransaction(long txid, Object prevMetadata, Object currMetadata) { if(currMetadata!=null) { return currMetadata; } else { @@ -82,8 +82,8 @@ public EmitterPartitionState(RotatingTransactionalState s, ISpoutPartition p) { } } - class Emitter implements ITridentSpout.Emitter { - private IPartitionedTridentSpout.Emitter _emitter; + class Emitter implements ITridentSpout.Emitter { + private IPartitionedTridentSpout.Emitter _emitter; private TransactionalState _state; private Map _partitionStates = new HashMap<>(); private int _index; @@ -100,7 +100,7 @@ public Emitter(String txStateId, Map conf, TopologyContext context) { @Override - public void emitBatch(final TransactionAttempt tx, final Integer coordinatorMeta, + public void emitBatch(final TransactionAttempt tx, final Object coordinatorMeta, final TridentCollector collector) { if(_savedCoordinatorMeta == null || !_savedCoordinatorMeta.equals(coordinatorMeta)) { List partitions = _emitter.getOrderedPartitions(coordinatorMeta); @@ -150,12 +150,12 @@ public void close() { } @Override - public ITridentSpout.BatchCoordinator getCoordinator(String txStateId, Map conf, TopologyContext context) { + public ITridentSpout.BatchCoordinator getCoordinator(String txStateId, Map conf, TopologyContext context) { return new Coordinator(conf, context); } @Override - public ITridentSpout.Emitter getEmitter(String txStateId, Map conf, TopologyContext context) { + public ITridentSpout.Emitter getEmitter(String txStateId, Map conf, TopologyContext context) { return new Emitter(txStateId, conf, context); } From 2a132dacdf2210a6827ab63b1363957563308b5e Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Tue, 24 May 2016 15:45:20 -0400 Subject: [PATCH 46/58] add STORM-1728 to changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 9c6da9d5881..537585f2721 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.1.0 + * Storm-1728: TransactionalTridentKafkaSpout error * STORM-1850: State Checkpointing Documentation update * STORM-1674: Idle KafkaSpout consumes more bandwidth than needed * STORM-1842: Forward references in storm.thrift cause tooling issues From 4f7e03420f9ad4b3cb2c2cadb15614825178b54e Mon Sep 17 00:00:00 2001 From: Pavel Grochal Date: Tue, 17 May 2016 15:11:29 +0200 Subject: [PATCH 47/58] FluxShellBolt can emit into custom stream. Signed-off-by: P. Taylor Goetz --- .../flux/wrappers/bolts/FluxShellBolt.java | 77 +++++++++++++++++-- 1 file changed, 71 insertions(+), 6 deletions(-) diff --git a/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/FluxShellBolt.java b/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/FluxShellBolt.java index 1af1f427193..7c96c64dc80 100644 --- a/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/FluxShellBolt.java +++ b/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/FluxShellBolt.java @@ -26,27 +26,92 @@ /** * A generic `ShellBolt` implementation that allows you specify output fields - * without having to subclass `ShellBolt` to do so. + * and even streams without having to subclass `ShellBolt` to do so. * */ public class FluxShellBolt extends ShellBolt implements IRichBolt{ - private String[] outputFields; + private Map outputFields; private Map componentConfig; + + /** + * Create a ShellBolt with command line arguments + * @param command Command line arguments for the bolt + */ + public FluxShellBolt(String[] command){ + super(command); + this.outputFields = new HashMap(); + } /** * Create a ShellBolt with command line arguments and output fields + * + * Keep this constructor for backward compatibility. + * * @param command Command line arguments for the bolt * @param outputFields Names of fields the bolt will emit (if any). */ - public FluxShellBolt(String[] command, String[] outputFields){ - super(command); - this.outputFields = outputFields; + this(command); + this.setDefaultStream(outputFields); + } + + + /** + * Set default stream outputFields, this method is called from YAML file: + * + * ``` + * bolts: + * - className: org.apache.storm.flux.wrappers.bolts.FluxShellBolt + * id: my_bolt + * constructorArgs: + * - [python, my_bolt.py] + * configMethods: + * - name: setDefaultStream + * args: + * - [word, count] + * ``` + * + * @param outputFields Names of fields the bolt will emit (if any) in default stream. + */ + public void setDefaultStream(String[] outputFields) { + this.setNamedStream("default", outputFields); + } + + /** + * Set custom *named* stream outputFields, this method is called from YAML file: + * + * ``` + * bolts: + * - className: org.apache.storm.flux.wrappers.bolts.FluxShellBolt + * id: my_bolt + * constructorArgs: + * - [python, my_bolt.py] + * configMethods: + * - name: setNamedStream + * args: + * - first + * - [word, count] + * ``` + * @param name Name of stream the bolt will emit into. + * @param outputFields Names of fields the bolt will emit in custom *named* stream. + */ + public void setNamedStream(String name, String[] outputFields) { + this.outputFields.put(name, outputFields); } @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { - declarer.declare(new Fields(this.outputFields)); + Iterator it = this.outputFields.entrySet().iterator(); + while (it.hasNext()) { + Map.Entry entryTuple = (Map.Entry)it.next(); + String key = (String)entryTuple.getKey(); + String[] value = (String[])entryTuple.getValue(); + if(key.equals("default")) { + declarer.declare(new Fields(value)); + } else { + declarer.declareStream(key, new Fields(value)); + } + } } @Override From 06ed21cc0cb6cfe3102711da2c368e4e8f88a10f Mon Sep 17 00:00:00 2001 From: Pavel Grochal Date: Tue, 17 May 2016 15:12:13 +0200 Subject: [PATCH 48/58] Fixed imports Signed-off-by: P. Taylor Goetz --- .../org/apache/storm/flux/wrappers/bolts/FluxShellBolt.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/FluxShellBolt.java b/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/FluxShellBolt.java index 7c96c64dc80..457b8dd1f19 100644 --- a/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/FluxShellBolt.java +++ b/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/FluxShellBolt.java @@ -23,6 +23,8 @@ import org.apache.storm.tuple.Fields; import java.util.Map; +import java.util.HashMap; +import java.util.Iterator; /** * A generic `ShellBolt` implementation that allows you specify output fields From e61110db1548e3ad4c25c31faa26e5450bbbd851 Mon Sep 17 00:00:00 2001 From: Pavel Grochal Date: Tue, 17 May 2016 15:17:50 +0200 Subject: [PATCH 49/58] FluxShellSpout can emit into streams. Signed-off-by: P. Taylor Goetz --- .../flux/wrappers/spouts/FluxShellSpout.java | 77 +++++++++++++++++-- 1 file changed, 72 insertions(+), 5 deletions(-) diff --git a/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/spouts/FluxShellSpout.java b/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/spouts/FluxShellSpout.java index 1951c1e6cd4..d855f123b3e 100644 --- a/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/spouts/FluxShellSpout.java +++ b/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/spouts/FluxShellSpout.java @@ -23,29 +23,96 @@ import org.apache.storm.tuple.Fields; import java.util.Map; +import java.util.HashMap; +import java.util.Iterator; /** * A generic `ShellSpout` implementation that allows you specify output fields - * without having to subclass `ShellSpout` to do so. + * and even streams without having to subclass `ShellSpout` to do so. * */ public class FluxShellSpout extends ShellSpout implements IRichSpout { - private String[] outputFields; + private Map outputFields; private Map componentConfig; + + /** + * Create a ShellSpout with command line arguments + * @param command Command line arguments for the bolt + */ + public FluxShellSpout(String[] command){ + super(command); + this.outputFields = new HashMap(); + } /** * Create a ShellSpout with command line arguments and output fields + * + * Keep this constructor for backward compatibility. + * * @param args Command line arguments for the spout * @param outputFields Names of fields the spout will emit. */ public FluxShellSpout(String[] args, String[] outputFields){ - super(args); - this.outputFields = outputFields; + this(args); + this.setDefaultStream(outputFields); + } + + /** + * Set default stream outputFields, this method is called from YAML file: + * + * ``` + * spouts: + * - className: org.apache.storm.flux.wrappers.bolts.FluxShellSpout + * id: my_spout + * constructorArgs: + * - [python, my_spout.py] + * configMethods: + * - name: setDefaultStream + * args: + * - [word, count] + * ``` + * + * @param outputFields Names of fields the spout will emit (if any) in default stream. + */ + public void setDefaultStream(String[] outputFields) { + this.setNamedStream("default", outputFields); + } + + /** + * Set custom *named* stream outputFields, this method is called from YAML file: + * + * ``` + * spouts: + * - className: org.apache.storm.flux.wrappers.bolts.FluxShellSpout + * id: my_spout + * constructorArgs: + * - [python, my_spout.py] + * configMethods: + * - name: setNamedStream + * args: + * - first + * - [word, count] + * ``` + * @param name Name of stream the spout will emit into. + * @param outputFields Names of fields the spout will emit in custom *named* stream. + */ + public void setNamedStream(String name, String[] outputFields) { + this.outputFields.put(name, outputFields); } @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { - declarer.declare(new Fields(this.outputFields)); + Iterator it = this.outputFields.entrySet().iterator(); + while (it.hasNext()) { + Map.Entry entryTuple = (Map.Entry)it.next(); + String key = (String)entryTuple.getKey(); + String[] value = (String[])entryTuple.getValue(); + if(key.equals("default")) { + declarer.declare(new Fields(value)); + } else { + declarer.declareStream(key, new Fields(value)); + } + } } @Override From 6397fbae377de1906ef01d79ef422dfd146931f3 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Tue, 24 May 2016 16:21:03 -0400 Subject: [PATCH 50/58] add STORM-1862 to changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 537585f2721..80e88eae8ba 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.1.0 + * STORM-1862: Flux ShellSpout and ShellBolt can't emit to named streams * Storm-1728: TransactionalTridentKafkaSpout error * STORM-1850: State Checkpointing Documentation update * STORM-1674: Idle KafkaSpout consumes more bandwidth than needed From b70f36db9051196c84737200a6f4148038413ebf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stig=20Rohde=20D=C3=B8ssing?= Date: Wed, 18 May 2016 13:13:15 +0200 Subject: [PATCH 51/58] STORM-1848: Make KafkaMessageId and Partition serializable to support eventlogging Signed-off-by: P. Taylor Goetz --- .../storm-kafka/src/jvm/org/apache/storm/kafka/Partition.java | 3 ++- .../src/jvm/org/apache/storm/kafka/PartitionManager.java | 4 ++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/Partition.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/Partition.java index afdf8afc1b6..99bb9d31f8c 100644 --- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/Partition.java +++ b/external/storm-kafka/src/jvm/org/apache/storm/kafka/Partition.java @@ -18,10 +18,11 @@ package org.apache.storm.kafka; import com.google.common.base.Objects; +import java.io.Serializable; import org.apache.storm.trident.spout.ISpoutPartition; -public class Partition implements ISpoutPartition { +public class Partition implements ISpoutPartition, Serializable { public Broker host; public int partition; 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 4fb5685ff33..c20f237cbf1 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 @@ -19,6 +19,7 @@ import com.google.common.base.Strings; import com.google.common.collect.ImmutableMap; +import java.io.Serializable; import org.apache.storm.Config; import org.apache.storm.kafka.KafkaSpout.EmitState; @@ -323,11 +324,10 @@ public void close() { _connections.unregister(_partition.host, _partition.topic , _partition.partition); } - static class KafkaMessageId { + static class KafkaMessageId implements Serializable { public Partition partition; public long offset; - public KafkaMessageId(Partition partition, long offset) { this.partition = partition; this.offset = offset; From bce3036d475b835fec9eb37988c77ba1c6b532a7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stig=20Rohde=20D=C3=B8ssing?= Date: Wed, 18 May 2016 21:13:06 +0200 Subject: [PATCH 52/58] STORM-1848: Add comment to SpoutOutputCollector javadoc mentioning that message ids should be serializable to work with event logging Signed-off-by: P. Taylor Goetz --- .../storm/spout/SpoutOutputCollector.java | 20 +++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/storm-core/src/jvm/org/apache/storm/spout/SpoutOutputCollector.java b/storm-core/src/jvm/org/apache/storm/spout/SpoutOutputCollector.java index 13482ba03f7..4b475759cba 100644 --- a/storm-core/src/jvm/org/apache/storm/spout/SpoutOutputCollector.java +++ b/storm-core/src/jvm/org/apache/storm/spout/SpoutOutputCollector.java @@ -40,8 +40,9 @@ public SpoutOutputCollector(ISpoutOutputCollector delegate) { * When Storm detects that this tuple has been fully processed, or has failed * to be fully processed, the spout will receive an ack or fail callback respectively * with the messageId as long as the messageId was not null. If the messageId was null, - * Storm will not track the tuple and no callback will be received. The emitted values must be - * immutable. + * Storm will not track the tuple and no callback will be received. + * Note that Storm's event logging functionality will only work if the messageId + * is serializable via Kryo or the Serializable interface. The emitted values must be immutable. * * @return the list of task ids that this tuple was sent to */ @@ -54,8 +55,9 @@ public List emit(String streamId, List tuple, Object messageId) * When Storm detects that this tuple has been fully processed, or has failed * to be fully processed, the spout will receive an ack or fail callback respectively * with the messageId as long as the messageId was not null. If the messageId was null, - * Storm will not track the tuple and no callback will be received. The emitted values must be - * immutable. + * Storm will not track the tuple and no callback will be received. + * Note that Storm's event logging functionality will only work if the messageId + * is serializable via Kryo or the Serializable interface. The emitted values must be immutable. * * @return the list of task ids that this tuple was sent to */ @@ -84,8 +86,9 @@ public List emit(String streamId, List tuple) { /** * Emits a tuple to the specified task on the specified output stream. This output * stream must have been declared as a direct stream, and the specified task must - * use a direct grouping on this stream to receive the message. The emitted values must be - * immutable. + * use a direct grouping on this stream to receive the message. Note that Storm's event + * logging functionality will only work if the messageId is serializable via Kryo + * or the Serializable interface. The emitted values must be immutable. */ public void emitDirect(int taskId, String streamId, List tuple, Object messageId) { _delegate.emitDirect(taskId, streamId, tuple, messageId); @@ -94,8 +97,9 @@ public void emitDirect(int taskId, String streamId, List tuple, Object m /** * Emits a tuple to the specified task on the default output stream. This output * stream must have been declared as a direct stream, and the specified task must - * use a direct grouping on this stream to receive the message. The emitted values must be - * immutable. + * use a direct grouping on this stream to receive the message. Note that Storm's event + * logging functionality will only work if the messageId is serializable via Kryo + * or the Serializable interface. The emitted values must be immutable. */ public void emitDirect(int taskId, List tuple, Object messageId) { emitDirect(taskId, Utils.DEFAULT_STREAM_ID, tuple, messageId); From 7b8561d8dd3bcab08d18b4cf3faaa24ef3ea5f4a Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Tue, 24 May 2016 16:34:29 -0400 Subject: [PATCH 53/58] add STORM-1848 to changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 80e88eae8ba..694fcb0732a 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.1.0 + * STORM-1848: Make KafkaMessageId and Partition serializable to support * STORM-1862: Flux ShellSpout and ShellBolt can't emit to named streams * Storm-1728: TransactionalTridentKafkaSpout error * STORM-1850: State Checkpointing Documentation update From 79a6c2dd83ee45b68d0609eb24d3bb9c2852e537 Mon Sep 17 00:00:00 2001 From: Arun Mahadevan Date: Thu, 19 May 2016 17:04:05 +0530 Subject: [PATCH 54/58] [STORM-1851] Fix default nimbus impersonation authorizer config "nimbus.impersonation.authorizer" is set to "ImpersonationAuthorizer" by default and this causes issues when a user tries to submit topology as a different user in secure mode since the "nimbus.impersonation.acl" configuration is not set by default. Users need to set nimbus.impersonation.acl first before they can submit topology as any user other than "storm" in secure mode. Removing this config allows users to submit topologies in secure mode as any user by default. Users can set up impersonation by providing both authorizer and the acls via storm.yaml. Signed-off-by: P. Taylor Goetz --- conf/defaults.yaml | 1 - docs/SECURITY.md | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/conf/defaults.yaml b/conf/defaults.yaml index 9e731d02396..9b081a42ebd 100644 --- a/conf/defaults.yaml +++ b/conf/defaults.yaml @@ -76,7 +76,6 @@ nimbus.topology.validator: "org.apache.storm.nimbus.DefaultTopologyValidator" 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 diff --git a/docs/SECURITY.md b/docs/SECURITY.md index b3ec303b2d8..98c55d39c86 100644 --- a/docs/SECURITY.md +++ b/docs/SECURITY.md @@ -391,7 +391,7 @@ A storm client may submit requests on behalf of another user. For example, if a it can do so by leveraging the impersonation feature.In order to submit topology as some other user , you can use `StormSubmitter.submitTopologyAs` API. Alternatively you can use `NimbusClient.getConfiguredClientAs` to get a nimbus client as some other user and perform any nimbus action(i.e. kill/rebalance/activate/deactivate) using this client. -To ensure only authorized users can perform impersonation you should start nimbus with `nimbus.impersonation.authorizer` set to `org.apache.storm.security.auth.authorizer.ImpersonationAuthorizer`. +Impersonation authorization is disabled by default which means any user can perform impersonation. To ensure only authorized users can perform impersonation you should start nimbus with `nimbus.impersonation.authorizer` set to `org.apache.storm.security.auth.authorizer.ImpersonationAuthorizer`. The `ImpersonationAuthorizer` uses `nimbus.impersonation.acl` as the acl to authorize users. Following is a sample nimbus config for supporting impersonation: ```yaml From 1a5b1dc566a6cce6902dbdf7330fe4e4d936fd3d Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Tue, 24 May 2016 16:47:44 -0400 Subject: [PATCH 55/58] add STORM-1851 to changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 694fcb0732a..0815c6c8310 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.1.0 + * STORM-1851: Fix default nimbus impersonation authorizer config * STORM-1848: Make KafkaMessageId and Partition serializable to support * STORM-1862: Flux ShellSpout and ShellBolt can't emit to named streams * Storm-1728: TransactionalTridentKafkaSpout error From 0d5ed75b7a05ce026cf3180f229de0ccdd3c2cb2 Mon Sep 17 00:00:00 2001 From: Balazs Kossovics Date: Fri, 20 May 2016 17:36:42 +0200 Subject: [PATCH 56/58] Ack late tuples in windowed mode The current implementation simply ignores late tuples without acking them, which causes timeouts and replays after TOPOLOGY_MESSAGE_TIMEOUT_SECS. A tuple which was late at a some time is going to be late at any moment in the future, so there is no point of replaying it, because the lingering late tuples will just block the topology (especially if TOPOLOGY_MAX_SPOUT_PENDING is set). --- .../src/jvm/org/apache/storm/topology/WindowedBoltExecutor.java | 1 + 1 file changed, 1 insertion(+) diff --git a/storm-core/src/jvm/org/apache/storm/topology/WindowedBoltExecutor.java b/storm-core/src/jvm/org/apache/storm/topology/WindowedBoltExecutor.java index ab2fe86a475..9439ef24f90 100644 --- a/storm-core/src/jvm/org/apache/storm/topology/WindowedBoltExecutor.java +++ b/storm-core/src/jvm/org/apache/storm/topology/WindowedBoltExecutor.java @@ -268,6 +268,7 @@ public void execute(Tuple input) { if (waterMarkEventGenerator.track(input.getSourceGlobalStreamId(), ts)) { windowManager.add(input, ts); } else { + windowedOutputCollector.ack(input); LOG.info("Received a late tuple {} with ts {}. This will not processed.", input, ts); } } else { From 2425f27f1d9cb179513f6b46de47039b95a3d8a7 Mon Sep 17 00:00:00 2001 From: Arun Mahadevan Date: Wed, 25 May 2016 21:25:24 +0530 Subject: [PATCH 57/58] Added STORM-1859 to Changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 0815c6c8310..a97dd348121 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.1.0 + * STORM-1859: Ack late tuples in windowed mode * STORM-1851: Fix default nimbus impersonation authorizer config * STORM-1848: Make KafkaMessageId and Partition serializable to support * STORM-1862: Flux ShellSpout and ShellBolt can't emit to named streams From 5b4827273fb1fc0e0b564c0d865751dbcc465d26 Mon Sep 17 00:00:00 2001 From: Sivaguru Kannan Date: Mon, 23 May 2016 12:50:48 -0700 Subject: [PATCH 58/58] STORM-1861 Fix storm script bug to not fork java process --- bin/storm.py | 22 ++++++++++++---------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/bin/storm.py b/bin/storm.py index bc44f56c636..a990de7d530 100755 --- a/bin/storm.py +++ b/bin/storm.py @@ -209,8 +209,9 @@ def exec_storm_class(klass, jvmtype="-server", jvmopts=[], extrajars=[], args=[] ] + jvmopts + [klass] + list(args) print("Running: " + " ".join(all_args)) sys.stdout.flush() + exit_code = 0 if fork: - os.spawnvp(os.P_WAIT, JAVA_CMD, all_args) + exit_code = os.spawnvp(os.P_WAIT, JAVA_CMD, all_args) elif is_windows(): # handling whitespaces in JAVA_CMD try: @@ -220,7 +221,7 @@ def exec_storm_class(klass, jvmtype="-server", jvmopts=[], extrajars=[], args=[] sys.exit(e.returncode) else: os.execvp(JAVA_CMD, all_args) - os._exit() + return exit_code def jar(jarfile, klass, *args): """Syntax: [storm jar topology-jar-path class ...] @@ -235,15 +236,16 @@ def jar(jarfile, klass, *args): if (transform_class != None and transform_class != "nil"): tmpjar = os.path.join(tempfile.gettempdir(), uuid.uuid1().hex+".jar") exec_storm_class("org.apache.storm.daemon.ClientJarTransformerRunner", args=[transform_class, jarfile, tmpjar], fork=True, daemon=False) - exec_storm_class( - klass, - jvmtype="-client", - extrajars=[tmpjar, USER_CONF_DIR, STORM_BIN_DIR], - args=args, - daemon=False, - fork=True, - jvmopts=JAR_JVM_OPTS + ["-Dstorm.jar=" + tmpjar]) + topology_runner_exit_code = exec_storm_class( + klass, + jvmtype="-client", + extrajars=[tmpjar, USER_CONF_DIR, STORM_BIN_DIR], + args=args, + daemon=False, + fork=True, + jvmopts=JAR_JVM_OPTS + ["-Dstorm.jar=" + tmpjar]) os.remove(tmpjar) + sys.exit(topology_runner_exit_code) else: exec_storm_class( klass,