blob: 7f3afb1c808b28ff3223d73007646f6f058c84bb [file] [log] [blame]
/**
* 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.hbase.topology;
import java.util.HashMap;
import java.util.Map;
import org.apache.storm.Config;
import org.apache.storm.StormSubmitter;
import org.apache.storm.hbase.bolt.HBaseLookupBolt;
import org.apache.storm.hbase.bolt.mapper.HBaseProjectionCriteria;
import org.apache.storm.hbase.bolt.mapper.SimpleHBaseMapper;
import org.apache.storm.topology.TopologyBuilder;
import org.apache.storm.tuple.Fields;
public class LookupWordCount {
private static final String WORD_SPOUT = "WORD_SPOUT";
private static final String LOOKUP_BOLT = "LOOKUP_BOLT";
private static final String TOTAL_COUNT_BOLT = "TOTAL_COUNT_BOLT";
public static void main(String[] args) throws Exception {
Config config = new Config();
Map<String, Object> hbConf = new HashMap<String, Object>();
if (args.length > 0) {
hbConf.put("hbase.rootdir", args[0]);
}
config.put("hbase.conf", hbConf);
WordSpout spout = new WordSpout();
SimpleHBaseMapper mapper = new SimpleHBaseMapper().withRowKeyField("word");
HBaseProjectionCriteria projectionCriteria = new HBaseProjectionCriteria();
projectionCriteria.addColumn(new HBaseProjectionCriteria.ColumnMetaData("cf", "count"));
WordCountValueMapper rowToTupleMapper = new WordCountValueMapper();
HBaseLookupBolt lookupBolt = new HBaseLookupBolt("WordCount", mapper, rowToTupleMapper)
.withConfigKey("hbase.conf")
.withProjectionCriteria(projectionCriteria);
//wordspout -> lookupbolt -> totalCountBolt
TopologyBuilder builder = new TopologyBuilder();
builder.setSpout(WORD_SPOUT, spout, 1);
builder.setBolt(LOOKUP_BOLT, lookupBolt, 1).shuffleGrouping(WORD_SPOUT);
TotalWordCounter totalBolt = new TotalWordCounter();
builder.setBolt(TOTAL_COUNT_BOLT, totalBolt, 1).fieldsGrouping(LOOKUP_BOLT, new Fields("columnName"));
String topoName = "test";
if (args.length == 1) {
topoName = args[1];
} else if (args.length > 1) {
System.out.println("Usage: LookupWordCount <hbase.rootdir>");
return;
}
StormSubmitter.submitTopology(topoName, config, builder.createTopology());
}
}