/* * Copyright 2013-2014 Amazon.com, Inc. or its affiliates. All Rights Reserved. * * Licensed under the Amazon Software License (the "License"). * You may not use this file except in compliance with the License. * A copy of the License is located at * * http://aws.amazon.com/asl/ * * or in the "license" file accompanying this file. This file 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 KinesisStormClickstreamApp; import backtype.storm.Config; import backtype.storm.task.OutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.topology.base.BaseRichBolt; import backtype.storm.tuple.Fields; import backtype.storm.tuple.Tuple; import com.amazonaws.util.json.JSONException; import org.apache.log4j.Logger; import storm.starter.tools.NthLastModifiedTimeTracker; import storm.starter.tools.SlidingWindowCounter; import storm.starter.util.TupleHelpers; import com.amazonaws.util.json.JSONObject; import java.util.HashMap; import java.util.Map; import java.util.Map.Entry; import redis.clients.jedis.Jedis; public class RollingCountBolt extends BaseRichBolt { private static final long serialVersionUID = 5537727428628598519L; private static final Logger LOG = Logger.getLogger(RollingCountBolt.class); private static final int NUM_WINDOW_CHUNKS = 5; private static final int DEFAULT_SLIDING_WINDOW_IN_SECONDS = NUM_WINDOW_CHUNKS * 60; private static final int DEFAULT_EMIT_FREQUENCY_IN_SECONDS = DEFAULT_SLIDING_WINDOW_IN_SECONDS / NUM_WINDOW_CHUNKS; private static final String WINDOW_LENGTH_WARNING_TEMPLATE = "Actual window length is %d seconds when it should be %d seconds"; private final SlidingWindowCounter<Object> counter; private final String elasticCacheRedisEndpoint; private final int windowLengthInSeconds; private final int emitFrequencyInSeconds; private OutputCollector collector; private NthLastModifiedTimeTracker lastModifiedTracker; private Jedis jedis; //public RollingCountBolt() { // this(DEFAULT_SLIDING_WINDOW_IN_SECONDS, DEFAULT_EMIT_FREQUENCY_IN_SECONDS); //} public RollingCountBolt(int windowLengthInSeconds, int emitFrequencyInSeconds, String elasticCacheRedisEndpoint) { this.windowLengthInSeconds = windowLengthInSeconds; this.emitFrequencyInSeconds = emitFrequencyInSeconds; counter = new SlidingWindowCounter<Object>(deriveNumWindowChunksFrom(this.windowLengthInSeconds, this.emitFrequencyInSeconds)); this.elasticCacheRedisEndpoint = elasticCacheRedisEndpoint; } private int deriveNumWindowChunksFrom(int windowLengthInSeconds, int windowUpdateFrequencyInSeconds) { return windowLengthInSeconds / windowUpdateFrequencyInSeconds; } @SuppressWarnings("rawtypes") @Override public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { this.collector = collector; lastModifiedTracker = new NthLastModifiedTimeTracker(deriveNumWindowChunksFrom(this.windowLengthInSeconds, this.emitFrequencyInSeconds)); jedis = new Jedis(elasticCacheRedisEndpoint); } @Override public void execute(Tuple tuple) { if (TupleHelpers.isTickTuple(tuple)) { LOG.debug("Received tick tuple, triggering emit of current window counts"); emitCurrentWindowCounts(); } else { countObjAndAck(tuple); } } private void emitCurrentWindowCounts() { Map<Object, Long> counts = counter.getCountsThenAdvanceWindow(); int actualWindowLengthInSeconds = lastModifiedTracker.secondsSinceOldestModification(); lastModifiedTracker.markAsModified(); if (actualWindowLengthInSeconds != windowLengthInSeconds) { LOG.warn(String.format(WINDOW_LENGTH_WARNING_TEMPLATE, actualWindowLengthInSeconds, windowLengthInSeconds)); } emit(counts, actualWindowLengthInSeconds); } private void emit(Map<Object, Long> counts, int actualWindowLengthInSeconds) { for (Entry<Object, Long> entry : counts.entrySet()) { String referrer = entry.getKey().toString(); Long count = entry.getValue(); long currentEPOCH = java.lang.System.currentTimeMillis(); if(jedis.llen("l:"+referrer) > 100 ) { String lastEPOCH = jedis.lpop("l:"+referrer); jedis.hdel("h:"+referrer,lastEPOCH); } jedis.hset("h:" + referrer, String.valueOf(currentEPOCH), count.toString()); jedis.rpush("l:"+referrer,String.valueOf(currentEPOCH)); JSONObject msg = new JSONObject(); try { msg.put("name", referrer); msg.put("time", currentEPOCH); msg.put("count", count); } catch (JSONException e) { LOG.error("Exception when creating JSON Message", e); } jedis.publish("pubsubCounters",msg.toString()); } } private void countObjAndAck(Tuple tuple) { Object obj = tuple.getValue(0); counter.incrementCount(obj); collector.ack(tuple); } @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { declarer.declare(new Fields("obj", "count", "actualWindowLengthInSeconds")); } @Override public Map<String, Object> getComponentConfiguration() { Map<String, Object> conf = new HashMap<String, Object>(); conf.put(Config.TOPOLOGY_TICK_TUPLE_FREQ_SECS, emitFrequencyInSeconds); return conf; } @Override public void cleanup() { jedis.quit(); } }