diff --git a/external/storm-redis/src/main/java/org/apache/storm/redis/bolt/RedisFilterBolt.java b/external/storm-redis/src/main/java/org/apache/storm/redis/bolt/RedisFilterBolt.java index 4c858bcfdf2..e0e9da67a4e 100644 --- a/external/storm-redis/src/main/java/org/apache/storm/redis/bolt/RedisFilterBolt.java +++ b/external/storm-redis/src/main/java/org/apache/storm/redis/bolt/RedisFilterBolt.java @@ -17,17 +17,20 @@ */ package org.apache.storm.redis.bolt; +import java.util.List; import org.apache.storm.redis.common.config.JedisClusterConfig; import org.apache.storm.redis.common.config.JedisPoolConfig; +import org.apache.storm.redis.common.mapper.BasicStreamMapper; +import org.apache.storm.redis.common.mapper.DefaultStreamMapper; import org.apache.storm.redis.common.mapper.RedisDataTypeDescription; import org.apache.storm.redis.common.mapper.RedisFilterMapper; +import org.apache.storm.redis.common.mapper.StreamMapper; import org.apache.storm.topology.OutputFieldsDeclarer; import org.apache.storm.tuple.Tuple; +import org.apache.storm.tuple.Values; import redis.clients.jedis.GeoCoordinate; import redis.clients.jedis.JedisCommands; -import java.util.List; - /** * Basic bolt for querying from Redis and filters out if key/field doesn't exist. * If key/field exists on Redis, this bolt just forwards input tuple to default stream. @@ -45,18 +48,41 @@ */ public class RedisFilterBolt extends AbstractRedisBolt { private final RedisFilterMapper filterMapper; + private final StreamMapper streamMapper; private final RedisDataTypeDescription.RedisDataType dataType; private final String additionalKey; /** - * Constructor for single Redis environment (JedisPool) + * Constructor for single Redis environment (JedisPool). + * Tuples will be emitted to Storm's default streamId. * @param config configuration for initializing JedisPool * @param filterMapper mapper containing which datatype, query key that Bolt uses */ public RedisFilterBolt(JedisPoolConfig config, RedisFilterMapper filterMapper) { + this(config, filterMapper, new DefaultStreamMapper()); + } + + /** + * Constructor for single Redis environment (JedisPool). + * @param config configuration for initializing JedisCluster + * @param filterMapper mapper containing which datatype, query key that Bolt uses + * @param streamId the stream to which tuples that make it through the filter should be emitted. + */ + public RedisFilterBolt(JedisPoolConfig config, RedisFilterMapper filterMapper, String streamId) { + this(config, filterMapper, new BasicStreamMapper(streamId)); + } + + /** + * Constructor for single Redis environment (JedisPool). + * @param config configuration for initializing JedisPool + * @param filterMapper mapper containing which datatype, query key that Bolt uses + * @param streamMapper mapper to which stream a given Tuple/Values pair should be emitted. + */ + public RedisFilterBolt(JedisPoolConfig config, RedisFilterMapper filterMapper, StreamMapper streamMapper) { super(config); this.filterMapper = filterMapper; + this.streamMapper = streamMapper; RedisDataTypeDescription dataTypeDescription = filterMapper.getDataTypeDescription(); this.dataType = dataTypeDescription.getDataType(); @@ -69,14 +95,36 @@ public RedisFilterBolt(JedisPoolConfig config, RedisFilterMapper filterMapper) { } /** - * Constructor for Redis Cluster environment (JedisCluster) + * Constructor for Redis Cluster environment (JedisCluster). + * Tuples will be emitted to Storm's default streamId. * @param config configuration for initializing JedisCluster * @param filterMapper mapper containing which datatype, query key that Bolt uses */ public RedisFilterBolt(JedisClusterConfig config, RedisFilterMapper filterMapper) { + this(config, filterMapper, new DefaultStreamMapper()); + } + + /** + * Constructor for Redis Cluster environment (JedisCluster). + * @param config configuration for initializing JedisCluster + * @param filterMapper mapper containing which datatype, query key that Bolt uses + * @param streamId the stream to which tuples that make it through the filter should be emitted. + */ + public RedisFilterBolt(JedisClusterConfig config, RedisFilterMapper filterMapper, String streamId) { + this(config, filterMapper, new BasicStreamMapper(streamId)); + } + + /** + * Constructor for Redis Cluster environment (JedisCluster). + * @param config configuration for initializing JedisCluster + * @param filterMapper mapper containing which datatype, query key that Bolt uses + * @param streamMapper mapper to which stream a given Tuple/Values pair should be emitted. + */ + public RedisFilterBolt(JedisClusterConfig config, RedisFilterMapper filterMapper, StreamMapper streamMapper) { super(config); this.filterMapper = filterMapper; + this.streamMapper = streamMapper; RedisDataTypeDescription dataTypeDescription = filterMapper.getDataTypeDescription(); this.dataType = dataTypeDescription.getDataType(); @@ -126,7 +174,9 @@ public void execute(Tuple input) { } if (found) { - collector.emit(input, input.getValues()); + Values values = new Values(input.getValues().toArray()); + String streamId = streamMapper.getStreamId(input, values); + collector.emit(streamId, input, values); } collector.ack(input); 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 06529232ee3..13b66c7d407 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 @@ -17,17 +17,19 @@ */ package org.apache.storm.redis.bolt; +import java.util.List; +import org.apache.storm.redis.common.config.JedisClusterConfig; +import org.apache.storm.redis.common.config.JedisPoolConfig; +import org.apache.storm.redis.common.mapper.BasicStreamMapper; +import org.apache.storm.redis.common.mapper.DefaultStreamMapper; +import org.apache.storm.redis.common.mapper.RedisDataTypeDescription; +import org.apache.storm.redis.common.mapper.RedisLookupMapper; +import org.apache.storm.redis.common.mapper.StreamMapper; import org.apache.storm.topology.OutputFieldsDeclarer; import org.apache.storm.tuple.Tuple; import org.apache.storm.tuple.Values; -import org.apache.storm.redis.common.mapper.RedisDataTypeDescription; -import org.apache.storm.redis.common.mapper.RedisLookupMapper; -import org.apache.storm.redis.common.config.JedisClusterConfig; -import org.apache.storm.redis.common.config.JedisPoolConfig; import redis.clients.jedis.JedisCommands; -import java.util.List; - /** * Basic bolt for querying from Redis and emits response as tuple. *
@@ -35,18 +37,41 @@ */ public class RedisLookupBolt extends AbstractRedisBolt { private final RedisLookupMapper lookupMapper; + private final StreamMapper streamMapper; private final RedisDataTypeDescription.RedisDataType dataType; private final String additionalKey; /** - * Constructor for single Redis environment (JedisPool) + * Constructor for single Redis environment (JedisPool). + * Emits tuples to Storm's default stream. * @param config configuration for initializing JedisPool * @param lookupMapper mapper containing which datatype, query key, output key that Bolt uses */ public RedisLookupBolt(JedisPoolConfig config, RedisLookupMapper lookupMapper) { + this(config, lookupMapper, new DefaultStreamMapper()); + } + + /** + * Constructor for single Redis environment (JedisPool). + * @param config configuration for initializing JedisPool + * @param lookupMapper mapper containing which datatype, query key, output key that Bolt uses + * @param streamId the streamId to which this bolt should emit tuples + */ + public RedisLookupBolt(JedisPoolConfig config, RedisLookupMapper lookupMapper, String streamId) { + this(config, lookupMapper, new BasicStreamMapper(streamId)); + } + + /** + * Constructor for single Redis environment (JedisPool). + * @param config configuration for initializing JedisPool + * @param lookupMapper mapper containing which datatype, query key, output key that Bolt uses + * @param streamMapper mapper to which stream a given Tuple/Values pair should be emitted. + */ + public RedisLookupBolt(JedisPoolConfig config, RedisLookupMapper lookupMapper, StreamMapper streamMapper) { super(config); this.lookupMapper = lookupMapper; + this.streamMapper = streamMapper; RedisDataTypeDescription dataTypeDescription = lookupMapper.getDataTypeDescription(); this.dataType = dataTypeDescription.getDataType(); @@ -54,14 +79,36 @@ public RedisLookupBolt(JedisPoolConfig config, RedisLookupMapper lookupMapper) { } /** - * Constructor for Redis Cluster environment (JedisCluster) + * Constructor for Redis Cluster environment (JedisCluster). + * Emits tuples to Storm's default stream. * @param config configuration for initializing JedisCluster * @param lookupMapper mapper containing which datatype, query key, output key that Bolt uses */ public RedisLookupBolt(JedisClusterConfig config, RedisLookupMapper lookupMapper) { + this(config, lookupMapper, new DefaultStreamMapper()); + } + + /** + * Constructor for single Redis environment (JedisPool). + * @param config configuration for initializing JedisPool + * @param lookupMapper mapper containing which datatype, query key, output key that Bolt uses + * @param streamId the streamId to which this bolt should emit tuples + */ + public RedisLookupBolt(JedisClusterConfig config, RedisLookupMapper lookupMapper, String streamId) { + this(config, lookupMapper, new BasicStreamMapper(streamId)); + } + + /** + * Constructor for Redis Cluster environment (JedisCluster). + * @param config configuration for initializing JedisCluster + * @param lookupMapper mapper containing which datatype, query key, output key that Bolt uses + * @param streamMapper mapper to which stream a given Tuple/Values pair should be emitted. + */ + public RedisLookupBolt(JedisClusterConfig config, RedisLookupMapper lookupMapper, StreamMapper streamMapper) { super(config); this.lookupMapper = lookupMapper; + this.streamMapper = streamMapper; RedisDataTypeDescription dataTypeDescription = lookupMapper.getDataTypeDescription(); this.dataType = dataTypeDescription.getDataType(); @@ -115,7 +162,8 @@ public void execute(Tuple input) { List