You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by bo...@apache.org on 2015/02/23 17:26:18 UTC

[05/23] storm git commit: Introduce Basic Redis Bolt with some test topologies

Introduce Basic Redis Bolt with some test topologies


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/171a8bf4
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/171a8bf4
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/171a8bf4

Branch: refs/heads/master
Commit: 171a8bf461a2fa97dcfa8439680f4043d1c2dd3a
Parents: 28ebb82
Author: Jungtaek Lim <ka...@gmail.com>
Authored: Tue Dec 30 17:50:35 2014 +0900
Committer: Jungtaek Lim <ka...@gmail.com>
Committed: Tue Dec 30 17:50:35 2014 +0900

----------------------------------------------------------------------
 .../storm/redis/bolt/AbstractRedisBolt.java     |  50 +++++++
 .../redis/util/config/JedisClusterConfig.java   |  59 +++++++++
 .../redis/util/config/JedisPoolConfig.java      |  80 +++++++++++
 .../util/container/JedisClusterContainer.java   |  30 +++++
 .../JedisCommandsContainerBuilder.java          |  22 ++++
 .../JedisCommandsInstanceContainer.java         |   8 ++
 .../redis/util/container/JedisContainer.java    |  44 +++++++
 .../storm/redis/topology/LookupWordCount.java   | 131 +++++++++++++++++++
 .../redis/topology/PersistentWordCount.java     | 117 +++++++++++++++++
 .../storm/redis/topology/WordCounter.java       |  58 ++++++++
 .../apache/storm/redis/topology/WordSpout.java  |  88 +++++++++++++
 11 files changed, 687 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/171a8bf4/external/storm-redis/src/main/java/org/apache/storm/redis/bolt/AbstractRedisBolt.java
