| /** |
| * 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 com.google.common.collect.Lists; |
| |
| import java.util.List; |
| import java.util.Map; |
| import java.util.Random; |
| |
| import org.apache.storm.Config; |
| import org.apache.storm.StormSubmitter; |
| import org.apache.storm.redis.bolt.RedisLookupBolt; |
| import org.apache.storm.redis.common.config.JedisPoolConfig; |
| import org.apache.storm.redis.common.mapper.RedisDataTypeDescription; |
| import org.apache.storm.redis.common.mapper.RedisLookupMapper; |
| import org.apache.storm.task.OutputCollector; |
| import org.apache.storm.task.TopologyContext; |
| import org.apache.storm.topology.OutputFieldsDeclarer; |
| import org.apache.storm.topology.TopologyBuilder; |
| import org.apache.storm.topology.base.BaseRichBolt; |
| import org.apache.storm.tuple.Fields; |
| import org.apache.storm.tuple.ITuple; |
| import org.apache.storm.tuple.Tuple; |
| import org.apache.storm.tuple.Values; |
| |
| import org.slf4j.Logger; |
| import org.slf4j.LoggerFactory; |
| |
| public class LookupWordCount { |
| private static final String WORD_SPOUT = "WORD_SPOUT"; |
| private static final String LOOKUP_BOLT = "LOOKUP_BOLT"; |
| private static final String PRINT_BOLT = "PRINT_BOLT"; |
| |
| private static final String TEST_REDIS_HOST = "127.0.0.1"; |
| private static final int TEST_REDIS_PORT = 6379; |
| |
| public static class PrintWordTotalCountBolt extends BaseRichBolt { |
| private static final Logger LOG = LoggerFactory.getLogger(PrintWordTotalCountBolt.class); |
| private static final Random RANDOM = new Random(); |
| private OutputCollector collector; |
| |
| @Override |
| public void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector) { |
| this.collector = collector; |
| } |
| |
| @Override |
| public void execute(Tuple input) { |
| String wordName = input.getStringByField("wordName"); |
| String countStr = input.getStringByField("count"); |
| |
| // print lookup result with low probability |
| if (RANDOM.nextInt(1000) > 995) { |
| int count = 0; |
| if (countStr != null) { |
| count = Integer.parseInt(countStr); |
| } |
| LOG.info("Lookup result - word : " + wordName + " / count : " + count); |
| } |
| |
| collector.ack(input); |
| } |
| |
| @Override |
| public void declareOutputFields(OutputFieldsDeclarer declarer) { |
| } |
| } |
| |
| public static void main(String[] args) throws Exception { |
| 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(); |
| RedisLookupMapper lookupMapper = setupLookupMapper(); |
| RedisLookupBolt lookupBolt = new RedisLookupBolt(poolConfig, lookupMapper); |
| |
| PrintWordTotalCountBolt printBolt = new PrintWordTotalCountBolt(); |
| |
| //wordspout -> lookupbolt |
| TopologyBuilder builder = new TopologyBuilder(); |
| builder.setSpout(WORD_SPOUT, spout, 1); |
| builder.setBolt(LOOKUP_BOLT, lookupBolt, 1).shuffleGrouping(WORD_SPOUT); |
| builder.setBolt(PRINT_BOLT, printBolt, 1).shuffleGrouping(LOOKUP_BOLT); |
| |
| String topoName = "test"; |
| if (args.length == 3) { |
| topoName = args[2]; |
| } else if (args.length > 3) { |
| System.out.println("Usage: LookupWordCount <redis host> <redis port> (topology name)"); |
| return; |
| } |
| Config config = new Config(); |
| StormSubmitter.submitTopology(topoName, config, builder.createTopology()); |
| } |
| |
| private static RedisLookupMapper setupLookupMapper() { |
| return new WordCountRedisLookupMapper(); |
| } |
| |
| private static class WordCountRedisLookupMapper implements RedisLookupMapper { |
| private RedisDataTypeDescription description; |
| private final String hashKey = "wordCount"; |
| |
| WordCountRedisLookupMapper() { |
| description = new RedisDataTypeDescription( |
| RedisDataTypeDescription.RedisDataType.HASH, hashKey); |
| } |
| |
| @Override |
| public List<Values> toTuple(ITuple input, Object value) { |
| String member = getKeyFromTuple(input); |
| List<Values> values = Lists.newArrayList(); |
| values.add(new Values(member, value)); |
| return values; |
| } |
| |
| @Override |
| public void declareOutputFields(OutputFieldsDeclarer declarer) { |
| declarer.declare(new Fields("wordName", "count")); |
| } |
| |
| @Override |
| public RedisDataTypeDescription getDataTypeDescription() { |
| return description; |
| } |
| |
| @Override |
| public String getKeyFromTuple(ITuple tuple) { |
| return tuple.getStringByField("word"); |
| } |
| |
| @Override |
| public String getValueFromTuple(ITuple tuple) { |
| return null; |
| } |
| } |
| } |