blob: 3f30f4866c0248bb4b97a66432a4e20971390a3d [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.starter;
import java.util.Map;
import org.apache.storm.Config;
import org.apache.storm.StormSubmitter;
import org.apache.storm.starter.spout.RandomIntegerSpout;
import org.apache.storm.state.KeyValueState;
import org.apache.storm.task.OutputCollector;
import org.apache.storm.task.TopologyContext;
import org.apache.storm.topology.BasicOutputCollector;
import org.apache.storm.topology.OutputFieldsDeclarer;
import org.apache.storm.topology.TopologyBuilder;
import org.apache.storm.topology.base.BaseBasicBolt;
import org.apache.storm.topology.base.BaseStatefulBolt;
import org.apache.storm.tuple.Fields;
import org.apache.storm.tuple.Tuple;
import org.apache.storm.tuple.Values;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* An example topology that demonstrates the use of {@link org.apache.storm.topology.IStatefulBolt} to manage state. To run the example,
* <pre>
* $ storm jar examples/storm-starter/storm-starter-topologies-*.jar storm.starter.StatefulTopology statetopology
* </pre>
* <p/>
* The default state used is 'InMemoryKeyValueState' which does not persist the state across restarts. You could use 'RedisKeyValueState' to
* test state persistence by setting below property in conf/storm.yaml
* <pre>
* topology.state.provider: org.apache.storm.redis.state.RedisKeyValueStateProvider
* </pre>
* <p/>
* You should also start a local redis instance before running the 'storm jar' command. The default RedisKeyValueStateProvider parameters
* can be overridden in conf/storm.yaml, for e.g.
* <p/>
* <pre>
* topology.state.provider.config: '{"keyClass":"...", "valueClass":"...",
* "keySerializerClass":"...", "valueSerializerClass":"...",
* "jedisPoolConfig":{"host":"localhost", "port":6379,
* "timeout":2000, "database":0, "password":"xyz"}}'
*
* </pre>
* </p>
*/
public class StatefulTopology {
private static final Logger LOG = LoggerFactory.getLogger(StatefulTopology.class);
public static void main(String[] args) throws Exception {
TopologyBuilder builder = new TopologyBuilder();
builder.setSpout("spout", new RandomIntegerSpout());
builder.setBolt("partialsum", new StatefulSumBolt("partial"), 1).shuffleGrouping("spout");
builder.setBolt("printer", new PrinterBolt(), 2).shuffleGrouping("partialsum");
builder.setBolt("total", new StatefulSumBolt("total"), 1).shuffleGrouping("printer");
Config conf = new Config();
conf.setDebug(false);
String topoName = "test";
if (args != null && args.length > 0) {
topoName = args[0];
}
conf.setNumWorkers(1);
StormSubmitter.submitTopologyWithProgressBar(topoName, conf, builder.createTopology());
}
/**
* A bolt that uses {@link KeyValueState} to save its state.
*/
private static class StatefulSumBolt extends BaseStatefulBolt<KeyValueState<String, Long>> {
String name;
KeyValueState<String, Long> kvState;
long sum;
private OutputCollector collector;
StatefulSumBolt(String name) {
this.name = name;
}
@Override
public void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector) {
this.collector = collector;
}
@Override
public void execute(Tuple input) {
sum += ((Number) input.getValueByField("value")).longValue();
LOG.debug("{} sum = {}", name, sum);
kvState.put("sum", sum);
collector.emit(input, new Values(sum));
collector.ack(input);
}
@Override
public void initState(KeyValueState<String, Long> state) {
kvState = state;
sum = kvState.get("sum", 0L);
LOG.debug("Initstate, sum from saved state = {} ", sum);
}
@Override
public void declareOutputFields(OutputFieldsDeclarer declarer) {
declarer.declare(new Fields("value"));
}
}
public static class PrinterBolt extends BaseBasicBolt {
@Override
public void execute(Tuple tuple, BasicOutputCollector collector) {
System.out.println(tuple);
LOG.debug("Got tuple {}", tuple);
collector.emit(tuple.getValues());
}
@Override
public void declareOutputFields(OutputFieldsDeclarer ofd) {
ofd.declare(new Fields("value"));
}
}
}