----------------------------------------------------------------------
diff --git a/external/storm-redis/src/main/java/org/apache/storm/redis/bolt/AbstractRedisBolt.java b/external/storm-redis/src/main/java/org/apache/storm/redis/bolt/AbstractRedisBolt.java
new file mode 100644
index 0000000..100503a
--- /dev/null
+++ b/external/storm-redis/src/main/java/org/apache/storm/redis/bolt/AbstractRedisBolt.java
@@ -0,0 +1,50 @@
+package org.apache.storm.redis.bolt;
+
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.base.BaseRichBolt;
+import org.apache.storm.redis.util.config.JedisClusterConfig;
+import org.apache.storm.redis.util.config.JedisPoolConfig;
+import org.apache.storm.redis.util.container.JedisCommandsContainerBuilder;
+import org.apache.storm.redis.util.container.JedisCommandsInstanceContainer;
+import redis.clients.jedis.JedisCommands;
+
+import java.util.Map;
+
+public abstract class AbstractRedisBolt extends BaseRichBolt {
+    protected OutputCollector collector;
+
+    private transient JedisCommandsInstanceContainer container;
+
+    private JedisPoolConfig jedisPoolConfig;
+    private JedisClusterConfig jedisClusterConfig;
+
+    public AbstractRedisBolt(JedisPoolConfig config) {
+        this.jedisPoolConfig = config;
+    }
+
+    public AbstractRedisBolt(JedisClusterConfig config) {
+        this.jedisClusterConfig = config;
+    }
+
+    @Override
+    public void prepare(Map map, TopologyContext topologyContext, OutputCollector collector) {
+        this.collector = collector;
+
+        if (jedisPoolConfig != null) {
+            this.container = JedisCommandsContainerBuilder.build(jedisPoolConfig);
+        } else if (jedisClusterConfig != null) {
+            this.container = JedisCommandsContainerBuilder.build(jedisClusterConfig);
+        } else {
+            throw new IllegalArgumentException("Jedis configuration not found");
+        }
+    }
+
+    protected JedisCommands getInstance() {
+        return this.container.getInstance();
+    }
+
+    protected void returnInstance(JedisCommands instance) {
+        this.container.returnInstance(instance);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/171a8bf4/external/storm-redis/src/main/java/org/apache/storm/redis/util/config/JedisClusterConfig.java
----------------------------------------------------------------------
diff --git a/external/storm-redis/src/main/java/org/apache/storm/redis/util/config/JedisClusterConfig.java b/external/storm-redis/src/main/java/org/apache/storm/redis/util/config/JedisClusterConfig.java
new file mode 100644
index 0000000..9ec6b20
--- /dev/null
+++ b/external/storm-redis/src/main/java/org/apache/storm/redis/util/config/JedisClusterConfig.java
@@ -0,0 +1,59 @@
+package org.apache.storm.redis.util.config;
+
+import com.google.common.base.Preconditions;
+import redis.clients.jedis.HostAndPort;
+import redis.clients.jedis.Protocol;
+
+import java.io.Serializable;
+import java.util.Set;
+
+public class JedisClusterConfig implements Serializable {
+    private Set<HostAndPort> nodes;
+    private int timeout;
+    private int maxRedirections;
+
+    public JedisClusterConfig(Set<HostAndPort> nodes, int timeout, int maxRedirections) {
+        this.nodes = nodes;
+        this.timeout = timeout;
+        this.maxRedirections = maxRedirections;
+    }
+
+    public Set<HostAndPort> getNodes() {
+        return nodes;
+    }
+
+    public int getTimeout() {
+        return timeout;
+    }
+
+    public int getMaxRedirections() {
+        return maxRedirections;
+    }
+
+    static class Builder {
+        private Set<HostAndPort> nodes;
+        private int timeout = Protocol.DEFAULT_TIMEOUT;
+        private int maxRedirections = 5;
+
+        public Builder setNodes(Set<HostAndPort> nodes) {
+            this.nodes = nodes;
+            return this;
+        }
+
+        public Builder setTimeout(int timeout) {
+            this.timeout = timeout;
+            return this;
+        }
+
+        public Builder setMaxRedirections(int maxRedirections) {
+            this.maxRedirections = maxRedirections;
+            return this;
+        }
+
+        public JedisClusterConfig build() {
+            Preconditions.checkNotNull(this.nodes, "Node information should be presented");
+
+            return new JedisClusterConfig(nodes, timeout, maxRedirections);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/171a8bf4/external/storm-redis/src/main/java/org/apache/storm/redis/util/config/JedisPoolConfig.java
----------------------------------------------------------------------
diff --git a/external/storm-redis/src/main/java/org/apache/storm/redis/util/config/JedisPoolConfig.java b/external/storm-redis/src/main/java/org/apache/storm/redis/util/config/JedisPoolConfig.java
new file mode 100644
index 0000000..21a79dd
--- /dev/null
+++ b/external/storm-redis/src/main/java/org/apache/storm/redis/util/config/JedisPoolConfig.java
@@ -0,0 +1,80 @@
+package org.apache.storm.redis.util.config;
+
+import redis.clients.jedis.Protocol;
+
+import java.io.Serializable;
+
+public class JedisPoolConfig implements Serializable {
+    public static final String DEFAULT_HOST = "127.0.0.1";
+
+    private String host;
+    private int port;
+    private int timeout;
+    private int database;
+    private String password;
+
+    public JedisPoolConfig(String host, int port, int timeout, String password, int database) {
+        this.host = host;
+        this.port = port;
+        this.timeout = timeout;
+        this.database = database;
+        this.password = password;
+    }
+
+    public String getHost() {
+        return host;
+    }
+
+    public int getPort() {
+        return port;
+    }
+
+    public int getTimeout() {
+        return timeout;
+    }
+
+    public int getDatabase() {
+        return database;
+    }
+
+    public String getPassword() {
+        return password;
+    }
+
+    public static class Builder {
+        private String host = DEFAULT_HOST;
+        private int port = Protocol.DEFAULT_PORT;
+        private int timeout = Protocol.DEFAULT_TIMEOUT;
+        private int database = Protocol.DEFAULT_DATABASE;
+        private String password;
+
+        public Builder setHost(String host) {
+            this.host = host;
+            return this;
+        }
+
+        public Builder setPort(int port) {
+            this.port = port;
+            return this;
+        }
+
+        public Builder setTimeout(int timeout) {
+            this.timeout = timeout;
+            return this;
+        }
+
+        public Builder setDatabase(int database) {
+            this.database = database;
+            return this;
+        }
+
+        public Builder setPassword(String password) {
+            this.password = password;
+            return this;
+        }
+
+        public JedisPoolConfig build() {
+            return new JedisPoolConfig(host, port, timeout, password, database);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/171a8bf4/external/storm-redis/src/main/java/org/apache/storm/redis/util/container/JedisClusterContainer.java
----------------------------------------------------------------------
diff --git a/external/storm-redis/src/main/java/org/apache/storm/redis/util/container/JedisClusterContainer.java b/external/storm-redis/src/main/java/org/apache/storm/redis/util/container/JedisClusterContainer.java
new file mode 100644
index 0000000..be6694a
--- /dev/null
+++ b/external/storm-redis/src/main/java/org/apache/storm/redis/util/container/JedisClusterContainer.java
@@ -0,0 +1,30 @@
+package org.apache.storm.redis.util.container;
+
+import redis.clients.jedis.JedisCluster;
+import redis.clients.jedis.JedisCommands;
+
+import java.io.Closeable;
+
+public class JedisClusterContainer implements JedisCommandsInstanceContainer, Closeable {
+
+    private JedisCluster jedisCluster;
+
+    public JedisClusterContainer(JedisCluster jedisCluster) {
+        this.jedisCluster = jedisCluster;
+    }
+
+    @Override
+    public JedisCommands getInstance() {
+        return this.jedisCluster;
+    }
+
+    @Override
+    public void returnInstance(JedisCommands jedisCommands) {
+        // do nothing
+    }
+
+    @Override
+    public void close() {
+        this.jedisCluster.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/171a8bf4/external/storm-redis/src/main/java/org/apache/storm/redis/util/container/JedisCommandsContainerBuilder.java
----------------------------------------------------------------------
diff --git a/external/storm-redis/src/main/java/org/apache/storm/redis/util/container/JedisCommandsContainerBuilder.java b/external/storm-redis/src/main/java/org/apache/storm/redis/util/container/JedisCommandsContainerBuilder.java
new file mode 100644
index 0000000..1f10524
--- /dev/null
+++ b/external/storm-redis/src/main/java/org/apache/storm/redis/util/container/JedisCommandsContainerBuilder.java
@@ -0,0 +1,22 @@
+package org.apache.storm.redis.util.container;
+
+import org.apache.storm.redis.util.config.JedisClusterConfig;
+import org.apache.storm.redis.util.config.JedisPoolConfig;
+import redis.clients.jedis.JedisCluster;
+import redis.clients.jedis.JedisPool;
+
+public class JedisCommandsContainerBuilder {
+
+    // TODO : serialize redis.clients.jedis.JedisPoolConfig
+    public static final redis.clients.jedis.JedisPoolConfig DEFAULT_POOL_CONFIG = new redis.clients.jedis.JedisPoolConfig();
+
+    public static JedisCommandsInstanceContainer build(JedisPoolConfig config) {
+        JedisPool jedisPool = new JedisPool(DEFAULT_POOL_CONFIG, config.getHost(), config.getPort(), config.getTimeout(), config.getPassword(), config.getDatabase());
+        return new JedisContainer(jedisPool);
+    }
+
+    public static JedisCommandsInstanceContainer build(JedisClusterConfig config) {
+        JedisCluster jedisCluster = new JedisCluster(config.getNodes(), config.getTimeout(), config.getMaxRedirections(), DEFAULT_POOL_CONFIG);
+        return new JedisClusterContainer(jedisCluster);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/171a8bf4/external/storm-redis/src/main/java/org/apache/storm/redis/util/container/JedisCommandsInstanceContainer.java
----------------------------------------------------------------------
diff --git a/external/storm-redis/src/main/java/org/apache/storm/redis/util/container/JedisCommandsInstanceContainer.java b/external/storm-redis/src/main/java/org/apache/storm/redis/util/container/JedisCommandsInstanceContainer.java
new file mode 100644
index 0000000..6f1a148
--- /dev/null
+++ b/external/storm-redis/src/main/java/org/apache/storm/redis/util/container/JedisCommandsInstanceContainer.java
@@ -0,0 +1,8 @@
+package org.apache.storm.redis.util.container;
+
+import redis.clients.jedis.JedisCommands;
+
+public interface JedisCommandsInstanceContainer {
+    JedisCommands getInstance();
+    void returnInstance(JedisCommands jedisCommands);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/171a8bf4/external/storm-redis/src/main/java/org/apache/storm/redis/util/container/JedisContainer.java
----------------------------------------------------------------------
diff --git a/external/storm-redis/src/main/java/org/apache/storm/redis/util/container/JedisContainer.java b/external/storm-redis/src/main/java/org/apache/storm/redis/util/container/JedisContainer.java
new file mode 100644
index 0000000..58177e4
--- /dev/null
+++ b/external/storm-redis/src/main/java/org/apache/storm/redis/util/container/JedisContainer.java
@@ -0,0 +1,44 @@
+package org.apache.storm.redis.util.container;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import redis.clients.jedis.Jedis;
+import redis.clients.jedis.JedisCommands;
+import redis.clients.jedis.JedisPool;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+public class JedisContainer implements JedisCommandsInstanceContainer, Closeable {
+    private static final Logger LOG = LoggerFactory.getLogger(JedisContainer.class);
+
+    private JedisPool jedisPool;
+
+    public JedisContainer(JedisPool jedisPool) {
+        this.jedisPool = jedisPool;
+    }
+
+    @Override
+    public JedisCommands getInstance() {
+        return jedisPool.getResource();
+    }
+
+    @Override
+    public void returnInstance(JedisCommands jedisCommands) {
+        try {
+            ((Closeable) jedisCommands).close();
+        } catch (IOException e) {
+            LOG.warn("Failed to close (return) instance to pool");
+            try {
+                jedisPool.returnBrokenResource((Jedis) jedisCommands);
+            } catch (Exception e2) {
+                LOG.error("Failed to discard instance from pool");
+            }
+        }
+    }
+
+    @Override
+    public void close() {
+        jedisPool.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/171a8bf4/external/storm-redis/src/test/java/org/apache/storm/redis/topology/LookupWordCount.java
----------------------------------------------------------------------
diff --git a/external/storm-redis/src/test/java/org/apache/storm/redis/topology/LookupWordCount.java b/external/storm-redis/src/test/java/org/apache/storm/redis/topology/LookupWordCount.java
new file mode 100644
index 0000000..42d3800
--- /dev/null
+++ b/external/storm-redis/src/test/java/org/apache/storm/redis/topology/LookupWordCount.java
@@ -0,0 +1,131 @@
+/**
+ * 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.redis.topology;
+
+import backtype.storm.Config;
+import backtype.storm.LocalCluster;
+import backtype.storm.StormSubmitter;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.topology.TopologyBuilder;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.tuple.Values;
+import org.apache.storm.redis.bolt.AbstractRedisBolt;
+import org.apache.storm.redis.util.config.JedisClusterConfig;
+import org.apache.storm.redis.util.config.JedisPoolConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import redis.clients.jedis.JedisCommands;
+import redis.clients.jedis.exceptions.JedisConnectionException;
+import redis.clients.jedis.exceptions.JedisException;
+
+import java.util.Random;
+
+public class LookupWordCount {
+    private static final String WORD_SPOUT = "WORD_SPOUT";
+    private static final String LOOKUP_BOLT = "LOOKUP_BOLT";
+
+    private static final String TEST_REDIS_HOST = "127.0.0.1";
+    private static final int TEST_REDIS_PORT = 6379;
+
+    public static class LookupWordTotalCountBolt extends AbstractRedisBolt {
+        private static final Logger LOG = LoggerFactory.getLogger(LookupWordTotalCountBolt.class);
+        private static final Random RANDOM = new Random();
+
+        public LookupWordTotalCountBolt(JedisPoolConfig config) {
+            super(config);
+        }
+
+        public LookupWordTotalCountBolt(JedisClusterConfig config) {
+            super(config);
+        }
+
+        @Override
+        public void execute(Tuple input) {
+            JedisCommands jedisCommands = null;
+            try {
+                jedisCommands = getInstance();
+                String wordName = input.getStringByField("word");
+                String countStr = jedisCommands.get(wordName);
+                if (countStr != null) {
+                    int count = Integer.parseInt(countStr);
+                    this.collector.emit(new Values(wordName, count));
+
+                    // print lookup result with low probability
+                    if(RANDOM.nextInt(1000) > 995) {
+                        LOG.info("Lookup result - word : " + wordName + " / count : " + count);
+                    }
+                } else {
+                    // skip
+                    LOG.warn("Word not found in Redis - word : " + wordName);
+                }
+            } catch (NumberFormatException e) {
+                LOG.error("Counter Type seems not stored to integer", e);
+            } catch (JedisConnectionException e) {
+                throw new RuntimeException("Unfortunately, this test requires redis-server running", e);
+            } catch (JedisException e) {
+                LOG.error("Exception occurred from Jedis/Redis", e);
+            } finally {
+                returnInstance(jedisCommands);
+                this.collector.ack(input);
+            }
+        }
+
+        @Override
+        public void declareOutputFields(OutputFieldsDeclarer declarer) {
+            // wordName, count
+            declarer.declare(new Fields("wordName", "count"));
+        }
+    }
+
+    public static void main(String[] args) throws Exception {
+        Config config = new Config();
+
+        String host = TEST_REDIS_HOST;
+        int port = TEST_REDIS_PORT;
+
+        if (args.length > 2) {
+            host = args[0];
+            port = Integer.parseInt(args[1]);
+        }
+
+        JedisPoolConfig poolConfig = new JedisPoolConfig.Builder()
+                .setHost(host).setPort(port).build();
+
+        WordSpout spout = new WordSpout();
+        LookupWordTotalCountBolt redisLookupBolt = new LookupWordTotalCountBolt(poolConfig);
+
+        //wordspout -> lookupbolt
+        TopologyBuilder builder = new TopologyBuilder();
+        builder.setSpout(WORD_SPOUT, spout, 1);
+        builder.setBolt(LOOKUP_BOLT, redisLookupBolt, 1).shuffleGrouping(WORD_SPOUT);
+
+        if (args.length == 2) {
+            LocalCluster cluster = new LocalCluster();
+            cluster.submitTopology("test", config, builder.createTopology());
+            Thread.sleep(30000);
+            cluster.killTopology("test");
+            cluster.shutdown();
+            System.exit(0);
+        } else if (args.length == 3) {
+            StormSubmitter.submitTopology(args[2], config, builder.createTopology());
+        } else{
+            System.out.println("Usage: LookupWordCount <redis host> <redis port> (topology name)");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/171a8bf4/external/storm-redis/src/test/java/org/apache/storm/redis/topology/PersistentWordCount.java
----------------------------------------------------------------------
diff --git a/external/storm-redis/src/test/java/org/apache/storm/redis/topology/PersistentWordCount.java b/external/storm-redis/src/test/java/org/apache/storm/redis/topology/PersistentWordCount.java
new file mode 100644
index 0000000..a96696b
--- /dev/null
+++ b/external/storm-redis/src/test/java/org/apache/storm/redis/topology/PersistentWordCount.java
@@ -0,0 +1,117 @@
+/**
+ * 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.redis.topology;
+
+import backtype.storm.Config;
+import backtype.storm.LocalCluster;
+import backtype.storm.StormSubmitter;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.topology.TopologyBuilder;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Tuple;
+import org.apache.storm.redis.bolt.AbstractRedisBolt;
+import org.apache.storm.redis.util.config.JedisClusterConfig;
+import org.apache.storm.redis.util.config.JedisPoolConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import redis.clients.jedis.JedisCommands;
+import redis.clients.jedis.exceptions.JedisConnectionException;
+import redis.clients.jedis.exceptions.JedisException;
+
+public class PersistentWordCount {
+    private static final String WORD_SPOUT = "WORD_SPOUT";
+    private static final String COUNT_BOLT = "COUNT_BOLT";
+    private static final String REDIS_BOLT = "REDIS_BOLT";
+
+    private static final String TEST_REDIS_HOST = "127.0.0.1";
+    private static final int TEST_REDIS_PORT = 6379;
+
+    public static class StoreCountRedisBolt extends AbstractRedisBolt {
+        private static final Logger LOG = LoggerFactory.getLogger(StoreCountRedisBolt.class);
+
+        public StoreCountRedisBolt(JedisPoolConfig config) {
+            super(config);
+        }
+
+        public StoreCountRedisBolt(JedisClusterConfig config) {
+            super(config);
+        }
+
+        @Override
+        public void execute(Tuple input) {
+            String word = input.getStringByField("word");
+            int count = input.getIntegerByField("count");
+
+            JedisCommands commands = null;
+            try {
+                commands = getInstance();
+                commands.incrBy(word, count);
+            } catch (JedisConnectionException e) {
+                throw new RuntimeException("Unfortunately, this test requires redis-server running", e);
+            } catch (JedisException e) {
+                LOG.error("Exception occurred from Jedis/Redis", e);
+            } finally {
+                returnInstance(commands);
+                this.collector.ack(input);
+            }
+        }
+
+        @Override
+        public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        }
+    }
+
+    public static void main(String[] args) throws Exception {
+        Config config = new Config();
+
+        String host = TEST_REDIS_HOST;
+        int port = TEST_REDIS_PORT;
+
+        if (args.length > 2) {
+            host = args[0];
+            port = Integer.parseInt(args[1]);
+        }
+
+        JedisPoolConfig poolConfig = new JedisPoolConfig.Builder()
+                .setHost(host).setPort(port).build();
+
+        WordSpout spout = new WordSpout();
+        WordCounter bolt = new WordCounter();
+        StoreCountRedisBolt redisBolt = new StoreCountRedisBolt(poolConfig);
+
+        // wordSpout ==> countBolt ==> RedisBolt
+        TopologyBuilder builder = new TopologyBuilder();
+
+        builder.setSpout(WORD_SPOUT, spout, 1);
+        builder.setBolt(COUNT_BOLT, bolt, 1).shuffleGrouping(WORD_SPOUT);
+        builder.setBolt(REDIS_BOLT, redisBolt, 1).fieldsGrouping(COUNT_BOLT, new Fields("word"));
+
+        if (args.length == 2) {
+            LocalCluster cluster = new LocalCluster();
+            cluster.submitTopology("test", config, builder.createTopology());
+            Thread.sleep(30000);
+            cluster.killTopology("test");
+            cluster.shutdown();
+            System.exit(0);
+        } else if (args.length == 3) {
+            StormSubmitter.submitTopology(args[2], config, builder.createTopology());
+        } else {
+            System.out.println("Usage: PersistentWordCount <redis host> <redis port> (topology name)");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/171a8bf4/external/storm-redis/src/test/java/org/apache/storm/redis/topology/WordCounter.java
----------------------------------------------------------------------
diff --git a/external/storm-redis/src/test/java/org/apache/storm/redis/topology/WordCounter.java b/external/storm-redis/src/test/java/org/apache/storm/redis/topology/WordCounter.java
new file mode 100644
index 0000000..6a0548d
--- /dev/null
+++ b/external/storm-redis/src/test/java/org/apache/storm/redis/topology/WordCounter.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.redis.topology;
+
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.BasicOutputCollector;
+import backtype.storm.topology.IBasicBolt;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Tuple;
+
+import java.util.Map;
+
+import static backtype.storm.utils.Utils.tuple;
+
+public class WordCounter implements IBasicBolt {
+
+
+    @SuppressWarnings("rawtypes")
+    public void prepare(Map stormConf, TopologyContext context) {
+    }
+
+    /*
+     * Just output the word value with a count of 1.
+     */
+    public void execute(Tuple input, BasicOutputCollector collector) {
+        collector.emit(tuple(input.getValues().get(0), 1));
+    }
+
+    public void cleanup() {
+
+    }
+
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        declarer.declare(new Fields("word", "count"));
+    }
+
+    @Override
+    public Map<String, Object> getComponentConfiguration() {
+        return null;
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/171a8bf4/external/storm-redis/src/test/java/org/apache/storm/redis/topology/WordSpout.java
----------------------------------------------------------------------
diff --git a/external/storm-redis/src/test/java/org/apache/storm/redis/topology/WordSpout.java b/external/storm-redis/src/test/java/org/apache/storm/redis/topology/WordSpout.java
new file mode 100644
index 0000000..bb9c2d7
--- /dev/null
+++ b/external/storm-redis/src/test/java/org/apache/storm/redis/topology/WordSpout.java
@@ -0,0 +1,88 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.redis.topology;
+
+import backtype.storm.spout.SpoutOutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.IRichSpout;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Values;
+
+import java.util.Map;
+import java.util.Random;
+import java.util.UUID;
+
+public class WordSpout implements IRichSpout {
+    boolean isDistributed;
+    SpoutOutputCollector collector;
+    public static final String[] words = new String[] { "apple", "orange", "pineapple", "banana", "watermelon" };
+
+    public WordSpout() {
+        this(true);
+    }
+
+    public WordSpout(boolean isDistributed) {
+        this.isDistributed = isDistributed;
+    }
+
+    public boolean isDistributed() {
+        return this.isDistributed;
+    }
+
+    @SuppressWarnings("rawtypes")
+    public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
+        this.collector = collector;
+    }
+
+    public void close() {
+
+    }
+
+    public void nextTuple() {
+        final Random rand = new Random();
+        final String word = words[rand.nextInt(words.length)];
+        this.collector.emit(new Values(word), UUID.randomUUID());
+        Thread.yield();
+    }
+
+    public void ack(Object msgId) {
+
+    }
+
+    public void fail(Object msgId) {
+
+    }
+
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        declarer.declare(new Fields("word"));
+    }
+
+    @Override
+    public void activate() {
+    }
+
+    @Override
+    public void deactivate() {
+    }
+
+    @Override
+    public Map<String, Object> getComponentConfiguration() {
+        return null;
+    }
+}