You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by pt...@apache.org on 2016/01/12 21:01:21 UTC

[01/10] storm git commit: [STORM-1175] State store for windowing operations

Repository: storm
Updated Branches:
  refs/heads/master 3db968092 -> b24d7d93e


http://git-wip-us.apache.org/repos/asf/storm/blob/4623d8f8/storm-core/src/jvm/backtype/storm/topology/IStatefulComponent.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/topology/IStatefulComponent.java b/storm-core/src/jvm/backtype/storm/topology/IStatefulComponent.java
new file mode 100644
index 0000000..887c91a
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/topology/IStatefulComponent.java
@@ -0,0 +1,50 @@
+/**
+ * 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 backtype.storm.topology;
+
+import backtype.storm.state.State;
+
+/**
+ * <p>
+ * Common methods for stateful components in the topology.
+ * </p>
+ * A stateful component is one that has state (e.g. the result of some computation in a bolt)
+ * and wants the framework to manage its state.
+ */
+public interface IStatefulComponent<T extends State> extends IComponent {
+    /**
+     * This method is invoked by the framework with the previously
+     * saved state of the component. This is invoked after prepare but before
+     * the component starts processing tuples.
+     *
+     * @param state the previously saved state of the component.
+     */
+    void initState(T state);
+
+    /**
+     * This is a hook for the component to perform some actions just before the
+     * framework commits its state.
+     */
+    void preCommit();
+
+    /**
+     * This is a hook for the component to perform some actions just after the
+     * framework commits its state.
+     */
+    void postCommit();
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4623d8f8/storm-core/src/jvm/backtype/storm/topology/StatefulBoltExecutor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/topology/StatefulBoltExecutor.java b/storm-core/src/jvm/backtype/storm/topology/StatefulBoltExecutor.java
new file mode 100644
index 0000000..bdb45e3
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/topology/StatefulBoltExecutor.java
@@ -0,0 +1,98 @@
+/**
+ * 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 backtype.storm.topology;
+
+import backtype.storm.spout.CheckpointSpout;
+import backtype.storm.state.State;
+import backtype.storm.state.StateFactory;
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.tuple.Values;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import static backtype.storm.spout.CheckpointSpout.CHECKPOINT_ACTION_COMMIT;
+import static backtype.storm.spout.CheckpointSpout.CHECKPOINT_ACTION_PREPARE;
+import static backtype.storm.spout.CheckpointSpout.CHECKPOINT_ACTION_ROLLBACK;
+import static backtype.storm.spout.CheckpointSpout.CHECKPOINT_ACTION_INITSTATE;
+
+/**
+ * Wraps a {@link IStatefulBolt} and manages the state of the bolt.
+ */
+public class StatefulBoltExecutor<T extends State> extends CheckpointTupleForwarder {
+    private static final Logger LOG = LoggerFactory.getLogger(StatefulBoltExecutor.class);
+    private final IStatefulBolt<T> bolt;
+    private State state;
+    private boolean boltInitialized = false;
+    private List<Tuple> pendingTuples = new ArrayList<>();
+
+    public StatefulBoltExecutor(IStatefulBolt<T> bolt) {
+        super(bolt);
+        this.bolt = bolt;
+    }
+
+    @Override
+    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+        // get the last successfully committed state from state store
+        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId();
+        prepare(stormConf, context, collector, StateFactory.getState(namespace, stormConf, context));
+    }
+
+    // package access for unit tests
+    void prepare(Map stormConf, TopologyContext context, OutputCollector collector, State state) {
+        super.prepare(stormConf, context, collector);
+        this.state = state;
+    }
+
+    @Override
+    protected void handleCheckpoint(Tuple input, String action, long txid) {
+        LOG.debug("handleCheckPoint with tuple {}, action {}, txid {}", input, action, txid);
+        if (action.equals(CHECKPOINT_ACTION_PREPARE)) {
+            state.prepareCommit(txid);
+        } else if (action.equals(CHECKPOINT_ACTION_COMMIT)) {
+            bolt.preCommit();
+            state.commit(txid);
+            bolt.postCommit();
+        } else if (action.equals(CHECKPOINT_ACTION_ROLLBACK)) {
+            state.rollback();
+        } else if (action.equals(CHECKPOINT_ACTION_INITSTATE)) {
+            bolt.initState((T) state);
+            boltInitialized = true;
+            LOG.debug("{} pending tuples to process", pendingTuples.size());
+            for (Tuple tuple : pendingTuples) {
+                bolt.execute(tuple);
+            }
+        }
+        collector.emit(CheckpointSpout.CHECKPOINT_STREAM_ID, input, new Values(txid, action));
+    }
+
+    @Override
+    protected void handleTuple(Tuple input) {
+        if (boltInitialized) {
+            bolt.execute(input);
+        } else {
+            LOG.debug("Bolt state not initialized, adding tuple {} to pending tuples", input);
+            pendingTuples.add(input);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4623d8f8/storm-core/src/jvm/backtype/storm/topology/base/BaseStatefulBolt.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/topology/base/BaseStatefulBolt.java b/storm-core/src/jvm/backtype/storm/topology/base/BaseStatefulBolt.java
new file mode 100644
index 0000000..93bcd57
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/topology/base/BaseStatefulBolt.java
@@ -0,0 +1,59 @@
+/**
+ * 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 backtype.storm.topology.base;
+
+import backtype.storm.state.State;
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.IStatefulBolt;
+import backtype.storm.topology.OutputFieldsDeclarer;
+
+import java.util.Map;
+
+public abstract class BaseStatefulBolt<T extends State> implements IStatefulBolt<T> {
+
+    @Override
+    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+        // NOOP
+    }
+
+    @Override
+    public void cleanup() {
+        // NOOP
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        // NOOP
+    }
+
+    @Override
+    public Map<String, Object> getComponentConfiguration() {
+        return null;
+    }
+
+    @Override
+    public void preCommit() {
+        // NOOP
+    }
+
+    @Override
+    public void postCommit() {
+        // NOOP
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4623d8f8/storm-core/src/jvm/org/apache/storm/Config.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/Config.java b/storm-core/src/jvm/org/apache/storm/Config.java
index c4f7552..1bd4a54 100644
--- a/storm-core/src/jvm/org/apache/storm/Config.java
+++ b/storm-core/src/jvm/org/apache/storm/Config.java
@@ -1539,6 +1539,29 @@ public class Config extends HashMap<String, Object> {
     public static final String TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT = "topology.component.cpu.pcore.percent";
 
     /**
+     * The class name of the {@link backtype.storm.state.StateProvider} implementation. If not specified
+     * defaults to {@link backtype.storm.state.InMemoryKeyValueStateProvider}. This can be overridden
+     * at the component level.
+     */
+    @isString
+    public static final String TOPOLOGY_STATE_PROVIDER = "topology.state.provider";
+
+    /**
+     * The configuration specific to the {@link backtype.storm.state.StateProvider} implementation.
+     * This can be overridden at the component level.
+     */
+    @isString
+    public static final String TOPOLOGY_STATE_PROVIDER_CONFIG = "topology.state.provider.config";
+
+    /**
+     * Topology configuration to specify the checkpoint interval (in millis) at which the
+     * topology state is saved when {@link backtype.storm.topology.IStatefulBolt} bolts are involved.
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String TOPOLOGY_STATE_CHECKPOINT_INTERVAL = "topology.state.checkpoint.interval.ms";
+
+    /**
      * A per topology config that specifies the maximum amount of memory a worker can use for that specific topology
      */
     @isPositiveNumber

http://git-wip-us.apache.org/repos/asf/storm/blob/4623d8f8/storm-core/src/jvm/org/apache/storm/topology/TopologyBuilder.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/topology/TopologyBuilder.java b/storm-core/src/jvm/org/apache/storm/topology/TopologyBuilder.java
index 0f7522f..2c002cc 100644
--- a/storm-core/src/jvm/org/apache/storm/topology/TopologyBuilder.java
+++ b/storm-core/src/jvm/org/apache/storm/topology/TopologyBuilder.java
@@ -22,7 +22,12 @@ import org.apache.storm.generated.*;
 import org.apache.storm.grouping.CustomStreamGrouping;
 import org.apache.storm.grouping.PartialKeyGrouping;
 import org.apache.storm.hooks.IWorkerHook;
+import org.apache.storm.spout.CheckpointSpout;
+import org.apache.storm.state.State;
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
 import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
 import org.apache.storm.utils.Utils;
 import org.json.simple.JSONValue;
 
@@ -33,6 +38,9 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import org.apache.storm.windowing.TupleWindow;
+import org.json.simple.JSONValue;
+import static org.apache.storm.spout.CheckpointSpout.CHECKPOINT_COMPONENT_ID;
+import static org.apache.storm.spout.CheckpointSpout.CHECKPOINT_STREAM_ID;
 
 /**
  * TopologyBuilder exposes the Java API for specifying a topology for Storm
@@ -90,6 +98,7 @@ public class TopologyBuilder {
     private Map<String, IRichBolt> _bolts = new HashMap<>();
     private Map<String, IRichSpout> _spouts = new HashMap<>();
     private Map<String, ComponentCommon> _commons = new HashMap<>();
+    private boolean hasStatefulBolt = false;
 
 //    private Map<String, Map<GlobalStreamId, Grouping>> _inputs = new HashMap<String, Map<GlobalStreamId, Grouping>>();
 
@@ -100,11 +109,13 @@ public class TopologyBuilder {
     public StormTopology createTopology() {
         Map<String, Bolt> boltSpecs = new HashMap<>();
         Map<String, SpoutSpec> spoutSpecs = new HashMap<>();
-
+        maybeAddCheckpointSpout();
         for(String boltId: _bolts.keySet()) {
             IRichBolt bolt = _bolts.get(boltId);
+            bolt = maybeAddCheckpointTupleForwarder(bolt);
             ComponentCommon common = getComponentCommon(boltId, bolt);
             try{
+                maybeAddCheckpointInputs(common);
                 boltSpecs.put(boltId, new Bolt(ComponentObject.serialized_java(Utils.javaSerialize(bolt)), common));
             }catch(RuntimeException wrapperCause){
                 if (wrapperCause.getCause() != null && NotSerializableException.class.equals(wrapperCause.getCause().getClass())){
@@ -216,6 +227,23 @@ public class TopologyBuilder {
     }
 
     /**
+     * Define a new bolt in this topology. This defines a stateful bolt, that requires its
+     * state (of computation) to be saved. When this bolt is initialized, the {@link IStatefulBolt#initState(State)} method
+     * is invoked after {@link IStatefulBolt#prepare(Map, TopologyContext, OutputCollector)} but before {@link IStatefulBolt#execute(Tuple)}
+     * with its previously saved state.
+     *
+     * @param id the id of this component. This id is referenced by other components that want to consume this bolt's outputs.
+     * @param bolt the stateful bolt
+     * @param parallelism_hint the number of tasks that should be assigned to execute this bolt. Each task will run on a thread in a process somwehere around the cluster.
+     * @return use the returned object to declare the inputs to this component
+     * @throws IllegalArgumentException if {@code parallelism_hint} is not positive
+     */
+    public <T extends State> BoltDeclarer setBolt(String id, IStatefulBolt<T> bolt, Number parallelism_hint) throws IllegalArgumentException {
+        hasStatefulBolt = true;
+        return setBolt(id, new StatefulBoltExecutor<T>(bolt), parallelism_hint);
+    }
+
+    /**
      * Define a new spout in this topology.
      *
      * @param id the id of this component. This id is referenced by other components that want to consume this spout's outputs.
@@ -277,15 +305,60 @@ public class TopologyBuilder {
         }
     }
 
+    /**
+     * If the topology has at least one stateful bolt
+     * add a {@link CheckpointSpout} component to the topology.
+     */
+    private void maybeAddCheckpointSpout() {
+        if (hasStatefulBolt) {
+            setSpout(CHECKPOINT_COMPONENT_ID, new CheckpointSpout(), 1);
+        }
+    }
+
+    private void maybeAddCheckpointInputs(ComponentCommon common) {
+        if (hasStatefulBolt) {
+            addCheckPointInputs(common);
+        }
+    }
+
+    /**
+     * If the topology has at least one stateful bolt all the non-stateful bolts
+     * are wrapped in {@link CheckpointTupleForwarder} so that the checkpoint
+     * tuples can flow through the topology.
+     */
+    private IRichBolt maybeAddCheckpointTupleForwarder(IRichBolt bolt) {
+        if (hasStatefulBolt && !(bolt instanceof StatefulBoltExecutor)) {
+            bolt = new CheckpointTupleForwarder(bolt);
+        }
+        return bolt;
+    }
+
+    /**
+     * For bolts that has incoming streams from spouts (the root bolts),
+     * add checkpoint stream from checkpoint spout to its input. For other bolts,
+     * add checkpoint stream from the previous bolt to its input.
+     */
+    private void addCheckPointInputs(ComponentCommon component) {
+        for (GlobalStreamId inputStream : component.get_inputs().keySet()) {
+            String sourceId = inputStream.get_componentId();
+            if (_spouts.containsKey(sourceId)) {
+                GlobalStreamId checkPointStream = new GlobalStreamId(CHECKPOINT_COMPONENT_ID, CHECKPOINT_STREAM_ID);
+                component.put_to_inputs(checkPointStream, Grouping.all(new NullStruct()));
+            } else {
+                GlobalStreamId checkPointStream = new GlobalStreamId(sourceId, CHECKPOINT_STREAM_ID);
+                component.put_to_inputs(checkPointStream, Grouping.all(new NullStruct()));
+            }
+        }
+    }
+
     private ComponentCommon getComponentCommon(String id, IComponent component) {
         ComponentCommon ret = new ComponentCommon(_commons.get(id));
-        
         OutputFieldsGetter getter = new OutputFieldsGetter();
         component.declareOutputFields(getter);
         ret.set_streams(getter.getFieldsDeclaration());
-        return ret;        
+        return ret;
     }
-    
+
     private void initCommon(String id, IComponent component, Number parallelism) throws IllegalArgumentException {
         ComponentCommon common = new ComponentCommon();
         common.set_inputs(new HashMap<GlobalStreamId, Grouping>());

http://git-wip-us.apache.org/repos/asf/storm/blob/4623d8f8/storm-core/test/jvm/backtype/storm/spout/CheckpointSpoutTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/spout/CheckpointSpoutTest.java b/storm-core/test/jvm/backtype/storm/spout/CheckpointSpoutTest.java
new file mode 100644
index 0000000..15bba91
--- /dev/null
+++ b/storm-core/test/jvm/backtype/storm/spout/CheckpointSpoutTest.java
@@ -0,0 +1,245 @@
+/**
+ * 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 backtype.storm.spout;
+
+import backtype.storm.Config;
+import backtype.storm.state.KeyValueState;
+import backtype.storm.state.StateFactory;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.tuple.Values;
+import backtype.storm.utils.Utils;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.ArgumentMatcher;
+import org.mockito.Mockito;
+import org.mockito.internal.matchers.Equals;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.*;
+
+/**
+ * Unit test for {@link CheckpointSpout}
+ */
+public class CheckpointSpoutTest {
+    CheckpointSpout spout = new CheckpointSpout();
+    TopologyContext mockTopologyContext;
+    SpoutOutputCollector mockOutputCollector;
+
+    @Before
+    public void setUp() throws Exception {
+        mockTopologyContext = Mockito.mock(TopologyContext.class);
+        Mockito.when(mockTopologyContext.getThisComponentId()).thenReturn("test");
+        Mockito.when(mockTopologyContext.getThisTaskId()).thenReturn(1);
+        mockOutputCollector = Mockito.mock(SpoutOutputCollector.class);
+    }
+
+    @Test
+    public void testInitState() throws Exception {
+        spout.open(new HashMap(), mockTopologyContext, mockOutputCollector);
+
+        spout.nextTuple();
+        Values expectedTuple = new Values(-1L, CheckpointSpout.CHECKPOINT_ACTION_INITSTATE);
+        ArgumentCaptor<String> stream = ArgumentCaptor.forClass(String.class);
+        ArgumentCaptor<Values> values = ArgumentCaptor.forClass(Values.class);
+        ArgumentCaptor<Object> msgId = ArgumentCaptor.forClass(Object.class);
+        Mockito.verify(mockOutputCollector).emit(stream.capture(),
+                                                 values.capture(),
+                                                 msgId.capture());
+
+        assertEquals(CheckpointSpout.CHECKPOINT_STREAM_ID, stream.getValue());
+        assertEquals(expectedTuple, values.getValue());
+        assertEquals(-1L, msgId.getValue());
+
+        spout.ack(-1L);
+
+        Mockito.verify(mockOutputCollector).emit(stream.capture(),
+                                                 values.capture(),
+                                                 msgId.capture());
+
+        expectedTuple = new Values(-1L, CheckpointSpout.CHECKPOINT_ACTION_INITSTATE);
+        assertEquals(CheckpointSpout.CHECKPOINT_STREAM_ID, stream.getValue());
+        assertEquals(expectedTuple, values.getValue());
+        assertEquals(-1L, msgId.getValue());
+
+    }
+
+    @Test
+    public void testPrepare() throws Exception {
+        spout.open(new HashMap(), mockTopologyContext, mockOutputCollector);
+        ArgumentCaptor<String> stream = ArgumentCaptor.forClass(String.class);
+        ArgumentCaptor<Values> values = ArgumentCaptor.forClass(Values.class);
+        ArgumentCaptor<Object> msgId = ArgumentCaptor.forClass(Object.class);
+
+        spout.nextTuple();
+        spout.ack(-1L);
+        spout.nextTuple();
+        Mockito.verify(mockOutputCollector, Mockito.times(2)).emit(stream.capture(),
+                                                 values.capture(),
+                                                 msgId.capture());
+
+        Values expectedTuple = new Values(0L, CheckpointSpout.CHECKPOINT_ACTION_PREPARE);
+        assertEquals(CheckpointSpout.CHECKPOINT_STREAM_ID, stream.getValue());
+        assertEquals(expectedTuple, values.getValue());
+        assertEquals(0L, msgId.getValue());
+
+    }
+
+    @Test
+    public void testPrepareWithFail() throws Exception {
+        Map<String, Object> stormConf = new HashMap<>();
+        stormConf.put(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL, 0);
+        spout.open(stormConf, mockTopologyContext, mockOutputCollector);
+        ArgumentCaptor<String> stream = ArgumentCaptor.forClass(String.class);
+        ArgumentCaptor<Values> values = ArgumentCaptor.forClass(Values.class);
+        ArgumentCaptor<Object> msgId = ArgumentCaptor.forClass(Object.class);
+
+        spout.nextTuple();
+        spout.ack(-1L);
+        spout.nextTuple();
+        spout.fail(0L);
+        Utils.sleep(10);
+        spout.nextTuple();
+        spout.fail(0L);
+        Utils.sleep(10);
+        spout.nextTuple();
+        Mockito.verify(mockOutputCollector, Mockito.times(4)).emit(stream.capture(),
+                                                                   values.capture(),
+                                                                   msgId.capture());
+
+        Values expectedTuple = new Values(0L, CheckpointSpout.CHECKPOINT_ACTION_PREPARE);
+        assertEquals(CheckpointSpout.CHECKPOINT_STREAM_ID, stream.getValue());
+        assertEquals(expectedTuple, values.getValue());
+        assertEquals(0L, msgId.getValue());
+
+    }
+
+    @Test
+    public void testCommit() throws Exception {
+        Map<String, Object> stormConf = new HashMap();
+        stormConf.put(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL, 0);
+        spout.open(stormConf, mockTopologyContext, mockOutputCollector);
+        ArgumentCaptor<String> stream = ArgumentCaptor.forClass(String.class);
+        ArgumentCaptor<Values> values = ArgumentCaptor.forClass(Values.class);
+        ArgumentCaptor<Object> msgId = ArgumentCaptor.forClass(Object.class);
+
+        spout.nextTuple();
+        spout.ack(-1L);
+        spout.nextTuple();
+        spout.ack(0L);
+        Utils.sleep(10);
+        spout.nextTuple();
+        spout.fail(0L);
+        Utils.sleep(10);
+        spout.nextTuple();
+        Mockito.verify(mockOutputCollector, Mockito.times(4)).emit(stream.capture(),
+                                                                   values.capture(),
+                                                                   msgId.capture());
+
+        Values expectedTuple = new Values(0L, CheckpointSpout.CHECKPOINT_ACTION_COMMIT);
+        assertEquals(CheckpointSpout.CHECKPOINT_STREAM_ID, stream.getValue());
+        assertEquals(expectedTuple, values.getValue());
+        assertEquals(0L, msgId.getValue());
+
+    }
+
+    @Test
+    public void testRecoveryRollback() throws Exception {
+        Map<String, Object> stormConf = new HashMap();
+
+        KeyValueState<String, CheckPointState> state =
+                (KeyValueState<String, CheckPointState>) StateFactory.getState("test-1", stormConf, mockTopologyContext);
+
+        CheckPointState checkPointState = new CheckPointState(100, CheckPointState.State.PREPARING);
+        state.put("__state", checkPointState);
+        spout.open(mockTopologyContext, mockOutputCollector, 0, state);
+        ArgumentCaptor<String> stream = ArgumentCaptor.forClass(String.class);
+        ArgumentCaptor<Values> values = ArgumentCaptor.forClass(Values.class);
+        ArgumentCaptor<Object> msgId = ArgumentCaptor.forClass(Object.class);
+
+        spout.nextTuple();
+        Mockito.verify(mockOutputCollector, Mockito.times(1)).emit(stream.capture(),
+                                                                   values.capture(),
+                                                                   msgId.capture());
+
+        Values expectedTuple = new Values(100L, CheckpointSpout.CHECKPOINT_ACTION_ROLLBACK);
+        assertEquals(CheckpointSpout.CHECKPOINT_STREAM_ID, stream.getValue());
+        assertEquals(expectedTuple, values.getValue());
+        assertEquals(100L, msgId.getValue());
+
+    }
+
+    @Test
+    public void testRecoveryRollbackAck() throws Exception {
+        Map<String, Object> stormConf = new HashMap();
+
+        KeyValueState<String, CheckPointState> state =
+                (KeyValueState<String, CheckPointState>) StateFactory.getState("test-1", stormConf, mockTopologyContext);
+
+        CheckPointState checkPointState = new CheckPointState(100, CheckPointState.State.PREPARING);
+        state.put("__state", checkPointState);
+        spout.open(mockTopologyContext, mockOutputCollector, 0, state);
+        ArgumentCaptor<String> stream = ArgumentCaptor.forClass(String.class);
+        ArgumentCaptor<Values> values = ArgumentCaptor.forClass(Values.class);
+        ArgumentCaptor<Object> msgId = ArgumentCaptor.forClass(Object.class);
+
+        spout.nextTuple();
+        spout.ack(100L);
+        spout.nextTuple();
+        spout.ack(99L);
+        spout.nextTuple();
+        Mockito.verify(mockOutputCollector, Mockito.times(3)).emit(stream.capture(),
+                                                                   values.capture(),
+                                                                   msgId.capture());
+
+        Values expectedTuple = new Values(100L, CheckpointSpout.CHECKPOINT_ACTION_PREPARE);
+        assertEquals(CheckpointSpout.CHECKPOINT_STREAM_ID, stream.getValue());
+        assertEquals(expectedTuple, values.getValue());
+        assertEquals(100L, msgId.getValue());
+
+    }
+
+    @Test
+    public void testRecoveryCommit() throws Exception {
+        Map<String, Object> stormConf = new HashMap();
+
+        KeyValueState<String, CheckPointState> state =
+                (KeyValueState<String, CheckPointState>) StateFactory.getState("test-1", stormConf, mockTopologyContext);
+
+        CheckPointState checkPointState = new CheckPointState(100, CheckPointState.State.COMMITTING);
+        state.put("__state", checkPointState);
+        spout.open(mockTopologyContext, mockOutputCollector, 0, state);
+        ArgumentCaptor<String> stream = ArgumentCaptor.forClass(String.class);
+        ArgumentCaptor<Values> values = ArgumentCaptor.forClass(Values.class);
+        ArgumentCaptor<Object> msgId = ArgumentCaptor.forClass(Object.class);
+
+        spout.nextTuple();
+        Mockito.verify(mockOutputCollector, Mockito.times(1)).emit(stream.capture(),
+                                                                   values.capture(),
+                                                                   msgId.capture());
+
+        Values expectedTuple = new Values(100L, CheckpointSpout.CHECKPOINT_ACTION_COMMIT);
+        assertEquals(CheckpointSpout.CHECKPOINT_STREAM_ID, stream.getValue());
+        assertEquals(expectedTuple, values.getValue());
+        assertEquals(100L, msgId.getValue());
+
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/4623d8f8/storm-core/test/jvm/backtype/storm/state/InMemoryKeyValueStateTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/state/InMemoryKeyValueStateTest.java b/storm-core/test/jvm/backtype/storm/state/InMemoryKeyValueStateTest.java
new file mode 100644
index 0000000..31dbc79
--- /dev/null
+++ b/storm-core/test/jvm/backtype/storm/state/InMemoryKeyValueStateTest.java
@@ -0,0 +1,73 @@
+/**
+ * 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 backtype.storm.state;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+/**
+ * Unit tests for {@link InMemoryKeyValueState}
+ */
+public class InMemoryKeyValueStateTest {
+
+    KeyValueState<String, String> keyValueState;
+
+    @Before
+    public void setUp() {
+        keyValueState = new InMemoryKeyValueState<>();
+    }
+
+    @Test
+    public void testPutAndGet() throws Exception {
+        keyValueState.put("a", "1");
+        keyValueState.put("b", "2");
+        assertEquals("1", keyValueState.get("a"));
+        assertEquals("2", keyValueState.get("b"));
+        assertEquals(null, keyValueState.get("c"));
+    }
+
+    @Test
+    public void testPrepareCommitRollback() throws Exception {
+        keyValueState.put("a", "1");
+        keyValueState.put("b", "2");
+        keyValueState.prepareCommit(1);
+        keyValueState.put("c", "3");
+        assertEquals("1", keyValueState.get("a"));
+        assertEquals("2", keyValueState.get("b"));
+        assertEquals("3", keyValueState.get("c"));
+        keyValueState.rollback();
+        assertEquals(null, keyValueState.get("a"));
+        assertEquals(null, keyValueState.get("b"));
+        assertEquals(null, keyValueState.get("c"));
+        keyValueState.put("a", "1");
+        keyValueState.put("b", "2");
+        keyValueState.prepareCommit(1);
+        keyValueState.commit();
+        keyValueState.put("c", "3");
+        assertEquals("1", keyValueState.get("a"));
+        assertEquals("2", keyValueState.get("b"));
+        assertEquals("3", keyValueState.get("c"));
+        keyValueState.rollback();
+        assertEquals("1", keyValueState.get("a"));
+        assertEquals("2", keyValueState.get("b"));
+        assertEquals(null, keyValueState.get("c"));
+
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/4623d8f8/storm-core/test/jvm/backtype/storm/topology/StatefulBoltExecutorTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/topology/StatefulBoltExecutorTest.java b/storm-core/test/jvm/backtype/storm/topology/StatefulBoltExecutorTest.java
new file mode 100644
index 0000000..72d2ac7
--- /dev/null
+++ b/storm-core/test/jvm/backtype/storm/topology/StatefulBoltExecutorTest.java
@@ -0,0 +1,122 @@
+/**
+ * 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 backtype.storm.topology;
+
+import backtype.storm.generated.GlobalStreamId;
+import backtype.storm.generated.Grouping;
+import backtype.storm.spout.CheckpointSpout;
+import backtype.storm.state.KeyValueState;
+import backtype.storm.state.State;
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.tuple.Tuple;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import static backtype.storm.spout.CheckpointSpout.*;
+
+/**
+ * Unit tests for {@link StatefulBoltExecutor}
+ */
+public class StatefulBoltExecutorTest {
+    StatefulBoltExecutor<KeyValueState<String, String>> executor;
+    IStatefulBolt<KeyValueState<String, String>> mockBolt;
+    TopologyContext mockTopologyContext;
+    Tuple mockTuple;
+    Tuple mockCheckpointTuple;
+    Map<String, Object> mockStormConf = new HashMap<>();
+    OutputCollector mockOutputCollector;
+    State mockState;
+    Map<GlobalStreamId, Grouping> mockGlobalStream;
+    Set<GlobalStreamId> mockStreamIds;
+    @Before
+    public void setUp() throws Exception {
+        mockBolt = Mockito.mock(IStatefulBolt.class);
+        executor = new StatefulBoltExecutor<>(mockBolt);
+        GlobalStreamId mockGlobalStreamId = Mockito.mock(GlobalStreamId.class);
+        Mockito.when(mockGlobalStreamId.get_streamId()).thenReturn(CheckpointSpout.CHECKPOINT_STREAM_ID);
+        mockStreamIds = new HashSet<>();
+        mockStreamIds.add(mockGlobalStreamId);
+        mockTopologyContext = Mockito.mock(TopologyContext.class);
+        mockOutputCollector = Mockito.mock(OutputCollector.class);
+        mockGlobalStream = Mockito.mock(Map.class);
+        mockState = Mockito.mock(State.class);
+        Mockito.when(mockTopologyContext.getThisComponentId()).thenReturn("test");
+        Mockito.when(mockTopologyContext.getThisTaskId()).thenReturn(1);
+        Mockito.when(mockTopologyContext.getThisSources()).thenReturn(mockGlobalStream);
+        Mockito.when(mockTopologyContext.getComponentTasks(Mockito.anyString())).thenReturn(Collections.singletonList(1));
+        Mockito.when(mockGlobalStream.keySet()).thenReturn(mockStreamIds);
+        mockTuple = Mockito.mock(Tuple.class);
+        mockCheckpointTuple = Mockito.mock(Tuple.class);
+        executor.prepare(mockStormConf, mockTopologyContext, mockOutputCollector, mockState);
+    }
+
+    @Test
+    public void testHandleTupleBeforeInit() throws Exception {
+        Mockito.when(mockTuple.getSourceStreamId()).thenReturn("default");
+        executor.execute(mockTuple);
+        Mockito.verify(mockBolt, Mockito.times(0)).execute(Mockito.any(Tuple.class));
+    }
+
+
+    @Test
+    public void testHandleTuple() throws Exception {
+        Mockito.when(mockTuple.getSourceStreamId()).thenReturn("default");
+        executor.execute(mockTuple);
+        Mockito.when(mockCheckpointTuple.getSourceStreamId()).thenReturn(CheckpointSpout.CHECKPOINT_STREAM_ID);
+        Mockito.when(mockCheckpointTuple.getStringByField(CHECKPOINT_FIELD_ACTION)).thenReturn(CHECKPOINT_ACTION_INITSTATE);
+        Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(0));
+        Mockito.doNothing().when(mockOutputCollector).ack(mockCheckpointTuple);
+        executor.execute(mockCheckpointTuple);
+        Mockito.verify(mockBolt, Mockito.times(1)).execute(mockTuple);
+        Mockito.verify(mockBolt, Mockito.times(1)).initState(Mockito.any(KeyValueState.class));
+    }
+
+    @Test
+    public void testRollback() throws Exception {
+        Mockito.when(mockTuple.getSourceStreamId()).thenReturn("default");
+        executor.execute(mockTuple);
+        Mockito.when(mockCheckpointTuple.getSourceStreamId()).thenReturn(CheckpointSpout.CHECKPOINT_STREAM_ID);
+        Mockito.when(mockCheckpointTuple.getStringByField(CHECKPOINT_FIELD_ACTION)).thenReturn(CHECKPOINT_ACTION_ROLLBACK);
+        Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(0));
+        Mockito.doNothing().when(mockOutputCollector).ack(mockCheckpointTuple);
+        executor.execute(mockCheckpointTuple);
+        Mockito.verify(mockState, Mockito.times(1)).rollback();
+    }
+
+    @Test
+    public void testCommit() throws Exception {
+        Mockito.when(mockTuple.getSourceStreamId()).thenReturn("default");
+        executor.execute(mockTuple);
+        Mockito.when(mockCheckpointTuple.getSourceStreamId()).thenReturn(CheckpointSpout.CHECKPOINT_STREAM_ID);
+        Mockito.when(mockCheckpointTuple.getStringByField(CHECKPOINT_FIELD_ACTION)).thenReturn(CHECKPOINT_ACTION_COMMIT);
+        Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(0));
+        Mockito.doNothing().when(mockOutputCollector).ack(mockCheckpointTuple);
+        executor.execute(mockCheckpointTuple);
+        Mockito.verify(mockBolt, Mockito.times(1)).preCommit();
+        Mockito.verify(mockState, Mockito.times(1)).commit(new Long(0));
+        Mockito.verify(mockBolt, Mockito.times(1)).postCommit();
+    }
+}
\ No newline at end of file


[05/10] storm git commit: Refactoring for accomodating windowing with state

Posted by pt...@apache.org.
Refactoring for accomodating windowing with state


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

Branch: refs/heads/master
Commit: 40a1c7e05962de6e767fb3b37db4eb759fdc9b2d
Parents: 3b59ec5
Author: Arun Mahadevan <ai...@hortonworks.com>
Authored: Tue Dec 22 12:28:30 2015 +0530
Committer: Arun Mahadevan <ai...@hortonworks.com>
Committed: Tue Jan 12 10:21:00 2016 +0530

----------------------------------------------------------------------
 .../storm/redis/state/RedisKeyValueState.java      | 17 +++++++----------
 .../storm/state/InMemoryKeyValueState.java         | 10 +++++-----
 .../storm/state/InMemoryKeyValueStateProvider.java | 12 +++++++++++-
 .../storm/topology/IStatefulComponent.java         | 14 ++++++++++----
 .../storm/topology/StatefulBoltExecutor.java       |  5 +++--
 .../storm/topology/base/BaseStatefulBolt.java      |  9 +++++++--
 .../storm/topology/StatefulBoltExecutorTest.java   |  3 +--
 7 files changed, 44 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/40a1c7e0/external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueState.java
----------------------------------------------------------------------
diff --git a/external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueState.java b/external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueState.java
index db54989..57b30cc 100644
--- a/external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueState.java
+++ b/external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueState.java
@@ -33,6 +33,7 @@ import java.io.IOException;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 
 /**
  * A redis based implementation that persists the state in Redis.
@@ -76,7 +77,7 @@ public class RedisKeyValueState<K, V> implements KeyValueState<K, V> {
         this.keySerializer = keySerializer;
         this.valueSerializer = valueSerializer;
         this.jedisContainer = jedisContainer;
-        this.pendingPrepare = new HashMap<>();
+        this.pendingPrepare = new ConcurrentHashMap<>();
         initTxids();
         initPendingCommit();
     }
@@ -101,10 +102,10 @@ public class RedisKeyValueState<K, V> implements KeyValueState<K, V> {
             commands = jedisContainer.getInstance();
             if (commands.exists(prepareNamespace)) {
                 LOG.debug("Loading previously prepared commit from {}", prepareNamespace);
-                pendingCommit = commands.hgetAll(prepareNamespace);
+                pendingCommit = Collections.unmodifiableMap(commands.hgetAll(prepareNamespace));
             } else {
                 LOG.debug("No previously prepared commits.");
-                pendingCommit = new HashMap<>();
+                pendingCommit = Collections.emptyMap();
             }
         } finally {
             jedisContainer.returnInstance(commands);
@@ -169,7 +170,7 @@ public class RedisKeyValueState<K, V> implements KeyValueState<K, V> {
             txIds.put(PREPARE_TXID_KEY, String.valueOf(txid));
             commands.hmset(txidNamespace, txIds);
             pendingCommit = Collections.unmodifiableMap(pendingPrepare);
-            pendingPrepare = new HashMap<>();
+            pendingPrepare = new ConcurrentHashMap<>();
         } finally {
             jedisContainer.returnInstance(commands);
         }
@@ -206,7 +207,7 @@ public class RedisKeyValueState<K, V> implements KeyValueState<K, V> {
             } else {
                 LOG.debug("Nothing to save for commit");
             }
-            pendingPrepare = new HashMap<>();
+            pendingPrepare = new ConcurrentHashMap<>();
         } finally {
             jedisContainer.returnInstance(commands);
         }
@@ -231,7 +232,7 @@ public class RedisKeyValueState<K, V> implements KeyValueState<K, V> {
             }
             commands.hmset(txidNamespace, txIds);
             pendingCommit = Collections.emptyMap();
-            pendingPrepare = new HashMap<>();
+            pendingPrepare = new ConcurrentHashMap<>();
         } finally {
             jedisContainer.returnInstance(commands);
         }
@@ -248,10 +249,6 @@ public class RedisKeyValueState<K, V> implements KeyValueState<K, V> {
                 throw new RuntimeException("Invalid txid '" + txid + "' for prepare. Txid '" + committedTxid +
                                                    "' is already committed");
             }
-            if (txid > committedTxid + 1) {
-                throw new RuntimeException("Cannot prepare a txn with id '" + txid +
-                                                   "' when last committed txid is '" + committedTxid + "'");
-            }
         }
     }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/40a1c7e0/storm-core/src/jvm/backtype/storm/state/InMemoryKeyValueState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/state/InMemoryKeyValueState.java b/storm-core/src/jvm/backtype/storm/state/InMemoryKeyValueState.java
index a516d34..394cf5d 100644
--- a/storm-core/src/jvm/backtype/storm/state/InMemoryKeyValueState.java
+++ b/storm-core/src/jvm/backtype/storm/state/InMemoryKeyValueState.java
@@ -20,8 +20,8 @@ package backtype.storm.state;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.HashMap;
 import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 
 /**
  * An in-memory implementation of the {@link State}
@@ -31,7 +31,7 @@ public class InMemoryKeyValueState<K, V> implements KeyValueState<K, V> {
     private static final long DEFAULT_TXID = -1;
     private TxIdState<K, V> commitedState;
     private TxIdState<K, V> preparedState;
-    private Map<K, V> state = new HashMap<>();
+    private Map<K, V> state = new ConcurrentHashMap<>();
 
     private static class TxIdState<K, V> {
         private long txid;
@@ -69,7 +69,7 @@ public class InMemoryKeyValueState<K, V> implements KeyValueState<K, V> {
 
     @Override
     public void commit() {
-        commitedState = new TxIdState<>(DEFAULT_TXID, new HashMap<K, V>(state));
+        commitedState = new TxIdState<>(DEFAULT_TXID, new ConcurrentHashMap<>(state));
     }
 
     @Override
@@ -78,7 +78,7 @@ public class InMemoryKeyValueState<K, V> implements KeyValueState<K, V> {
         if (preparedState != null && txid > preparedState.txid) {
             throw new RuntimeException("Cannot prepare a new txn while there is a pending txn");
         }
-        preparedState = new TxIdState<>(txid, new HashMap<K, V>(state));
+        preparedState = new TxIdState<>(txid, new ConcurrentHashMap<K, V>(state));
     }
 
     @Override
@@ -99,7 +99,7 @@ public class InMemoryKeyValueState<K, V> implements KeyValueState<K, V> {
         if (commitedState != null) {
             state = commitedState.state;
         } else {
-            state = new HashMap<>();
+            state = new ConcurrentHashMap<>();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/40a1c7e0/storm-core/src/jvm/backtype/storm/state/InMemoryKeyValueStateProvider.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/state/InMemoryKeyValueStateProvider.java b/storm-core/src/jvm/backtype/storm/state/InMemoryKeyValueStateProvider.java
index 1a79e72..aaedbb6 100644
--- a/storm-core/src/jvm/backtype/storm/state/InMemoryKeyValueStateProvider.java
+++ b/storm-core/src/jvm/backtype/storm/state/InMemoryKeyValueStateProvider.java
@@ -20,14 +20,24 @@ package backtype.storm.state;
 import backtype.storm.task.TopologyContext;
 
 import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 
 /**
  * Provides {@link InMemoryKeyValueState}
  */
 public class InMemoryKeyValueStateProvider implements StateProvider {
+    private final ConcurrentHashMap<String, State> states = new ConcurrentHashMap<>();
 
     @Override
     public State newState(String namespace, Map stormConf, TopologyContext context) {
-        return new InMemoryKeyValueState();
+        State state = states.get(namespace);
+        if (state == null) {
+            State newState = new InMemoryKeyValueState<>();
+            state = states.putIfAbsent(namespace, newState);
+            if (state == null) {
+                state = newState;
+            }
+        }
+        return state;
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/40a1c7e0/storm-core/src/jvm/backtype/storm/topology/IStatefulComponent.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/topology/IStatefulComponent.java b/storm-core/src/jvm/backtype/storm/topology/IStatefulComponent.java
index 887c91a..ea88d90 100644
--- a/storm-core/src/jvm/backtype/storm/topology/IStatefulComponent.java
+++ b/storm-core/src/jvm/backtype/storm/topology/IStatefulComponent.java
@@ -40,11 +40,17 @@ public interface IStatefulComponent<T extends State> extends IComponent {
      * This is a hook for the component to perform some actions just before the
      * framework commits its state.
      */
-    void preCommit();
+    void preCommit(long txid);
 
     /**
-     * This is a hook for the component to perform some actions just after the
-     * framework commits its state.
+     * This is a hook for the component to perform some actions just before the
+     * framework prepares its state.
+     */
+    void prePrepare(long txid);
+
+    /**
+     * This is a hook for the component to perform some actions just before the
+     * framework rolls back the prepared state.
      */
-    void postCommit();
+    void preRollback();
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/40a1c7e0/storm-core/src/jvm/backtype/storm/topology/StatefulBoltExecutor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/topology/StatefulBoltExecutor.java b/storm-core/src/jvm/backtype/storm/topology/StatefulBoltExecutor.java
index bdb45e3..a7c5b2e 100644
--- a/storm-core/src/jvm/backtype/storm/topology/StatefulBoltExecutor.java
+++ b/storm-core/src/jvm/backtype/storm/topology/StatefulBoltExecutor.java
@@ -68,12 +68,13 @@ public class StatefulBoltExecutor<T extends State> extends CheckpointTupleForwar
     protected void handleCheckpoint(Tuple input, String action, long txid) {
         LOG.debug("handleCheckPoint with tuple {}, action {}, txid {}", input, action, txid);
         if (action.equals(CHECKPOINT_ACTION_PREPARE)) {
+            bolt.prePrepare(txid);
             state.prepareCommit(txid);
         } else if (action.equals(CHECKPOINT_ACTION_COMMIT)) {
-            bolt.preCommit();
+            bolt.preCommit(txid);
             state.commit(txid);
-            bolt.postCommit();
         } else if (action.equals(CHECKPOINT_ACTION_ROLLBACK)) {
+            bolt.preRollback();
             state.rollback();
         } else if (action.equals(CHECKPOINT_ACTION_INITSTATE)) {
             bolt.initState((T) state);

http://git-wip-us.apache.org/repos/asf/storm/blob/40a1c7e0/storm-core/src/jvm/backtype/storm/topology/base/BaseStatefulBolt.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/topology/base/BaseStatefulBolt.java b/storm-core/src/jvm/backtype/storm/topology/base/BaseStatefulBolt.java
index 93bcd57..80734ab 100644
--- a/storm-core/src/jvm/backtype/storm/topology/base/BaseStatefulBolt.java
+++ b/storm-core/src/jvm/backtype/storm/topology/base/BaseStatefulBolt.java
@@ -48,12 +48,17 @@ public abstract class BaseStatefulBolt<T extends State> implements IStatefulBolt
     }
 
     @Override
-    public void preCommit() {
+    public void preCommit(long txid) {
         // NOOP
     }
 
     @Override
-    public void postCommit() {
+    public void prePrepare(long txid) {
+        // NOOP
+    }
+
+    @Override
+    public void preRollback() {
         // NOOP
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/40a1c7e0/storm-core/test/jvm/backtype/storm/topology/StatefulBoltExecutorTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/topology/StatefulBoltExecutorTest.java b/storm-core/test/jvm/backtype/storm/topology/StatefulBoltExecutorTest.java
index 72d2ac7..570258a 100644
--- a/storm-core/test/jvm/backtype/storm/topology/StatefulBoltExecutorTest.java
+++ b/storm-core/test/jvm/backtype/storm/topology/StatefulBoltExecutorTest.java
@@ -115,8 +115,7 @@ public class StatefulBoltExecutorTest {
         Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(0));
         Mockito.doNothing().when(mockOutputCollector).ack(mockCheckpointTuple);
         executor.execute(mockCheckpointTuple);
-        Mockito.verify(mockBolt, Mockito.times(1)).preCommit();
+        Mockito.verify(mockBolt, Mockito.times(1)).preCommit(new Long(0));
         Mockito.verify(mockState, Mockito.times(1)).commit(new Long(0));
-        Mockito.verify(mockBolt, Mockito.times(1)).postCommit();
     }
 }
\ No newline at end of file


[07/10] storm git commit: Package name fixes

Posted by pt...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/test/jvm/org/apache/storm/topology/StatefulBoltExecutorTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/topology/StatefulBoltExecutorTest.java b/storm-core/test/jvm/org/apache/storm/topology/StatefulBoltExecutorTest.java
new file mode 100644
index 0000000..69c541b
--- /dev/null
+++ b/storm-core/test/jvm/org/apache/storm/topology/StatefulBoltExecutorTest.java
@@ -0,0 +1,177 @@
+/**
+ * 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.topology;
+
+import org.apache.storm.generated.GlobalStreamId;
+import org.apache.storm.generated.Grouping;
+import org.apache.storm.spout.CheckpointSpout;
+import org.apache.storm.state.KeyValueState;
+import org.apache.storm.state.State;
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.topology.IStatefulBolt;
+import org.apache.storm.topology.StatefulBoltExecutor;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.storm.spout.CheckPointState.Action.*;
+import static org.apache.storm.spout.CheckpointSpout.*;
+
+/**
+ * Unit tests for {@link StatefulBoltExecutor}
+ */
+public class StatefulBoltExecutorTest {
+    StatefulBoltExecutor<KeyValueState<String, String>> executor;
+    IStatefulBolt<KeyValueState<String, String>> mockBolt;
+    TopologyContext mockTopologyContext;
+    Tuple mockTuple;
+    Tuple mockCheckpointTuple;
+    Map<String, Object> mockStormConf = new HashMap<>();
+    OutputCollector mockOutputCollector;
+    State mockState;
+    Map<GlobalStreamId, Grouping> mockGlobalStream;
+    Set<GlobalStreamId> mockStreamIds;
+    @Before
+    public void setUp() throws Exception {
+        mockBolt = Mockito.mock(IStatefulBolt.class);
+        executor = new StatefulBoltExecutor<>(mockBolt);
+        GlobalStreamId mockGlobalStreamId = Mockito.mock(GlobalStreamId.class);
+        Mockito.when(mockGlobalStreamId.get_streamId()).thenReturn(CheckpointSpout.CHECKPOINT_STREAM_ID);
+        mockStreamIds = new HashSet<>();
+        mockStreamIds.add(mockGlobalStreamId);
+        mockTopologyContext = Mockito.mock(TopologyContext.class);
+        mockOutputCollector = Mockito.mock(OutputCollector.class);
+        mockGlobalStream = Mockito.mock(Map.class);
+        mockState = Mockito.mock(State.class);
+        Mockito.when(mockTopologyContext.getThisComponentId()).thenReturn("test");
+        Mockito.when(mockTopologyContext.getThisTaskId()).thenReturn(1);
+        Mockito.when(mockTopologyContext.getThisSources()).thenReturn(mockGlobalStream);
+        Mockito.when(mockTopologyContext.getComponentTasks(Mockito.anyString())).thenReturn(Collections.singletonList(1));
+        Mockito.when(mockGlobalStream.keySet()).thenReturn(mockStreamIds);
+        mockTuple = Mockito.mock(Tuple.class);
+        mockCheckpointTuple = Mockito.mock(Tuple.class);
+        executor.prepare(mockStormConf, mockTopologyContext, mockOutputCollector, mockState);
+    }
+
+    @Test
+    public void testHandleTupleBeforeInit() throws Exception {
+        Mockito.when(mockTuple.getSourceStreamId()).thenReturn("default");
+        executor.execute(mockTuple);
+        Mockito.verify(mockBolt, Mockito.times(0)).execute(Mockito.any(Tuple.class));
+    }
+
+    @Test
+    public void testHandleTuple() throws Exception {
+        Mockito.when(mockTuple.getSourceStreamId()).thenReturn("default");
+        executor.execute(mockTuple);
+        Mockito.when(mockCheckpointTuple.getSourceStreamId()).thenReturn(CheckpointSpout.CHECKPOINT_STREAM_ID);
+        Mockito.when(mockCheckpointTuple.getValueByField(CHECKPOINT_FIELD_ACTION)).thenReturn(INITSTATE);
+        Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(0));
+        Mockito.doNothing().when(mockOutputCollector).ack(mockCheckpointTuple);
+        executor.execute(mockCheckpointTuple);
+        Mockito.verify(mockBolt, Mockito.times(1)).execute(mockTuple);
+        Mockito.verify(mockBolt, Mockito.times(1)).initState(Mockito.any(KeyValueState.class));
+    }
+
+    @Test
+    public void testRollback() throws Exception {
+        Mockito.when(mockTuple.getSourceStreamId()).thenReturn("default");
+        executor.execute(mockTuple);
+        Mockito.when(mockCheckpointTuple.getSourceStreamId()).thenReturn(CheckpointSpout.CHECKPOINT_STREAM_ID);
+        Mockito.when(mockCheckpointTuple.getValueByField(CHECKPOINT_FIELD_ACTION)).thenReturn(ROLLBACK);
+        Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(0));
+        Mockito.doNothing().when(mockOutputCollector).ack(mockCheckpointTuple);
+        executor.execute(mockCheckpointTuple);
+        Mockito.verify(mockState, Mockito.times(1)).rollback();
+    }
+
+    @Test
+    public void testCommit() throws Exception {
+        Mockito.when(mockTuple.getSourceStreamId()).thenReturn("default");
+        executor.execute(mockTuple);
+        Mockito.when(mockCheckpointTuple.getSourceStreamId()).thenReturn(CheckpointSpout.CHECKPOINT_STREAM_ID);
+        Mockito.when(mockCheckpointTuple.getValueByField(CHECKPOINT_FIELD_ACTION)).thenReturn(COMMIT);
+        Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(0));
+        Mockito.doNothing().when(mockOutputCollector).ack(mockCheckpointTuple);
+        executor.execute(mockCheckpointTuple);
+        Mockito.verify(mockBolt, Mockito.times(1)).preCommit(new Long(0));
+        Mockito.verify(mockState, Mockito.times(1)).commit(new Long(0));
+    }
+
+    @Test
+    public void testPrepareAndRollbackBeforeInitstate() throws Exception {
+        Mockito.when(mockTuple.getSourceStreamId()).thenReturn("default");
+        executor.execute(mockTuple);
+        Mockito.when(mockCheckpointTuple.getSourceStreamId()).thenReturn(CheckpointSpout.CHECKPOINT_STREAM_ID);
+        Mockito.when(mockCheckpointTuple.getValueByField(CHECKPOINT_FIELD_ACTION)).thenReturn(PREPARE);
+        Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(100));
+        executor.execute(mockCheckpointTuple);
+        Mockito.verify(mockOutputCollector, Mockito.times(1)).fail(mockCheckpointTuple);
+
+        Mockito.when(mockCheckpointTuple.getValueByField(CHECKPOINT_FIELD_ACTION)).thenReturn(ROLLBACK);
+        Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(100));
+        Mockito.doNothing().when(mockOutputCollector).ack(mockCheckpointTuple);
+        executor.execute(mockCheckpointTuple);
+        Mockito.verify(mockState, Mockito.times(1)).rollback();
+    }
+
+    @Test
+    public void testCommitBeforeInitstate() throws Exception {
+        Mockito.when(mockTuple.getSourceStreamId()).thenReturn("default");
+        Mockito.when(mockCheckpointTuple.getSourceStreamId()).thenReturn(CheckpointSpout.CHECKPOINT_STREAM_ID);
+        Mockito.when(mockCheckpointTuple.getValueByField(CHECKPOINT_FIELD_ACTION)).thenReturn(COMMIT);
+        Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(100));
+        executor.execute(mockCheckpointTuple);
+        Mockito.verify(mockOutputCollector, Mockito.times(1)).ack(mockCheckpointTuple);
+
+        Mockito.when(mockCheckpointTuple.getValueByField(CHECKPOINT_FIELD_ACTION)).thenReturn(ROLLBACK);
+        Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(100));
+        executor.execute(mockCheckpointTuple);
+        Mockito.verify(mockState, Mockito.times(1)).rollback();
+    }
+
+    @Test
+    public void testPrepareAndCommit() throws Exception {
+        Mockito.when(mockTuple.getSourceStreamId()).thenReturn("default");
+        Mockito.when(mockCheckpointTuple.getSourceStreamId()).thenReturn(CheckpointSpout.CHECKPOINT_STREAM_ID);
+        Mockito.when(mockCheckpointTuple.getValueByField(CHECKPOINT_FIELD_ACTION)).thenReturn(INITSTATE);
+        Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(0));
+        executor.execute(mockCheckpointTuple);
+
+        executor.execute(mockTuple);
+        Mockito.when(mockCheckpointTuple.getSourceStreamId()).thenReturn(CheckpointSpout.CHECKPOINT_STREAM_ID);
+        Mockito.when(mockCheckpointTuple.getValueByField(CHECKPOINT_FIELD_ACTION)).thenReturn(PREPARE);
+        Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(100));
+        executor.execute(mockCheckpointTuple);
+        executor.execute(mockTuple);
+        Mockito.when(mockCheckpointTuple.getValueByField(CHECKPOINT_FIELD_ACTION)).thenReturn(COMMIT);
+        Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(100));
+        executor.execute(mockCheckpointTuple);
+        Mockito.verify(mockState, Mockito.times(1)).commit(new Long(100));
+        Mockito.verify(mockBolt, Mockito.times(2)).execute(mockTuple);
+        Mockito.verify(mockOutputCollector, Mockito.times(1)).ack(mockTuple);
+    }
+}
\ No newline at end of file


[10/10] storm git commit: add STORM-1175 to changelog

Posted by pt...@apache.org.
add STORM-1175 to changelog


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

Branch: refs/heads/master
Commit: b24d7d93eea1b1a5700aacb4bc83cce8cef7ce6e
Parents: bfc5ffd
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Tue Jan 12 14:59:20 2016 -0500
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Tue Jan 12 14:59:20 2016 -0500

----------------------------------------------------------------------
 CHANGELOG.md | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/b24d7d93/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 52e2ba4..1890e69 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,5 +1,6 @@
 ## 2.0.0
 ## 1.0.0
+ * STORM-1175: State store for windowing operations
  * STORM-1202: Migrate APIs to org.apache.storm, but try to provide some form of backwards compatability
  * STORM-468: java.io.NotSerializableException should be explained
  * STORM-1348: refactor API to remove Insert/Update builder in Cassandra connector


[03/10] storm git commit: Refactored unit tests

Posted by pt...@apache.org.
Refactored unit tests


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

Branch: refs/heads/master
Commit: 3b59ec5d7f0eb505522495f2f67bba506246554e
Parents: 4623d8f
Author: Arun Mahadevan <ai...@hortonworks.com>
Authored: Fri Dec 11 01:39:12 2015 +0530
Committer: Arun Mahadevan <ai...@hortonworks.com>
Committed: Tue Jan 12 10:20:59 2016 +0530

----------------------------------------------------------------------
 .../redis/state/RedisKeyValueStateTest.java     | 25 ++++++++---------
 .../storm/state/InMemoryKeyValueStateTest.java  | 29 +++++++++-----------
 2 files changed, 24 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/3b59ec5d/external/storm-redis/src/test/java/org/apache/storm/redis/state/RedisKeyValueStateTest.java
----------------------------------------------------------------------
diff --git a/external/storm-redis/src/test/java/org/apache/storm/redis/state/RedisKeyValueStateTest.java b/external/storm-redis/src/test/java/org/apache/storm/redis/state/RedisKeyValueStateTest.java
index 80bab4c..9d8286c 100644
--- a/external/storm-redis/src/test/java/org/apache/storm/redis/state/RedisKeyValueStateTest.java
+++ b/external/storm-redis/src/test/java/org/apache/storm/redis/state/RedisKeyValueStateTest.java
@@ -113,29 +113,26 @@ public class RedisKeyValueStateTest {
         keyValueState.put("b", "2");
         keyValueState.prepareCommit(1);
         keyValueState.put("c", "3");
-        assertEquals("1", keyValueState.get("a"));
-        assertEquals("2", keyValueState.get("b"));
-        assertEquals("3", keyValueState.get("c"));
+        assertArrayEquals(new String[]{"1", "2", "3"}, getValues());
         keyValueState.rollback();
-        assertEquals(null, keyValueState.get("a"));
-        assertEquals(null, keyValueState.get("b"));
-        assertEquals(null, keyValueState.get("c"));
-
+        assertArrayEquals(new String[]{null, null, null}, getValues());
         keyValueState.put("a", "1");
         keyValueState.put("b", "2");
         keyValueState.prepareCommit(1);
         keyValueState.commit(1);
         keyValueState.put("c", "3");
-        assertEquals("1", keyValueState.get("a"));
-        assertEquals("2", keyValueState.get("b"));
-        assertEquals("3", keyValueState.get("c"));
+        assertArrayEquals(new String[]{"1", "2", "3"}, getValues());
         keyValueState.rollback();
-        assertEquals("1", keyValueState.get("a"));
-        assertEquals("2", keyValueState.get("b"));
-        assertEquals(null, keyValueState.get("c"));
-
+        assertArrayEquals(new String[]{"1", "2", null}, getValues());
     }
 
+    private String[] getValues() {
+        return new String[]{
+                keyValueState.get("a"),
+                keyValueState.get("b"),
+                keyValueState.get("c")
+        };
+    }
 
     private String hmset(Map<String, Map<String, String>> mockMap, String key, Map value) {
         mockMap.put(key, value);

http://git-wip-us.apache.org/repos/asf/storm/blob/3b59ec5d/storm-core/test/jvm/backtype/storm/state/InMemoryKeyValueStateTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/state/InMemoryKeyValueStateTest.java b/storm-core/test/jvm/backtype/storm/state/InMemoryKeyValueStateTest.java
index 31dbc79..78ace77 100644
--- a/storm-core/test/jvm/backtype/storm/state/InMemoryKeyValueStateTest.java
+++ b/storm-core/test/jvm/backtype/storm/state/InMemoryKeyValueStateTest.java
@@ -38,9 +38,7 @@ public class InMemoryKeyValueStateTest {
     public void testPutAndGet() throws Exception {
         keyValueState.put("a", "1");
         keyValueState.put("b", "2");
-        assertEquals("1", keyValueState.get("a"));
-        assertEquals("2", keyValueState.get("b"));
-        assertEquals(null, keyValueState.get("c"));
+        assertArrayEquals(new String[]{"1", "2", null}, getValues());
     }
 
     @Test
@@ -49,25 +47,24 @@ public class InMemoryKeyValueStateTest {
         keyValueState.put("b", "2");
         keyValueState.prepareCommit(1);
         keyValueState.put("c", "3");
-        assertEquals("1", keyValueState.get("a"));
-        assertEquals("2", keyValueState.get("b"));
-        assertEquals("3", keyValueState.get("c"));
+        assertArrayEquals(new String[]{"1", "2", "3"}, getValues());
         keyValueState.rollback();
-        assertEquals(null, keyValueState.get("a"));
-        assertEquals(null, keyValueState.get("b"));
-        assertEquals(null, keyValueState.get("c"));
+        assertArrayEquals(new String[]{null, null, null}, getValues());
         keyValueState.put("a", "1");
         keyValueState.put("b", "2");
         keyValueState.prepareCommit(1);
-        keyValueState.commit();
+        keyValueState.commit(1);
         keyValueState.put("c", "3");
-        assertEquals("1", keyValueState.get("a"));
-        assertEquals("2", keyValueState.get("b"));
-        assertEquals("3", keyValueState.get("c"));
+        assertArrayEquals(new String[]{"1", "2", "3"}, getValues());
         keyValueState.rollback();
-        assertEquals("1", keyValueState.get("a"));
-        assertEquals("2", keyValueState.get("b"));
-        assertEquals(null, keyValueState.get("c"));
+        assertArrayEquals(new String[]{"1", "2", null}, getValues());
+    }
 
+    private String[] getValues() {
+        return new String[]{
+                keyValueState.get("a"),
+                keyValueState.get("b"),
+                keyValueState.get("c")
+        };
     }
 }
\ No newline at end of file


[02/10] storm git commit: [STORM-1175] State store for windowing operations

Posted by pt...@apache.org.
[STORM-1175] State store for windowing operations

Added IStatefulBolt abstraction that can be implemented by bolts which requires
its state to be checkpointed periodically. State implementations based on
key-value mapping store are added. There is a default in-memory based implementation
and optional redis based implementation that provides state persistence. An internal
CheckpointSpout periodically emits checkpoint tuples which flows through the
topology DAG to take a consistent snapshot of the state across all components.

There is still pending work to capture the evaluated/expired state of the tuples
in the Window and use it to prune duplicate window evaluations during restart. This can
be built on top of the stateful bolt abstractions and will be done as part of STORM-1176


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

Branch: refs/heads/master
Commit: 4623d8f87f044a9a8737a55bd32f955485900ce1
Parents: 3db9680
Author: Arun Mahadevan <ai...@hortonworks.com>
Authored: Mon Nov 30 16:36:10 2015 +0530
Committer: Arun Mahadevan <ai...@hortonworks.com>
Committed: Tue Jan 12 10:20:51 2016 +0530

----------------------------------------------------------------------
 examples/storm-starter/pom.xml                  |   5 +
 .../src/jvm/storm/starter/StatefulTopology.java | 144 +++++++++
 external/storm-redis/pom.xml                    |  21 ++
 .../redis/common/config/JedisPoolConfig.java    |  14 +
 .../storm/redis/state/RedisKeyValueState.java   | 305 +++++++++++++++++++
 .../redis/state/RedisKeyValueStateProvider.java | 120 ++++++++
 .../redis/state/DefaultStateSerializerTest.java |  56 ++++
 .../state/RedisKeyValueStateProviderTest.java   |  62 ++++
 .../redis/state/RedisKeyValueStateTest.java     | 157 ++++++++++
 .../clj/org/apache/storm/daemon/executor.clj    |   2 +
 .../backtype/storm/spout/CheckPointState.java   |  78 +++++
 .../backtype/storm/spout/CheckpointSpout.java   | 280 +++++++++++++++++
 .../storm/state/DefaultStateSerializer.java     |  65 ++++
 .../storm/state/InMemoryKeyValueState.java      | 114 +++++++
 .../state/InMemoryKeyValueStateProvider.java    |  33 ++
 .../jvm/backtype/storm/state/KeyValueState.java |  48 +++
 .../jvm/backtype/storm/state/Serializer.java    |  30 ++
 .../src/jvm/backtype/storm/state/State.java     |  53 ++++
 .../jvm/backtype/storm/state/StateFactory.java  |  69 +++++
 .../jvm/backtype/storm/state/StateProvider.java |  38 +++
 .../topology/CheckpointTupleForwarder.java      | 218 +++++++++++++
 .../backtype/storm/topology/IStatefulBolt.java  |  26 ++
 .../storm/topology/IStatefulComponent.java      |  50 +++
 .../storm/topology/StatefulBoltExecutor.java    |  98 ++++++
 .../storm/topology/base/BaseStatefulBolt.java   |  59 ++++
 storm-core/src/jvm/org/apache/storm/Config.java |  23 ++
 .../apache/storm/topology/TopologyBuilder.java  |  81 ++++-
 .../storm/spout/CheckpointSpoutTest.java        | 245 +++++++++++++++
 .../storm/state/InMemoryKeyValueStateTest.java  |  73 +++++
 .../topology/StatefulBoltExecutorTest.java      | 122 ++++++++
 30 files changed, 2685 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/4623d8f8/examples/storm-starter/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml
index da03688..8d6752d 100644
--- a/examples/storm-starter/pom.xml
+++ b/examples/storm-starter/pom.xml
@@ -157,6 +157,11 @@
       <version>0.8.2.1</version>
       <scope>provided</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.storm</groupId>
+      <artifactId>storm-redis</artifactId>
+      <version>${project.version}</version>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/storm/blob/4623d8f8/examples/storm-starter/src/jvm/storm/starter/StatefulTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/storm/starter/StatefulTopology.java b/examples/storm-starter/src/jvm/storm/starter/StatefulTopology.java
new file mode 100644
index 0000000..fa106dd
--- /dev/null
+++ b/examples/storm-starter/src/jvm/storm/starter/StatefulTopology.java
@@ -0,0 +1,144 @@
+/**
+ * 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 storm.starter;
+
+import backtype.storm.Config;
+import backtype.storm.LocalCluster;
+import backtype.storm.StormSubmitter;
+import backtype.storm.generated.StormTopology;
+import backtype.storm.state.KeyValueState;
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.BasicOutputCollector;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.topology.TopologyBuilder;
+import backtype.storm.topology.base.BaseBasicBolt;
+import backtype.storm.topology.base.BaseStatefulBolt;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.tuple.Values;
+import backtype.storm.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import storm.starter.spout.RandomIntegerSpout;
+
+import java.util.Map;
+
+/**
+ * An example topology that demonstrates the use of {@link backtype.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);
+    /**
+     * 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 stormConf, 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));
+        }
+
+        @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"));
+        }
+
+    }
+
+    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);
+
+        if (args != null && args.length > 0) {
+            conf.setNumWorkers(1);
+            StormSubmitter.submitTopologyWithProgressBar(args[0], conf, builder.createTopology());
+        } else {
+            LocalCluster cluster = new LocalCluster();
+            StormTopology topology = builder.createTopology();
+            cluster.submitTopology("test", conf, topology);
+            Utils.sleep(40000);
+            cluster.killTopology("test");
+            cluster.shutdown();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4623d8f8/external/storm-redis/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-redis/pom.xml b/external/storm-redis/pom.xml
index 6024b0f..530f010 100644
--- a/external/storm-redis/pom.xml
+++ b/external/storm-redis/pom.xml
@@ -61,5 +61,26 @@
             <artifactId>guava</artifactId>
             <version>18.0</version>
         </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-core</artifactId>
+            <version>${jackson.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+            <version>${jackson.version}</version>
+        </dependency>
+        <!--test dependencies -->
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-all</artifactId>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/storm/blob/4623d8f8/external/storm-redis/src/main/java/org/apache/storm/redis/common/config/JedisPoolConfig.java
----------------------------------------------------------------------
diff --git a/external/storm-redis/src/main/java/org/apache/storm/redis/common/config/JedisPoolConfig.java b/external/storm-redis/src/main/java/org/apache/storm/redis/common/config/JedisPoolConfig.java
index 8b3a2e5..d555a2a 100644
--- a/external/storm-redis/src/main/java/org/apache/storm/redis/common/config/JedisPoolConfig.java
+++ b/external/storm-redis/src/main/java/org/apache/storm/redis/common/config/JedisPoolConfig.java
@@ -33,6 +33,9 @@ public class JedisPoolConfig implements Serializable {
     private int database;
     private String password;
 
+    // for serialization
+    public JedisPoolConfig() {
+    }
     /**
      * Constructor
      * <p/>
@@ -160,4 +163,15 @@ public class JedisPoolConfig implements Serializable {
             return new JedisPoolConfig(host, port, timeout, password, database);
         }
     }
+
+    @Override
+    public String toString() {
+        return "JedisPoolConfig{" +
+                "host='" + host + '\'' +
+                ", port=" + port +
+                ", timeout=" + timeout +
+                ", database=" + database +
+                ", password='" + password + '\'' +
+                '}';
+    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/4623d8f8/external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueState.java
----------------------------------------------------------------------
diff --git a/external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueState.java b/external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueState.java
new file mode 100644
index 0000000..db54989
--- /dev/null
+++ b/external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueState.java
@@ -0,0 +1,305 @@
+/**
+ * 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.state;
+
+import backtype.storm.state.DefaultStateSerializer;
+import backtype.storm.state.KeyValueState;
+import backtype.storm.state.Serializer;
+import org.apache.storm.redis.common.config.JedisPoolConfig;
+import org.apache.storm.redis.common.container.JedisCommandsContainerBuilder;
+import org.apache.storm.redis.common.container.JedisCommandsInstanceContainer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import redis.clients.jedis.JedisCommands;
+import sun.misc.BASE64Decoder;
+import sun.misc.BASE64Encoder;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * A redis based implementation that persists the state in Redis.
+ */
+public class RedisKeyValueState<K, V> implements KeyValueState<K, V> {
+    private static final Logger LOG = LoggerFactory.getLogger(RedisKeyValueState.class);
+    private static final String COMMIT_TXID_KEY = "$commit";
+    private static final String PREPARE_TXID_KEY = "$prepare";
+
+    private final BASE64Encoder base64Encoder;
+    private final BASE64Decoder base64Decoder;
+    private final String namespace;
+    private final String prepareNamespace;
+    private final String txidNamespace;
+    private final Serializer<K> keySerializer;
+    private final Serializer<V> valueSerializer;
+    private final JedisCommandsInstanceContainer jedisContainer;
+    private Map<String, String> pendingPrepare;
+    private Map<String, String> pendingCommit;
+    private Map<String, String> txIds;
+
+    public RedisKeyValueState(String namespace) {
+        this(namespace, new JedisPoolConfig.Builder().build());
+    }
+
+    public RedisKeyValueState(String namespace, JedisPoolConfig poolConfig) {
+        this(namespace, poolConfig, new DefaultStateSerializer<K>(), new DefaultStateSerializer<V>());
+    }
+
+    public RedisKeyValueState(String namespace, JedisPoolConfig poolConfig, Serializer<K> keySerializer, Serializer<V> valueSerializer) {
+        this(namespace, JedisCommandsContainerBuilder.build(poolConfig), keySerializer, valueSerializer);
+    }
+
+    public RedisKeyValueState(String namespace, JedisCommandsInstanceContainer jedisContainer,
+                              Serializer<K> keySerializer, Serializer<V> valueSerializer) {
+        base64Encoder = new BASE64Encoder();
+        base64Decoder = new BASE64Decoder();
+        this.namespace = namespace;
+        this.prepareNamespace = namespace + "$prepare";
+        this.txidNamespace = namespace + "$txid";
+        this.keySerializer = keySerializer;
+        this.valueSerializer = valueSerializer;
+        this.jedisContainer = jedisContainer;
+        this.pendingPrepare = new HashMap<>();
+        initTxids();
+        initPendingCommit();
+    }
+
+    private void initTxids() {
+        JedisCommands commands = null;
+        try {
+            commands = jedisContainer.getInstance();
+            if (commands.exists(txidNamespace)) {
+                txIds = commands.hgetAll(txidNamespace);
+            } else {
+                txIds = new HashMap<>();
+            }
+        } finally {
+            jedisContainer.returnInstance(commands);
+        }
+    }
+
+    private void initPendingCommit() {
+        JedisCommands commands = null;
+        try {
+            commands = jedisContainer.getInstance();
+            if (commands.exists(prepareNamespace)) {
+                LOG.debug("Loading previously prepared commit from {}", prepareNamespace);
+                pendingCommit = commands.hgetAll(prepareNamespace);
+            } else {
+                LOG.debug("No previously prepared commits.");
+                pendingCommit = new HashMap<>();
+            }
+        } finally {
+            jedisContainer.returnInstance(commands);
+        }
+    }
+
+    @Override
+    public void put(K key, V value) {
+        LOG.debug("put key '{}', value '{}'", key, value);
+        pendingPrepare.put(encode(keySerializer.serialize(key)),
+                           encode(valueSerializer.serialize(value)));
+    }
+
+    @Override
+    public V get(K key) {
+        LOG.debug("get key '{}'", key);
+        String redisKey = encode(keySerializer.serialize(key));
+        String redisValue = null;
+        if (pendingPrepare.containsKey(redisKey)) {
+            redisValue = pendingPrepare.get(redisKey);
+        } else if (pendingCommit.containsKey(redisKey)) {
+            redisValue = pendingCommit.get(redisKey);
+        } else {
+            JedisCommands commands = null;
+            try {
+                commands = jedisContainer.getInstance();
+                redisValue = commands.hget(namespace, redisKey);
+            } finally {
+                jedisContainer.returnInstance(commands);
+            }
+        }
+        V value = null;
+        if (redisValue != null) {
+            value = valueSerializer.deserialize(decode(redisValue));
+        }
+        LOG.debug("Value for key '{}' is '{}'", key, value);
+        return value;
+    }
+
+    @Override
+    public V get(K key, V defaultValue) {
+        V val = get(key);
+        return val != null ? val : defaultValue;
+    }
+
+    @Override
+    public void prepareCommit(long txid) {
+        LOG.debug("prepareCommit txid {}", txid);
+        validatePrepareTxid(txid);
+        JedisCommands commands = null;
+        try {
+            commands = jedisContainer.getInstance();
+            if (commands.exists(prepareNamespace)) {
+                LOG.debug("Prepared txn already exists, will merge", txid);
+                pendingPrepare.putAll(pendingCommit);
+            }
+            if (!pendingPrepare.isEmpty()) {
+                commands.hmset(prepareNamespace, pendingPrepare);
+            } else {
+                LOG.debug("Nothing to save for prepareCommit, txid {}.", txid);
+            }
+            txIds.put(PREPARE_TXID_KEY, String.valueOf(txid));
+            commands.hmset(txidNamespace, txIds);
+            pendingCommit = Collections.unmodifiableMap(pendingPrepare);
+            pendingPrepare = new HashMap<>();
+        } finally {
+            jedisContainer.returnInstance(commands);
+        }
+    }
+
+    @Override
+    public void commit(long txid) {
+        LOG.debug("commit txid {}", txid);
+        validateCommitTxid(txid);
+        JedisCommands commands = null;
+        try {
+            commands = jedisContainer.getInstance();
+            if (!pendingCommit.isEmpty()) {
+                commands.hmset(namespace, pendingCommit);
+            } else {
+                LOG.debug("Nothing to save for commit, txid {}.", txid);
+            }
+            txIds.put(COMMIT_TXID_KEY, String.valueOf(txid));
+            commands.hmset(txidNamespace, txIds);
+            commands.del(prepareNamespace);
+            pendingCommit = Collections.emptyMap();
+        } finally {
+            jedisContainer.returnInstance(commands);
+        }
+    }
+
+    @Override
+    public void commit() {
+        JedisCommands commands = null;
+        try {
+            commands = jedisContainer.getInstance();
+            if (!pendingPrepare.isEmpty()) {
+                commands.hmset(namespace, pendingPrepare);
+            } else {
+                LOG.debug("Nothing to save for commit");
+            }
+            pendingPrepare = new HashMap<>();
+        } finally {
+            jedisContainer.returnInstance(commands);
+        }
+    }
+
+    @Override
+    public void rollback() {
+        LOG.debug("rollback");
+        JedisCommands commands = null;
+        try {
+            commands = jedisContainer.getInstance();
+            if (commands.exists(prepareNamespace)) {
+                commands.del(prepareNamespace);
+            } else {
+                LOG.debug("Nothing to rollback, prepared data is empty");
+            }
+            Long lastCommittedId = lastCommittedTxid();
+            if (lastCommittedId != null) {
+                txIds.put(PREPARE_TXID_KEY, String.valueOf(lastCommittedId));
+            } else {
+                txIds.remove(PREPARE_TXID_KEY);
+            }
+            commands.hmset(txidNamespace, txIds);
+            pendingCommit = Collections.emptyMap();
+            pendingPrepare = new HashMap<>();
+        } finally {
+            jedisContainer.returnInstance(commands);
+        }
+    }
+
+    /*
+     * Same txid can be prepared again, but the next txid cannot be prepared
+     * when previous one is not committed yet.
+     */
+    private void validatePrepareTxid(long txid) {
+        Long committedTxid = lastCommittedTxid();
+        if (committedTxid != null) {
+            if (txid <= committedTxid) {
+                throw new RuntimeException("Invalid txid '" + txid + "' for prepare. Txid '" + committedTxid +
+                                                   "' is already committed");
+            }
+            if (txid > committedTxid + 1) {
+                throw new RuntimeException("Cannot prepare a txn with id '" + txid +
+                                                   "' when last committed txid is '" + committedTxid + "'");
+            }
+        }
+    }
+
+    /*
+     * Same txid can be committed again but the
+     * txid to be committed must be the last prepared one.
+     */
+    private void validateCommitTxid(long txid) {
+        Long committedTxid = lastCommittedTxid();
+        if (committedTxid != null) {
+            if (txid < committedTxid) {
+                throw new RuntimeException("Invalid txid '" + txid + "' txid '" + committedTxid + "' is already committed");
+            }
+        }
+        Long preparedTxid = lastPreparedTxid();
+        if (preparedTxid != null) {
+            if (txid != preparedTxid) {
+                throw new RuntimeException("Invalid txid '" + txid + "' not same as prepared txid '" + preparedTxid + "'");
+            }
+        }
+    }
+
+    private Long lastCommittedTxid() {
+        return lastId(COMMIT_TXID_KEY);
+    }
+
+    private Long lastPreparedTxid() {
+        return lastId(PREPARE_TXID_KEY);
+    }
+
+    private Long lastId(String key) {
+        Long lastId = null;
+        String str = txIds.get(key);
+        if (str != null) {
+            lastId = Long.valueOf(str);
+        }
+        return lastId;
+    }
+
+    private String encode(byte[] bytes) {
+        return base64Encoder.encode(bytes);
+    }
+
+    private byte[] decode(String s) {
+        try {
+            return base64Decoder.decodeBuffer(s);
+        } catch (IOException ex) {
+            throw new RuntimeException("Error while decoding string " + s);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4623d8f8/external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueStateProvider.java
----------------------------------------------------------------------
diff --git a/external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueStateProvider.java b/external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueStateProvider.java
new file mode 100644
index 0000000..175110f
--- /dev/null
+++ b/external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueStateProvider.java
@@ -0,0 +1,120 @@
+/**
+ * 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.state;
+
+import backtype.storm.state.DefaultStateSerializer;
+import backtype.storm.state.Serializer;
+import backtype.storm.state.State;
+import backtype.storm.state.StateProvider;
+import backtype.storm.task.TopologyContext;
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.PropertyAccessor;
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.storm.redis.common.config.JedisPoolConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Provides {@link RedisKeyValueState}
+ */
+public class RedisKeyValueStateProvider implements StateProvider {
+    private static final Logger LOG = LoggerFactory.getLogger(RedisKeyValueStateProvider.class);
+
+    @Override
+    public State newState(String namespace, Map stormConf, TopologyContext context) {
+        try {
+            return getRedisKeyValueState(namespace, getStateConfig(stormConf));
+        } catch (Exception ex) {
+            LOG.error("Error loading config from storm conf {}", stormConf);
+            throw new RuntimeException(ex);
+        }
+    }
+
+    StateConfig getStateConfig(Map stormConf) throws Exception {
+        StateConfig stateConfig = null;
+        String providerConfig = null;
+        ObjectMapper mapper = new ObjectMapper();
+        mapper.setVisibility(PropertyAccessor.FIELD, JsonAutoDetect.Visibility.ANY);
+        if (stormConf.containsKey(backtype.storm.Config.TOPOLOGY_STATE_PROVIDER_CONFIG)) {
+            providerConfig = (String) stormConf.get(backtype.storm.Config.TOPOLOGY_STATE_PROVIDER_CONFIG);
+            stateConfig = mapper.readValue(providerConfig, StateConfig.class);
+        } else {
+            stateConfig = new StateConfig();
+        }
+        return stateConfig;
+    }
+
+    private RedisKeyValueState getRedisKeyValueState(String namespace, StateConfig config) throws Exception {
+        return new RedisKeyValueState(namespace, getJedisPoolConfig(config), getKeySerializer(config), getValueSerializer(config));
+    }
+
+    private Serializer getKeySerializer(StateConfig config) throws Exception {
+        Serializer serializer = null;
+        if (config.keySerializerClass != null) {
+            Class<?> klass = (Class<?>) Class.forName(config.keySerializerClass);
+            serializer = (Serializer) klass.newInstance();
+        } else if (config.keyClass != null) {
+            serializer = new DefaultStateSerializer(Collections.singletonList(Class.forName(config.keyClass)));
+        } else {
+            serializer = new DefaultStateSerializer();
+        }
+        return serializer;
+    }
+
+    private Serializer getValueSerializer(StateConfig config) throws Exception {
+        Serializer serializer = null;
+        if (config.valueSerializerClass != null) {
+            Class<?> klass = (Class<?>) Class.forName(config.valueSerializerClass);
+            serializer = (Serializer) klass.newInstance();
+        } else if (config.valueClass != null) {
+            serializer = new DefaultStateSerializer(Collections.singletonList(Class.forName(config.valueClass)));
+        } else {
+            serializer = new DefaultStateSerializer();
+        }
+        return serializer;
+    }
+
+    private JedisPoolConfig getJedisPoolConfig(StateConfig config) {
+        return config.jedisPoolConfig != null ? config.jedisPoolConfig : new JedisPoolConfig.Builder().build();
+    }
+
+    static class StateConfig {
+        String keyClass;
+        String valueClass;
+        String keySerializerClass;
+        String valueSerializerClass;
+        JedisPoolConfig jedisPoolConfig;
+
+        @Override
+        public String toString() {
+            return "StateConfig{" +
+                    "keyClass='" + keyClass + '\'' +
+                    ", valueClass='" + valueClass + '\'' +
+                    ", keySerializerClass='" + keySerializerClass + '\'' +
+                    ", valueSerializerClass='" + valueSerializerClass + '\'' +
+                    ", jedisPoolConfig=" + jedisPoolConfig +
+                    '}';
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4623d8f8/external/storm-redis/src/test/java/org/apache/storm/redis/state/DefaultStateSerializerTest.java
----------------------------------------------------------------------
diff --git a/external/storm-redis/src/test/java/org/apache/storm/redis/state/DefaultStateSerializerTest.java b/external/storm-redis/src/test/java/org/apache/storm/redis/state/DefaultStateSerializerTest.java
new file mode 100644
index 0000000..d8fbebe
--- /dev/null
+++ b/external/storm-redis/src/test/java/org/apache/storm/redis/state/DefaultStateSerializerTest.java
@@ -0,0 +1,56 @@
+/**
+ * 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.state;
+
+import backtype.storm.spout.CheckPointState;
+import backtype.storm.state.DefaultStateSerializer;
+import backtype.storm.state.Serializer;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import static org.junit.Assert.*;
+
+/**
+ * Unit tests for {@link DefaultStateSerializer}
+ */
+public class DefaultStateSerializerTest {
+
+    @Test
+    public void testSerializeDeserialize() throws Exception {
+        Serializer<Long> s1 = new DefaultStateSerializer<Long>();
+        byte[] bytes;
+        long val = 100;
+        bytes = s1.serialize(val);
+        assertEquals(val, (long) s1.deserialize(bytes));
+
+        Serializer<CheckPointState> s2 = new DefaultStateSerializer<CheckPointState>();
+        CheckPointState cs = new CheckPointState(100, CheckPointState.State.COMMITTED);
+        bytes = s2.serialize(cs);
+        assertEquals(cs, (CheckPointState) s2.deserialize(bytes));
+
+        List<Class<?>> classesToRegister = new ArrayList<>();
+        classesToRegister.add(CheckPointState.class);
+        Serializer<CheckPointState> s3 = new DefaultStateSerializer<CheckPointState>(classesToRegister);
+        bytes = s2.serialize(cs);
+        assertEquals(cs, (CheckPointState) s2.deserialize(bytes));
+
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/4623d8f8/external/storm-redis/src/test/java/org/apache/storm/redis/state/RedisKeyValueStateProviderTest.java
----------------------------------------------------------------------
diff --git a/external/storm-redis/src/test/java/org/apache/storm/redis/state/RedisKeyValueStateProviderTest.java b/external/storm-redis/src/test/java/org/apache/storm/redis/state/RedisKeyValueStateProviderTest.java
new file mode 100644
index 0000000..1746031
--- /dev/null
+++ b/external/storm-redis/src/test/java/org/apache/storm/redis/state/RedisKeyValueStateProviderTest.java
@@ -0,0 +1,62 @@
+/**
+ * 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.state;
+
+import backtype.storm.Config;
+import backtype.storm.state.State;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.*;
+
+/**
+ * Unit tests for {@link RedisKeyValueStateProvider}
+ */
+public class RedisKeyValueStateProviderTest {
+
+    @Test
+    public void testgetDefaultConfig() throws Exception {
+
+        RedisKeyValueStateProvider provider = new RedisKeyValueStateProvider();
+        Map<String, String> stormConf = new HashMap<>();
+        //stormConf.put(Config.TOPOLOGY_STATE_PROVIDER_CONFIG, "{\"keyClass\":\"String\"}");
+        RedisKeyValueStateProvider.StateConfig config = provider.getStateConfig(stormConf);
+        assertNotNull(config);
+    }
+
+
+    @Test
+    public void testgetConfigWithProviderConfig() throws Exception {
+        RedisKeyValueStateProvider provider = new RedisKeyValueStateProvider();
+        Map<String, String> stormConf = new HashMap<>();
+        stormConf.put(Config.TOPOLOGY_STATE_PROVIDER_CONFIG, "{\"keyClass\":\"String\", \"valueClass\":\"String\"," +
+                " \"jedisPoolConfig\":" +
+                "{\"host\":\"localhost\", \"port\":1000}}");
+
+        RedisKeyValueStateProvider.StateConfig config = provider.getStateConfig(stormConf);
+        //System.out.println(config);
+        assertEquals("String", config.keyClass);
+        assertEquals("String", config.valueClass);
+        assertEquals("localhost", config.jedisPoolConfig.getHost());
+        assertEquals(1000, config.jedisPoolConfig.getPort());
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/4623d8f8/external/storm-redis/src/test/java/org/apache/storm/redis/state/RedisKeyValueStateTest.java
----------------------------------------------------------------------
diff --git a/external/storm-redis/src/test/java/org/apache/storm/redis/state/RedisKeyValueStateTest.java b/external/storm-redis/src/test/java/org/apache/storm/redis/state/RedisKeyValueStateTest.java
new file mode 100644
index 0000000..80bab4c
--- /dev/null
+++ b/external/storm-redis/src/test/java/org/apache/storm/redis/state/RedisKeyValueStateTest.java
@@ -0,0 +1,157 @@
+/**
+ * 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.state;
+
+import backtype.storm.state.DefaultStateSerializer;
+import org.apache.storm.redis.common.container.JedisCommandsInstanceContainer;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import redis.clients.jedis.BinaryClient;
+import redis.clients.jedis.JedisCommands;
+import redis.clients.jedis.ScanResult;
+import redis.clients.jedis.SortingParams;
+import redis.clients.jedis.Tuple;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.junit.Assert.*;
+
+/**
+ * Unit tests for {@link RedisKeyValueState}
+ */
+public class RedisKeyValueStateTest {
+    JedisCommandsInstanceContainer mockContainer;
+    JedisCommands mockCommands;
+    RedisKeyValueState<String, String> keyValueState;
+
+    @Before
+    public void setUp() {
+        final Map<String, Map<String, String>> mockMap = new HashMap<>();
+        mockContainer = Mockito.mock(JedisCommandsInstanceContainer.class);
+        mockCommands = Mockito.mock(JedisCommands.class);
+        Mockito.when(mockContainer.getInstance()).thenReturn(mockCommands);
+        ArgumentCaptor<String> stringArgumentCaptor = ArgumentCaptor.forClass(String.class);
+        ArgumentCaptor<String> stringArgumentCaptor2 = ArgumentCaptor.forClass(String.class);
+        ArgumentCaptor<Map> mapArgumentCaptor = ArgumentCaptor.forClass(Map.class);
+
+        Mockito.when(mockCommands.exists(Mockito.anyString()))
+                .thenAnswer(new Answer<Boolean>() {
+                    @Override
+                    public Boolean answer(InvocationOnMock invocation) throws Throwable {
+                        Object[] args = invocation.getArguments();
+                        return mockMap.containsKey((String) args[0]);
+                    }
+                });
+
+
+        Mockito.when(mockCommands.hmset(Mockito.anyString(), Mockito.anyMap()))
+                .thenAnswer(new Answer<String>() {
+                    @Override
+                    public String answer(InvocationOnMock invocation) throws Throwable {
+                        Object[] args = invocation.getArguments();
+                        return hmset(mockMap, (String) args[0], (Map) args[1]);
+                    }
+                });
+
+        Mockito.when(mockCommands.del(Mockito.anyString()))
+                .thenAnswer(new Answer<Long>() {
+                    @Override
+                    public Long answer(InvocationOnMock invocation) throws Throwable {
+                        Object[] args = invocation.getArguments();
+                        return del(mockMap, (String) args[0]);
+                    }
+                });
+
+        Mockito.when(mockCommands.hget(Mockito.anyString(), Mockito.anyString()))
+                .thenAnswer(new Answer<String>() {
+                    @Override
+                    public String answer(InvocationOnMock invocation) throws Throwable {
+                        Object[] args = invocation.getArguments();
+                        return hget(mockMap, (String) args[0], (String) args[1]);
+                    }
+                });
+
+        keyValueState = new RedisKeyValueState<String, String>("test", mockContainer, new DefaultStateSerializer<String>(),
+                                                               new DefaultStateSerializer<String>());
+    }
+
+
+    @Test
+    public void testPutAndGet() throws Exception {
+        keyValueState.put("a", "1");
+        keyValueState.put("b", "2");
+        assertEquals("1", keyValueState.get("a"));
+        assertEquals("2", keyValueState.get("b"));
+        assertEquals(null, keyValueState.get("c"));
+    }
+
+    @Test
+    public void testPrepareCommitRollback() throws Exception {
+        keyValueState.put("a", "1");
+        keyValueState.put("b", "2");
+        keyValueState.prepareCommit(1);
+        keyValueState.put("c", "3");
+        assertEquals("1", keyValueState.get("a"));
+        assertEquals("2", keyValueState.get("b"));
+        assertEquals("3", keyValueState.get("c"));
+        keyValueState.rollback();
+        assertEquals(null, keyValueState.get("a"));
+        assertEquals(null, keyValueState.get("b"));
+        assertEquals(null, keyValueState.get("c"));
+
+        keyValueState.put("a", "1");
+        keyValueState.put("b", "2");
+        keyValueState.prepareCommit(1);
+        keyValueState.commit(1);
+        keyValueState.put("c", "3");
+        assertEquals("1", keyValueState.get("a"));
+        assertEquals("2", keyValueState.get("b"));
+        assertEquals("3", keyValueState.get("c"));
+        keyValueState.rollback();
+        assertEquals("1", keyValueState.get("a"));
+        assertEquals("2", keyValueState.get("b"));
+        assertEquals(null, keyValueState.get("c"));
+
+    }
+
+
+    private String hmset(Map<String, Map<String, String>> mockMap, String key, Map value) {
+        mockMap.put(key, value);
+        return "";
+    }
+
+    private Long del(Map<String, Map<String, String>> mockMap, String key) {
+        mockMap.remove(key);
+        return 0L;
+    }
+
+    private String hget(Map<String, Map<String, String>> mockMap, String namespace, String key) {
+        if (mockMap.containsKey(namespace)) {
+            return mockMap.get(namespace).get(key);
+        }
+        return null;
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/4623d8f8/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/executor.clj b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
index c65f5d8..8052835 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
@@ -174,6 +174,8 @@
                         TOPOLOGY-BOLTS-SLIDING-INTERVAL-DURATION-MS
                         TOPOLOGY-BOLTS-TUPLE-TIMESTAMP-FIELD-NAME
                         TOPOLOGY-BOLTS-TUPLE-TIMESTAMP-MAX-LAG-MS
+                        TOPOLOGY-STATE-PROVIDER
+                        TOPOLOGY-STATE-PROVIDER-CONFIG
                         )
         spec-conf (-> general-context
                       (.getComponentCommon component-id)

http://git-wip-us.apache.org/repos/asf/storm/blob/4623d8f8/storm-core/src/jvm/backtype/storm/spout/CheckPointState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/spout/CheckPointState.java b/storm-core/src/jvm/backtype/storm/spout/CheckPointState.java
new file mode 100644
index 0000000..5ad9772
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/spout/CheckPointState.java
@@ -0,0 +1,78 @@
+/**
+ * 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 backtype.storm.spout;
+
+/**
+ * Captures the current state of the transaction in
+ * {@link CheckpointSpout}
+ */
+public class CheckPointState {
+    public long txid;
+    public State state;
+
+    public enum State {
+        /**
+         * The checkpoint spout has committed the transaction.
+         */
+        COMMITTED,
+        /**
+         * The checkpoint spout has started committing the transaction.
+         */
+        COMMITTING,
+        /**
+         * The checkpoint spout has started preparing the transaction for commit.
+         */
+        PREPARING
+    }
+
+    public CheckPointState(long txid, State state) {
+        this.txid = txid;
+        this.state = state;
+    }
+
+    // for kryo
+    public CheckPointState() {
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        CheckPointState that = (CheckPointState) o;
+
+        if (txid != that.txid) return false;
+        return state == that.state;
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result = (int) (txid ^ (txid >>> 32));
+        result = 31 * result + (state != null ? state.hashCode() : 0);
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "CheckPointState{" +
+                "txid=" + txid +
+                ", state=" + state +
+                '}';
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4623d8f8/storm-core/src/jvm/backtype/storm/spout/CheckpointSpout.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/spout/CheckpointSpout.java b/storm-core/src/jvm/backtype/storm/spout/CheckpointSpout.java
new file mode 100644
index 0000000..929dd12
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/spout/CheckpointSpout.java
@@ -0,0 +1,280 @@
+/**
+ * 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 backtype.storm.spout;
+
+import backtype.storm.Config;
+import backtype.storm.state.KeyValueState;
+import backtype.storm.state.StateFactory;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.topology.base.BaseRichSpout;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.tuple.Values;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import static backtype.storm.spout.CheckPointState.State.COMMITTED;
+import static backtype.storm.spout.CheckPointState.State.COMMITTING;
+import static backtype.storm.spout.CheckPointState.State.PREPARING;
+
+/**
+ * Emits checkpoint tuples which is used to save the state of the {@link backtype.storm.topology.IStatefulComponent}
+ * across the topology. If a topology contains Stateful bolts, Checkpoint spouts are automatically added
+ * to the topology. There is only one Checkpoint task per topology.
+ * <p/>
+ * Checkpoint spout stores its internal state in a {@link KeyValueState}. The state transitions are as follows.
+ * <p/>
+ * <pre>
+ *                  ROLLBACK(tx2)
+ *               <-------------                  PREPARE(tx2)                     COMMIT(tx2)
+ * COMMITTED(tx1)-------------> PREPARING(tx2) --------------> COMMITTING(tx2) -----------------> COMMITTED (tx2)
+ *
+ *
+ * </pre>
+ */
+public class CheckpointSpout extends BaseRichSpout {
+    private static final Logger LOG = LoggerFactory.getLogger(CheckpointSpout.class);
+
+    public static final String CHECKPOINT_STREAM_ID = "$checkpoint";
+    public static final String CHECKPOINT_COMPONENT_ID = "$checkpointspout";
+    public static final String CHECKPOINT_FIELD_TXID = "txid";
+    public static final String CHECKPOINT_FIELD_ACTION = "action";
+    public static final String CHECKPOINT_ACTION_PREPARE = "prepare";
+    public static final String CHECKPOINT_ACTION_COMMIT = "commit";
+    public static final String CHECKPOINT_ACTION_ROLLBACK = "rollback";
+    public static final String CHECKPOINT_ACTION_INITSTATE = "initstate";
+
+    private static final String TX_STATE_KEY = "__state";
+    private static final int DEFAULT_CHECKPOINT_INTERVAL = 1000; // every sec
+
+    private TopologyContext context;
+    private SpoutOutputCollector collector;
+    private long lastCheckpointTs;
+    private int checkpointInterval;
+    private boolean recoveryStepInProgress;
+    private boolean checkpointStepInProgress;
+    private boolean recovering;
+    private KeyValueState<String, CheckPointState> checkpointState;
+
+    @Override
+    public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
+        open(context, collector, loadCheckpointInterval(conf), loadCheckpointState(conf, context));
+    }
+
+    // package access for unit test
+    void open(TopologyContext context, SpoutOutputCollector collector,
+              int checkpointInterval, KeyValueState<String, CheckPointState> checkpointState) {
+        this.context = context;
+        this.collector = collector;
+        this.checkpointInterval = checkpointInterval;
+        this.checkpointState = checkpointState;
+        lastCheckpointTs = 0;
+        recoveryStepInProgress = false;
+        checkpointStepInProgress = false;
+        recovering = true;
+    }
+
+    @Override
+    public void nextTuple() {
+        if (shouldRecover()) {
+            LOG.debug("In recovery");
+            handleRecovery();
+            startProgress();
+        } else if (shouldCheckpoint()) {
+            LOG.debug("In checkpoint");
+            doCheckpoint();
+            startProgress();
+        }
+    }
+
+    @Override
+    public void ack(Object msgId) {
+        CheckPointState txState = getTxState();
+        LOG.debug("Got ack with txid {}, current txState {}", msgId, txState);
+        if (txState.txid == ((Number) msgId).longValue()) {
+            if (recovering) {
+                handleRecoveryAck();
+            } else {
+                handleCheckpointAck();
+            }
+        } else {
+            LOG.warn("Ack msgid {}, txState.txid {} mismatch", msgId, txState.txid);
+        }
+        resetProgress();
+    }
+
+    @Override
+    public void fail(Object msgId) {
+        LOG.debug("Got fail with msgid {}", msgId);
+        resetProgress();
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        declarer.declareStream(CHECKPOINT_STREAM_ID, new Fields(CHECKPOINT_FIELD_TXID, CHECKPOINT_FIELD_ACTION));
+    }
+
+    @Override
+    public Map<String, Object> getComponentConfiguration() {
+        Config conf = new Config();
+        conf.put(Config.TOPOLOGY_SLEEP_SPOUT_WAIT_STRATEGY_TIME_MS, 100);
+        return conf;
+    }
+
+    public static boolean isCheckpoint(Tuple input) {
+        return CHECKPOINT_STREAM_ID.equals(input.getSourceStreamId());
+    }
+
+    /**
+     * Loads the last saved checkpoint state the from persistent storage.
+     */
+    private KeyValueState<String, CheckPointState> loadCheckpointState(Map conf, TopologyContext ctx) {
+        String namespace = ctx.getThisComponentId() + "-" + ctx.getThisTaskId();
+        KeyValueState<String, CheckPointState> state =
+                (KeyValueState<String, CheckPointState>) StateFactory.getState(namespace, conf, ctx);
+        if (state.get(TX_STATE_KEY) == null) {
+            CheckPointState txState = new CheckPointState(-1, COMMITTED);
+            state.put(TX_STATE_KEY, txState);
+            state.commit();
+            LOG.debug("Initialized checkpoint spout state with txState {}", txState);
+        } else {
+            LOG.debug("Got checkpoint spout state {}", state.get(TX_STATE_KEY));
+        }
+        return state;
+    }
+
+    private int loadCheckpointInterval(Map stormConf) {
+        int interval;
+        if (stormConf.containsKey(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL)) {
+            interval = ((Number) stormConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL)).intValue();
+        } else {
+            interval = DEFAULT_CHECKPOINT_INTERVAL;
+        }
+        return interval;
+    }
+
+    private boolean shouldRecover() {
+        return recovering && !recoveryStepInProgress;
+    }
+
+    private boolean shouldCheckpoint() {
+        return !recovering && !checkpointStepInProgress
+                && (System.currentTimeMillis() - lastCheckpointTs) > checkpointInterval;
+    }
+
+    private boolean shouldRollback(CheckPointState txState) {
+        return txState.state == PREPARING;
+    }
+
+    private boolean shouldCommit(CheckPointState txState) {
+        return txState.state == COMMITTING;
+    }
+
+    private boolean shouldInitState(CheckPointState txState) {
+        return txState.state == COMMITTED;
+    }
+
+    private void handleRecovery() {
+        CheckPointState txState = getTxState();
+        LOG.debug("Current txState is {}", txState);
+        if (shouldRollback(txState)) {
+            LOG.debug("Emitting rollback with txid {}", txState.txid);
+            collector.emit(CHECKPOINT_STREAM_ID, new Values(txState.txid, CHECKPOINT_ACTION_ROLLBACK), txState.txid);
+        } else if (shouldCommit(txState)) {
+            LOG.debug("Emitting commit with txid {}", txState.txid);
+            collector.emit(CHECKPOINT_STREAM_ID, new Values(txState.txid, CHECKPOINT_ACTION_COMMIT), txState.txid);
+        } else if (shouldInitState(txState)) {
+            LOG.debug("Emitting init state with txid {}", txState.txid);
+            collector.emit(CHECKPOINT_STREAM_ID, new Values(txState.txid, CHECKPOINT_ACTION_INITSTATE), txState.txid);
+        }
+        startProgress();
+    }
+
+    private void handleRecoveryAck() {
+        CheckPointState txState = getTxState();
+        if (shouldRollback(txState)) {
+            txState.state = COMMITTED;
+            --txState.txid;
+            saveTxState(txState);
+        } else if (shouldCommit(txState)) {
+            txState.state = COMMITTED;
+            saveTxState(txState);
+        } else if (shouldInitState(txState)) {
+            LOG.debug("Recovery complete, current state {}", txState);
+            recovering = false;
+        }
+    }
+
+    private void doCheckpoint() {
+        CheckPointState txState = getTxState();
+        if (txState.state == COMMITTED) {
+            txState.txid++;
+            txState.state = PREPARING;
+            saveTxState(txState);
+            lastCheckpointTs = System.currentTimeMillis();
+            LOG.debug("Emitting prepare with txid {}", txState.txid);
+            collector.emit(CHECKPOINT_STREAM_ID, new Values(txState.txid, CHECKPOINT_ACTION_PREPARE), txState.txid);
+        } else if (txState.state == PREPARING) {
+            LOG.debug("Emitting prepare with txid {}", txState.txid);
+            collector.emit(CHECKPOINT_STREAM_ID, new Values(txState.txid, CHECKPOINT_ACTION_PREPARE), txState.txid);
+        } else if (txState.state == COMMITTING) {
+            LOG.debug("Emitting commit with txid {}", txState.txid);
+            collector.emit(CHECKPOINT_STREAM_ID, new Values(txState.txid, CHECKPOINT_ACTION_COMMIT), txState.txid);
+        }
+        startProgress();
+    }
+
+    private void handleCheckpointAck() {
+        CheckPointState txState = getTxState();
+        if (txState.state == PREPARING) {
+            txState.state = COMMITTING;
+            LOG.debug("Prepare txid {} complete", txState.txid);
+        } else if (txState.state == COMMITTING) {
+            txState.state = COMMITTED;
+            LOG.debug("Commit txid {} complete", txState.txid);
+        }
+        saveTxState(txState);
+    }
+
+    private void saveTxState(CheckPointState txState) {
+        checkpointState.put(TX_STATE_KEY, txState);
+        checkpointState.commit();
+    }
+
+    private CheckPointState getTxState() {
+        return checkpointState.get(TX_STATE_KEY);
+    }
+
+    private void startProgress() {
+        if (recovering) {
+            recoveryStepInProgress = true;
+        } else {
+            checkpointStepInProgress = true;
+        }
+    }
+
+    private void resetProgress() {
+        if (recovering) {
+            recoveryStepInProgress = false;
+        } else {
+            checkpointStepInProgress = false;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4623d8f8/storm-core/src/jvm/backtype/storm/state/DefaultStateSerializer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/state/DefaultStateSerializer.java b/storm-core/src/jvm/backtype/storm/state/DefaultStateSerializer.java
new file mode 100644
index 0000000..1e57a1d
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/state/DefaultStateSerializer.java
@@ -0,0 +1,65 @@
+/**
+ * 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 backtype.storm.state;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * A default implementation that uses Kryo to serialize and de-serialize
+ * the state.
+ */
+public class DefaultStateSerializer<T> implements Serializer<T> {
+    private final Kryo kryo;
+    private final Output output;
+
+    /**
+     * Constructs a {@link DefaultStateSerializer} instance with the given list
+     * of classes registered in kryo.
+     *
+     * @param classesToRegister the classes to register.
+     */
+    public DefaultStateSerializer(List<Class<?>> classesToRegister) {
+        kryo = new Kryo();
+        output = new Output(2000, 2000000000);
+        for (Class<?> klazz : classesToRegister) {
+            kryo.register(klazz);
+        }
+    }
+
+    public DefaultStateSerializer() {
+        this(Collections.<Class<?>>emptyList());
+    }
+
+    @Override
+    public byte[] serialize(T obj) {
+        output.clear();
+        kryo.writeClassAndObject(output, obj);
+        return output.toBytes();
+    }
+
+    @Override
+    public T deserialize(byte[] b) {
+        Input input = new Input(b);
+        return (T) kryo.readClassAndObject(input);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4623d8f8/storm-core/src/jvm/backtype/storm/state/InMemoryKeyValueState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/state/InMemoryKeyValueState.java b/storm-core/src/jvm/backtype/storm/state/InMemoryKeyValueState.java
new file mode 100644
index 0000000..a516d34
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/state/InMemoryKeyValueState.java
@@ -0,0 +1,114 @@
+/**
+ * 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 backtype.storm.state;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * An in-memory implementation of the {@link State}
+ */
+public class InMemoryKeyValueState<K, V> implements KeyValueState<K, V> {
+    private static final Logger LOG = LoggerFactory.getLogger(InMemoryKeyValueState.class);
+    private static final long DEFAULT_TXID = -1;
+    private TxIdState<K, V> commitedState;
+    private TxIdState<K, V> preparedState;
+    private Map<K, V> state = new HashMap<>();
+
+    private static class TxIdState<K, V> {
+        private long txid;
+        private Map<K, V> state;
+
+        TxIdState(long txid, Map<K, V> state) {
+            this.txid = txid;
+            this.state = state;
+        }
+
+        @Override
+        public String toString() {
+            return "TxIdState{" +
+                    "txid=" + txid +
+                    ", state=" + state +
+                    '}';
+        }
+    }
+
+    @Override
+    public void put(K key, V value) {
+        state.put(key, value);
+    }
+
+    @Override
+    public V get(K key) {
+        return state.get(key);
+    }
+
+    @Override
+    public V get(K key, V defaultValue) {
+        V val = get(key);
+        return val != null ? val : defaultValue;
+    }
+
+    @Override
+    public void commit() {
+        commitedState = new TxIdState<>(DEFAULT_TXID, new HashMap<K, V>(state));
+    }
+
+    @Override
+    public void prepareCommit(long txid) {
+        LOG.debug("prepare commit, txid {}", txid);
+        if (preparedState != null && txid > preparedState.txid) {
+            throw new RuntimeException("Cannot prepare a new txn while there is a pending txn");
+        }
+        preparedState = new TxIdState<>(txid, new HashMap<K, V>(state));
+    }
+
+    @Override
+    public void commit(long txid) {
+        LOG.debug("commit, txid {}", txid);
+        if (preparedState != null && txid == preparedState.txid) {
+            commitedState = preparedState;
+            preparedState = null;
+        } else {
+            throw new RuntimeException("Invalid prepared state for commit, " +
+                                               "preparedState " + preparedState + " txid " + txid);
+        }
+    }
+
+    @Override
+    public void rollback() {
+        preparedState = null;
+        if (commitedState != null) {
+            state = commitedState.state;
+        } else {
+            state = new HashMap<>();
+        }
+    }
+
+    @Override
+    public String toString() {
+        return "InMemoryKeyValueState{" +
+                "commitedState=" + commitedState +
+                ", preparedState=" + preparedState +
+                ", state=" + state +
+                '}';
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4623d8f8/storm-core/src/jvm/backtype/storm/state/InMemoryKeyValueStateProvider.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/state/InMemoryKeyValueStateProvider.java b/storm-core/src/jvm/backtype/storm/state/InMemoryKeyValueStateProvider.java
new file mode 100644
index 0000000..1a79e72
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/state/InMemoryKeyValueStateProvider.java
@@ -0,0 +1,33 @@
+/**
+ * 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 backtype.storm.state;
+
+import backtype.storm.task.TopologyContext;
+
+import java.util.Map;
+
+/**
+ * Provides {@link InMemoryKeyValueState}
+ */
+public class InMemoryKeyValueStateProvider implements StateProvider {
+
+    @Override
+    public State newState(String namespace, Map stormConf, TopologyContext context) {
+        return new InMemoryKeyValueState();
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4623d8f8/storm-core/src/jvm/backtype/storm/state/KeyValueState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/state/KeyValueState.java b/storm-core/src/jvm/backtype/storm/state/KeyValueState.java
new file mode 100644
index 0000000..3ef0707
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/state/KeyValueState.java
@@ -0,0 +1,48 @@
+/**
+ * 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 backtype.storm.state;
+
+/**
+ * A state that supports key-value mappings.
+ */
+public interface KeyValueState<K, V> extends State {
+    /**
+     * Maps the value with the key
+     *
+     * @param key   the key
+     * @param value the value
+     */
+    void put(K key, V value);
+
+    /**
+     * Returns the value mapped to the key
+     *
+     * @param key the key
+     * @return the value or null if no mapping is found
+     */
+    V get(K key);
+
+    /**
+     * Returns the value mapped to the key or defaultValue if no mapping is found.
+     *
+     * @param key          the key
+     * @param defaultValue the value to return if no mapping is found
+     * @return the value or defaultValue if no mapping is found
+     */
+    V get(K key, V defaultValue);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4623d8f8/storm-core/src/jvm/backtype/storm/state/Serializer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/state/Serializer.java b/storm-core/src/jvm/backtype/storm/state/Serializer.java
new file mode 100644
index 0000000..1761621
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/state/Serializer.java
@@ -0,0 +1,30 @@
+/**
+ * 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 backtype.storm.state;
+
+import java.io.Serializable;
+
+/**
+ * Interface to be implemented for serlializing and de-serializing the
+ * state.
+ */
+public interface Serializer<T> extends Serializable {
+    byte[] serialize(T obj);
+
+    T deserialize(byte[] b);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4623d8f8/storm-core/src/jvm/backtype/storm/state/State.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/state/State.java b/storm-core/src/jvm/backtype/storm/state/State.java
new file mode 100644
index 0000000..4a837b4
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/state/State.java
@@ -0,0 +1,53 @@
+/**
+ * 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 backtype.storm.state;
+
+/**
+ * The state of the component that is either managed by the framework (e.g in case of {@link backtype.storm.topology.IStatefulBolt})
+ * or managed by the the individual components themselves.
+ */
+public interface State {
+    /**
+     * Invoked by the framework to prepare a transaction for commit. It should be possible
+     * to commit the prepared state later.
+     * <p>
+     * The same txid can be prepared again, but the next txid cannot be prepared
+     * when previous one is not yet committed.
+     * </p>
+     *
+     * @param txid the transaction id
+     */
+    void prepareCommit(long txid);
+
+    /**
+     * Commit a previously prepared transaction. It should be possible to retrieve a committed state later.
+     *
+     * @param txid the transaction id
+     */
+    void commit(long txid);
+
+    /**
+     * Persist the current state. This is used when the component manages the state.
+     */
+    void commit();
+
+    /**
+     * Rollback a prepared transaction to the previously committed state.
+     */
+    void rollback();
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4623d8f8/storm-core/src/jvm/backtype/storm/state/StateFactory.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/state/StateFactory.java b/storm-core/src/jvm/backtype/storm/state/StateFactory.java
new file mode 100644
index 0000000..e0ea548
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/state/StateFactory.java
@@ -0,0 +1,69 @@
+/**
+ * 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 backtype.storm.state;
+
+import backtype.storm.Config;
+import backtype.storm.task.TopologyContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+
+/**
+ * A factory for creating {@link State} instances
+ */
+public class StateFactory {
+    private static final Logger LOG = LoggerFactory.getLogger(StateFactory.class);
+
+    private static final String DEFAULT_PROVIDER = "backtype.storm.state.InMemoryKeyValueStateProvider";
+
+    /**
+     * Returns a new state instance using the {@link Config#TOPOLOGY_STATE_PROVIDER} or a
+     * {@link InMemoryKeyValueState} if no provider is configured.
+     *
+     * @param namespace the state namespace
+     * @param stormConf the storm conf
+     * @param context   the topology context
+     * @return the state instance
+     */
+    public static State getState(String namespace, Map stormConf, TopologyContext context) {
+        State state;
+        try {
+            String provider = null;
+            if (stormConf.containsKey(Config.TOPOLOGY_STATE_PROVIDER)) {
+                provider = (String) stormConf.get(Config.TOPOLOGY_STATE_PROVIDER);
+            } else {
+                provider = DEFAULT_PROVIDER;
+            }
+            Class<?> klazz = Class.forName(provider);
+            Object object = klazz.newInstance();
+            if (object instanceof StateProvider) {
+                state = ((StateProvider) object).newState(namespace, stormConf, context);
+            } else {
+                String msg = "Invalid state provider '" + provider +
+                        "'. Should implement backtype.storm.state.StateProvider";
+                LOG.error(msg);
+                throw new RuntimeException(msg);
+            }
+        } catch (Exception ex) {
+            LOG.error("Got exception while loading the state provider", ex);
+            throw new RuntimeException(ex);
+        }
+        return state;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4623d8f8/storm-core/src/jvm/backtype/storm/state/StateProvider.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/state/StateProvider.java b/storm-core/src/jvm/backtype/storm/state/StateProvider.java
new file mode 100644
index 0000000..d10141e
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/state/StateProvider.java
@@ -0,0 +1,38 @@
+/**
+ * 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 backtype.storm.state;
+
+import backtype.storm.task.TopologyContext;
+
+import java.util.Map;
+
+/**
+ * Used by the {@link StateFactory} to create a new state instances.
+ */
+public interface StateProvider {
+    /**
+     * Returns a new state instance. Each state belongs unique namespace which is typically
+     * the componentid-task of the task, so that each task can have its own unique state.
+     *
+     * @param namespace a namespace of the state
+     * @param stormConf the storm topology configuration
+     * @param context   the {@link TopologyContext}
+     * @return a previously saved state instance
+     */
+    State newState(String namespace, Map stormConf, TopologyContext context);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4623d8f8/storm-core/src/jvm/backtype/storm/topology/CheckpointTupleForwarder.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/topology/CheckpointTupleForwarder.java b/storm-core/src/jvm/backtype/storm/topology/CheckpointTupleForwarder.java
new file mode 100644
index 0000000..2d25e72
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/topology/CheckpointTupleForwarder.java
@@ -0,0 +1,218 @@
+/**
+ * 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 backtype.storm.topology;
+
+import backtype.storm.generated.GlobalStreamId;
+import backtype.storm.spout.CheckpointSpout;
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.tuple.Values;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static backtype.storm.spout.CheckpointSpout.*;
+
+/**
+ * Wraps {@link IRichBolt} and forwards checkpoint tuples in a
+ * stateful topology.
+ * <p>
+ * When a storm topology contains one or more {@link IStatefulBolt} all non-stateful
+ * bolts are wrapped in {@link CheckpointTupleForwarder} so that the checkpoint tuples
+ * can flow through the entire topology DAG.
+ * </p>
+ */
+public class CheckpointTupleForwarder implements IRichBolt {
+    private static final Logger LOG = LoggerFactory.getLogger(CheckpointTupleForwarder.class);
+    private final IRichBolt bolt;
+    private final Map<TransactionRequest, Integer> transactionRequestCount;
+    private int checkPointInputTaskCount;
+    private long lastTxid = Long.MIN_VALUE;
+    protected OutputCollector collector;
+
+    public CheckpointTupleForwarder(IRichBolt bolt) {
+        this.bolt = bolt;
+        transactionRequestCount = new HashMap<>();
+    }
+
+    @Override
+    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+        bolt.prepare(stormConf, context, collector);
+        this.collector = collector;
+        checkPointInputTaskCount = getCheckpointInputTaskCount(context);
+    }
+
+    @Override
+    public void execute(Tuple input) {
+        if (CheckpointSpout.isCheckpoint(input)) {
+            processCheckpoint(input);
+        } else {
+            handleTuple(input);
+        }
+    }
+
+    @Override
+    public void cleanup() {
+        bolt.cleanup();
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        bolt.declareOutputFields(declarer);
+        declarer.declareStream(CHECKPOINT_STREAM_ID, new Fields(CHECKPOINT_FIELD_TXID, CHECKPOINT_FIELD_ACTION));
+    }
+
+    @Override
+    public Map<String, Object> getComponentConfiguration() {
+        return bolt.getComponentConfiguration();
+    }
+
+    /**
+     * Forwards the checkpoint tuple downstream. Sub-classes can override
+     * with the logic for handling checkpoint tuple.
+     *
+     * @param input  the checkpoint tuple
+     * @param action the action (prepare, commit, rollback or initstate)
+     * @param txid   the transaction id.
+     */
+    protected void handleCheckpoint(Tuple input, String action, long txid) {
+        collector.emit(CHECKPOINT_STREAM_ID, input, new Values(txid, action));
+    }
+
+    /**
+     * Hands off tuple to the wrapped bolt to execute. Sub-classes can
+     * override the behavior.
+     * <p>
+     * Right now tuples continue to get forwarded while waiting for checkpoints to arrive on other streams
+     * after checkpoint arrives on one of the streams. This can cause duplicates but still at least once.
+     * </p>
+     *
+     * @param input the input tuple
+     */
+    protected void handleTuple(Tuple input) {
+        bolt.execute(input);
+    }
+
+    /**
+     * Invokes handleCheckpoint once checkpoint tuple is received on
+     * all input checkpoint streams to this component.
+     */
+    private void processCheckpoint(Tuple input) {
+        String action = input.getStringByField(CHECKPOINT_FIELD_ACTION);
+        long txid = input.getLongByField(CHECKPOINT_FIELD_TXID);
+        if (shouldProcessTransaction(action, txid)) {
+            LOG.debug("Processing action {}, txid {}", action, txid);
+            try {
+                if (txid >= lastTxid) {
+                    handleCheckpoint(input, action, txid);
+                    if (CHECKPOINT_ACTION_ROLLBACK.equals(action)) {
+                        lastTxid = txid - 1;
+                    } else {
+                        lastTxid = txid;
+                    }
+                } else {
+                    LOG.debug("Ignoring old transaction. Action {}, txid {}", action, txid);
+                }
+            } catch (Throwable th) {
+                LOG.error("Got error while processing checkpoint tuple", th);
+                collector.fail(input);
+                collector.reportError(th);
+            }
+        } else {
+            LOG.debug("Waiting for action {}, txid {} from all input tasks. checkPointInputTaskCount {}, " +
+                              "transactionRequestCount {}", action, txid, checkPointInputTaskCount, transactionRequestCount);
+        }
+        collector.ack(input);
+    }
+
+    /**
+     * returns the total number of input checkpoint streams across
+     * all input tasks to this component.
+     */
+    private int getCheckpointInputTaskCount(TopologyContext context) {
+        int count = 0;
+        for (GlobalStreamId inputStream : context.getThisSources().keySet()) {
+            if (CHECKPOINT_STREAM_ID.equals(inputStream.get_streamId())) {
+                count += context.getComponentTasks(inputStream.get_componentId()).size();
+            }
+        }
+        return count;
+    }
+
+    /**
+     * Checks if check points have been received from all tasks across
+     * all input streams to this component
+     */
+    private boolean shouldProcessTransaction(String action, long txid) {
+        TransactionRequest request = new TransactionRequest(action, txid);
+        Integer count;
+        if ((count = transactionRequestCount.get(request)) == null) {
+            transactionRequestCount.put(request, 1);
+            count = 1;
+        } else {
+            transactionRequestCount.put(request, ++count);
+        }
+        if (count == checkPointInputTaskCount) {
+            transactionRequestCount.remove(request);
+            return true;
+        }
+        return false;
+    }
+
+    private static class TransactionRequest {
+        private final String action;
+        private final long txid;
+
+        TransactionRequest(String action, long txid) {
+            this.action = action;
+            this.txid = txid;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+
+            TransactionRequest that = (TransactionRequest) o;
+
+            if (txid != that.txid) return false;
+            return !(action != null ? !action.equals(that.action) : that.action != null);
+
+        }
+
+        @Override
+        public int hashCode() {
+            int result = action != null ? action.hashCode() : 0;
+            result = 31 * result + (int) (txid ^ (txid >>> 32));
+            return result;
+        }
+
+        @Override
+        public String toString() {
+            return "TransactionRequest{" +
+                    "action='" + action + '\'' +
+                    ", txid=" + txid +
+                    '}';
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4623d8f8/storm-core/src/jvm/backtype/storm/topology/IStatefulBolt.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/topology/IStatefulBolt.java b/storm-core/src/jvm/backtype/storm/topology/IStatefulBolt.java
new file mode 100644
index 0000000..56408bd
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/topology/IStatefulBolt.java
@@ -0,0 +1,26 @@
+/**
+ * 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 backtype.storm.topology;
+
+import backtype.storm.state.State;
+
+/**
+ * A bolt abstraction for supporting stateful computation.
+ */
+public interface IStatefulBolt<T extends State> extends IStatefulComponent<T>, IRichBolt {
+}


[09/10] storm git commit: Package name fixes

Posted by pt...@apache.org.
Package name fixes

Since backtype.storm got renamed to org.apache.storm in master


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

Branch: refs/heads/master
Commit: bfc5ffd57d7373a477744663401a1c66c4f93c7a
Parents: 16d58e3
Author: Arun Mahadevan <ai...@hortonworks.com>
Authored: Tue Jan 12 11:25:30 2016 +0530
Committer: Arun Mahadevan <ai...@hortonworks.com>
Committed: Tue Jan 12 11:25:30 2016 +0530

----------------------------------------------------------------------
 docs/documentation/State-checkpointing.md       |   4 +-
 .../src/jvm/storm/starter/StatefulTopology.java |  36 +--
 .../storm/redis/state/RedisKeyValueState.java   |   6 +-
 .../redis/state/RedisKeyValueStateProvider.java |  14 +-
 .../redis/state/DefaultStateSerializerTest.java |   6 +-
 .../state/RedisKeyValueStateProviderTest.java   |   4 +-
 .../redis/state/RedisKeyValueStateTest.java     |   2 +-
 .../backtype/storm/spout/CheckPointState.java   | 172 ------------
 .../backtype/storm/spout/CheckpointSpout.java   | 232 ----------------
 .../storm/state/DefaultStateSerializer.java     |  65 -----
 .../storm/state/InMemoryKeyValueState.java      | 114 --------
 .../state/InMemoryKeyValueStateProvider.java    |  43 ---
 .../jvm/backtype/storm/state/KeyValueState.java |  48 ----
 .../jvm/backtype/storm/state/Serializer.java    |  30 ---
 .../src/jvm/backtype/storm/state/State.java     |  53 ----
 .../jvm/backtype/storm/state/StateFactory.java  |  69 -----
 .../jvm/backtype/storm/state/StateProvider.java |  38 ---
 .../topology/CheckpointTupleForwarder.java      | 252 ------------------
 .../backtype/storm/topology/IStatefulBolt.java  |  26 --
 .../storm/topology/IStatefulComponent.java      |  56 ----
 .../storm/topology/StatefulBoltExecutor.java    | 151 -----------
 .../storm/topology/base/BaseStatefulBolt.java   |  64 -----
 storm-core/src/jvm/org/apache/storm/Config.java |   8 +-
 .../org/apache/storm/spout/CheckPointState.java | 172 ++++++++++++
 .../org/apache/storm/spout/CheckpointSpout.java | 232 ++++++++++++++++
 .../storm/state/DefaultStateSerializer.java     |  65 +++++
 .../storm/state/InMemoryKeyValueState.java      | 114 ++++++++
 .../state/InMemoryKeyValueStateProvider.java    |  43 +++
 .../org/apache/storm/state/KeyValueState.java   |  48 ++++
 .../jvm/org/apache/storm/state/Serializer.java  |  30 +++
 .../src/jvm/org/apache/storm/state/State.java   |  55 ++++
 .../org/apache/storm/state/StateFactory.java    |  69 +++++
 .../org/apache/storm/state/StateProvider.java   |  38 +++
 .../topology/CheckpointTupleForwarder.java      | 250 ++++++++++++++++++
 .../apache/storm/topology/IStatefulBolt.java    |  26 ++
 .../storm/topology/IStatefulComponent.java      |  56 ++++
 .../storm/topology/StatefulBoltExecutor.java    | 151 +++++++++++
 .../storm/topology/base/BaseStatefulBolt.java   |  64 +++++
 .../storm/spout/CheckpointSpoutTest.java        | 263 -------------------
 .../storm/state/InMemoryKeyValueStateTest.java  |  70 -----
 .../topology/StatefulBoltExecutorTest.java      | 175 ------------
 .../apache/storm/spout/CheckpointSpoutTest.java | 262 ++++++++++++++++++
 .../storm/state/InMemoryKeyValueStateTest.java  |  72 +++++
 .../topology/StatefulBoltExecutorTest.java      | 177 +++++++++++++
 44 files changed, 1964 insertions(+), 1961 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/docs/documentation/State-checkpointing.md
----------------------------------------------------------------------
diff --git a/docs/documentation/State-checkpointing.md b/docs/documentation/State-checkpointing.md
index 88e7d68..889b387 100644
--- a/docs/documentation/State-checkpointing.md
+++ b/docs/documentation/State-checkpointing.md
@@ -133,12 +133,12 @@ stateful bolt's state is prepared, committed or rolled back.
 ## Providing custom state implementations
 Currently the only kind of `State` implementation supported is `KeyValueState` which provides key-value mapping.
 
-Custom state implementations should provide implementations for the methods defined in the `backtype.storm.State` interface.
+Custom state implementations should provide implementations for the methods defined in the `org.apache.storm.State` interface.
 These are the `void prepareCommit(long txid)`, `void commit(long txid)`, `rollback()` methods. `commit()` method is optional
 and is useful if the bolt manages the state on its own. This is currently used only by the internal system bolts,
 for e.g. the CheckpointSpout to save its state.
 
-`KeyValueState` implementation should also implement the methods defined in the `backtype.storm.state.KeyValueState` interface.
+`KeyValueState` implementation should also implement the methods defined in the `org.apache.storm.state.KeyValueState` interface.
 
 ### State provider
 The framework instantiates the state via the corresponding `StateProvider` implementation. A custom state should also provide

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/examples/storm-starter/src/jvm/storm/starter/StatefulTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/storm/starter/StatefulTopology.java b/examples/storm-starter/src/jvm/storm/starter/StatefulTopology.java
index fa106dd..d09ceea 100644
--- a/examples/storm-starter/src/jvm/storm/starter/StatefulTopology.java
+++ b/examples/storm-starter/src/jvm/storm/starter/StatefulTopology.java
@@ -17,30 +17,30 @@
  */
 package storm.starter;
 
-import backtype.storm.Config;
-import backtype.storm.LocalCluster;
-import backtype.storm.StormSubmitter;
-import backtype.storm.generated.StormTopology;
-import backtype.storm.state.KeyValueState;
-import backtype.storm.task.OutputCollector;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.topology.BasicOutputCollector;
-import backtype.storm.topology.OutputFieldsDeclarer;
-import backtype.storm.topology.TopologyBuilder;
-import backtype.storm.topology.base.BaseBasicBolt;
-import backtype.storm.topology.base.BaseStatefulBolt;
-import backtype.storm.tuple.Fields;
-import backtype.storm.tuple.Tuple;
-import backtype.storm.tuple.Values;
-import backtype.storm.utils.Utils;
+import org.apache.storm.Config;
+import org.apache.storm.LocalCluster;
+import org.apache.storm.StormSubmitter;
+import org.apache.storm.generated.StormTopology;
+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.apache.storm.utils.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import storm.starter.spout.RandomIntegerSpout;
 
 import java.util.Map;
 
 /**
- * An example topology that demonstrates the use of {@link backtype.storm.topology.IStatefulBolt}
+ * 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

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueState.java
----------------------------------------------------------------------
diff --git a/external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueState.java b/external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueState.java
index 175f705..6071757 100644
--- a/external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueState.java
+++ b/external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueState.java
@@ -17,9 +17,9 @@
  */
 package org.apache.storm.redis.state;
 
-import backtype.storm.state.DefaultStateSerializer;
-import backtype.storm.state.KeyValueState;
-import backtype.storm.state.Serializer;
+import org.apache.storm.state.DefaultStateSerializer;
+import org.apache.storm.state.KeyValueState;
+import org.apache.storm.state.Serializer;
 import org.apache.storm.redis.common.config.JedisPoolConfig;
 import org.apache.storm.redis.common.container.JedisCommandsContainerBuilder;
 import org.apache.storm.redis.common.container.JedisCommandsInstanceContainer;

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueStateProvider.java
----------------------------------------------------------------------
diff --git a/external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueStateProvider.java b/external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueStateProvider.java
index 175110f..68a11d8 100644
--- a/external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueStateProvider.java
+++ b/external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueStateProvider.java
@@ -17,11 +17,11 @@
  */
 package org.apache.storm.redis.state;
 
-import backtype.storm.state.DefaultStateSerializer;
-import backtype.storm.state.Serializer;
-import backtype.storm.state.State;
-import backtype.storm.state.StateProvider;
-import backtype.storm.task.TopologyContext;
+import org.apache.storm.state.DefaultStateSerializer;
+import org.apache.storm.state.Serializer;
+import org.apache.storm.state.State;
+import org.apache.storm.state.StateProvider;
+import org.apache.storm.task.TopologyContext;
 import com.fasterxml.jackson.annotation.JsonAutoDetect;
 import com.fasterxml.jackson.annotation.PropertyAccessor;
 import com.fasterxml.jackson.core.type.TypeReference;
@@ -55,8 +55,8 @@ public class RedisKeyValueStateProvider implements StateProvider {
         String providerConfig = null;
         ObjectMapper mapper = new ObjectMapper();
         mapper.setVisibility(PropertyAccessor.FIELD, JsonAutoDetect.Visibility.ANY);
-        if (stormConf.containsKey(backtype.storm.Config.TOPOLOGY_STATE_PROVIDER_CONFIG)) {
-            providerConfig = (String) stormConf.get(backtype.storm.Config.TOPOLOGY_STATE_PROVIDER_CONFIG);
+        if (stormConf.containsKey(org.apache.storm.Config.TOPOLOGY_STATE_PROVIDER_CONFIG)) {
+            providerConfig = (String) stormConf.get(org.apache.storm.Config.TOPOLOGY_STATE_PROVIDER_CONFIG);
             stateConfig = mapper.readValue(providerConfig, StateConfig.class);
         } else {
             stateConfig = new StateConfig();

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/external/storm-redis/src/test/java/org/apache/storm/redis/state/DefaultStateSerializerTest.java
----------------------------------------------------------------------
diff --git a/external/storm-redis/src/test/java/org/apache/storm/redis/state/DefaultStateSerializerTest.java b/external/storm-redis/src/test/java/org/apache/storm/redis/state/DefaultStateSerializerTest.java
index d8fbebe..7346989 100644
--- a/external/storm-redis/src/test/java/org/apache/storm/redis/state/DefaultStateSerializerTest.java
+++ b/external/storm-redis/src/test/java/org/apache/storm/redis/state/DefaultStateSerializerTest.java
@@ -17,9 +17,9 @@
  */
 package org.apache.storm.redis.state;
 
-import backtype.storm.spout.CheckPointState;
-import backtype.storm.state.DefaultStateSerializer;
-import backtype.storm.state.Serializer;
+import org.apache.storm.spout.CheckPointState;
+import org.apache.storm.state.DefaultStateSerializer;
+import org.apache.storm.state.Serializer;
 import org.junit.Test;
 
 import java.util.ArrayList;

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/external/storm-redis/src/test/java/org/apache/storm/redis/state/RedisKeyValueStateProviderTest.java
----------------------------------------------------------------------
diff --git a/external/storm-redis/src/test/java/org/apache/storm/redis/state/RedisKeyValueStateProviderTest.java b/external/storm-redis/src/test/java/org/apache/storm/redis/state/RedisKeyValueStateProviderTest.java
index 1746031..ec6507a 100644
--- a/external/storm-redis/src/test/java/org/apache/storm/redis/state/RedisKeyValueStateProviderTest.java
+++ b/external/storm-redis/src/test/java/org/apache/storm/redis/state/RedisKeyValueStateProviderTest.java
@@ -17,8 +17,8 @@
  */
 package org.apache.storm.redis.state;
 
-import backtype.storm.Config;
-import backtype.storm.state.State;
+import org.apache.storm.Config;
+import org.apache.storm.state.State;
 import org.junit.Assert;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/external/storm-redis/src/test/java/org/apache/storm/redis/state/RedisKeyValueStateTest.java
----------------------------------------------------------------------
diff --git a/external/storm-redis/src/test/java/org/apache/storm/redis/state/RedisKeyValueStateTest.java b/external/storm-redis/src/test/java/org/apache/storm/redis/state/RedisKeyValueStateTest.java
index 9d8286c..ea8cc15 100644
--- a/external/storm-redis/src/test/java/org/apache/storm/redis/state/RedisKeyValueStateTest.java
+++ b/external/storm-redis/src/test/java/org/apache/storm/redis/state/RedisKeyValueStateTest.java
@@ -17,7 +17,7 @@
  */
 package org.apache.storm.redis.state;
 
-import backtype.storm.state.DefaultStateSerializer;
+import org.apache.storm.state.DefaultStateSerializer;
 import org.apache.storm.redis.common.container.JedisCommandsInstanceContainer;
 import org.junit.Before;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/src/jvm/backtype/storm/spout/CheckPointState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/spout/CheckPointState.java b/storm-core/src/jvm/backtype/storm/spout/CheckPointState.java
deleted file mode 100644
index ff0e088..0000000
--- a/storm-core/src/jvm/backtype/storm/spout/CheckPointState.java
+++ /dev/null
@@ -1,172 +0,0 @@
-/**
- * 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 backtype.storm.spout;
-
-import static backtype.storm.spout.CheckPointState.State.COMMITTED;
-import static backtype.storm.spout.CheckPointState.State.COMMITTING;
-import static backtype.storm.spout.CheckPointState.State.PREPARING;
-
-/**
- * Captures the current state of the transaction in {@link CheckpointSpout}. The state transitions are as follows.
- * <pre>
- *                  ROLLBACK(tx2)
- *               <-------------                  PREPARE(tx2)                     COMMIT(tx2)
- * COMMITTED(tx1)-------------> PREPARING(tx2) --------------> COMMITTING(tx2) -----------------> COMMITTED (tx2)
- *
- * </pre>
- *
- * During recovery, if a previous transaction is in PREPARING state, it is rolled back since all bolts in the topology
- * might not have prepared (saved) the data for commit. If the previous transaction is in COMMITTING state, it is
- * rolled forward (committed) since some bolts might have already committed the data.
- * <p>
- * During normal flow, the state transitions from PREPARING to COMMITTING to COMMITTED. In case of failures the
- * prepare/commit operation is retried.
- * </p>
- */
-public class CheckPointState {
-    private long txid;
-    private State state;
-
-    public enum State {
-        /**
-         * The checkpoint spout has committed the transaction.
-         */
-        COMMITTED,
-        /**
-         * The checkpoint spout has started committing the transaction
-         * and the commit is in progress.
-         */
-        COMMITTING,
-        /**
-         * The checkpoint spout has started preparing the transaction for commit
-         * and the prepare is in progress.
-         */
-        PREPARING
-    }
-
-    public enum Action {
-        /**
-         * prepare transaction for commit
-         */
-        PREPARE,
-        /**
-         * commit the previously prepared transaction
-         */
-        COMMIT,
-        /**
-         * rollback the previously prepared transaction
-         */
-        ROLLBACK,
-        /**
-         * initialize the state
-         */
-        INITSTATE
-    }
-
-    public CheckPointState(long txid, State state) {
-        this.txid = txid;
-        this.state = state;
-    }
-
-    // for kryo
-    public CheckPointState() {
-    }
-
-    public long getTxid() {
-        return txid;
-    }
-
-    public State getState() {
-        return state;
-    }
-
-    /**
-     * Get the next state based on this checkpoint state.
-     *
-     * @param recovering if in recovering phase
-     * @return the next checkpoint state based on this state.
-     */
-    public CheckPointState nextState(boolean recovering) {
-        CheckPointState nextState;
-        switch (state) {
-            case PREPARING:
-                nextState = recovering ? new CheckPointState(txid - 1, COMMITTED) : new CheckPointState(txid, COMMITTING);
-                break;
-            case COMMITTING:
-                nextState = new CheckPointState(txid, COMMITTED);
-                break;
-            case COMMITTED:
-                nextState = recovering ? this : new CheckPointState(txid + 1, PREPARING);
-                break;
-            default:
-                throw new IllegalStateException("Unknown state " + state);
-        }
-        return nextState;
-    }
-
-    /**
-     * Get the next action to perform based on this checkpoint state.
-     *
-     * @param recovering if in recovering phase
-     * @return the next action to perform based on this state
-     */
-    public Action nextAction(boolean recovering) {
-        Action action;
-        switch (state) {
-            case PREPARING:
-                action = recovering ? Action.ROLLBACK : Action.PREPARE;
-                break;
-            case COMMITTING:
-                action = Action.COMMIT;
-                break;
-            case COMMITTED:
-                action = recovering ? Action.INITSTATE : Action.PREPARE;
-                break;
-            default:
-                throw new IllegalStateException("Unknown state " + state);
-        }
-        return action;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-
-        CheckPointState that = (CheckPointState) o;
-
-        if (txid != that.txid) return false;
-        return state == that.state;
-
-    }
-
-    @Override
-    public int hashCode() {
-        int result = (int) (txid ^ (txid >>> 32));
-        result = 31 * result + (state != null ? state.hashCode() : 0);
-        return result;
-    }
-
-    @Override
-    public String toString() {
-        return "CheckPointState{" +
-                "txid=" + txid +
-                ", state=" + state +
-                '}';
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/src/jvm/backtype/storm/spout/CheckpointSpout.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/spout/CheckpointSpout.java b/storm-core/src/jvm/backtype/storm/spout/CheckpointSpout.java
deleted file mode 100644
index 178770c..0000000
--- a/storm-core/src/jvm/backtype/storm/spout/CheckpointSpout.java
+++ /dev/null
@@ -1,232 +0,0 @@
-/**
- * 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 backtype.storm.spout;
-
-import backtype.storm.Config;
-import backtype.storm.state.KeyValueState;
-import backtype.storm.state.StateFactory;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.topology.OutputFieldsDeclarer;
-import backtype.storm.topology.base.BaseRichSpout;
-import backtype.storm.tuple.Fields;
-import backtype.storm.tuple.Tuple;
-import backtype.storm.tuple.Values;
-import backtype.storm.utils.Utils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Map;
-
-import static backtype.storm.spout.CheckPointState.State.COMMITTED;
-import static backtype.storm.spout.CheckPointState.Action;
-
-/**
- * Emits checkpoint tuples which is used to save the state of the {@link backtype.storm.topology.IStatefulComponent}
- * across the topology. If a topology contains Stateful bolts, Checkpoint spouts are automatically added
- * to the topology. There is only one Checkpoint task per topology.
- * Checkpoint spout stores its internal state in a {@link KeyValueState}.
- *
- * @see CheckPointState
- */
-public class CheckpointSpout extends BaseRichSpout {
-    private static final Logger LOG = LoggerFactory.getLogger(CheckpointSpout.class);
-
-    public static final String CHECKPOINT_STREAM_ID = "$checkpoint";
-    public static final String CHECKPOINT_COMPONENT_ID = "$checkpointspout";
-    public static final String CHECKPOINT_FIELD_TXID = "txid";
-    public static final String CHECKPOINT_FIELD_ACTION = "action";
-    private static final String TX_STATE_KEY = "__state";
-    private TopologyContext context;
-    private SpoutOutputCollector collector;
-    private long lastCheckpointTs;
-    private int checkpointInterval;
-    private int sleepInterval;
-    private boolean recoveryStepInProgress;
-    private boolean checkpointStepInProgress;
-    private boolean recovering;
-    private KeyValueState<String, CheckPointState> checkpointState;
-    private CheckPointState curTxState;
-
-    @Override
-    public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
-        open(context, collector, loadCheckpointInterval(conf), loadCheckpointState(conf, context));
-    }
-
-    // package access for unit test
-    void open(TopologyContext context, SpoutOutputCollector collector,
-              int checkpointInterval, KeyValueState<String, CheckPointState> checkpointState) {
-        this.context = context;
-        this.collector = collector;
-        this.checkpointInterval = checkpointInterval;
-        this.sleepInterval = checkpointInterval / 10;
-        this.checkpointState = checkpointState;
-        this.curTxState = checkpointState.get(TX_STATE_KEY);
-        lastCheckpointTs = 0;
-        recoveryStepInProgress = false;
-        checkpointStepInProgress = false;
-        recovering = true;
-    }
-
-    @Override
-    public void nextTuple() {
-        if (shouldRecover()) {
-            handleRecovery();
-            startProgress();
-        } else if (shouldCheckpoint()) {
-            doCheckpoint();
-            startProgress();
-        } else {
-            Utils.sleep(sleepInterval);
-        }
-    }
-
-    @Override
-    public void ack(Object msgId) {
-        LOG.debug("Got ack with txid {}, current txState {}", msgId, curTxState);
-        if (curTxState.getTxid() == ((Number) msgId).longValue()) {
-            if (recovering) {
-                handleRecoveryAck();
-            } else {
-                handleCheckpointAck();
-            }
-        } else {
-            LOG.warn("Ack msgid {}, txState.txid {} mismatch", msgId, curTxState.getTxid());
-        }
-        resetProgress();
-    }
-
-    @Override
-    public void fail(Object msgId) {
-        LOG.debug("Got fail with msgid {}", msgId);
-        if (!recovering) {
-            LOG.debug("Checkpoint failed, will trigger recovery");
-            recovering = true;
-        }
-        resetProgress();
-    }
-
-    @Override
-    public void declareOutputFields(OutputFieldsDeclarer declarer) {
-        declarer.declareStream(CHECKPOINT_STREAM_ID, new Fields(CHECKPOINT_FIELD_TXID, CHECKPOINT_FIELD_ACTION));
-    }
-
-    public static boolean isCheckpoint(Tuple input) {
-        return CHECKPOINT_STREAM_ID.equals(input.getSourceStreamId());
-    }
-
-    /**
-     * Loads the last saved checkpoint state the from persistent storage.
-     */
-    private KeyValueState<String, CheckPointState> loadCheckpointState(Map conf, TopologyContext ctx) {
-        String namespace = ctx.getThisComponentId() + "-" + ctx.getThisTaskId();
-        KeyValueState<String, CheckPointState> state =
-                (KeyValueState<String, CheckPointState>) StateFactory.getState(namespace, conf, ctx);
-        if (state.get(TX_STATE_KEY) == null) {
-            CheckPointState txState = new CheckPointState(-1, COMMITTED);
-            state.put(TX_STATE_KEY, txState);
-            state.commit();
-            LOG.debug("Initialized checkpoint spout state with txState {}", txState);
-        } else {
-            LOG.debug("Got checkpoint spout state {}", state.get(TX_STATE_KEY));
-        }
-        return state;
-    }
-
-    private int loadCheckpointInterval(Map stormConf) {
-        int interval = 0;
-        if (stormConf.containsKey(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL)) {
-            interval = ((Number) stormConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL)).intValue();
-        }
-        // ensure checkpoint interval is not less than a sane low value.
-        interval = Math.max(100, interval);
-        LOG.info("Checkpoint interval is {} millis", interval);
-        return interval;
-    }
-
-    private boolean shouldRecover() {
-        return recovering && !recoveryStepInProgress;
-    }
-
-    private boolean shouldCheckpoint() {
-        return !recovering && !checkpointStepInProgress &&
-                (curTxState.getState() != COMMITTED || checkpointIntervalElapsed());
-    }
-
-    private boolean checkpointIntervalElapsed() {
-        return (System.currentTimeMillis() - lastCheckpointTs) > checkpointInterval;
-    }
-
-    private void handleRecovery() {
-        LOG.debug("In recovery");
-        Action action = curTxState.nextAction(true);
-        emit(curTxState.getTxid(), action);
-    }
-
-    private void handleRecoveryAck() {
-        CheckPointState nextState = curTxState.nextState(true);
-        if (curTxState != nextState) {
-            saveTxState(nextState);
-        } else {
-            LOG.debug("Recovery complete, current state {}", curTxState);
-            recovering = false;
-        }
-    }
-
-    private void doCheckpoint() {
-        LOG.debug("In checkpoint");
-        if (curTxState.getState() == COMMITTED) {
-            saveTxState(curTxState.nextState(false));
-            lastCheckpointTs = System.currentTimeMillis();
-        }
-        Action action = curTxState.nextAction(false);
-        emit(curTxState.getTxid(), action);
-    }
-
-    private void handleCheckpointAck() {
-        CheckPointState nextState = curTxState.nextState(false);
-        saveTxState(nextState);
-    }
-
-    private void emit(long txid, Action action) {
-        LOG.debug("Current state {}, emitting txid {}, action {}", curTxState, txid, action);
-        collector.emit(CHECKPOINT_STREAM_ID, new Values(txid, action), txid);
-    }
-
-    private void saveTxState(CheckPointState txState) {
-        LOG.debug("saveTxState, current state {} -> new state {}", curTxState, txState);
-        checkpointState.put(TX_STATE_KEY, txState);
-        checkpointState.commit();
-        curTxState = txState;
-    }
-
-    private void startProgress() {
-        if (recovering) {
-            recoveryStepInProgress = true;
-        } else {
-            checkpointStepInProgress = true;
-        }
-    }
-
-    private void resetProgress() {
-        if (recovering) {
-            recoveryStepInProgress = false;
-        } else {
-            checkpointStepInProgress = false;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/src/jvm/backtype/storm/state/DefaultStateSerializer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/state/DefaultStateSerializer.java b/storm-core/src/jvm/backtype/storm/state/DefaultStateSerializer.java
deleted file mode 100644
index 1e57a1d..0000000
--- a/storm-core/src/jvm/backtype/storm/state/DefaultStateSerializer.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/**
- * 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 backtype.storm.state;
-
-import com.esotericsoftware.kryo.Kryo;
-import com.esotericsoftware.kryo.io.Input;
-import com.esotericsoftware.kryo.io.Output;
-
-import java.util.Collections;
-import java.util.List;
-
-/**
- * A default implementation that uses Kryo to serialize and de-serialize
- * the state.
- */
-public class DefaultStateSerializer<T> implements Serializer<T> {
-    private final Kryo kryo;
-    private final Output output;
-
-    /**
-     * Constructs a {@link DefaultStateSerializer} instance with the given list
-     * of classes registered in kryo.
-     *
-     * @param classesToRegister the classes to register.
-     */
-    public DefaultStateSerializer(List<Class<?>> classesToRegister) {
-        kryo = new Kryo();
-        output = new Output(2000, 2000000000);
-        for (Class<?> klazz : classesToRegister) {
-            kryo.register(klazz);
-        }
-    }
-
-    public DefaultStateSerializer() {
-        this(Collections.<Class<?>>emptyList());
-    }
-
-    @Override
-    public byte[] serialize(T obj) {
-        output.clear();
-        kryo.writeClassAndObject(output, obj);
-        return output.toBytes();
-    }
-
-    @Override
-    public T deserialize(byte[] b) {
-        Input input = new Input(b);
-        return (T) kryo.readClassAndObject(input);
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/src/jvm/backtype/storm/state/InMemoryKeyValueState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/state/InMemoryKeyValueState.java b/storm-core/src/jvm/backtype/storm/state/InMemoryKeyValueState.java
deleted file mode 100644
index 394cf5d..0000000
--- a/storm-core/src/jvm/backtype/storm/state/InMemoryKeyValueState.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/**
- * 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 backtype.storm.state;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-/**
- * An in-memory implementation of the {@link State}
- */
-public class InMemoryKeyValueState<K, V> implements KeyValueState<K, V> {
-    private static final Logger LOG = LoggerFactory.getLogger(InMemoryKeyValueState.class);
-    private static final long DEFAULT_TXID = -1;
-    private TxIdState<K, V> commitedState;
-    private TxIdState<K, V> preparedState;
-    private Map<K, V> state = new ConcurrentHashMap<>();
-
-    private static class TxIdState<K, V> {
-        private long txid;
-        private Map<K, V> state;
-
-        TxIdState(long txid, Map<K, V> state) {
-            this.txid = txid;
-            this.state = state;
-        }
-
-        @Override
-        public String toString() {
-            return "TxIdState{" +
-                    "txid=" + txid +
-                    ", state=" + state +
-                    '}';
-        }
-    }
-
-    @Override
-    public void put(K key, V value) {
-        state.put(key, value);
-    }
-
-    @Override
-    public V get(K key) {
-        return state.get(key);
-    }
-
-    @Override
-    public V get(K key, V defaultValue) {
-        V val = get(key);
-        return val != null ? val : defaultValue;
-    }
-
-    @Override
-    public void commit() {
-        commitedState = new TxIdState<>(DEFAULT_TXID, new ConcurrentHashMap<>(state));
-    }
-
-    @Override
-    public void prepareCommit(long txid) {
-        LOG.debug("prepare commit, txid {}", txid);
-        if (preparedState != null && txid > preparedState.txid) {
-            throw new RuntimeException("Cannot prepare a new txn while there is a pending txn");
-        }
-        preparedState = new TxIdState<>(txid, new ConcurrentHashMap<K, V>(state));
-    }
-
-    @Override
-    public void commit(long txid) {
-        LOG.debug("commit, txid {}", txid);
-        if (preparedState != null && txid == preparedState.txid) {
-            commitedState = preparedState;
-            preparedState = null;
-        } else {
-            throw new RuntimeException("Invalid prepared state for commit, " +
-                                               "preparedState " + preparedState + " txid " + txid);
-        }
-    }
-
-    @Override
-    public void rollback() {
-        preparedState = null;
-        if (commitedState != null) {
-            state = commitedState.state;
-        } else {
-            state = new ConcurrentHashMap<>();
-        }
-    }
-
-    @Override
-    public String toString() {
-        return "InMemoryKeyValueState{" +
-                "commitedState=" + commitedState +
-                ", preparedState=" + preparedState +
-                ", state=" + state +
-                '}';
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/src/jvm/backtype/storm/state/InMemoryKeyValueStateProvider.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/state/InMemoryKeyValueStateProvider.java b/storm-core/src/jvm/backtype/storm/state/InMemoryKeyValueStateProvider.java
deleted file mode 100644
index aaedbb6..0000000
--- a/storm-core/src/jvm/backtype/storm/state/InMemoryKeyValueStateProvider.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * 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 backtype.storm.state;
-
-import backtype.storm.task.TopologyContext;
-
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-/**
- * Provides {@link InMemoryKeyValueState}
- */
-public class InMemoryKeyValueStateProvider implements StateProvider {
-    private final ConcurrentHashMap<String, State> states = new ConcurrentHashMap<>();
-
-    @Override
-    public State newState(String namespace, Map stormConf, TopologyContext context) {
-        State state = states.get(namespace);
-        if (state == null) {
-            State newState = new InMemoryKeyValueState<>();
-            state = states.putIfAbsent(namespace, newState);
-            if (state == null) {
-                state = newState;
-            }
-        }
-        return state;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/src/jvm/backtype/storm/state/KeyValueState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/state/KeyValueState.java b/storm-core/src/jvm/backtype/storm/state/KeyValueState.java
deleted file mode 100644
index 3ef0707..0000000
--- a/storm-core/src/jvm/backtype/storm/state/KeyValueState.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/**
- * 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 backtype.storm.state;
-
-/**
- * A state that supports key-value mappings.
- */
-public interface KeyValueState<K, V> extends State {
-    /**
-     * Maps the value with the key
-     *
-     * @param key   the key
-     * @param value the value
-     */
-    void put(K key, V value);
-
-    /**
-     * Returns the value mapped to the key
-     *
-     * @param key the key
-     * @return the value or null if no mapping is found
-     */
-    V get(K key);
-
-    /**
-     * Returns the value mapped to the key or defaultValue if no mapping is found.
-     *
-     * @param key          the key
-     * @param defaultValue the value to return if no mapping is found
-     * @return the value or defaultValue if no mapping is found
-     */
-    V get(K key, V defaultValue);
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/src/jvm/backtype/storm/state/Serializer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/state/Serializer.java b/storm-core/src/jvm/backtype/storm/state/Serializer.java
deleted file mode 100644
index 1761621..0000000
--- a/storm-core/src/jvm/backtype/storm/state/Serializer.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/**
- * 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 backtype.storm.state;
-
-import java.io.Serializable;
-
-/**
- * Interface to be implemented for serlializing and de-serializing the
- * state.
- */
-public interface Serializer<T> extends Serializable {
-    byte[] serialize(T obj);
-
-    T deserialize(byte[] b);
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/src/jvm/backtype/storm/state/State.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/state/State.java b/storm-core/src/jvm/backtype/storm/state/State.java
deleted file mode 100644
index 4a837b4..0000000
--- a/storm-core/src/jvm/backtype/storm/state/State.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/**
- * 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 backtype.storm.state;
-
-/**
- * The state of the component that is either managed by the framework (e.g in case of {@link backtype.storm.topology.IStatefulBolt})
- * or managed by the the individual components themselves.
- */
-public interface State {
-    /**
-     * Invoked by the framework to prepare a transaction for commit. It should be possible
-     * to commit the prepared state later.
-     * <p>
-     * The same txid can be prepared again, but the next txid cannot be prepared
-     * when previous one is not yet committed.
-     * </p>
-     *
-     * @param txid the transaction id
-     */
-    void prepareCommit(long txid);
-
-    /**
-     * Commit a previously prepared transaction. It should be possible to retrieve a committed state later.
-     *
-     * @param txid the transaction id
-     */
-    void commit(long txid);
-
-    /**
-     * Persist the current state. This is used when the component manages the state.
-     */
-    void commit();
-
-    /**
-     * Rollback a prepared transaction to the previously committed state.
-     */
-    void rollback();
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/src/jvm/backtype/storm/state/StateFactory.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/state/StateFactory.java b/storm-core/src/jvm/backtype/storm/state/StateFactory.java
deleted file mode 100644
index e0ea548..0000000
--- a/storm-core/src/jvm/backtype/storm/state/StateFactory.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/**
- * 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 backtype.storm.state;
-
-import backtype.storm.Config;
-import backtype.storm.task.TopologyContext;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Map;
-
-/**
- * A factory for creating {@link State} instances
- */
-public class StateFactory {
-    private static final Logger LOG = LoggerFactory.getLogger(StateFactory.class);
-
-    private static final String DEFAULT_PROVIDER = "backtype.storm.state.InMemoryKeyValueStateProvider";
-
-    /**
-     * Returns a new state instance using the {@link Config#TOPOLOGY_STATE_PROVIDER} or a
-     * {@link InMemoryKeyValueState} if no provider is configured.
-     *
-     * @param namespace the state namespace
-     * @param stormConf the storm conf
-     * @param context   the topology context
-     * @return the state instance
-     */
-    public static State getState(String namespace, Map stormConf, TopologyContext context) {
-        State state;
-        try {
-            String provider = null;
-            if (stormConf.containsKey(Config.TOPOLOGY_STATE_PROVIDER)) {
-                provider = (String) stormConf.get(Config.TOPOLOGY_STATE_PROVIDER);
-            } else {
-                provider = DEFAULT_PROVIDER;
-            }
-            Class<?> klazz = Class.forName(provider);
-            Object object = klazz.newInstance();
-            if (object instanceof StateProvider) {
-                state = ((StateProvider) object).newState(namespace, stormConf, context);
-            } else {
-                String msg = "Invalid state provider '" + provider +
-                        "'. Should implement backtype.storm.state.StateProvider";
-                LOG.error(msg);
-                throw new RuntimeException(msg);
-            }
-        } catch (Exception ex) {
-            LOG.error("Got exception while loading the state provider", ex);
-            throw new RuntimeException(ex);
-        }
-        return state;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/src/jvm/backtype/storm/state/StateProvider.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/state/StateProvider.java b/storm-core/src/jvm/backtype/storm/state/StateProvider.java
deleted file mode 100644
index d10141e..0000000
--- a/storm-core/src/jvm/backtype/storm/state/StateProvider.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/**
- * 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 backtype.storm.state;
-
-import backtype.storm.task.TopologyContext;
-
-import java.util.Map;
-
-/**
- * Used by the {@link StateFactory} to create a new state instances.
- */
-public interface StateProvider {
-    /**
-     * Returns a new state instance. Each state belongs unique namespace which is typically
-     * the componentid-task of the task, so that each task can have its own unique state.
-     *
-     * @param namespace a namespace of the state
-     * @param stormConf the storm topology configuration
-     * @param context   the {@link TopologyContext}
-     * @return a previously saved state instance
-     */
-    State newState(String namespace, Map stormConf, TopologyContext context);
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/src/jvm/backtype/storm/topology/CheckpointTupleForwarder.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/topology/CheckpointTupleForwarder.java b/storm-core/src/jvm/backtype/storm/topology/CheckpointTupleForwarder.java
deleted file mode 100644
index 1e0b12e..0000000
--- a/storm-core/src/jvm/backtype/storm/topology/CheckpointTupleForwarder.java
+++ /dev/null
@@ -1,252 +0,0 @@
-/**
- * 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 backtype.storm.topology;
-
-import backtype.storm.generated.GlobalStreamId;
-import backtype.storm.spout.CheckpointSpout;
-import backtype.storm.task.IOutputCollector;
-import backtype.storm.task.OutputCollector;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.tuple.Fields;
-import backtype.storm.tuple.Tuple;
-import backtype.storm.tuple.Values;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import static backtype.storm.spout.CheckpointSpout.CHECKPOINT_STREAM_ID;
-import static backtype.storm.spout.CheckpointSpout.CHECKPOINT_FIELD_TXID;
-import static backtype.storm.spout.CheckpointSpout.CHECKPOINT_FIELD_ACTION;
-import static backtype.storm.spout.CheckPointState.Action;
-import static backtype.storm.spout.CheckPointState.Action.ROLLBACK;
-
-/**
- * Wraps {@link IRichBolt} and forwards checkpoint tuples in a
- * stateful topology.
- * <p>
- * When a storm topology contains one or more {@link IStatefulBolt} all non-stateful
- * bolts are wrapped in {@link CheckpointTupleForwarder} so that the checkpoint tuples
- * can flow through the entire topology DAG.
- * </p>
- */
-public class CheckpointTupleForwarder implements IRichBolt {
-    private static final Logger LOG = LoggerFactory.getLogger(CheckpointTupleForwarder.class);
-    private final IRichBolt bolt;
-    private final Map<TransactionRequest, Integer> transactionRequestCount;
-    private int checkPointInputTaskCount;
-    private long lastTxid = Long.MIN_VALUE;
-    protected AnchoringOutputCollector collector;
-
-    public CheckpointTupleForwarder(IRichBolt bolt) {
-        this.bolt = bolt;
-        transactionRequestCount = new HashMap<>();
-    }
-
-    @Override
-    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
-        this.collector = new AnchoringOutputCollector(collector);
-        bolt.prepare(stormConf, context, this.collector);
-        checkPointInputTaskCount = getCheckpointInputTaskCount(context);
-    }
-
-    @Override
-    public void execute(Tuple input) {
-        if (CheckpointSpout.isCheckpoint(input)) {
-            processCheckpoint(input);
-        } else {
-            handleTuple(input);
-        }
-    }
-
-    @Override
-    public void cleanup() {
-        bolt.cleanup();
-    }
-
-    @Override
-    public void declareOutputFields(OutputFieldsDeclarer declarer) {
-        bolt.declareOutputFields(declarer);
-        declarer.declareStream(CHECKPOINT_STREAM_ID, new Fields(CHECKPOINT_FIELD_TXID, CHECKPOINT_FIELD_ACTION));
-    }
-
-    @Override
-    public Map<String, Object> getComponentConfiguration() {
-        return bolt.getComponentConfiguration();
-    }
-
-    /**
-     * Forwards the checkpoint tuple downstream. Sub-classes can override
-     * with the logic for handling checkpoint tuple.
-     *
-     * @param checkpointTuple  the checkpoint tuple
-     * @param action the action (prepare, commit, rollback or initstate)
-     * @param txid   the transaction id.
-     */
-    protected void handleCheckpoint(Tuple checkpointTuple, Action action, long txid) {
-        collector.emit(CHECKPOINT_STREAM_ID, checkpointTuple, new Values(txid, action));
-        collector.ack(checkpointTuple);
-    }
-
-    /**
-     * Hands off tuple to the wrapped bolt to execute. Sub-classes can
-     * override the behavior.
-     * <p>
-     * Right now tuples continue to get forwarded while waiting for checkpoints to arrive on other streams
-     * after checkpoint arrives on one of the streams. This can cause duplicates but still at least once.
-     * </p>
-     *
-     * @param input the input tuple
-     */
-    protected void handleTuple(Tuple input) {
-        collector.setContext(input);
-        bolt.execute(input);
-        collector.ack(input);
-    }
-
-    /**
-     * Invokes handleCheckpoint once checkpoint tuple is received on
-     * all input checkpoint streams to this component.
-     */
-    private void processCheckpoint(Tuple input) {
-        Action action = (Action) input.getValueByField(CHECKPOINT_FIELD_ACTION);
-        long txid = input.getLongByField(CHECKPOINT_FIELD_TXID);
-        if (shouldProcessTransaction(action, txid)) {
-            LOG.debug("Processing action {}, txid {}", action, txid);
-            try {
-                if (txid >= lastTxid) {
-                    handleCheckpoint(input, action, txid);
-                    if (action == ROLLBACK) {
-                        lastTxid = txid - 1;
-                    } else {
-                        lastTxid = txid;
-                    }
-                } else {
-                    LOG.debug("Ignoring old transaction. Action {}, txid {}", action, txid);
-                    collector.ack(input);
-                }
-            } catch (Throwable th) {
-                LOG.error("Got error while processing checkpoint tuple", th);
-                collector.fail(input);
-                collector.reportError(th);
-            }
-        } else {
-            LOG.debug("Waiting for action {}, txid {} from all input tasks. checkPointInputTaskCount {}, " +
-                              "transactionRequestCount {}", action, txid, checkPointInputTaskCount, transactionRequestCount);
-            collector.ack(input);
-        }
-    }
-
-    /**
-     * returns the total number of input checkpoint streams across
-     * all input tasks to this component.
-     */
-    private int getCheckpointInputTaskCount(TopologyContext context) {
-        int count = 0;
-        for (GlobalStreamId inputStream : context.getThisSources().keySet()) {
-            if (CHECKPOINT_STREAM_ID.equals(inputStream.get_streamId())) {
-                count += context.getComponentTasks(inputStream.get_componentId()).size();
-            }
-        }
-        return count;
-    }
-
-    /**
-     * Checks if check points have been received from all tasks across
-     * all input streams to this component
-     */
-    private boolean shouldProcessTransaction(Action action, long txid) {
-        TransactionRequest request = new TransactionRequest(action, txid);
-        Integer count;
-        if ((count = transactionRequestCount.get(request)) == null) {
-            transactionRequestCount.put(request, 1);
-            count = 1;
-        } else {
-            transactionRequestCount.put(request, ++count);
-        }
-        if (count == checkPointInputTaskCount) {
-            transactionRequestCount.remove(request);
-            return true;
-        }
-        return false;
-    }
-
-    private static class TransactionRequest {
-        private final Action action;
-        private final long txid;
-
-        TransactionRequest(Action action, long txid) {
-            this.action = action;
-            this.txid = txid;
-        }
-
-        @Override
-        public boolean equals(Object o) {
-            if (this == o) return true;
-            if (o == null || getClass() != o.getClass()) return false;
-
-            TransactionRequest that = (TransactionRequest) o;
-
-            if (txid != that.txid) return false;
-            return !(action != null ? !action.equals(that.action) : that.action != null);
-
-        }
-
-        @Override
-        public int hashCode() {
-            int result = action != null ? action.hashCode() : 0;
-            result = 31 * result + (int) (txid ^ (txid >>> 32));
-            return result;
-        }
-
-        @Override
-        public String toString() {
-            return "TransactionRequest{" +
-                    "action='" + action + '\'' +
-                    ", txid=" + txid +
-                    '}';
-        }
-    }
-
-
-    protected static class AnchoringOutputCollector extends OutputCollector {
-        private Tuple inputTuple;
-
-        AnchoringOutputCollector(IOutputCollector delegate) {
-            super(delegate);
-        }
-
-        void setContext(Tuple inputTuple) {
-            this.inputTuple = inputTuple;
-        }
-
-        @Override
-        public List<Integer> emit(String streamId, List<Object> tuple) {
-            return emit(streamId, inputTuple, tuple);
-        }
-
-        @Override
-        public void emitDirect(int taskId, String streamId, List<Object> tuple) {
-            emitDirect(taskId, streamId, inputTuple, tuple);
-        }
-
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/src/jvm/backtype/storm/topology/IStatefulBolt.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/topology/IStatefulBolt.java b/storm-core/src/jvm/backtype/storm/topology/IStatefulBolt.java
deleted file mode 100644
index 56408bd..0000000
--- a/storm-core/src/jvm/backtype/storm/topology/IStatefulBolt.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/**
- * 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 backtype.storm.topology;
-
-import backtype.storm.state.State;
-
-/**
- * A bolt abstraction for supporting stateful computation.
- */
-public interface IStatefulBolt<T extends State> extends IStatefulComponent<T>, IRichBolt {
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/src/jvm/backtype/storm/topology/IStatefulComponent.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/topology/IStatefulComponent.java b/storm-core/src/jvm/backtype/storm/topology/IStatefulComponent.java
deleted file mode 100644
index ea88d90..0000000
--- a/storm-core/src/jvm/backtype/storm/topology/IStatefulComponent.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/**
- * 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 backtype.storm.topology;
-
-import backtype.storm.state.State;
-
-/**
- * <p>
- * Common methods for stateful components in the topology.
- * </p>
- * A stateful component is one that has state (e.g. the result of some computation in a bolt)
- * and wants the framework to manage its state.
- */
-public interface IStatefulComponent<T extends State> extends IComponent {
-    /**
-     * This method is invoked by the framework with the previously
-     * saved state of the component. This is invoked after prepare but before
-     * the component starts processing tuples.
-     *
-     * @param state the previously saved state of the component.
-     */
-    void initState(T state);
-
-    /**
-     * This is a hook for the component to perform some actions just before the
-     * framework commits its state.
-     */
-    void preCommit(long txid);
-
-    /**
-     * This is a hook for the component to perform some actions just before the
-     * framework prepares its state.
-     */
-    void prePrepare(long txid);
-
-    /**
-     * This is a hook for the component to perform some actions just before the
-     * framework rolls back the prepared state.
-     */
-    void preRollback();
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/src/jvm/backtype/storm/topology/StatefulBoltExecutor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/topology/StatefulBoltExecutor.java b/storm-core/src/jvm/backtype/storm/topology/StatefulBoltExecutor.java
deleted file mode 100644
index f20df33..0000000
--- a/storm-core/src/jvm/backtype/storm/topology/StatefulBoltExecutor.java
+++ /dev/null
@@ -1,151 +0,0 @@
-/**
- * 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 backtype.storm.topology;
-
-import backtype.storm.spout.CheckpointSpout;
-import backtype.storm.state.State;
-import backtype.storm.state.StateFactory;
-import backtype.storm.task.OutputCollector;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.tuple.Tuple;
-import backtype.storm.tuple.Values;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import static backtype.storm.spout.CheckPointState.Action;
-import static backtype.storm.spout.CheckPointState.Action.COMMIT;
-import static backtype.storm.spout.CheckPointState.Action.PREPARE;
-import static backtype.storm.spout.CheckPointState.Action.ROLLBACK;
-import static backtype.storm.spout.CheckPointState.Action.INITSTATE;
-
-/**
- * Wraps a {@link IStatefulBolt} and manages the state of the bolt.
- */
-public class StatefulBoltExecutor<T extends State> extends CheckpointTupleForwarder {
-    private static final Logger LOG = LoggerFactory.getLogger(StatefulBoltExecutor.class);
-    private final IStatefulBolt<T> bolt;
-    private State state;
-    private boolean boltInitialized = false;
-    private List<Tuple> pendingTuples = new ArrayList<>();
-    private List<Tuple> preparedTuples = new ArrayList<>();
-    private List<Tuple> executedTuples = new ArrayList<>();
-
-    public StatefulBoltExecutor(IStatefulBolt<T> bolt) {
-        super(bolt);
-        this.bolt = bolt;
-    }
-
-    @Override
-    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
-        // get the last successfully committed state from state store
-        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId();
-        prepare(stormConf, context, collector, StateFactory.getState(namespace, stormConf, context));
-    }
-
-    // package access for unit tests
-    void prepare(Map stormConf, TopologyContext context, OutputCollector collector, State state) {
-        super.prepare(stormConf, context, collector);
-        this.state = state;
-    }
-
-    @Override
-    protected void handleCheckpoint(Tuple checkpointTuple, Action action, long txid) {
-        LOG.debug("handleCheckPoint with tuple {}, action {}, txid {}", checkpointTuple, action, txid);
-        if (action == PREPARE) {
-            if (boltInitialized) {
-                bolt.prePrepare(txid);
-                state.prepareCommit(txid);
-                preparedTuples.addAll(executedTuples);
-                executedTuples.clear();
-            } else {
-                /*
-                 * May be the task restarted in the middle and the state needs be initialized.
-                 * Fail fast and trigger recovery.
-                  */
-                LOG.debug("Failing checkpointTuple, PREPARE received when bolt state is not initialized.");
-                collector.fail(checkpointTuple);
-                return;
-            }
-        } else if (action == COMMIT) {
-            bolt.preCommit(txid);
-            state.commit(txid);
-            ack(preparedTuples);
-        } else if (action == ROLLBACK) {
-            bolt.preRollback();
-            state.rollback();
-            fail(preparedTuples);
-            fail(executedTuples);
-        } else if (action == INITSTATE) {
-            if (!boltInitialized) {
-                bolt.initState((T) state);
-                boltInitialized = true;
-                LOG.debug("{} pending tuples to process", pendingTuples.size());
-                for (Tuple tuple : pendingTuples) {
-                    doExecute(tuple);
-                }
-                pendingTuples.clear();
-            } else {
-                LOG.debug("Bolt state is already initialized, ignoring tuple {}, action {}, txid {}",
-                          checkpointTuple, action, txid);
-            }
-        }
-        collector.emit(CheckpointSpout.CHECKPOINT_STREAM_ID, checkpointTuple, new Values(txid, action));
-        collector.ack(checkpointTuple);
-    }
-
-    @Override
-    protected void handleTuple(Tuple input) {
-        if (boltInitialized) {
-            doExecute(input);
-        } else {
-            LOG.debug("Bolt state not initialized, adding tuple {} to pending tuples", input);
-            pendingTuples.add(input);
-        }
-    }
-
-    private void doExecute(Tuple tuple) {
-        collector.setContext(tuple);
-        bolt.execute(tuple);
-        executedTuples.add(tuple);
-    }
-
-    private void ack(List<Tuple> tuples) {
-        if (!tuples.isEmpty()) {
-            LOG.debug("Acking {} tuples", tuples.size());
-            for (Tuple tuple : tuples) {
-                collector.ack(tuple);
-            }
-            tuples.clear();
-        }
-    }
-
-    private void fail(List<Tuple> tuples) {
-        if (!tuples.isEmpty()) {
-            LOG.debug("Failing {} tuples", tuples.size());
-            for (Tuple tuple : tuples) {
-                collector.fail(tuple);
-            }
-            tuples.clear();
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/src/jvm/backtype/storm/topology/base/BaseStatefulBolt.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/topology/base/BaseStatefulBolt.java b/storm-core/src/jvm/backtype/storm/topology/base/BaseStatefulBolt.java
deleted file mode 100644
index 80734ab..0000000
--- a/storm-core/src/jvm/backtype/storm/topology/base/BaseStatefulBolt.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/**
- * 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 backtype.storm.topology.base;
-
-import backtype.storm.state.State;
-import backtype.storm.task.OutputCollector;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.topology.IStatefulBolt;
-import backtype.storm.topology.OutputFieldsDeclarer;
-
-import java.util.Map;
-
-public abstract class BaseStatefulBolt<T extends State> implements IStatefulBolt<T> {
-
-    @Override
-    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
-        // NOOP
-    }
-
-    @Override
-    public void cleanup() {
-        // NOOP
-    }
-
-    @Override
-    public void declareOutputFields(OutputFieldsDeclarer declarer) {
-        // NOOP
-    }
-
-    @Override
-    public Map<String, Object> getComponentConfiguration() {
-        return null;
-    }
-
-    @Override
-    public void preCommit(long txid) {
-        // NOOP
-    }
-
-    @Override
-    public void prePrepare(long txid) {
-        // NOOP
-    }
-
-    @Override
-    public void preRollback() {
-        // NOOP
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/src/jvm/org/apache/storm/Config.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/Config.java b/storm-core/src/jvm/org/apache/storm/Config.java
index c18be2f..e1f2281 100644
--- a/storm-core/src/jvm/org/apache/storm/Config.java
+++ b/storm-core/src/jvm/org/apache/storm/Config.java
@@ -1539,15 +1539,15 @@ public class Config extends HashMap<String, Object> {
     public static final String TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT = "topology.component.cpu.pcore.percent";
 
     /**
-     * The class name of the {@link backtype.storm.state.StateProvider} implementation. If not specified
-     * defaults to {@link backtype.storm.state.InMemoryKeyValueStateProvider}. This can be overridden
+     * The class name of the {@link org.apache.storm.state.StateProvider} implementation. If not specified
+     * defaults to {@link org.apache.storm.state.InMemoryKeyValueStateProvider}. This can be overridden
      * at the component level.
      */
     @isString
     public static final String TOPOLOGY_STATE_PROVIDER = "topology.state.provider";
 
     /**
-     * The configuration specific to the {@link backtype.storm.state.StateProvider} implementation.
+     * The configuration specific to the {@link org.apache.storm.state.StateProvider} implementation.
      * This can be overridden at the component level. The value and the interpretation of this config
      * is based on the state provider implementation. For e.g. this could be just a config file name
      * which contains the config for the state provider implementation.
@@ -1557,7 +1557,7 @@ public class Config extends HashMap<String, Object> {
 
     /**
      * Topology configuration to specify the checkpoint interval (in millis) at which the
-     * topology state is saved when {@link backtype.storm.topology.IStatefulBolt} bolts are involved.
+     * topology state is saved when {@link org.apache.storm.topology.IStatefulBolt} bolts are involved.
      */
     @isInteger
     @isPositiveNumber

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/src/jvm/org/apache/storm/spout/CheckPointState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/spout/CheckPointState.java b/storm-core/src/jvm/org/apache/storm/spout/CheckPointState.java
new file mode 100644
index 0000000..c386362
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/spout/CheckPointState.java
@@ -0,0 +1,172 @@
+/**
+ * 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.spout;
+
+import static org.apache.storm.spout.CheckPointState.State.COMMITTED;
+import static org.apache.storm.spout.CheckPointState.State.COMMITTING;
+import static org.apache.storm.spout.CheckPointState.State.PREPARING;
+
+/**
+ * Captures the current state of the transaction in {@link CheckpointSpout}. The state transitions are as follows.
+ * <pre>
+ *                  ROLLBACK(tx2)
+ *               <-------------                  PREPARE(tx2)                     COMMIT(tx2)
+ * COMMITTED(tx1)-------------> PREPARING(tx2) --------------> COMMITTING(tx2) -----------------> COMMITTED (tx2)
+ *
+ * </pre>
+ *
+ * During recovery, if a previous transaction is in PREPARING state, it is rolled back since all bolts in the topology
+ * might not have prepared (saved) the data for commit. If the previous transaction is in COMMITTING state, it is
+ * rolled forward (committed) since some bolts might have already committed the data.
+ * <p>
+ * During normal flow, the state transitions from PREPARING to COMMITTING to COMMITTED. In case of failures the
+ * prepare/commit operation is retried.
+ * </p>
+ */
+public class CheckPointState {
+    private long txid;
+    private State state;
+
+    public enum State {
+        /**
+         * The checkpoint spout has committed the transaction.
+         */
+        COMMITTED,
+        /**
+         * The checkpoint spout has started committing the transaction
+         * and the commit is in progress.
+         */
+        COMMITTING,
+        /**
+         * The checkpoint spout has started preparing the transaction for commit
+         * and the prepare is in progress.
+         */
+        PREPARING
+    }
+
+    public enum Action {
+        /**
+         * prepare transaction for commit
+         */
+        PREPARE,
+        /**
+         * commit the previously prepared transaction
+         */
+        COMMIT,
+        /**
+         * rollback the previously prepared transaction
+         */
+        ROLLBACK,
+        /**
+         * initialize the state
+         */
+        INITSTATE
+    }
+
+    public CheckPointState(long txid, State state) {
+        this.txid = txid;
+        this.state = state;
+    }
+
+    // for kryo
+    public CheckPointState() {
+    }
+
+    public long getTxid() {
+        return txid;
+    }
+
+    public State getState() {
+        return state;
+    }
+
+    /**
+     * Get the next state based on this checkpoint state.
+     *
+     * @param recovering if in recovering phase
+     * @return the next checkpoint state based on this state.
+     */
+    public CheckPointState nextState(boolean recovering) {
+        CheckPointState nextState;
+        switch (state) {
+            case PREPARING:
+                nextState = recovering ? new CheckPointState(txid - 1, COMMITTED) : new CheckPointState(txid, COMMITTING);
+                break;
+            case COMMITTING:
+                nextState = new CheckPointState(txid, COMMITTED);
+                break;
+            case COMMITTED:
+                nextState = recovering ? this : new CheckPointState(txid + 1, PREPARING);
+                break;
+            default:
+                throw new IllegalStateException("Unknown state " + state);
+        }
+        return nextState;
+    }
+
+    /**
+     * Get the next action to perform based on this checkpoint state.
+     *
+     * @param recovering if in recovering phase
+     * @return the next action to perform based on this state
+     */
+    public Action nextAction(boolean recovering) {
+        Action action;
+        switch (state) {
+            case PREPARING:
+                action = recovering ? Action.ROLLBACK : Action.PREPARE;
+                break;
+            case COMMITTING:
+                action = Action.COMMIT;
+                break;
+            case COMMITTED:
+                action = recovering ? Action.INITSTATE : Action.PREPARE;
+                break;
+            default:
+                throw new IllegalStateException("Unknown state " + state);
+        }
+        return action;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        CheckPointState that = (CheckPointState) o;
+
+        if (txid != that.txid) return false;
+        return state == that.state;
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result = (int) (txid ^ (txid >>> 32));
+        result = 31 * result + (state != null ? state.hashCode() : 0);
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "CheckPointState{" +
+                "txid=" + txid +
+                ", state=" + state +
+                '}';
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/src/jvm/org/apache/storm/spout/CheckpointSpout.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/spout/CheckpointSpout.java b/storm-core/src/jvm/org/apache/storm/spout/CheckpointSpout.java
new file mode 100644
index 0000000..dfebf90
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/spout/CheckpointSpout.java
@@ -0,0 +1,232 @@
+/**
+ * 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.spout;
+
+import org.apache.storm.Config;
+import org.apache.storm.state.KeyValueState;
+import org.apache.storm.state.StateFactory;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.base.BaseRichSpout;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+
+import static org.apache.storm.spout.CheckPointState.State.COMMITTED;
+import static org.apache.storm.spout.CheckPointState.Action;
+
+/**
+ * Emits checkpoint tuples which is used to save the state of the {@link org.apache.storm.topology.IStatefulComponent}
+ * across the topology. If a topology contains Stateful bolts, Checkpoint spouts are automatically added
+ * to the topology. There is only one Checkpoint task per topology.
+ * Checkpoint spout stores its internal state in a {@link KeyValueState}.
+ *
+ * @see CheckPointState
+ */
+public class CheckpointSpout extends BaseRichSpout {
+    private static final Logger LOG = LoggerFactory.getLogger(CheckpointSpout.class);
+
+    public static final String CHECKPOINT_STREAM_ID = "$checkpoint";
+    public static final String CHECKPOINT_COMPONENT_ID = "$checkpointspout";
+    public static final String CHECKPOINT_FIELD_TXID = "txid";
+    public static final String CHECKPOINT_FIELD_ACTION = "action";
+    private static final String TX_STATE_KEY = "__state";
+    private TopologyContext context;
+    private SpoutOutputCollector collector;
+    private long lastCheckpointTs;
+    private int checkpointInterval;
+    private int sleepInterval;
+    private boolean recoveryStepInProgress;
+    private boolean checkpointStepInProgress;
+    private boolean recovering;
+    private KeyValueState<String, CheckPointState> checkpointState;
+    private CheckPointState curTxState;
+
+    @Override
+    public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
+        open(context, collector, loadCheckpointInterval(conf), loadCheckpointState(conf, context));
+    }
+
+    // package access for unit test
+    void open(TopologyContext context, SpoutOutputCollector collector,
+              int checkpointInterval, KeyValueState<String, CheckPointState> checkpointState) {
+        this.context = context;
+        this.collector = collector;
+        this.checkpointInterval = checkpointInterval;
+        this.sleepInterval = checkpointInterval / 10;
+        this.checkpointState = checkpointState;
+        this.curTxState = checkpointState.get(TX_STATE_KEY);
+        lastCheckpointTs = 0;
+        recoveryStepInProgress = false;
+        checkpointStepInProgress = false;
+        recovering = true;
+    }
+
+    @Override
+    public void nextTuple() {
+        if (shouldRecover()) {
+            handleRecovery();
+            startProgress();
+        } else if (shouldCheckpoint()) {
+            doCheckpoint();
+            startProgress();
+        } else {
+            Utils.sleep(sleepInterval);
+        }
+    }
+
+    @Override
+    public void ack(Object msgId) {
+        LOG.debug("Got ack with txid {}, current txState {}", msgId, curTxState);
+        if (curTxState.getTxid() == ((Number) msgId).longValue()) {
+            if (recovering) {
+                handleRecoveryAck();
+            } else {
+                handleCheckpointAck();
+            }
+        } else {
+            LOG.warn("Ack msgid {}, txState.txid {} mismatch", msgId, curTxState.getTxid());
+        }
+        resetProgress();
+    }
+
+    @Override
+    public void fail(Object msgId) {
+        LOG.debug("Got fail with msgid {}", msgId);
+        if (!recovering) {
+            LOG.debug("Checkpoint failed, will trigger recovery");
+            recovering = true;
+        }
+        resetProgress();
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        declarer.declareStream(CHECKPOINT_STREAM_ID, new Fields(CHECKPOINT_FIELD_TXID, CHECKPOINT_FIELD_ACTION));
+    }
+
+    public static boolean isCheckpoint(Tuple input) {
+        return CHECKPOINT_STREAM_ID.equals(input.getSourceStreamId());
+    }
+
+    /**
+     * Loads the last saved checkpoint state the from persistent storage.
+     */
+    private KeyValueState<String, CheckPointState> loadCheckpointState(Map conf, TopologyContext ctx) {
+        String namespace = ctx.getThisComponentId() + "-" + ctx.getThisTaskId();
+        KeyValueState<String, CheckPointState> state =
+                (KeyValueState<String, CheckPointState>) StateFactory.getState(namespace, conf, ctx);
+        if (state.get(TX_STATE_KEY) == null) {
+            CheckPointState txState = new CheckPointState(-1, COMMITTED);
+            state.put(TX_STATE_KEY, txState);
+            state.commit();
+            LOG.debug("Initialized checkpoint spout state with txState {}", txState);
+        } else {
+            LOG.debug("Got checkpoint spout state {}", state.get(TX_STATE_KEY));
+        }
+        return state;
+    }
+
+    private int loadCheckpointInterval(Map stormConf) {
+        int interval = 0;
+        if (stormConf.containsKey(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL)) {
+            interval = ((Number) stormConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL)).intValue();
+        }
+        // ensure checkpoint interval is not less than a sane low value.
+        interval = Math.max(100, interval);
+        LOG.info("Checkpoint interval is {} millis", interval);
+        return interval;
+    }
+
+    private boolean shouldRecover() {
+        return recovering && !recoveryStepInProgress;
+    }
+
+    private boolean shouldCheckpoint() {
+        return !recovering && !checkpointStepInProgress &&
+                (curTxState.getState() != COMMITTED || checkpointIntervalElapsed());
+    }
+
+    private boolean checkpointIntervalElapsed() {
+        return (System.currentTimeMillis() - lastCheckpointTs) > checkpointInterval;
+    }
+
+    private void handleRecovery() {
+        LOG.debug("In recovery");
+        Action action = curTxState.nextAction(true);
+        emit(curTxState.getTxid(), action);
+    }
+
+    private void handleRecoveryAck() {
+        CheckPointState nextState = curTxState.nextState(true);
+        if (curTxState != nextState) {
+            saveTxState(nextState);
+        } else {
+            LOG.debug("Recovery complete, current state {}", curTxState);
+            recovering = false;
+        }
+    }
+
+    private void doCheckpoint() {
+        LOG.debug("In checkpoint");
+        if (curTxState.getState() == COMMITTED) {
+            saveTxState(curTxState.nextState(false));
+            lastCheckpointTs = System.currentTimeMillis();
+        }
+        Action action = curTxState.nextAction(false);
+        emit(curTxState.getTxid(), action);
+    }
+
+    private void handleCheckpointAck() {
+        CheckPointState nextState = curTxState.nextState(false);
+        saveTxState(nextState);
+    }
+
+    private void emit(long txid, Action action) {
+        LOG.debug("Current state {}, emitting txid {}, action {}", curTxState, txid, action);
+        collector.emit(CHECKPOINT_STREAM_ID, new Values(txid, action), txid);
+    }
+
+    private void saveTxState(CheckPointState txState) {
+        LOG.debug("saveTxState, current state {} -> new state {}", curTxState, txState);
+        checkpointState.put(TX_STATE_KEY, txState);
+        checkpointState.commit();
+        curTxState = txState;
+    }
+
+    private void startProgress() {
+        if (recovering) {
+            recoveryStepInProgress = true;
+        } else {
+            checkpointStepInProgress = true;
+        }
+    }
+
+    private void resetProgress() {
+        if (recovering) {
+            recoveryStepInProgress = false;
+        } else {
+            checkpointStepInProgress = false;
+        }
+    }
+}


[06/10] storm git commit: Handle ack and failures in state checkpointing

Posted by pt...@apache.org.
Handle ack and failures in state checkpointing


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

Branch: refs/heads/master
Commit: 16d58e31d5e817e987cf9789139f31c344d6c3e9
Parents: d9a0698
Author: Arun Mahadevan <ai...@hortonworks.com>
Authored: Mon Dec 28 15:25:12 2015 +0530
Committer: Arun Mahadevan <ai...@hortonworks.com>
Committed: Tue Jan 12 10:21:01 2016 +0530

----------------------------------------------------------------------
 .../storm/redis/state/RedisKeyValueState.java   |  6 +-
 .../backtype/storm/spout/CheckpointSpout.java   |  4 ++
 .../topology/CheckpointTupleForwarder.java      | 44 ++++++++++--
 .../storm/topology/StatefulBoltExecutor.java    | 73 +++++++++++++++++---
 .../storm/spout/CheckpointSpoutTest.java        | 33 +++++++--
 .../topology/StatefulBoltExecutorTest.java      | 59 ++++++++++++++--
 6 files changed, 189 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/16d58e31/external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueState.java
----------------------------------------------------------------------
diff --git a/external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueState.java b/external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueState.java
index 29d33b7..175f705 100644
--- a/external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueState.java
+++ b/external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueState.java
@@ -91,6 +91,7 @@ public class RedisKeyValueState<K, V> implements KeyValueState<K, V> {
             } else {
                 txIds = new HashMap<>();
             }
+            LOG.debug("initTxids, txIds {}", txIds);
         } finally {
             jedisContainer.returnInstance(commands);
         }
@@ -230,7 +231,10 @@ public class RedisKeyValueState<K, V> implements KeyValueState<K, V> {
             } else {
                 txIds.remove(PREPARE_TXID_KEY);
             }
-            commands.hmset(txidNamespace, txIds);
+            if (!txIds.isEmpty()) {
+                LOG.debug("hmset txidNamespace {}, txIds {}", txidNamespace, txIds);
+                commands.hmset(txidNamespace, txIds);
+            }
             pendingCommit = Collections.emptyMap();
             pendingPrepare = new ConcurrentHashMap<>();
         } finally {

http://git-wip-us.apache.org/repos/asf/storm/blob/16d58e31/storm-core/src/jvm/backtype/storm/spout/CheckpointSpout.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/spout/CheckpointSpout.java b/storm-core/src/jvm/backtype/storm/spout/CheckpointSpout.java
index 32ed6e6..178770c 100644
--- a/storm-core/src/jvm/backtype/storm/spout/CheckpointSpout.java
+++ b/storm-core/src/jvm/backtype/storm/spout/CheckpointSpout.java
@@ -113,6 +113,10 @@ public class CheckpointSpout extends BaseRichSpout {
     @Override
     public void fail(Object msgId) {
         LOG.debug("Got fail with msgid {}", msgId);
+        if (!recovering) {
+            LOG.debug("Checkpoint failed, will trigger recovery");
+            recovering = true;
+        }
         resetProgress();
     }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/16d58e31/storm-core/src/jvm/backtype/storm/topology/CheckpointTupleForwarder.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/topology/CheckpointTupleForwarder.java b/storm-core/src/jvm/backtype/storm/topology/CheckpointTupleForwarder.java
index 6a0a055..1e0b12e 100644
--- a/storm-core/src/jvm/backtype/storm/topology/CheckpointTupleForwarder.java
+++ b/storm-core/src/jvm/backtype/storm/topology/CheckpointTupleForwarder.java
@@ -19,6 +19,7 @@ package backtype.storm.topology;
 
 import backtype.storm.generated.GlobalStreamId;
 import backtype.storm.spout.CheckpointSpout;
+import backtype.storm.task.IOutputCollector;
 import backtype.storm.task.OutputCollector;
 import backtype.storm.task.TopologyContext;
 import backtype.storm.tuple.Fields;
@@ -28,6 +29,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 
 import static backtype.storm.spout.CheckpointSpout.CHECKPOINT_STREAM_ID;
@@ -51,7 +53,7 @@ public class CheckpointTupleForwarder implements IRichBolt {
     private final Map<TransactionRequest, Integer> transactionRequestCount;
     private int checkPointInputTaskCount;
     private long lastTxid = Long.MIN_VALUE;
-    protected OutputCollector collector;
+    protected AnchoringOutputCollector collector;
 
     public CheckpointTupleForwarder(IRichBolt bolt) {
         this.bolt = bolt;
@@ -60,8 +62,8 @@ public class CheckpointTupleForwarder implements IRichBolt {
 
     @Override
     public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
-        bolt.prepare(stormConf, context, collector);
-        this.collector = collector;
+        this.collector = new AnchoringOutputCollector(collector);
+        bolt.prepare(stormConf, context, this.collector);
         checkPointInputTaskCount = getCheckpointInputTaskCount(context);
     }
 
@@ -94,12 +96,13 @@ public class CheckpointTupleForwarder implements IRichBolt {
      * Forwards the checkpoint tuple downstream. Sub-classes can override
      * with the logic for handling checkpoint tuple.
      *
-     * @param input  the checkpoint tuple
+     * @param checkpointTuple  the checkpoint tuple
      * @param action the action (prepare, commit, rollback or initstate)
      * @param txid   the transaction id.
      */
-    protected void handleCheckpoint(Tuple input, Action action, long txid) {
-        collector.emit(CHECKPOINT_STREAM_ID, input, new Values(txid, action));
+    protected void handleCheckpoint(Tuple checkpointTuple, Action action, long txid) {
+        collector.emit(CHECKPOINT_STREAM_ID, checkpointTuple, new Values(txid, action));
+        collector.ack(checkpointTuple);
     }
 
     /**
@@ -113,7 +116,9 @@ public class CheckpointTupleForwarder implements IRichBolt {
      * @param input the input tuple
      */
     protected void handleTuple(Tuple input) {
+        collector.setContext(input);
         bolt.execute(input);
+        collector.ack(input);
     }
 
     /**
@@ -135,6 +140,7 @@ public class CheckpointTupleForwarder implements IRichBolt {
                     }
                 } else {
                     LOG.debug("Ignoring old transaction. Action {}, txid {}", action, txid);
+                    collector.ack(input);
                 }
             } catch (Throwable th) {
                 LOG.error("Got error while processing checkpoint tuple", th);
@@ -144,8 +150,8 @@ public class CheckpointTupleForwarder implements IRichBolt {
         } else {
             LOG.debug("Waiting for action {}, txid {} from all input tasks. checkPointInputTaskCount {}, " +
                               "transactionRequestCount {}", action, txid, checkPointInputTaskCount, transactionRequestCount);
+            collector.ack(input);
         }
-        collector.ack(input);
     }
 
     /**
@@ -219,4 +225,28 @@ public class CheckpointTupleForwarder implements IRichBolt {
         }
     }
 
+
+    protected static class AnchoringOutputCollector extends OutputCollector {
+        private Tuple inputTuple;
+
+        AnchoringOutputCollector(IOutputCollector delegate) {
+            super(delegate);
+        }
+
+        void setContext(Tuple inputTuple) {
+            this.inputTuple = inputTuple;
+        }
+
+        @Override
+        public List<Integer> emit(String streamId, List<Object> tuple) {
+            return emit(streamId, inputTuple, tuple);
+        }
+
+        @Override
+        public void emitDirect(int taskId, String streamId, List<Object> tuple) {
+            emitDirect(taskId, streamId, inputTuple, tuple);
+        }
+
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/16d58e31/storm-core/src/jvm/backtype/storm/topology/StatefulBoltExecutor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/topology/StatefulBoltExecutor.java b/storm-core/src/jvm/backtype/storm/topology/StatefulBoltExecutor.java
index d44ad48..f20df33 100644
--- a/storm-core/src/jvm/backtype/storm/topology/StatefulBoltExecutor.java
+++ b/storm-core/src/jvm/backtype/storm/topology/StatefulBoltExecutor.java
@@ -46,6 +46,8 @@ public class StatefulBoltExecutor<T extends State> extends CheckpointTupleForwar
     private State state;
     private boolean boltInitialized = false;
     private List<Tuple> pendingTuples = new ArrayList<>();
+    private List<Tuple> preparedTuples = new ArrayList<>();
+    private List<Tuple> executedTuples = new ArrayList<>();
 
     public StatefulBoltExecutor(IStatefulBolt<T> bolt) {
         super(bolt);
@@ -66,35 +68,84 @@ public class StatefulBoltExecutor<T extends State> extends CheckpointTupleForwar
     }
 
     @Override
-    protected void handleCheckpoint(Tuple input, Action action, long txid) {
-        LOG.debug("handleCheckPoint with tuple {}, action {}, txid {}", input, action, txid);
+    protected void handleCheckpoint(Tuple checkpointTuple, Action action, long txid) {
+        LOG.debug("handleCheckPoint with tuple {}, action {}, txid {}", checkpointTuple, action, txid);
         if (action == PREPARE) {
-            bolt.prePrepare(txid);
-            state.prepareCommit(txid);
+            if (boltInitialized) {
+                bolt.prePrepare(txid);
+                state.prepareCommit(txid);
+                preparedTuples.addAll(executedTuples);
+                executedTuples.clear();
+            } else {
+                /*
+                 * May be the task restarted in the middle and the state needs be initialized.
+                 * Fail fast and trigger recovery.
+                  */
+                LOG.debug("Failing checkpointTuple, PREPARE received when bolt state is not initialized.");
+                collector.fail(checkpointTuple);
+                return;
+            }
         } else if (action == COMMIT) {
             bolt.preCommit(txid);
             state.commit(txid);
+            ack(preparedTuples);
         } else if (action == ROLLBACK) {
             bolt.preRollback();
             state.rollback();
+            fail(preparedTuples);
+            fail(executedTuples);
         } else if (action == INITSTATE) {
-            bolt.initState((T) state);
-            boltInitialized = true;
-            LOG.debug("{} pending tuples to process", pendingTuples.size());
-            for (Tuple tuple : pendingTuples) {
-                bolt.execute(tuple);
+            if (!boltInitialized) {
+                bolt.initState((T) state);
+                boltInitialized = true;
+                LOG.debug("{} pending tuples to process", pendingTuples.size());
+                for (Tuple tuple : pendingTuples) {
+                    doExecute(tuple);
+                }
+                pendingTuples.clear();
+            } else {
+                LOG.debug("Bolt state is already initialized, ignoring tuple {}, action {}, txid {}",
+                          checkpointTuple, action, txid);
             }
         }
-        collector.emit(CheckpointSpout.CHECKPOINT_STREAM_ID, input, new Values(txid, action));
+        collector.emit(CheckpointSpout.CHECKPOINT_STREAM_ID, checkpointTuple, new Values(txid, action));
+        collector.ack(checkpointTuple);
     }
 
     @Override
     protected void handleTuple(Tuple input) {
         if (boltInitialized) {
-            bolt.execute(input);
+            doExecute(input);
         } else {
             LOG.debug("Bolt state not initialized, adding tuple {} to pending tuples", input);
             pendingTuples.add(input);
         }
     }
+
+    private void doExecute(Tuple tuple) {
+        collector.setContext(tuple);
+        bolt.execute(tuple);
+        executedTuples.add(tuple);
+    }
+
+    private void ack(List<Tuple> tuples) {
+        if (!tuples.isEmpty()) {
+            LOG.debug("Acking {} tuples", tuples.size());
+            for (Tuple tuple : tuples) {
+                collector.ack(tuple);
+            }
+            tuples.clear();
+        }
+    }
+
+    private void fail(List<Tuple> tuples) {
+        if (!tuples.isEmpty()) {
+            LOG.debug("Failing {} tuples", tuples.size());
+            for (Tuple tuple : tuples) {
+                collector.fail(tuple);
+            }
+            tuples.clear();
+        }
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/16d58e31/storm-core/test/jvm/backtype/storm/spout/CheckpointSpoutTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/spout/CheckpointSpoutTest.java b/storm-core/test/jvm/backtype/storm/spout/CheckpointSpoutTest.java
index f341f70..1aa5ab2 100644
--- a/storm-core/test/jvm/backtype/storm/spout/CheckpointSpoutTest.java
+++ b/storm-core/test/jvm/backtype/storm/spout/CheckpointSpoutTest.java
@@ -19,6 +19,7 @@ package backtype.storm.spout;
 
 import backtype.storm.Config;
 import backtype.storm.state.KeyValueState;
+import backtype.storm.state.State;
 import backtype.storm.state.StateFactory;
 import backtype.storm.task.TopologyContext;
 import backtype.storm.tuple.Values;
@@ -31,6 +32,7 @@ import org.mockito.Mockito;
 import java.util.HashMap;
 import java.util.Map;
 
+import static backtype.storm.spout.CheckPointState.State.COMMITTED;
 import static org.junit.Assert.assertEquals;
 import static backtype.storm.spout.CheckPointState.Action;
 
@@ -104,29 +106,46 @@ public class CheckpointSpoutTest {
     @Test
     public void testPrepareWithFail() throws Exception {
         Map<String, Object> stormConf = new HashMap<>();
-        stormConf.put(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL, 0);
-        spout.open(stormConf, mockTopologyContext, mockOutputCollector);
+        KeyValueState<String, CheckPointState> state =
+                (KeyValueState<String, CheckPointState>) StateFactory.getState("__state", stormConf, mockTopologyContext);
+        CheckPointState txState = new CheckPointState(-1, COMMITTED);
+        state.put("__state", txState);
+
+        spout.open(mockTopologyContext, mockOutputCollector, 0, state);
         ArgumentCaptor<String> stream = ArgumentCaptor.forClass(String.class);
         ArgumentCaptor<Values> values = ArgumentCaptor.forClass(Values.class);
         ArgumentCaptor<Object> msgId = ArgumentCaptor.forClass(Object.class);
 
         spout.nextTuple();
         spout.ack(-1L);
+        Utils.sleep(10);
         spout.nextTuple();
-        spout.fail(0L);
+        spout.ack(0L);
         Utils.sleep(10);
         spout.nextTuple();
-        spout.fail(0L);
+        spout.ack(0L);
         Utils.sleep(10);
         spout.nextTuple();
-        Mockito.verify(mockOutputCollector, Mockito.times(4)).emit(stream.capture(),
+        spout.fail(1L);
+        Utils.sleep(10);
+        spout.nextTuple();
+        spout.fail(1L);
+        Utils.sleep(10);
+        spout.nextTuple();
+        spout.ack(1L);
+        Utils.sleep(10);
+        spout.nextTuple();
+        spout.ack(0L);
+        Utils.sleep(10);
+        spout.nextTuple();
+        Mockito.verify(mockOutputCollector, Mockito.times(8)).emit(stream.capture(),
                                                                    values.capture(),
                                                                    msgId.capture());
 
-        Values expectedTuple = new Values(0L, Action.PREPARE);
+        Values expectedTuple = new Values(1L, Action.PREPARE);
         assertEquals(CheckpointSpout.CHECKPOINT_STREAM_ID, stream.getValue());
         assertEquals(expectedTuple, values.getValue());
-        assertEquals(0L, msgId.getValue());
+        assertEquals(1L, msgId.getValue());
 
     }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/16d58e31/storm-core/test/jvm/backtype/storm/topology/StatefulBoltExecutorTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/topology/StatefulBoltExecutorTest.java b/storm-core/test/jvm/backtype/storm/topology/StatefulBoltExecutorTest.java
index 2959a10..6f73727 100644
--- a/storm-core/test/jvm/backtype/storm/topology/StatefulBoltExecutorTest.java
+++ b/storm-core/test/jvm/backtype/storm/topology/StatefulBoltExecutorTest.java
@@ -35,10 +35,8 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 
+import static backtype.storm.spout.CheckPointState.Action.*;
 import static backtype.storm.spout.CheckpointSpout.*;
-import static backtype.storm.spout.CheckPointState.Action.INITSTATE;
-import static backtype.storm.spout.CheckPointState.Action.ROLLBACK;
-import static backtype.storm.spout.CheckPointState.Action.COMMIT;
 
 /**
  * Unit tests for {@link StatefulBoltExecutor}
@@ -83,7 +81,6 @@ public class StatefulBoltExecutorTest {
         Mockito.verify(mockBolt, Mockito.times(0)).execute(Mockito.any(Tuple.class));
     }
 
-
     @Test
     public void testHandleTuple() throws Exception {
         Mockito.when(mockTuple.getSourceStreamId()).thenReturn("default");
@@ -121,4 +118,58 @@ public class StatefulBoltExecutorTest {
         Mockito.verify(mockBolt, Mockito.times(1)).preCommit(new Long(0));
         Mockito.verify(mockState, Mockito.times(1)).commit(new Long(0));
     }
+
+    @Test
+    public void testPrepareAndRollbackBeforeInitstate() throws Exception {
+        Mockito.when(mockTuple.getSourceStreamId()).thenReturn("default");
+        executor.execute(mockTuple);
+        Mockito.when(mockCheckpointTuple.getSourceStreamId()).thenReturn(CheckpointSpout.CHECKPOINT_STREAM_ID);
+        Mockito.when(mockCheckpointTuple.getValueByField(CHECKPOINT_FIELD_ACTION)).thenReturn(PREPARE);
+        Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(100));
+        executor.execute(mockCheckpointTuple);
+        Mockito.verify(mockOutputCollector, Mockito.times(1)).fail(mockCheckpointTuple);
+
+        Mockito.when(mockCheckpointTuple.getValueByField(CHECKPOINT_FIELD_ACTION)).thenReturn(ROLLBACK);
+        Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(100));
+        Mockito.doNothing().when(mockOutputCollector).ack(mockCheckpointTuple);
+        executor.execute(mockCheckpointTuple);
+        Mockito.verify(mockState, Mockito.times(1)).rollback();
+    }
+
+    @Test
+    public void testCommitBeforeInitstate() throws Exception {
+        Mockito.when(mockTuple.getSourceStreamId()).thenReturn("default");
+        Mockito.when(mockCheckpointTuple.getSourceStreamId()).thenReturn(CheckpointSpout.CHECKPOINT_STREAM_ID);
+        Mockito.when(mockCheckpointTuple.getValueByField(CHECKPOINT_FIELD_ACTION)).thenReturn(COMMIT);
+        Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(100));
+        executor.execute(mockCheckpointTuple);
+        Mockito.verify(mockOutputCollector, Mockito.times(1)).ack(mockCheckpointTuple);
+
+        Mockito.when(mockCheckpointTuple.getValueByField(CHECKPOINT_FIELD_ACTION)).thenReturn(ROLLBACK);
+        Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(100));
+        executor.execute(mockCheckpointTuple);
+        Mockito.verify(mockState, Mockito.times(1)).rollback();
+    }
+
+    @Test
+    public void testPrepareAndCommit() throws Exception {
+        Mockito.when(mockTuple.getSourceStreamId()).thenReturn("default");
+        Mockito.when(mockCheckpointTuple.getSourceStreamId()).thenReturn(CheckpointSpout.CHECKPOINT_STREAM_ID);
+        Mockito.when(mockCheckpointTuple.getValueByField(CHECKPOINT_FIELD_ACTION)).thenReturn(INITSTATE);
+        Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(0));
+        executor.execute(mockCheckpointTuple);
+
+        executor.execute(mockTuple);
+        Mockito.when(mockCheckpointTuple.getSourceStreamId()).thenReturn(CheckpointSpout.CHECKPOINT_STREAM_ID);
+        Mockito.when(mockCheckpointTuple.getValueByField(CHECKPOINT_FIELD_ACTION)).thenReturn(PREPARE);
+        Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(100));
+        executor.execute(mockCheckpointTuple);
+        executor.execute(mockTuple);
+        Mockito.when(mockCheckpointTuple.getValueByField(CHECKPOINT_FIELD_ACTION)).thenReturn(COMMIT);
+        Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(100));
+        executor.execute(mockCheckpointTuple);
+        Mockito.verify(mockState, Mockito.times(1)).commit(new Long(100));
+        Mockito.verify(mockBolt, Mockito.times(2)).execute(mockTuple);
+        Mockito.verify(mockOutputCollector, Mockito.times(1)).ack(mockTuple);
+    }
 }
\ No newline at end of file


[08/10] storm git commit: Package name fixes

Posted by pt...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/src/jvm/org/apache/storm/state/DefaultStateSerializer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/state/DefaultStateSerializer.java b/storm-core/src/jvm/org/apache/storm/state/DefaultStateSerializer.java
new file mode 100644
index 0000000..55e934e
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/state/DefaultStateSerializer.java
@@ -0,0 +1,65 @@
+/**
+ * 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.state;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * A default implementation that uses Kryo to serialize and de-serialize
+ * the state.
+ */
+public class DefaultStateSerializer<T> implements Serializer<T> {
+    private final Kryo kryo;
+    private final Output output;
+
+    /**
+     * Constructs a {@link DefaultStateSerializer} instance with the given list
+     * of classes registered in kryo.
+     *
+     * @param classesToRegister the classes to register.
+     */
+    public DefaultStateSerializer(List<Class<?>> classesToRegister) {
+        kryo = new Kryo();
+        output = new Output(2000, 2000000000);
+        for (Class<?> klazz : classesToRegister) {
+            kryo.register(klazz);
+        }
+    }
+
+    public DefaultStateSerializer() {
+        this(Collections.<Class<?>>emptyList());
+    }
+
+    @Override
+    public byte[] serialize(T obj) {
+        output.clear();
+        kryo.writeClassAndObject(output, obj);
+        return output.toBytes();
+    }
+
+    @Override
+    public T deserialize(byte[] b) {
+        Input input = new Input(b);
+        return (T) kryo.readClassAndObject(input);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/src/jvm/org/apache/storm/state/InMemoryKeyValueState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/state/InMemoryKeyValueState.java b/storm-core/src/jvm/org/apache/storm/state/InMemoryKeyValueState.java
new file mode 100644
index 0000000..4b116ba
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/state/InMemoryKeyValueState.java
@@ -0,0 +1,114 @@
+/**
+ * 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.state;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * An in-memory implementation of the {@link State}
+ */
+public class InMemoryKeyValueState<K, V> implements KeyValueState<K, V> {
+    private static final Logger LOG = LoggerFactory.getLogger(InMemoryKeyValueState.class);
+    private static final long DEFAULT_TXID = -1;
+    private TxIdState<K, V> commitedState;
+    private TxIdState<K, V> preparedState;
+    private Map<K, V> state = new ConcurrentHashMap<>();
+
+    private static class TxIdState<K, V> {
+        private long txid;
+        private Map<K, V> state;
+
+        TxIdState(long txid, Map<K, V> state) {
+            this.txid = txid;
+            this.state = state;
+        }
+
+        @Override
+        public String toString() {
+            return "TxIdState{" +
+                    "txid=" + txid +
+                    ", state=" + state +
+                    '}';
+        }
+    }
+
+    @Override
+    public void put(K key, V value) {
+        state.put(key, value);
+    }
+
+    @Override
+    public V get(K key) {
+        return state.get(key);
+    }
+
+    @Override
+    public V get(K key, V defaultValue) {
+        V val = get(key);
+        return val != null ? val : defaultValue;
+    }
+
+    @Override
+    public void commit() {
+        commitedState = new TxIdState<>(DEFAULT_TXID, new ConcurrentHashMap<>(state));
+    }
+
+    @Override
+    public void prepareCommit(long txid) {
+        LOG.debug("prepare commit, txid {}", txid);
+        if (preparedState != null && txid > preparedState.txid) {
+            throw new RuntimeException("Cannot prepare a new txn while there is a pending txn");
+        }
+        preparedState = new TxIdState<>(txid, new ConcurrentHashMap<K, V>(state));
+    }
+
+    @Override
+    public void commit(long txid) {
+        LOG.debug("commit, txid {}", txid);
+        if (preparedState != null && txid == preparedState.txid) {
+            commitedState = preparedState;
+            preparedState = null;
+        } else {
+            throw new RuntimeException("Invalid prepared state for commit, " +
+                                               "preparedState " + preparedState + " txid " + txid);
+        }
+    }
+
+    @Override
+    public void rollback() {
+        preparedState = null;
+        if (commitedState != null) {
+            state = commitedState.state;
+        } else {
+            state = new ConcurrentHashMap<>();
+        }
+    }
+
+    @Override
+    public String toString() {
+        return "InMemoryKeyValueState{" +
+                "commitedState=" + commitedState +
+                ", preparedState=" + preparedState +
+                ", state=" + state +
+                '}';
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/src/jvm/org/apache/storm/state/InMemoryKeyValueStateProvider.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/state/InMemoryKeyValueStateProvider.java b/storm-core/src/jvm/org/apache/storm/state/InMemoryKeyValueStateProvider.java
new file mode 100644
index 0000000..bc33241
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/state/InMemoryKeyValueStateProvider.java
@@ -0,0 +1,43 @@
+/**
+ * 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.state;
+
+import org.apache.storm.task.TopologyContext;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Provides {@link InMemoryKeyValueState}
+ */
+public class InMemoryKeyValueStateProvider implements StateProvider {
+    private final ConcurrentHashMap<String, State> states = new ConcurrentHashMap<>();
+
+    @Override
+    public State newState(String namespace, Map stormConf, TopologyContext context) {
+        State state = states.get(namespace);
+        if (state == null) {
+            State newState = new InMemoryKeyValueState<>();
+            state = states.putIfAbsent(namespace, newState);
+            if (state == null) {
+                state = newState;
+            }
+        }
+        return state;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/src/jvm/org/apache/storm/state/KeyValueState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/state/KeyValueState.java b/storm-core/src/jvm/org/apache/storm/state/KeyValueState.java
new file mode 100644
index 0000000..3ab60f1
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/state/KeyValueState.java
@@ -0,0 +1,48 @@
+/**
+ * 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.state;
+
+/**
+ * A state that supports key-value mappings.
+ */
+public interface KeyValueState<K, V> extends State {
+    /**
+     * Maps the value with the key
+     *
+     * @param key   the key
+     * @param value the value
+     */
+    void put(K key, V value);
+
+    /**
+     * Returns the value mapped to the key
+     *
+     * @param key the key
+     * @return the value or null if no mapping is found
+     */
+    V get(K key);
+
+    /**
+     * Returns the value mapped to the key or defaultValue if no mapping is found.
+     *
+     * @param key          the key
+     * @param defaultValue the value to return if no mapping is found
+     * @return the value or defaultValue if no mapping is found
+     */
+    V get(K key, V defaultValue);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/src/jvm/org/apache/storm/state/Serializer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/state/Serializer.java b/storm-core/src/jvm/org/apache/storm/state/Serializer.java
new file mode 100644
index 0000000..b442cec
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/state/Serializer.java
@@ -0,0 +1,30 @@
+/**
+ * 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.state;
+
+import java.io.Serializable;
+
+/**
+ * Interface to be implemented for serlializing and de-serializing the
+ * state.
+ */
+public interface Serializer<T> extends Serializable {
+    byte[] serialize(T obj);
+
+    T deserialize(byte[] b);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/src/jvm/org/apache/storm/state/State.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/state/State.java b/storm-core/src/jvm/org/apache/storm/state/State.java
new file mode 100644
index 0000000..2a7505c
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/state/State.java
@@ -0,0 +1,55 @@
+/**
+ * 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.state;
+
+import org.apache.storm.topology.IStatefulBolt;
+
+/**
+ * The state of the component that is either managed by the framework (e.g in case of {@link IStatefulBolt})
+ * or managed by the the individual components themselves.
+ */
+public interface State {
+    /**
+     * Invoked by the framework to prepare a transaction for commit. It should be possible
+     * to commit the prepared state later.
+     * <p>
+     * The same txid can be prepared again, but the next txid cannot be prepared
+     * when previous one is not yet committed.
+     * </p>
+     *
+     * @param txid the transaction id
+     */
+    void prepareCommit(long txid);
+
+    /**
+     * Commit a previously prepared transaction. It should be possible to retrieve a committed state later.
+     *
+     * @param txid the transaction id
+     */
+    void commit(long txid);
+
+    /**
+     * Persist the current state. This is used when the component manages the state.
+     */
+    void commit();
+
+    /**
+     * Rollback a prepared transaction to the previously committed state.
+     */
+    void rollback();
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/src/jvm/org/apache/storm/state/StateFactory.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/state/StateFactory.java b/storm-core/src/jvm/org/apache/storm/state/StateFactory.java
new file mode 100644
index 0000000..d5ec7ae
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/state/StateFactory.java
@@ -0,0 +1,69 @@
+/**
+ * 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.state;
+
+import org.apache.storm.Config;
+import org.apache.storm.task.TopologyContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+
+/**
+ * A factory for creating {@link State} instances
+ */
+public class StateFactory {
+    private static final Logger LOG = LoggerFactory.getLogger(StateFactory.class);
+
+    private static final String DEFAULT_PROVIDER = "org.apache.storm.state.InMemoryKeyValueStateProvider";
+
+    /**
+     * Returns a new state instance using the {@link Config#TOPOLOGY_STATE_PROVIDER} or a
+     * {@link InMemoryKeyValueState} if no provider is configured.
+     *
+     * @param namespace the state namespace
+     * @param stormConf the storm conf
+     * @param context   the topology context
+     * @return the state instance
+     */
+    public static State getState(String namespace, Map stormConf, TopologyContext context) {
+        State state;
+        try {
+            String provider = null;
+            if (stormConf.containsKey(Config.TOPOLOGY_STATE_PROVIDER)) {
+                provider = (String) stormConf.get(Config.TOPOLOGY_STATE_PROVIDER);
+            } else {
+                provider = DEFAULT_PROVIDER;
+            }
+            Class<?> klazz = Class.forName(provider);
+            Object object = klazz.newInstance();
+            if (object instanceof StateProvider) {
+                state = ((StateProvider) object).newState(namespace, stormConf, context);
+            } else {
+                String msg = "Invalid state provider '" + provider +
+                        "'. Should implement org.apache.storm.state.StateProvider";
+                LOG.error(msg);
+                throw new RuntimeException(msg);
+            }
+        } catch (Exception ex) {
+            LOG.error("Got exception while loading the state provider", ex);
+            throw new RuntimeException(ex);
+        }
+        return state;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/src/jvm/org/apache/storm/state/StateProvider.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/state/StateProvider.java b/storm-core/src/jvm/org/apache/storm/state/StateProvider.java
new file mode 100644
index 0000000..0dad13e
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/state/StateProvider.java
@@ -0,0 +1,38 @@
+/**
+ * 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.state;
+
+import org.apache.storm.task.TopologyContext;
+
+import java.util.Map;
+
+/**
+ * Used by the {@link StateFactory} to create a new state instances.
+ */
+public interface StateProvider {
+    /**
+     * Returns a new state instance. Each state belongs unique namespace which is typically
+     * the componentid-task of the task, so that each task can have its own unique state.
+     *
+     * @param namespace a namespace of the state
+     * @param stormConf the storm topology configuration
+     * @param context   the {@link TopologyContext}
+     * @return a previously saved state instance
+     */
+    State newState(String namespace, Map stormConf, TopologyContext context);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/src/jvm/org/apache/storm/topology/CheckpointTupleForwarder.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/topology/CheckpointTupleForwarder.java b/storm-core/src/jvm/org/apache/storm/topology/CheckpointTupleForwarder.java
new file mode 100644
index 0000000..675be57
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/topology/CheckpointTupleForwarder.java
@@ -0,0 +1,250 @@
+/**
+ * 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.topology;
+
+import org.apache.storm.generated.GlobalStreamId;
+import org.apache.storm.spout.CheckpointSpout;
+import org.apache.storm.task.IOutputCollector;
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+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;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.storm.spout.CheckPointState.Action;
+import static org.apache.storm.spout.CheckPointState.Action.ROLLBACK;
+import static org.apache.storm.spout.CheckpointSpout.*;
+
+/**
+ * Wraps {@link IRichBolt} and forwards checkpoint tuples in a
+ * stateful topology.
+ * <p>
+ * When a storm topology contains one or more {@link IStatefulBolt} all non-stateful
+ * bolts are wrapped in {@link CheckpointTupleForwarder} so that the checkpoint tuples
+ * can flow through the entire topology DAG.
+ * </p>
+ */
+public class CheckpointTupleForwarder implements IRichBolt {
+    private static final Logger LOG = LoggerFactory.getLogger(CheckpointTupleForwarder.class);
+    private final IRichBolt bolt;
+    private final Map<TransactionRequest, Integer> transactionRequestCount;
+    private int checkPointInputTaskCount;
+    private long lastTxid = Long.MIN_VALUE;
+    protected AnchoringOutputCollector collector;
+
+    public CheckpointTupleForwarder(IRichBolt bolt) {
+        this.bolt = bolt;
+        transactionRequestCount = new HashMap<>();
+    }
+
+    @Override
+    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+        this.collector = new AnchoringOutputCollector(collector);
+        bolt.prepare(stormConf, context, this.collector);
+        checkPointInputTaskCount = getCheckpointInputTaskCount(context);
+    }
+
+    @Override
+    public void execute(Tuple input) {
+        if (CheckpointSpout.isCheckpoint(input)) {
+            processCheckpoint(input);
+        } else {
+            handleTuple(input);
+        }
+    }
+
+    @Override
+    public void cleanup() {
+        bolt.cleanup();
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        bolt.declareOutputFields(declarer);
+        declarer.declareStream(CHECKPOINT_STREAM_ID, new Fields(CHECKPOINT_FIELD_TXID, CHECKPOINT_FIELD_ACTION));
+    }
+
+    @Override
+    public Map<String, Object> getComponentConfiguration() {
+        return bolt.getComponentConfiguration();
+    }
+
+    /**
+     * Forwards the checkpoint tuple downstream. Sub-classes can override
+     * with the logic for handling checkpoint tuple.
+     *
+     * @param checkpointTuple  the checkpoint tuple
+     * @param action the action (prepare, commit, rollback or initstate)
+     * @param txid   the transaction id.
+     */
+    protected void handleCheckpoint(Tuple checkpointTuple, Action action, long txid) {
+        collector.emit(CHECKPOINT_STREAM_ID, checkpointTuple, new Values(txid, action));
+        collector.ack(checkpointTuple);
+    }
+
+    /**
+     * Hands off tuple to the wrapped bolt to execute. Sub-classes can
+     * override the behavior.
+     * <p>
+     * Right now tuples continue to get forwarded while waiting for checkpoints to arrive on other streams
+     * after checkpoint arrives on one of the streams. This can cause duplicates but still at least once.
+     * </p>
+     *
+     * @param input the input tuple
+     */
+    protected void handleTuple(Tuple input) {
+        collector.setContext(input);
+        bolt.execute(input);
+        collector.ack(input);
+    }
+
+    /**
+     * Invokes handleCheckpoint once checkpoint tuple is received on
+     * all input checkpoint streams to this component.
+     */
+    private void processCheckpoint(Tuple input) {
+        Action action = (Action) input.getValueByField(CHECKPOINT_FIELD_ACTION);
+        long txid = input.getLongByField(CHECKPOINT_FIELD_TXID);
+        if (shouldProcessTransaction(action, txid)) {
+            LOG.debug("Processing action {}, txid {}", action, txid);
+            try {
+                if (txid >= lastTxid) {
+                    handleCheckpoint(input, action, txid);
+                    if (action == ROLLBACK) {
+                        lastTxid = txid - 1;
+                    } else {
+                        lastTxid = txid;
+                    }
+                } else {
+                    LOG.debug("Ignoring old transaction. Action {}, txid {}", action, txid);
+                    collector.ack(input);
+                }
+            } catch (Throwable th) {
+                LOG.error("Got error while processing checkpoint tuple", th);
+                collector.fail(input);
+                collector.reportError(th);
+            }
+        } else {
+            LOG.debug("Waiting for action {}, txid {} from all input tasks. checkPointInputTaskCount {}, " +
+                              "transactionRequestCount {}", action, txid, checkPointInputTaskCount, transactionRequestCount);
+            collector.ack(input);
+        }
+    }
+
+    /**
+     * returns the total number of input checkpoint streams across
+     * all input tasks to this component.
+     */
+    private int getCheckpointInputTaskCount(TopologyContext context) {
+        int count = 0;
+        for (GlobalStreamId inputStream : context.getThisSources().keySet()) {
+            if (CHECKPOINT_STREAM_ID.equals(inputStream.get_streamId())) {
+                count += context.getComponentTasks(inputStream.get_componentId()).size();
+            }
+        }
+        return count;
+    }
+
+    /**
+     * Checks if check points have been received from all tasks across
+     * all input streams to this component
+     */
+    private boolean shouldProcessTransaction(Action action, long txid) {
+        TransactionRequest request = new TransactionRequest(action, txid);
+        Integer count;
+        if ((count = transactionRequestCount.get(request)) == null) {
+            transactionRequestCount.put(request, 1);
+            count = 1;
+        } else {
+            transactionRequestCount.put(request, ++count);
+        }
+        if (count == checkPointInputTaskCount) {
+            transactionRequestCount.remove(request);
+            return true;
+        }
+        return false;
+    }
+
+    private static class TransactionRequest {
+        private final Action action;
+        private final long txid;
+
+        TransactionRequest(Action action, long txid) {
+            this.action = action;
+            this.txid = txid;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+
+            TransactionRequest that = (TransactionRequest) o;
+
+            if (txid != that.txid) return false;
+            return !(action != null ? !action.equals(that.action) : that.action != null);
+
+        }
+
+        @Override
+        public int hashCode() {
+            int result = action != null ? action.hashCode() : 0;
+            result = 31 * result + (int) (txid ^ (txid >>> 32));
+            return result;
+        }
+
+        @Override
+        public String toString() {
+            return "TransactionRequest{" +
+                    "action='" + action + '\'' +
+                    ", txid=" + txid +
+                    '}';
+        }
+    }
+
+
+    protected static class AnchoringOutputCollector extends OutputCollector {
+        private Tuple inputTuple;
+
+        AnchoringOutputCollector(IOutputCollector delegate) {
+            super(delegate);
+        }
+
+        void setContext(Tuple inputTuple) {
+            this.inputTuple = inputTuple;
+        }
+
+        @Override
+        public List<Integer> emit(String streamId, List<Object> tuple) {
+            return emit(streamId, inputTuple, tuple);
+        }
+
+        @Override
+        public void emitDirect(int taskId, String streamId, List<Object> tuple) {
+            emitDirect(taskId, streamId, inputTuple, tuple);
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/src/jvm/org/apache/storm/topology/IStatefulBolt.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/topology/IStatefulBolt.java b/storm-core/src/jvm/org/apache/storm/topology/IStatefulBolt.java
new file mode 100644
index 0000000..1c2c5fc
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/topology/IStatefulBolt.java
@@ -0,0 +1,26 @@
+/**
+ * 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.topology;
+
+import org.apache.storm.state.State;
+
+/**
+ * A bolt abstraction for supporting stateful computation.
+ */
+public interface IStatefulBolt<T extends State> extends IStatefulComponent<T>, IRichBolt {
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/src/jvm/org/apache/storm/topology/IStatefulComponent.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/topology/IStatefulComponent.java b/storm-core/src/jvm/org/apache/storm/topology/IStatefulComponent.java
new file mode 100644
index 0000000..c2f8789
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/topology/IStatefulComponent.java
@@ -0,0 +1,56 @@
+/**
+ * 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.topology;
+
+import org.apache.storm.state.State;
+
+/**
+ * <p>
+ * Common methods for stateful components in the topology.
+ * </p>
+ * A stateful component is one that has state (e.g. the result of some computation in a bolt)
+ * and wants the framework to manage its state.
+ */
+public interface IStatefulComponent<T extends State> extends IComponent {
+    /**
+     * This method is invoked by the framework with the previously
+     * saved state of the component. This is invoked after prepare but before
+     * the component starts processing tuples.
+     *
+     * @param state the previously saved state of the component.
+     */
+    void initState(T state);
+
+    /**
+     * This is a hook for the component to perform some actions just before the
+     * framework commits its state.
+     */
+    void preCommit(long txid);
+
+    /**
+     * This is a hook for the component to perform some actions just before the
+     * framework prepares its state.
+     */
+    void prePrepare(long txid);
+
+    /**
+     * This is a hook for the component to perform some actions just before the
+     * framework rolls back the prepared state.
+     */
+    void preRollback();
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/src/jvm/org/apache/storm/topology/StatefulBoltExecutor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/topology/StatefulBoltExecutor.java b/storm-core/src/jvm/org/apache/storm/topology/StatefulBoltExecutor.java
new file mode 100644
index 0000000..c9c36ee
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/topology/StatefulBoltExecutor.java
@@ -0,0 +1,151 @@
+/**
+ * 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.topology;
+
+import org.apache.storm.spout.CheckpointSpout;
+import org.apache.storm.state.State;
+import org.apache.storm.state.StateFactory;
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.Values;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.storm.spout.CheckPointState.Action;
+import static org.apache.storm.spout.CheckPointState.Action.COMMIT;
+import static org.apache.storm.spout.CheckPointState.Action.PREPARE;
+import static org.apache.storm.spout.CheckPointState.Action.ROLLBACK;
+import static org.apache.storm.spout.CheckPointState.Action.INITSTATE;
+
+/**
+ * Wraps a {@link IStatefulBolt} and manages the state of the bolt.
+ */
+public class StatefulBoltExecutor<T extends State> extends CheckpointTupleForwarder {
+    private static final Logger LOG = LoggerFactory.getLogger(StatefulBoltExecutor.class);
+    private final IStatefulBolt<T> bolt;
+    private State state;
+    private boolean boltInitialized = false;
+    private List<Tuple> pendingTuples = new ArrayList<>();
+    private List<Tuple> preparedTuples = new ArrayList<>();
+    private List<Tuple> executedTuples = new ArrayList<>();
+
+    public StatefulBoltExecutor(IStatefulBolt<T> bolt) {
+        super(bolt);
+        this.bolt = bolt;
+    }
+
+    @Override
+    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+        // get the last successfully committed state from state store
+        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId();
+        prepare(stormConf, context, collector, StateFactory.getState(namespace, stormConf, context));
+    }
+
+    // package access for unit tests
+    void prepare(Map stormConf, TopologyContext context, OutputCollector collector, State state) {
+        super.prepare(stormConf, context, collector);
+        this.state = state;
+    }
+
+    @Override
+    protected void handleCheckpoint(Tuple checkpointTuple, Action action, long txid) {
+        LOG.debug("handleCheckPoint with tuple {}, action {}, txid {}", checkpointTuple, action, txid);
+        if (action == PREPARE) {
+            if (boltInitialized) {
+                bolt.prePrepare(txid);
+                state.prepareCommit(txid);
+                preparedTuples.addAll(executedTuples);
+                executedTuples.clear();
+            } else {
+                /*
+                 * May be the task restarted in the middle and the state needs be initialized.
+                 * Fail fast and trigger recovery.
+                  */
+                LOG.debug("Failing checkpointTuple, PREPARE received when bolt state is not initialized.");
+                collector.fail(checkpointTuple);
+                return;
+            }
+        } else if (action == COMMIT) {
+            bolt.preCommit(txid);
+            state.commit(txid);
+            ack(preparedTuples);
+        } else if (action == ROLLBACK) {
+            bolt.preRollback();
+            state.rollback();
+            fail(preparedTuples);
+            fail(executedTuples);
+        } else if (action == INITSTATE) {
+            if (!boltInitialized) {
+                bolt.initState((T) state);
+                boltInitialized = true;
+                LOG.debug("{} pending tuples to process", pendingTuples.size());
+                for (Tuple tuple : pendingTuples) {
+                    doExecute(tuple);
+                }
+                pendingTuples.clear();
+            } else {
+                LOG.debug("Bolt state is already initialized, ignoring tuple {}, action {}, txid {}",
+                          checkpointTuple, action, txid);
+            }
+        }
+        collector.emit(CheckpointSpout.CHECKPOINT_STREAM_ID, checkpointTuple, new Values(txid, action));
+        collector.ack(checkpointTuple);
+    }
+
+    @Override
+    protected void handleTuple(Tuple input) {
+        if (boltInitialized) {
+            doExecute(input);
+        } else {
+            LOG.debug("Bolt state not initialized, adding tuple {} to pending tuples", input);
+            pendingTuples.add(input);
+        }
+    }
+
+    private void doExecute(Tuple tuple) {
+        collector.setContext(tuple);
+        bolt.execute(tuple);
+        executedTuples.add(tuple);
+    }
+
+    private void ack(List<Tuple> tuples) {
+        if (!tuples.isEmpty()) {
+            LOG.debug("Acking {} tuples", tuples.size());
+            for (Tuple tuple : tuples) {
+                collector.ack(tuple);
+            }
+            tuples.clear();
+        }
+    }
+
+    private void fail(List<Tuple> tuples) {
+        if (!tuples.isEmpty()) {
+            LOG.debug("Failing {} tuples", tuples.size());
+            for (Tuple tuple : tuples) {
+                collector.fail(tuple);
+            }
+            tuples.clear();
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/src/jvm/org/apache/storm/topology/base/BaseStatefulBolt.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/topology/base/BaseStatefulBolt.java b/storm-core/src/jvm/org/apache/storm/topology/base/BaseStatefulBolt.java
new file mode 100644
index 0000000..d84d18b
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/topology/base/BaseStatefulBolt.java
@@ -0,0 +1,64 @@
+/**
+ * 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.topology.base;
+
+import org.apache.storm.state.State;
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.IStatefulBolt;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+
+import java.util.Map;
+
+public abstract class BaseStatefulBolt<T extends State> implements IStatefulBolt<T> {
+
+    @Override
+    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+        // NOOP
+    }
+
+    @Override
+    public void cleanup() {
+        // NOOP
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        // NOOP
+    }
+
+    @Override
+    public Map<String, Object> getComponentConfiguration() {
+        return null;
+    }
+
+    @Override
+    public void preCommit(long txid) {
+        // NOOP
+    }
+
+    @Override
+    public void prePrepare(long txid) {
+        // NOOP
+    }
+
+    @Override
+    public void preRollback() {
+        // NOOP
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/test/jvm/backtype/storm/spout/CheckpointSpoutTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/spout/CheckpointSpoutTest.java b/storm-core/test/jvm/backtype/storm/spout/CheckpointSpoutTest.java
deleted file mode 100644
index 1aa5ab2..0000000
--- a/storm-core/test/jvm/backtype/storm/spout/CheckpointSpoutTest.java
+++ /dev/null
@@ -1,263 +0,0 @@
-/**
- * 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 backtype.storm.spout;
-
-import backtype.storm.Config;
-import backtype.storm.state.KeyValueState;
-import backtype.storm.state.State;
-import backtype.storm.state.StateFactory;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.tuple.Values;
-import backtype.storm.utils.Utils;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.ArgumentCaptor;
-import org.mockito.Mockito;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import static backtype.storm.spout.CheckPointState.State.COMMITTED;
-import static org.junit.Assert.assertEquals;
-import static backtype.storm.spout.CheckPointState.Action;
-
-/**
- * Unit test for {@link CheckpointSpout}
- */
-public class CheckpointSpoutTest {
-    CheckpointSpout spout = new CheckpointSpout();
-    TopologyContext mockTopologyContext;
-    SpoutOutputCollector mockOutputCollector;
-
-    @Before
-    public void setUp() throws Exception {
-        mockTopologyContext = Mockito.mock(TopologyContext.class);
-        Mockito.when(mockTopologyContext.getThisComponentId()).thenReturn("test");
-        Mockito.when(mockTopologyContext.getThisTaskId()).thenReturn(1);
-        mockOutputCollector = Mockito.mock(SpoutOutputCollector.class);
-    }
-
-    @Test
-    public void testInitState() throws Exception {
-        spout.open(new HashMap(), mockTopologyContext, mockOutputCollector);
-
-        spout.nextTuple();
-        Values expectedTuple = new Values(-1L, Action.INITSTATE);
-        ArgumentCaptor<String> stream = ArgumentCaptor.forClass(String.class);
-        ArgumentCaptor<Values> values = ArgumentCaptor.forClass(Values.class);
-        ArgumentCaptor<Object> msgId = ArgumentCaptor.forClass(Object.class);
-        Mockito.verify(mockOutputCollector).emit(stream.capture(),
-                                                 values.capture(),
-                                                 msgId.capture());
-
-        assertEquals(CheckpointSpout.CHECKPOINT_STREAM_ID, stream.getValue());
-        assertEquals(expectedTuple, values.getValue());
-        assertEquals(-1L, msgId.getValue());
-
-        spout.ack(-1L);
-
-        Mockito.verify(mockOutputCollector).emit(stream.capture(),
-                                                 values.capture(),
-                                                 msgId.capture());
-
-        expectedTuple = new Values(-1L, Action.INITSTATE);
-        assertEquals(CheckpointSpout.CHECKPOINT_STREAM_ID, stream.getValue());
-        assertEquals(expectedTuple, values.getValue());
-        assertEquals(-1L, msgId.getValue());
-
-    }
-
-    @Test
-    public void testPrepare() throws Exception {
-        spout.open(new HashMap(), mockTopologyContext, mockOutputCollector);
-        ArgumentCaptor<String> stream = ArgumentCaptor.forClass(String.class);
-        ArgumentCaptor<Values> values = ArgumentCaptor.forClass(Values.class);
-        ArgumentCaptor<Object> msgId = ArgumentCaptor.forClass(Object.class);
-
-        spout.nextTuple();
-        spout.ack(-1L);
-        spout.nextTuple();
-        Mockito.verify(mockOutputCollector, Mockito.times(2)).emit(stream.capture(),
-                                                 values.capture(),
-                                                 msgId.capture());
-
-        Values expectedTuple = new Values(0L, Action.PREPARE);
-        assertEquals(CheckpointSpout.CHECKPOINT_STREAM_ID, stream.getValue());
-        assertEquals(expectedTuple, values.getValue());
-        assertEquals(0L, msgId.getValue());
-
-    }
-
-    @Test
-    public void testPrepareWithFail() throws Exception {
-        Map<String, Object> stormConf = new HashMap<>();
-        KeyValueState<String, CheckPointState> state =
-                (KeyValueState<String, CheckPointState>) StateFactory.getState("__state", stormConf, mockTopologyContext);
-        CheckPointState txState = new CheckPointState(-1, COMMITTED);
-        state.put("__state", txState);
-
-        spout.open(mockTopologyContext, mockOutputCollector, 0, state);
-        ArgumentCaptor<String> stream = ArgumentCaptor.forClass(String.class);
-        ArgumentCaptor<Values> values = ArgumentCaptor.forClass(Values.class);
-        ArgumentCaptor<Object> msgId = ArgumentCaptor.forClass(Object.class);
-
-        spout.nextTuple();
-        spout.ack(-1L);
-        Utils.sleep(10);
-        spout.nextTuple();
-        spout.ack(0L);
-        Utils.sleep(10);
-        spout.nextTuple();
-        spout.ack(0L);
-        Utils.sleep(10);
-        spout.nextTuple();
-        spout.fail(1L);
-        Utils.sleep(10);
-        spout.nextTuple();
-        spout.fail(1L);
-        Utils.sleep(10);
-        spout.nextTuple();
-        spout.ack(1L);
-        Utils.sleep(10);
-        spout.nextTuple();
-        spout.ack(0L);
-        Utils.sleep(10);
-        spout.nextTuple();
-        Mockito.verify(mockOutputCollector, Mockito.times(8)).emit(stream.capture(),
-                                                                   values.capture(),
-                                                                   msgId.capture());
-
-        Values expectedTuple = new Values(1L, Action.PREPARE);
-        assertEquals(CheckpointSpout.CHECKPOINT_STREAM_ID, stream.getValue());
-        assertEquals(expectedTuple, values.getValue());
-        assertEquals(1L, msgId.getValue());
-
-    }
-
-    @Test
-    public void testCommit() throws Exception {
-        Map<String, Object> stormConf = new HashMap();
-        stormConf.put(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL, 0);
-        spout.open(stormConf, mockTopologyContext, mockOutputCollector);
-        ArgumentCaptor<String> stream = ArgumentCaptor.forClass(String.class);
-        ArgumentCaptor<Values> values = ArgumentCaptor.forClass(Values.class);
-        ArgumentCaptor<Object> msgId = ArgumentCaptor.forClass(Object.class);
-
-        spout.nextTuple();
-        spout.ack(-1L);
-        spout.nextTuple();
-        spout.ack(0L);
-        Utils.sleep(10);
-        spout.nextTuple();
-        spout.fail(0L);
-        Utils.sleep(10);
-        spout.nextTuple();
-        Mockito.verify(mockOutputCollector, Mockito.times(4)).emit(stream.capture(),
-                                                                   values.capture(),
-                                                                   msgId.capture());
-
-        Values expectedTuple = new Values(0L, Action.COMMIT);
-        assertEquals(CheckpointSpout.CHECKPOINT_STREAM_ID, stream.getValue());
-        assertEquals(expectedTuple, values.getValue());
-        assertEquals(0L, msgId.getValue());
-
-    }
-
-    @Test
-    public void testRecoveryRollback() throws Exception {
-        Map<String, Object> stormConf = new HashMap();
-
-        KeyValueState<String, CheckPointState> state =
-                (KeyValueState<String, CheckPointState>) StateFactory.getState("test-1", stormConf, mockTopologyContext);
-
-        CheckPointState checkPointState = new CheckPointState(100, CheckPointState.State.PREPARING);
-        state.put("__state", checkPointState);
-        spout.open(mockTopologyContext, mockOutputCollector, 0, state);
-        ArgumentCaptor<String> stream = ArgumentCaptor.forClass(String.class);
-        ArgumentCaptor<Values> values = ArgumentCaptor.forClass(Values.class);
-        ArgumentCaptor<Object> msgId = ArgumentCaptor.forClass(Object.class);
-
-        spout.nextTuple();
-        Mockito.verify(mockOutputCollector, Mockito.times(1)).emit(stream.capture(),
-                                                                   values.capture(),
-                                                                   msgId.capture());
-
-        Values expectedTuple = new Values(100L, Action.ROLLBACK);
-        assertEquals(CheckpointSpout.CHECKPOINT_STREAM_ID, stream.getValue());
-        assertEquals(expectedTuple, values.getValue());
-        assertEquals(100L, msgId.getValue());
-
-    }
-
-    @Test
-    public void testRecoveryRollbackAck() throws Exception {
-        Map<String, Object> stormConf = new HashMap();
-
-        KeyValueState<String, CheckPointState> state =
-                (KeyValueState<String, CheckPointState>) StateFactory.getState("test-1", stormConf, mockTopologyContext);
-
-        CheckPointState checkPointState = new CheckPointState(100, CheckPointState.State.PREPARING);
-        state.put("__state", checkPointState);
-        spout.open(mockTopologyContext, mockOutputCollector, 0, state);
-        ArgumentCaptor<String> stream = ArgumentCaptor.forClass(String.class);
-        ArgumentCaptor<Values> values = ArgumentCaptor.forClass(Values.class);
-        ArgumentCaptor<Object> msgId = ArgumentCaptor.forClass(Object.class);
-
-        spout.nextTuple();
-        spout.ack(100L);
-        spout.nextTuple();
-        spout.ack(99L);
-        spout.nextTuple();
-        Mockito.verify(mockOutputCollector, Mockito.times(3)).emit(stream.capture(),
-                                                                   values.capture(),
-                                                                   msgId.capture());
-
-        Values expectedTuple = new Values(100L, Action.PREPARE);
-        assertEquals(CheckpointSpout.CHECKPOINT_STREAM_ID, stream.getValue());
-        assertEquals(expectedTuple, values.getValue());
-        assertEquals(100L, msgId.getValue());
-
-    }
-
-    @Test
-    public void testRecoveryCommit() throws Exception {
-        Map<String, Object> stormConf = new HashMap();
-
-        KeyValueState<String, CheckPointState> state =
-                (KeyValueState<String, CheckPointState>) StateFactory.getState("test-1", stormConf, mockTopologyContext);
-
-        CheckPointState checkPointState = new CheckPointState(100, CheckPointState.State.COMMITTING);
-        state.put("__state", checkPointState);
-        spout.open(mockTopologyContext, mockOutputCollector, 0, state);
-        ArgumentCaptor<String> stream = ArgumentCaptor.forClass(String.class);
-        ArgumentCaptor<Values> values = ArgumentCaptor.forClass(Values.class);
-        ArgumentCaptor<Object> msgId = ArgumentCaptor.forClass(Object.class);
-
-        spout.nextTuple();
-        Mockito.verify(mockOutputCollector, Mockito.times(1)).emit(stream.capture(),
-                                                                   values.capture(),
-                                                                   msgId.capture());
-
-        Values expectedTuple = new Values(100L, Action.COMMIT);
-        assertEquals(CheckpointSpout.CHECKPOINT_STREAM_ID, stream.getValue());
-        assertEquals(expectedTuple, values.getValue());
-        assertEquals(100L, msgId.getValue());
-
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/test/jvm/backtype/storm/state/InMemoryKeyValueStateTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/state/InMemoryKeyValueStateTest.java b/storm-core/test/jvm/backtype/storm/state/InMemoryKeyValueStateTest.java
deleted file mode 100644
index 78ace77..0000000
--- a/storm-core/test/jvm/backtype/storm/state/InMemoryKeyValueStateTest.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/**
- * 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 backtype.storm.state;
-
-import org.junit.Before;
-import org.junit.Test;
-
-import static org.junit.Assert.*;
-
-/**
- * Unit tests for {@link InMemoryKeyValueState}
- */
-public class InMemoryKeyValueStateTest {
-
-    KeyValueState<String, String> keyValueState;
-
-    @Before
-    public void setUp() {
-        keyValueState = new InMemoryKeyValueState<>();
-    }
-
-    @Test
-    public void testPutAndGet() throws Exception {
-        keyValueState.put("a", "1");
-        keyValueState.put("b", "2");
-        assertArrayEquals(new String[]{"1", "2", null}, getValues());
-    }
-
-    @Test
-    public void testPrepareCommitRollback() throws Exception {
-        keyValueState.put("a", "1");
-        keyValueState.put("b", "2");
-        keyValueState.prepareCommit(1);
-        keyValueState.put("c", "3");
-        assertArrayEquals(new String[]{"1", "2", "3"}, getValues());
-        keyValueState.rollback();
-        assertArrayEquals(new String[]{null, null, null}, getValues());
-        keyValueState.put("a", "1");
-        keyValueState.put("b", "2");
-        keyValueState.prepareCommit(1);
-        keyValueState.commit(1);
-        keyValueState.put("c", "3");
-        assertArrayEquals(new String[]{"1", "2", "3"}, getValues());
-        keyValueState.rollback();
-        assertArrayEquals(new String[]{"1", "2", null}, getValues());
-    }
-
-    private String[] getValues() {
-        return new String[]{
-                keyValueState.get("a"),
-                keyValueState.get("b"),
-                keyValueState.get("c")
-        };
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/test/jvm/backtype/storm/topology/StatefulBoltExecutorTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/topology/StatefulBoltExecutorTest.java b/storm-core/test/jvm/backtype/storm/topology/StatefulBoltExecutorTest.java
deleted file mode 100644
index 6f73727..0000000
--- a/storm-core/test/jvm/backtype/storm/topology/StatefulBoltExecutorTest.java
+++ /dev/null
@@ -1,175 +0,0 @@
-/**
- * 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 backtype.storm.topology;
-
-import backtype.storm.generated.GlobalStreamId;
-import backtype.storm.generated.Grouping;
-import backtype.storm.spout.CheckpointSpout;
-import backtype.storm.state.KeyValueState;
-import backtype.storm.state.State;
-import backtype.storm.task.OutputCollector;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.tuple.Tuple;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.Mockito;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-import static backtype.storm.spout.CheckPointState.Action.*;
-import static backtype.storm.spout.CheckpointSpout.*;
-
-/**
- * Unit tests for {@link StatefulBoltExecutor}
- */
-public class StatefulBoltExecutorTest {
-    StatefulBoltExecutor<KeyValueState<String, String>> executor;
-    IStatefulBolt<KeyValueState<String, String>> mockBolt;
-    TopologyContext mockTopologyContext;
-    Tuple mockTuple;
-    Tuple mockCheckpointTuple;
-    Map<String, Object> mockStormConf = new HashMap<>();
-    OutputCollector mockOutputCollector;
-    State mockState;
-    Map<GlobalStreamId, Grouping> mockGlobalStream;
-    Set<GlobalStreamId> mockStreamIds;
-    @Before
-    public void setUp() throws Exception {
-        mockBolt = Mockito.mock(IStatefulBolt.class);
-        executor = new StatefulBoltExecutor<>(mockBolt);
-        GlobalStreamId mockGlobalStreamId = Mockito.mock(GlobalStreamId.class);
-        Mockito.when(mockGlobalStreamId.get_streamId()).thenReturn(CheckpointSpout.CHECKPOINT_STREAM_ID);
-        mockStreamIds = new HashSet<>();
-        mockStreamIds.add(mockGlobalStreamId);
-        mockTopologyContext = Mockito.mock(TopologyContext.class);
-        mockOutputCollector = Mockito.mock(OutputCollector.class);
-        mockGlobalStream = Mockito.mock(Map.class);
-        mockState = Mockito.mock(State.class);
-        Mockito.when(mockTopologyContext.getThisComponentId()).thenReturn("test");
-        Mockito.when(mockTopologyContext.getThisTaskId()).thenReturn(1);
-        Mockito.when(mockTopologyContext.getThisSources()).thenReturn(mockGlobalStream);
-        Mockito.when(mockTopologyContext.getComponentTasks(Mockito.anyString())).thenReturn(Collections.singletonList(1));
-        Mockito.when(mockGlobalStream.keySet()).thenReturn(mockStreamIds);
-        mockTuple = Mockito.mock(Tuple.class);
-        mockCheckpointTuple = Mockito.mock(Tuple.class);
-        executor.prepare(mockStormConf, mockTopologyContext, mockOutputCollector, mockState);
-    }
-
-    @Test
-    public void testHandleTupleBeforeInit() throws Exception {
-        Mockito.when(mockTuple.getSourceStreamId()).thenReturn("default");
-        executor.execute(mockTuple);
-        Mockito.verify(mockBolt, Mockito.times(0)).execute(Mockito.any(Tuple.class));
-    }
-
-    @Test
-    public void testHandleTuple() throws Exception {
-        Mockito.when(mockTuple.getSourceStreamId()).thenReturn("default");
-        executor.execute(mockTuple);
-        Mockito.when(mockCheckpointTuple.getSourceStreamId()).thenReturn(CheckpointSpout.CHECKPOINT_STREAM_ID);
-        Mockito.when(mockCheckpointTuple.getValueByField(CHECKPOINT_FIELD_ACTION)).thenReturn(INITSTATE);
-        Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(0));
-        Mockito.doNothing().when(mockOutputCollector).ack(mockCheckpointTuple);
-        executor.execute(mockCheckpointTuple);
-        Mockito.verify(mockBolt, Mockito.times(1)).execute(mockTuple);
-        Mockito.verify(mockBolt, Mockito.times(1)).initState(Mockito.any(KeyValueState.class));
-    }
-
-    @Test
-    public void testRollback() throws Exception {
-        Mockito.when(mockTuple.getSourceStreamId()).thenReturn("default");
-        executor.execute(mockTuple);
-        Mockito.when(mockCheckpointTuple.getSourceStreamId()).thenReturn(CheckpointSpout.CHECKPOINT_STREAM_ID);
-        Mockito.when(mockCheckpointTuple.getValueByField(CHECKPOINT_FIELD_ACTION)).thenReturn(ROLLBACK);
-        Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(0));
-        Mockito.doNothing().when(mockOutputCollector).ack(mockCheckpointTuple);
-        executor.execute(mockCheckpointTuple);
-        Mockito.verify(mockState, Mockito.times(1)).rollback();
-    }
-
-    @Test
-    public void testCommit() throws Exception {
-        Mockito.when(mockTuple.getSourceStreamId()).thenReturn("default");
-        executor.execute(mockTuple);
-        Mockito.when(mockCheckpointTuple.getSourceStreamId()).thenReturn(CheckpointSpout.CHECKPOINT_STREAM_ID);
-        Mockito.when(mockCheckpointTuple.getValueByField(CHECKPOINT_FIELD_ACTION)).thenReturn(COMMIT);
-        Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(0));
-        Mockito.doNothing().when(mockOutputCollector).ack(mockCheckpointTuple);
-        executor.execute(mockCheckpointTuple);
-        Mockito.verify(mockBolt, Mockito.times(1)).preCommit(new Long(0));
-        Mockito.verify(mockState, Mockito.times(1)).commit(new Long(0));
-    }
-
-    @Test
-    public void testPrepareAndRollbackBeforeInitstate() throws Exception {
-        Mockito.when(mockTuple.getSourceStreamId()).thenReturn("default");
-        executor.execute(mockTuple);
-        Mockito.when(mockCheckpointTuple.getSourceStreamId()).thenReturn(CheckpointSpout.CHECKPOINT_STREAM_ID);
-        Mockito.when(mockCheckpointTuple.getValueByField(CHECKPOINT_FIELD_ACTION)).thenReturn(PREPARE);
-        Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(100));
-        executor.execute(mockCheckpointTuple);
-        Mockito.verify(mockOutputCollector, Mockito.times(1)).fail(mockCheckpointTuple);
-
-        Mockito.when(mockCheckpointTuple.getValueByField(CHECKPOINT_FIELD_ACTION)).thenReturn(ROLLBACK);
-        Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(100));
-        Mockito.doNothing().when(mockOutputCollector).ack(mockCheckpointTuple);
-        executor.execute(mockCheckpointTuple);
-        Mockito.verify(mockState, Mockito.times(1)).rollback();
-    }
-
-    @Test
-    public void testCommitBeforeInitstate() throws Exception {
-        Mockito.when(mockTuple.getSourceStreamId()).thenReturn("default");
-        Mockito.when(mockCheckpointTuple.getSourceStreamId()).thenReturn(CheckpointSpout.CHECKPOINT_STREAM_ID);
-        Mockito.when(mockCheckpointTuple.getValueByField(CHECKPOINT_FIELD_ACTION)).thenReturn(COMMIT);
-        Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(100));
-        executor.execute(mockCheckpointTuple);
-        Mockito.verify(mockOutputCollector, Mockito.times(1)).ack(mockCheckpointTuple);
-
-        Mockito.when(mockCheckpointTuple.getValueByField(CHECKPOINT_FIELD_ACTION)).thenReturn(ROLLBACK);
-        Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(100));
-        executor.execute(mockCheckpointTuple);
-        Mockito.verify(mockState, Mockito.times(1)).rollback();
-    }
-
-    @Test
-    public void testPrepareAndCommit() throws Exception {
-        Mockito.when(mockTuple.getSourceStreamId()).thenReturn("default");
-        Mockito.when(mockCheckpointTuple.getSourceStreamId()).thenReturn(CheckpointSpout.CHECKPOINT_STREAM_ID);
-        Mockito.when(mockCheckpointTuple.getValueByField(CHECKPOINT_FIELD_ACTION)).thenReturn(INITSTATE);
-        Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(0));
-        executor.execute(mockCheckpointTuple);
-
-        executor.execute(mockTuple);
-        Mockito.when(mockCheckpointTuple.getSourceStreamId()).thenReturn(CheckpointSpout.CHECKPOINT_STREAM_ID);
-        Mockito.when(mockCheckpointTuple.getValueByField(CHECKPOINT_FIELD_ACTION)).thenReturn(PREPARE);
-        Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(100));
-        executor.execute(mockCheckpointTuple);
-        executor.execute(mockTuple);
-        Mockito.when(mockCheckpointTuple.getValueByField(CHECKPOINT_FIELD_ACTION)).thenReturn(COMMIT);
-        Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(100));
-        executor.execute(mockCheckpointTuple);
-        Mockito.verify(mockState, Mockito.times(1)).commit(new Long(100));
-        Mockito.verify(mockBolt, Mockito.times(2)).execute(mockTuple);
-        Mockito.verify(mockOutputCollector, Mockito.times(1)).ack(mockTuple);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/test/jvm/org/apache/storm/spout/CheckpointSpoutTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/spout/CheckpointSpoutTest.java b/storm-core/test/jvm/org/apache/storm/spout/CheckpointSpoutTest.java
new file mode 100644
index 0000000..d86f7f3
--- /dev/null
+++ b/storm-core/test/jvm/org/apache/storm/spout/CheckpointSpoutTest.java
@@ -0,0 +1,262 @@
+/**
+ * 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.spout;
+
+import org.apache.storm.Config;
+import org.apache.storm.state.KeyValueState;
+import org.apache.storm.state.StateFactory;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.Utils;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.storm.spout.CheckPointState.State.COMMITTED;
+import static org.junit.Assert.assertEquals;
+import static org.apache.storm.spout.CheckPointState.Action;
+
+/**
+ * Unit test for {@link CheckpointSpout}
+ */
+public class CheckpointSpoutTest {
+    CheckpointSpout spout = new CheckpointSpout();
+    TopologyContext mockTopologyContext;
+    SpoutOutputCollector mockOutputCollector;
+
+    @Before
+    public void setUp() throws Exception {
+        mockTopologyContext = Mockito.mock(TopologyContext.class);
+        Mockito.when(mockTopologyContext.getThisComponentId()).thenReturn("test");
+        Mockito.when(mockTopologyContext.getThisTaskId()).thenReturn(1);
+        mockOutputCollector = Mockito.mock(SpoutOutputCollector.class);
+    }
+
+    @Test
+    public void testInitState() throws Exception {
+        spout.open(new HashMap(), mockTopologyContext, mockOutputCollector);
+
+        spout.nextTuple();
+        Values expectedTuple = new Values(-1L, Action.INITSTATE);
+        ArgumentCaptor<String> stream = ArgumentCaptor.forClass(String.class);
+        ArgumentCaptor<Values> values = ArgumentCaptor.forClass(Values.class);
+        ArgumentCaptor<Object> msgId = ArgumentCaptor.forClass(Object.class);
+        Mockito.verify(mockOutputCollector).emit(stream.capture(),
+                                                 values.capture(),
+                                                 msgId.capture());
+
+        assertEquals(CheckpointSpout.CHECKPOINT_STREAM_ID, stream.getValue());
+        assertEquals(expectedTuple, values.getValue());
+        assertEquals(-1L, msgId.getValue());
+
+        spout.ack(-1L);
+
+        Mockito.verify(mockOutputCollector).emit(stream.capture(),
+                                                 values.capture(),
+                                                 msgId.capture());
+
+        expectedTuple = new Values(-1L, Action.INITSTATE);
+        assertEquals(CheckpointSpout.CHECKPOINT_STREAM_ID, stream.getValue());
+        assertEquals(expectedTuple, values.getValue());
+        assertEquals(-1L, msgId.getValue());
+
+    }
+
+    @Test
+    public void testPrepare() throws Exception {
+        spout.open(new HashMap(), mockTopologyContext, mockOutputCollector);
+        ArgumentCaptor<String> stream = ArgumentCaptor.forClass(String.class);
+        ArgumentCaptor<Values> values = ArgumentCaptor.forClass(Values.class);
+        ArgumentCaptor<Object> msgId = ArgumentCaptor.forClass(Object.class);
+
+        spout.nextTuple();
+        spout.ack(-1L);
+        spout.nextTuple();
+        Mockito.verify(mockOutputCollector, Mockito.times(2)).emit(stream.capture(),
+                                                 values.capture(),
+                                                 msgId.capture());
+
+        Values expectedTuple = new Values(0L, Action.PREPARE);
+        assertEquals(CheckpointSpout.CHECKPOINT_STREAM_ID, stream.getValue());
+        assertEquals(expectedTuple, values.getValue());
+        assertEquals(0L, msgId.getValue());
+
+    }
+
+    @Test
+    public void testPrepareWithFail() throws Exception {
+        Map<String, Object> stormConf = new HashMap<>();
+        KeyValueState<String, CheckPointState> state =
+                (KeyValueState<String, CheckPointState>) StateFactory.getState("__state", stormConf, mockTopologyContext);
+        CheckPointState txState = new CheckPointState(-1, COMMITTED);
+        state.put("__state", txState);
+
+        spout.open(mockTopologyContext, mockOutputCollector, 0, state);
+        ArgumentCaptor<String> stream = ArgumentCaptor.forClass(String.class);
+        ArgumentCaptor<Values> values = ArgumentCaptor.forClass(Values.class);
+        ArgumentCaptor<Object> msgId = ArgumentCaptor.forClass(Object.class);
+
+        spout.nextTuple();
+        spout.ack(-1L);
+        Utils.sleep(10);
+        spout.nextTuple();
+        spout.ack(0L);
+        Utils.sleep(10);
+        spout.nextTuple();
+        spout.ack(0L);
+        Utils.sleep(10);
+        spout.nextTuple();
+        spout.fail(1L);
+        Utils.sleep(10);
+        spout.nextTuple();
+        spout.fail(1L);
+        Utils.sleep(10);
+        spout.nextTuple();
+        spout.ack(1L);
+        Utils.sleep(10);
+        spout.nextTuple();
+        spout.ack(0L);
+        Utils.sleep(10);
+        spout.nextTuple();
+        Mockito.verify(mockOutputCollector, Mockito.times(8)).emit(stream.capture(),
+                                                                   values.capture(),
+                                                                   msgId.capture());
+
+        Values expectedTuple = new Values(1L, Action.PREPARE);
+        assertEquals(CheckpointSpout.CHECKPOINT_STREAM_ID, stream.getValue());
+        assertEquals(expectedTuple, values.getValue());
+        assertEquals(1L, msgId.getValue());
+
+    }
+
+    @Test
+    public void testCommit() throws Exception {
+        Map<String, Object> stormConf = new HashMap();
+        stormConf.put(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL, 0);
+        spout.open(stormConf, mockTopologyContext, mockOutputCollector);
+        ArgumentCaptor<String> stream = ArgumentCaptor.forClass(String.class);
+        ArgumentCaptor<Values> values = ArgumentCaptor.forClass(Values.class);
+        ArgumentCaptor<Object> msgId = ArgumentCaptor.forClass(Object.class);
+
+        spout.nextTuple();
+        spout.ack(-1L);
+        spout.nextTuple();
+        spout.ack(0L);
+        Utils.sleep(10);
+        spout.nextTuple();
+        spout.fail(0L);
+        Utils.sleep(10);
+        spout.nextTuple();
+        Mockito.verify(mockOutputCollector, Mockito.times(4)).emit(stream.capture(),
+                                                                   values.capture(),
+                                                                   msgId.capture());
+
+        Values expectedTuple = new Values(0L, Action.COMMIT);
+        assertEquals(CheckpointSpout.CHECKPOINT_STREAM_ID, stream.getValue());
+        assertEquals(expectedTuple, values.getValue());
+        assertEquals(0L, msgId.getValue());
+
+    }
+
+    @Test
+    public void testRecoveryRollback() throws Exception {
+        Map<String, Object> stormConf = new HashMap();
+
+        KeyValueState<String, CheckPointState> state =
+                (KeyValueState<String, CheckPointState>) StateFactory.getState("test-1", stormConf, mockTopologyContext);
+
+        CheckPointState checkPointState = new CheckPointState(100, CheckPointState.State.PREPARING);
+        state.put("__state", checkPointState);
+        spout.open(mockTopologyContext, mockOutputCollector, 0, state);
+        ArgumentCaptor<String> stream = ArgumentCaptor.forClass(String.class);
+        ArgumentCaptor<Values> values = ArgumentCaptor.forClass(Values.class);
+        ArgumentCaptor<Object> msgId = ArgumentCaptor.forClass(Object.class);
+
+        spout.nextTuple();
+        Mockito.verify(mockOutputCollector, Mockito.times(1)).emit(stream.capture(),
+                                                                   values.capture(),
+                                                                   msgId.capture());
+
+        Values expectedTuple = new Values(100L, Action.ROLLBACK);
+        assertEquals(CheckpointSpout.CHECKPOINT_STREAM_ID, stream.getValue());
+        assertEquals(expectedTuple, values.getValue());
+        assertEquals(100L, msgId.getValue());
+
+    }
+
+    @Test
+    public void testRecoveryRollbackAck() throws Exception {
+        Map<String, Object> stormConf = new HashMap();
+
+        KeyValueState<String, CheckPointState> state =
+                (KeyValueState<String, CheckPointState>) StateFactory.getState("test-1", stormConf, mockTopologyContext);
+
+        CheckPointState checkPointState = new CheckPointState(100, CheckPointState.State.PREPARING);
+        state.put("__state", checkPointState);
+        spout.open(mockTopologyContext, mockOutputCollector, 0, state);
+        ArgumentCaptor<String> stream = ArgumentCaptor.forClass(String.class);
+        ArgumentCaptor<Values> values = ArgumentCaptor.forClass(Values.class);
+        ArgumentCaptor<Object> msgId = ArgumentCaptor.forClass(Object.class);
+
+        spout.nextTuple();
+        spout.ack(100L);
+        spout.nextTuple();
+        spout.ack(99L);
+        spout.nextTuple();
+        Mockito.verify(mockOutputCollector, Mockito.times(3)).emit(stream.capture(),
+                                                                   values.capture(),
+                                                                   msgId.capture());
+
+        Values expectedTuple = new Values(100L, Action.PREPARE);
+        assertEquals(CheckpointSpout.CHECKPOINT_STREAM_ID, stream.getValue());
+        assertEquals(expectedTuple, values.getValue());
+        assertEquals(100L, msgId.getValue());
+
+    }
+
+    @Test
+    public void testRecoveryCommit() throws Exception {
+        Map<String, Object> stormConf = new HashMap();
+
+        KeyValueState<String, CheckPointState> state =
+                (KeyValueState<String, CheckPointState>) StateFactory.getState("test-1", stormConf, mockTopologyContext);
+
+        CheckPointState checkPointState = new CheckPointState(100, CheckPointState.State.COMMITTING);
+        state.put("__state", checkPointState);
+        spout.open(mockTopologyContext, mockOutputCollector, 0, state);
+        ArgumentCaptor<String> stream = ArgumentCaptor.forClass(String.class);
+        ArgumentCaptor<Values> values = ArgumentCaptor.forClass(Values.class);
+        ArgumentCaptor<Object> msgId = ArgumentCaptor.forClass(Object.class);
+
+        spout.nextTuple();
+        Mockito.verify(mockOutputCollector, Mockito.times(1)).emit(stream.capture(),
+                                                                   values.capture(),
+                                                                   msgId.capture());
+
+        Values expectedTuple = new Values(100L, Action.COMMIT);
+        assertEquals(CheckpointSpout.CHECKPOINT_STREAM_ID, stream.getValue());
+        assertEquals(expectedTuple, values.getValue());
+        assertEquals(100L, msgId.getValue());
+
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/bfc5ffd5/storm-core/test/jvm/org/apache/storm/state/InMemoryKeyValueStateTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/state/InMemoryKeyValueStateTest.java b/storm-core/test/jvm/org/apache/storm/state/InMemoryKeyValueStateTest.java
new file mode 100644
index 0000000..361865b
--- /dev/null
+++ b/storm-core/test/jvm/org/apache/storm/state/InMemoryKeyValueStateTest.java
@@ -0,0 +1,72 @@
+/**
+ * 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.state;
+
+import org.apache.storm.state.InMemoryKeyValueState;
+import org.apache.storm.state.KeyValueState;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+/**
+ * Unit tests for {@link InMemoryKeyValueState}
+ */
+public class InMemoryKeyValueStateTest {
+
+    KeyValueState<String, String> keyValueState;
+
+    @Before
+    public void setUp() {
+        keyValueState = new InMemoryKeyValueState<>();
+    }
+
+    @Test
+    public void testPutAndGet() throws Exception {
+        keyValueState.put("a", "1");
+        keyValueState.put("b", "2");
+        assertArrayEquals(new String[]{"1", "2", null}, getValues());
+    }
+
+    @Test
+    public void testPrepareCommitRollback() throws Exception {
+        keyValueState.put("a", "1");
+        keyValueState.put("b", "2");
+        keyValueState.prepareCommit(1);
+        keyValueState.put("c", "3");
+        assertArrayEquals(new String[]{"1", "2", "3"}, getValues());
+        keyValueState.rollback();
+        assertArrayEquals(new String[]{null, null, null}, getValues());
+        keyValueState.put("a", "1");
+        keyValueState.put("b", "2");
+        keyValueState.prepareCommit(1);
+        keyValueState.commit(1);
+        keyValueState.put("c", "3");
+        assertArrayEquals(new String[]{"1", "2", "3"}, getValues());
+        keyValueState.rollback();
+        assertArrayEquals(new String[]{"1", "2", null}, getValues());
+    }
+
+    private String[] getValues() {
+        return new String[]{
+                keyValueState.get("a"),
+                keyValueState.get("b"),
+                keyValueState.get("c")
+        };
+    }
+}
\ No newline at end of file


[04/10] storm git commit: Refactored state handling code and addressed review comments

Posted by pt...@apache.org.
Refactored state handling code and addressed review comments


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

Branch: refs/heads/master
Commit: d9a0698f079cd77fc7da40559eeffd7f9fc5b9a5
Parents: 40a1c7e
Author: Arun Mahadevan <ai...@hortonworks.com>
Authored: Wed Dec 23 18:18:13 2015 +0530
Committer: Arun Mahadevan <ai...@hortonworks.com>
Committed: Tue Jan 12 10:21:00 2016 +0530

----------------------------------------------------------------------
 conf/defaults.yaml                              |   1 +
 .../storm/redis/state/RedisKeyValueState.java   |   4 +-
 .../backtype/storm/spout/CheckPointState.java   | 106 +++++++++++++-
 .../backtype/storm/spout/CheckpointSpout.java   | 140 ++++++-------------
 .../topology/CheckpointTupleForwarder.java      |  18 ++-
 .../storm/topology/StatefulBoltExecutor.java    |  19 +--
 storm-core/src/jvm/org/apache/storm/Config.java |   4 +-
 .../storm/spout/CheckpointSpoutTest.java        |  21 ++-
 .../topology/StatefulBoltExecutorTest.java      |   9 +-
 9 files changed, 187 insertions(+), 135 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/d9a0698f/conf/defaults.yaml
----------------------------------------------------------------------
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index 735a83e..c15f0b3 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -253,6 +253,7 @@ topology.disruptor.wait.timeout.millis: 1000
 topology.disruptor.batch.size: 100
 topology.disruptor.batch.timeout.millis: 1
 topology.disable.loadaware: false
+topology.state.checkpoint.interval.ms: 1000
 
 # Configs for Resource Aware Scheduler
 # topology priority describing the importance of the topology in decreasing importance starting from 0 (i.e. 0 is the highest priority and the priority importance decreases as the priority number increases).

http://git-wip-us.apache.org/repos/asf/storm/blob/d9a0698f/external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueState.java
----------------------------------------------------------------------
diff --git a/external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueState.java b/external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueState.java
index 57b30cc..29d33b7 100644
--- a/external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueState.java
+++ b/external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueState.java
@@ -40,8 +40,8 @@ import java.util.concurrent.ConcurrentHashMap;
  */
 public class RedisKeyValueState<K, V> implements KeyValueState<K, V> {
     private static final Logger LOG = LoggerFactory.getLogger(RedisKeyValueState.class);
-    private static final String COMMIT_TXID_KEY = "$commit";
-    private static final String PREPARE_TXID_KEY = "$prepare";
+    private static final String COMMIT_TXID_KEY = "commit";
+    private static final String PREPARE_TXID_KEY = "prepare";
 
     private final BASE64Encoder base64Encoder;
     private final BASE64Decoder base64Decoder;

http://git-wip-us.apache.org/repos/asf/storm/blob/d9a0698f/storm-core/src/jvm/backtype/storm/spout/CheckPointState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/spout/CheckPointState.java b/storm-core/src/jvm/backtype/storm/spout/CheckPointState.java
index 5ad9772..ff0e088 100644
--- a/storm-core/src/jvm/backtype/storm/spout/CheckPointState.java
+++ b/storm-core/src/jvm/backtype/storm/spout/CheckPointState.java
@@ -17,13 +17,30 @@
  */
 package backtype.storm.spout;
 
+import static backtype.storm.spout.CheckPointState.State.COMMITTED;
+import static backtype.storm.spout.CheckPointState.State.COMMITTING;
+import static backtype.storm.spout.CheckPointState.State.PREPARING;
+
 /**
- * Captures the current state of the transaction in
- * {@link CheckpointSpout}
+ * Captures the current state of the transaction in {@link CheckpointSpout}. The state transitions are as follows.
+ * <pre>
+ *                  ROLLBACK(tx2)
+ *               <-------------                  PREPARE(tx2)                     COMMIT(tx2)
+ * COMMITTED(tx1)-------------> PREPARING(tx2) --------------> COMMITTING(tx2) -----------------> COMMITTED (tx2)
+ *
+ * </pre>
+ *
+ * During recovery, if a previous transaction is in PREPARING state, it is rolled back since all bolts in the topology
+ * might not have prepared (saved) the data for commit. If the previous transaction is in COMMITTING state, it is
+ * rolled forward (committed) since some bolts might have already committed the data.
+ * <p>
+ * During normal flow, the state transitions from PREPARING to COMMITTING to COMMITTED. In case of failures the
+ * prepare/commit operation is retried.
+ * </p>
  */
 public class CheckPointState {
-    public long txid;
-    public State state;
+    private long txid;
+    private State state;
 
     public enum State {
         /**
@@ -31,15 +48,36 @@ public class CheckPointState {
          */
         COMMITTED,
         /**
-         * The checkpoint spout has started committing the transaction.
+         * The checkpoint spout has started committing the transaction
+         * and the commit is in progress.
          */
         COMMITTING,
         /**
-         * The checkpoint spout has started preparing the transaction for commit.
+         * The checkpoint spout has started preparing the transaction for commit
+         * and the prepare is in progress.
          */
         PREPARING
     }
 
+    public enum Action {
+        /**
+         * prepare transaction for commit
+         */
+        PREPARE,
+        /**
+         * commit the previously prepared transaction
+         */
+        COMMIT,
+        /**
+         * rollback the previously prepared transaction
+         */
+        ROLLBACK,
+        /**
+         * initialize the state
+         */
+        INITSTATE
+    }
+
     public CheckPointState(long txid, State state) {
         this.txid = txid;
         this.state = state;
@@ -49,6 +87,62 @@ public class CheckPointState {
     public CheckPointState() {
     }
 
+    public long getTxid() {
+        return txid;
+    }
+
+    public State getState() {
+        return state;
+    }
+
+    /**
+     * Get the next state based on this checkpoint state.
+     *
+     * @param recovering if in recovering phase
+     * @return the next checkpoint state based on this state.
+     */
+    public CheckPointState nextState(boolean recovering) {
+        CheckPointState nextState;
+        switch (state) {
+            case PREPARING:
+                nextState = recovering ? new CheckPointState(txid - 1, COMMITTED) : new CheckPointState(txid, COMMITTING);
+                break;
+            case COMMITTING:
+                nextState = new CheckPointState(txid, COMMITTED);
+                break;
+            case COMMITTED:
+                nextState = recovering ? this : new CheckPointState(txid + 1, PREPARING);
+                break;
+            default:
+                throw new IllegalStateException("Unknown state " + state);
+        }
+        return nextState;
+    }
+
+    /**
+     * Get the next action to perform based on this checkpoint state.
+     *
+     * @param recovering if in recovering phase
+     * @return the next action to perform based on this state
+     */
+    public Action nextAction(boolean recovering) {
+        Action action;
+        switch (state) {
+            case PREPARING:
+                action = recovering ? Action.ROLLBACK : Action.PREPARE;
+                break;
+            case COMMITTING:
+                action = Action.COMMIT;
+                break;
+            case COMMITTED:
+                action = recovering ? Action.INITSTATE : Action.PREPARE;
+                break;
+            default:
+                throw new IllegalStateException("Unknown state " + state);
+        }
+        return action;
+    }
+
     @Override
     public boolean equals(Object o) {
         if (this == o) return true;

http://git-wip-us.apache.org/repos/asf/storm/blob/d9a0698f/storm-core/src/jvm/backtype/storm/spout/CheckpointSpout.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/spout/CheckpointSpout.java b/storm-core/src/jvm/backtype/storm/spout/CheckpointSpout.java
index 929dd12..32ed6e6 100644
--- a/storm-core/src/jvm/backtype/storm/spout/CheckpointSpout.java
+++ b/storm-core/src/jvm/backtype/storm/spout/CheckpointSpout.java
@@ -26,28 +26,22 @@ import backtype.storm.topology.base.BaseRichSpout;
 import backtype.storm.tuple.Fields;
 import backtype.storm.tuple.Tuple;
 import backtype.storm.tuple.Values;
+import backtype.storm.utils.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.util.Map;
+
 import static backtype.storm.spout.CheckPointState.State.COMMITTED;
-import static backtype.storm.spout.CheckPointState.State.COMMITTING;
-import static backtype.storm.spout.CheckPointState.State.PREPARING;
+import static backtype.storm.spout.CheckPointState.Action;
 
 /**
  * Emits checkpoint tuples which is used to save the state of the {@link backtype.storm.topology.IStatefulComponent}
  * across the topology. If a topology contains Stateful bolts, Checkpoint spouts are automatically added
  * to the topology. There is only one Checkpoint task per topology.
- * <p/>
- * Checkpoint spout stores its internal state in a {@link KeyValueState}. The state transitions are as follows.
- * <p/>
- * <pre>
- *                  ROLLBACK(tx2)
- *               <-------------                  PREPARE(tx2)                     COMMIT(tx2)
- * COMMITTED(tx1)-------------> PREPARING(tx2) --------------> COMMITTING(tx2) -----------------> COMMITTED (tx2)
- *
+ * Checkpoint spout stores its internal state in a {@link KeyValueState}.
  *
- * </pre>
+ * @see CheckPointState
  */
 public class CheckpointSpout extends BaseRichSpout {
     private static final Logger LOG = LoggerFactory.getLogger(CheckpointSpout.class);
@@ -56,22 +50,17 @@ public class CheckpointSpout extends BaseRichSpout {
     public static final String CHECKPOINT_COMPONENT_ID = "$checkpointspout";
     public static final String CHECKPOINT_FIELD_TXID = "txid";
     public static final String CHECKPOINT_FIELD_ACTION = "action";
-    public static final String CHECKPOINT_ACTION_PREPARE = "prepare";
-    public static final String CHECKPOINT_ACTION_COMMIT = "commit";
-    public static final String CHECKPOINT_ACTION_ROLLBACK = "rollback";
-    public static final String CHECKPOINT_ACTION_INITSTATE = "initstate";
-
     private static final String TX_STATE_KEY = "__state";
-    private static final int DEFAULT_CHECKPOINT_INTERVAL = 1000; // every sec
-
     private TopologyContext context;
     private SpoutOutputCollector collector;
     private long lastCheckpointTs;
     private int checkpointInterval;
+    private int sleepInterval;
     private boolean recoveryStepInProgress;
     private boolean checkpointStepInProgress;
     private boolean recovering;
     private KeyValueState<String, CheckPointState> checkpointState;
+    private CheckPointState curTxState;
 
     @Override
     public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
@@ -84,7 +73,9 @@ public class CheckpointSpout extends BaseRichSpout {
         this.context = context;
         this.collector = collector;
         this.checkpointInterval = checkpointInterval;
+        this.sleepInterval = checkpointInterval / 10;
         this.checkpointState = checkpointState;
+        this.curTxState = checkpointState.get(TX_STATE_KEY);
         lastCheckpointTs = 0;
         recoveryStepInProgress = false;
         checkpointStepInProgress = false;
@@ -94,28 +85,27 @@ public class CheckpointSpout extends BaseRichSpout {
     @Override
     public void nextTuple() {
         if (shouldRecover()) {
-            LOG.debug("In recovery");
             handleRecovery();
             startProgress();
         } else if (shouldCheckpoint()) {
-            LOG.debug("In checkpoint");
             doCheckpoint();
             startProgress();
+        } else {
+            Utils.sleep(sleepInterval);
         }
     }
 
     @Override
     public void ack(Object msgId) {
-        CheckPointState txState = getTxState();
-        LOG.debug("Got ack with txid {}, current txState {}", msgId, txState);
-        if (txState.txid == ((Number) msgId).longValue()) {
+        LOG.debug("Got ack with txid {}, current txState {}", msgId, curTxState);
+        if (curTxState.getTxid() == ((Number) msgId).longValue()) {
             if (recovering) {
                 handleRecoveryAck();
             } else {
                 handleCheckpointAck();
             }
         } else {
-            LOG.warn("Ack msgid {}, txState.txid {} mismatch", msgId, txState.txid);
+            LOG.warn("Ack msgid {}, txState.txid {} mismatch", msgId, curTxState.getTxid());
         }
         resetProgress();
     }
@@ -131,13 +121,6 @@ public class CheckpointSpout extends BaseRichSpout {
         declarer.declareStream(CHECKPOINT_STREAM_ID, new Fields(CHECKPOINT_FIELD_TXID, CHECKPOINT_FIELD_ACTION));
     }
 
-    @Override
-    public Map<String, Object> getComponentConfiguration() {
-        Config conf = new Config();
-        conf.put(Config.TOPOLOGY_SLEEP_SPOUT_WAIT_STRATEGY_TIME_MS, 100);
-        return conf;
-    }
-
     public static boolean isCheckpoint(Tuple input) {
         return CHECKPOINT_STREAM_ID.equals(input.getSourceStreamId());
     }
@@ -161,12 +144,13 @@ public class CheckpointSpout extends BaseRichSpout {
     }
 
     private int loadCheckpointInterval(Map stormConf) {
-        int interval;
+        int interval = 0;
         if (stormConf.containsKey(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL)) {
             interval = ((Number) stormConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL)).intValue();
-        } else {
-            interval = DEFAULT_CHECKPOINT_INTERVAL;
         }
+        // ensure checkpoint interval is not less than a sane low value.
+        interval = Math.max(100, interval);
+        LOG.info("Checkpoint interval is {} millis", interval);
         return interval;
     }
 
@@ -175,91 +159,55 @@ public class CheckpointSpout extends BaseRichSpout {
     }
 
     private boolean shouldCheckpoint() {
-        return !recovering && !checkpointStepInProgress
-                && (System.currentTimeMillis() - lastCheckpointTs) > checkpointInterval;
-    }
-
-    private boolean shouldRollback(CheckPointState txState) {
-        return txState.state == PREPARING;
-    }
-
-    private boolean shouldCommit(CheckPointState txState) {
-        return txState.state == COMMITTING;
+        return !recovering && !checkpointStepInProgress &&
+                (curTxState.getState() != COMMITTED || checkpointIntervalElapsed());
     }
 
-    private boolean shouldInitState(CheckPointState txState) {
-        return txState.state == COMMITTED;
+    private boolean checkpointIntervalElapsed() {
+        return (System.currentTimeMillis() - lastCheckpointTs) > checkpointInterval;
     }
 
     private void handleRecovery() {
-        CheckPointState txState = getTxState();
-        LOG.debug("Current txState is {}", txState);
-        if (shouldRollback(txState)) {
-            LOG.debug("Emitting rollback with txid {}", txState.txid);
-            collector.emit(CHECKPOINT_STREAM_ID, new Values(txState.txid, CHECKPOINT_ACTION_ROLLBACK), txState.txid);
-        } else if (shouldCommit(txState)) {
-            LOG.debug("Emitting commit with txid {}", txState.txid);
-            collector.emit(CHECKPOINT_STREAM_ID, new Values(txState.txid, CHECKPOINT_ACTION_COMMIT), txState.txid);
-        } else if (shouldInitState(txState)) {
-            LOG.debug("Emitting init state with txid {}", txState.txid);
-            collector.emit(CHECKPOINT_STREAM_ID, new Values(txState.txid, CHECKPOINT_ACTION_INITSTATE), txState.txid);
-        }
-        startProgress();
+        LOG.debug("In recovery");
+        Action action = curTxState.nextAction(true);
+        emit(curTxState.getTxid(), action);
     }
 
     private void handleRecoveryAck() {
-        CheckPointState txState = getTxState();
-        if (shouldRollback(txState)) {
-            txState.state = COMMITTED;
-            --txState.txid;
-            saveTxState(txState);
-        } else if (shouldCommit(txState)) {
-            txState.state = COMMITTED;
-            saveTxState(txState);
-        } else if (shouldInitState(txState)) {
-            LOG.debug("Recovery complete, current state {}", txState);
+        CheckPointState nextState = curTxState.nextState(true);
+        if (curTxState != nextState) {
+            saveTxState(nextState);
+        } else {
+            LOG.debug("Recovery complete, current state {}", curTxState);
             recovering = false;
         }
     }
 
     private void doCheckpoint() {
-        CheckPointState txState = getTxState();
-        if (txState.state == COMMITTED) {
-            txState.txid++;
-            txState.state = PREPARING;
-            saveTxState(txState);
+        LOG.debug("In checkpoint");
+        if (curTxState.getState() == COMMITTED) {
+            saveTxState(curTxState.nextState(false));
             lastCheckpointTs = System.currentTimeMillis();
-            LOG.debug("Emitting prepare with txid {}", txState.txid);
-            collector.emit(CHECKPOINT_STREAM_ID, new Values(txState.txid, CHECKPOINT_ACTION_PREPARE), txState.txid);
-        } else if (txState.state == PREPARING) {
-            LOG.debug("Emitting prepare with txid {}", txState.txid);
-            collector.emit(CHECKPOINT_STREAM_ID, new Values(txState.txid, CHECKPOINT_ACTION_PREPARE), txState.txid);
-        } else if (txState.state == COMMITTING) {
-            LOG.debug("Emitting commit with txid {}", txState.txid);
-            collector.emit(CHECKPOINT_STREAM_ID, new Values(txState.txid, CHECKPOINT_ACTION_COMMIT), txState.txid);
         }
-        startProgress();
+        Action action = curTxState.nextAction(false);
+        emit(curTxState.getTxid(), action);
     }
 
     private void handleCheckpointAck() {
-        CheckPointState txState = getTxState();
-        if (txState.state == PREPARING) {
-            txState.state = COMMITTING;
-            LOG.debug("Prepare txid {} complete", txState.txid);
-        } else if (txState.state == COMMITTING) {
-            txState.state = COMMITTED;
-            LOG.debug("Commit txid {} complete", txState.txid);
-        }
-        saveTxState(txState);
+        CheckPointState nextState = curTxState.nextState(false);
+        saveTxState(nextState);
+    }
+
+    private void emit(long txid, Action action) {
+        LOG.debug("Current state {}, emitting txid {}, action {}", curTxState, txid, action);
+        collector.emit(CHECKPOINT_STREAM_ID, new Values(txid, action), txid);
     }
 
     private void saveTxState(CheckPointState txState) {
+        LOG.debug("saveTxState, current state {} -> new state {}", curTxState, txState);
         checkpointState.put(TX_STATE_KEY, txState);
         checkpointState.commit();
-    }
-
-    private CheckPointState getTxState() {
-        return checkpointState.get(TX_STATE_KEY);
+        curTxState = txState;
     }
 
     private void startProgress() {

http://git-wip-us.apache.org/repos/asf/storm/blob/d9a0698f/storm-core/src/jvm/backtype/storm/topology/CheckpointTupleForwarder.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/topology/CheckpointTupleForwarder.java b/storm-core/src/jvm/backtype/storm/topology/CheckpointTupleForwarder.java
index 2d25e72..6a0a055 100644
--- a/storm-core/src/jvm/backtype/storm/topology/CheckpointTupleForwarder.java
+++ b/storm-core/src/jvm/backtype/storm/topology/CheckpointTupleForwarder.java
@@ -30,7 +30,11 @@ import org.slf4j.LoggerFactory;
 import java.util.HashMap;
 import java.util.Map;
 
-import static backtype.storm.spout.CheckpointSpout.*;
+import static backtype.storm.spout.CheckpointSpout.CHECKPOINT_STREAM_ID;
+import static backtype.storm.spout.CheckpointSpout.CHECKPOINT_FIELD_TXID;
+import static backtype.storm.spout.CheckpointSpout.CHECKPOINT_FIELD_ACTION;
+import static backtype.storm.spout.CheckPointState.Action;
+import static backtype.storm.spout.CheckPointState.Action.ROLLBACK;
 
 /**
  * Wraps {@link IRichBolt} and forwards checkpoint tuples in a
@@ -94,7 +98,7 @@ public class CheckpointTupleForwarder implements IRichBolt {
      * @param action the action (prepare, commit, rollback or initstate)
      * @param txid   the transaction id.
      */
-    protected void handleCheckpoint(Tuple input, String action, long txid) {
+    protected void handleCheckpoint(Tuple input, Action action, long txid) {
         collector.emit(CHECKPOINT_STREAM_ID, input, new Values(txid, action));
     }
 
@@ -117,14 +121,14 @@ public class CheckpointTupleForwarder implements IRichBolt {
      * all input checkpoint streams to this component.
      */
     private void processCheckpoint(Tuple input) {
-        String action = input.getStringByField(CHECKPOINT_FIELD_ACTION);
+        Action action = (Action) input.getValueByField(CHECKPOINT_FIELD_ACTION);
         long txid = input.getLongByField(CHECKPOINT_FIELD_TXID);
         if (shouldProcessTransaction(action, txid)) {
             LOG.debug("Processing action {}, txid {}", action, txid);
             try {
                 if (txid >= lastTxid) {
                     handleCheckpoint(input, action, txid);
-                    if (CHECKPOINT_ACTION_ROLLBACK.equals(action)) {
+                    if (action == ROLLBACK) {
                         lastTxid = txid - 1;
                     } else {
                         lastTxid = txid;
@@ -162,7 +166,7 @@ public class CheckpointTupleForwarder implements IRichBolt {
      * Checks if check points have been received from all tasks across
      * all input streams to this component
      */
-    private boolean shouldProcessTransaction(String action, long txid) {
+    private boolean shouldProcessTransaction(Action action, long txid) {
         TransactionRequest request = new TransactionRequest(action, txid);
         Integer count;
         if ((count = transactionRequestCount.get(request)) == null) {
@@ -179,10 +183,10 @@ public class CheckpointTupleForwarder implements IRichBolt {
     }
 
     private static class TransactionRequest {
-        private final String action;
+        private final Action action;
         private final long txid;
 
-        TransactionRequest(String action, long txid) {
+        TransactionRequest(Action action, long txid) {
             this.action = action;
             this.txid = txid;
         }

http://git-wip-us.apache.org/repos/asf/storm/blob/d9a0698f/storm-core/src/jvm/backtype/storm/topology/StatefulBoltExecutor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/topology/StatefulBoltExecutor.java b/storm-core/src/jvm/backtype/storm/topology/StatefulBoltExecutor.java
index a7c5b2e..d44ad48 100644
--- a/storm-core/src/jvm/backtype/storm/topology/StatefulBoltExecutor.java
+++ b/storm-core/src/jvm/backtype/storm/topology/StatefulBoltExecutor.java
@@ -31,10 +31,11 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
-import static backtype.storm.spout.CheckpointSpout.CHECKPOINT_ACTION_COMMIT;
-import static backtype.storm.spout.CheckpointSpout.CHECKPOINT_ACTION_PREPARE;
-import static backtype.storm.spout.CheckpointSpout.CHECKPOINT_ACTION_ROLLBACK;
-import static backtype.storm.spout.CheckpointSpout.CHECKPOINT_ACTION_INITSTATE;
+import static backtype.storm.spout.CheckPointState.Action;
+import static backtype.storm.spout.CheckPointState.Action.COMMIT;
+import static backtype.storm.spout.CheckPointState.Action.PREPARE;
+import static backtype.storm.spout.CheckPointState.Action.ROLLBACK;
+import static backtype.storm.spout.CheckPointState.Action.INITSTATE;
 
 /**
  * Wraps a {@link IStatefulBolt} and manages the state of the bolt.
@@ -65,18 +66,18 @@ public class StatefulBoltExecutor<T extends State> extends CheckpointTupleForwar
     }
 
     @Override
-    protected void handleCheckpoint(Tuple input, String action, long txid) {
+    protected void handleCheckpoint(Tuple input, Action action, long txid) {
         LOG.debug("handleCheckPoint with tuple {}, action {}, txid {}", input, action, txid);
-        if (action.equals(CHECKPOINT_ACTION_PREPARE)) {
+        if (action == PREPARE) {
             bolt.prePrepare(txid);
             state.prepareCommit(txid);
-        } else if (action.equals(CHECKPOINT_ACTION_COMMIT)) {
+        } else if (action == COMMIT) {
             bolt.preCommit(txid);
             state.commit(txid);
-        } else if (action.equals(CHECKPOINT_ACTION_ROLLBACK)) {
+        } else if (action == ROLLBACK) {
             bolt.preRollback();
             state.rollback();
-        } else if (action.equals(CHECKPOINT_ACTION_INITSTATE)) {
+        } else if (action == INITSTATE) {
             bolt.initState((T) state);
             boltInitialized = true;
             LOG.debug("{} pending tuples to process", pendingTuples.size());

http://git-wip-us.apache.org/repos/asf/storm/blob/d9a0698f/storm-core/src/jvm/org/apache/storm/Config.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/Config.java b/storm-core/src/jvm/org/apache/storm/Config.java
index 1bd4a54..c18be2f 100644
--- a/storm-core/src/jvm/org/apache/storm/Config.java
+++ b/storm-core/src/jvm/org/apache/storm/Config.java
@@ -1548,7 +1548,9 @@ public class Config extends HashMap<String, Object> {
 
     /**
      * The configuration specific to the {@link backtype.storm.state.StateProvider} implementation.
-     * This can be overridden at the component level.
+     * This can be overridden at the component level. The value and the interpretation of this config
+     * is based on the state provider implementation. For e.g. this could be just a config file name
+     * which contains the config for the state provider implementation.
      */
     @isString
     public static final String TOPOLOGY_STATE_PROVIDER_CONFIG = "topology.state.provider.config";

http://git-wip-us.apache.org/repos/asf/storm/blob/d9a0698f/storm-core/test/jvm/backtype/storm/spout/CheckpointSpoutTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/spout/CheckpointSpoutTest.java b/storm-core/test/jvm/backtype/storm/spout/CheckpointSpoutTest.java
index 15bba91..f341f70 100644
--- a/storm-core/test/jvm/backtype/storm/spout/CheckpointSpoutTest.java
+++ b/storm-core/test/jvm/backtype/storm/spout/CheckpointSpoutTest.java
@@ -26,14 +26,13 @@ import backtype.storm.utils.Utils;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
-import org.mockito.ArgumentMatcher;
 import org.mockito.Mockito;
-import org.mockito.internal.matchers.Equals;
 
 import java.util.HashMap;
 import java.util.Map;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static backtype.storm.spout.CheckPointState.Action;
 
 /**
  * Unit test for {@link CheckpointSpout}
@@ -56,7 +55,7 @@ public class CheckpointSpoutTest {
         spout.open(new HashMap(), mockTopologyContext, mockOutputCollector);
 
         spout.nextTuple();
-        Values expectedTuple = new Values(-1L, CheckpointSpout.CHECKPOINT_ACTION_INITSTATE);
+        Values expectedTuple = new Values(-1L, Action.INITSTATE);
         ArgumentCaptor<String> stream = ArgumentCaptor.forClass(String.class);
         ArgumentCaptor<Values> values = ArgumentCaptor.forClass(Values.class);
         ArgumentCaptor<Object> msgId = ArgumentCaptor.forClass(Object.class);
@@ -74,7 +73,7 @@ public class CheckpointSpoutTest {
                                                  values.capture(),
                                                  msgId.capture());
 
-        expectedTuple = new Values(-1L, CheckpointSpout.CHECKPOINT_ACTION_INITSTATE);
+        expectedTuple = new Values(-1L, Action.INITSTATE);
         assertEquals(CheckpointSpout.CHECKPOINT_STREAM_ID, stream.getValue());
         assertEquals(expectedTuple, values.getValue());
         assertEquals(-1L, msgId.getValue());
@@ -95,7 +94,7 @@ public class CheckpointSpoutTest {
                                                  values.capture(),
                                                  msgId.capture());
 
-        Values expectedTuple = new Values(0L, CheckpointSpout.CHECKPOINT_ACTION_PREPARE);
+        Values expectedTuple = new Values(0L, Action.PREPARE);
         assertEquals(CheckpointSpout.CHECKPOINT_STREAM_ID, stream.getValue());
         assertEquals(expectedTuple, values.getValue());
         assertEquals(0L, msgId.getValue());
@@ -124,7 +123,7 @@ public class CheckpointSpoutTest {
                                                                    values.capture(),
                                                                    msgId.capture());
 
-        Values expectedTuple = new Values(0L, CheckpointSpout.CHECKPOINT_ACTION_PREPARE);
+        Values expectedTuple = new Values(0L, Action.PREPARE);
         assertEquals(CheckpointSpout.CHECKPOINT_STREAM_ID, stream.getValue());
         assertEquals(expectedTuple, values.getValue());
         assertEquals(0L, msgId.getValue());
@@ -153,7 +152,7 @@ public class CheckpointSpoutTest {
                                                                    values.capture(),
                                                                    msgId.capture());
 
-        Values expectedTuple = new Values(0L, CheckpointSpout.CHECKPOINT_ACTION_COMMIT);
+        Values expectedTuple = new Values(0L, Action.COMMIT);
         assertEquals(CheckpointSpout.CHECKPOINT_STREAM_ID, stream.getValue());
         assertEquals(expectedTuple, values.getValue());
         assertEquals(0L, msgId.getValue());
@@ -179,7 +178,7 @@ public class CheckpointSpoutTest {
                                                                    values.capture(),
                                                                    msgId.capture());
 
-        Values expectedTuple = new Values(100L, CheckpointSpout.CHECKPOINT_ACTION_ROLLBACK);
+        Values expectedTuple = new Values(100L, Action.ROLLBACK);
         assertEquals(CheckpointSpout.CHECKPOINT_STREAM_ID, stream.getValue());
         assertEquals(expectedTuple, values.getValue());
         assertEquals(100L, msgId.getValue());
@@ -209,7 +208,7 @@ public class CheckpointSpoutTest {
                                                                    values.capture(),
                                                                    msgId.capture());
 
-        Values expectedTuple = new Values(100L, CheckpointSpout.CHECKPOINT_ACTION_PREPARE);
+        Values expectedTuple = new Values(100L, Action.PREPARE);
         assertEquals(CheckpointSpout.CHECKPOINT_STREAM_ID, stream.getValue());
         assertEquals(expectedTuple, values.getValue());
         assertEquals(100L, msgId.getValue());
@@ -235,7 +234,7 @@ public class CheckpointSpoutTest {
                                                                    values.capture(),
                                                                    msgId.capture());
 
-        Values expectedTuple = new Values(100L, CheckpointSpout.CHECKPOINT_ACTION_COMMIT);
+        Values expectedTuple = new Values(100L, Action.COMMIT);
         assertEquals(CheckpointSpout.CHECKPOINT_STREAM_ID, stream.getValue());
         assertEquals(expectedTuple, values.getValue());
         assertEquals(100L, msgId.getValue());

http://git-wip-us.apache.org/repos/asf/storm/blob/d9a0698f/storm-core/test/jvm/backtype/storm/topology/StatefulBoltExecutorTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/topology/StatefulBoltExecutorTest.java b/storm-core/test/jvm/backtype/storm/topology/StatefulBoltExecutorTest.java
index 570258a..2959a10 100644
--- a/storm-core/test/jvm/backtype/storm/topology/StatefulBoltExecutorTest.java
+++ b/storm-core/test/jvm/backtype/storm/topology/StatefulBoltExecutorTest.java
@@ -36,6 +36,9 @@ import java.util.Map;
 import java.util.Set;
 
 import static backtype.storm.spout.CheckpointSpout.*;
+import static backtype.storm.spout.CheckPointState.Action.INITSTATE;
+import static backtype.storm.spout.CheckPointState.Action.ROLLBACK;
+import static backtype.storm.spout.CheckPointState.Action.COMMIT;
 
 /**
  * Unit tests for {@link StatefulBoltExecutor}
@@ -86,7 +89,7 @@ public class StatefulBoltExecutorTest {
         Mockito.when(mockTuple.getSourceStreamId()).thenReturn("default");
         executor.execute(mockTuple);
         Mockito.when(mockCheckpointTuple.getSourceStreamId()).thenReturn(CheckpointSpout.CHECKPOINT_STREAM_ID);
-        Mockito.when(mockCheckpointTuple.getStringByField(CHECKPOINT_FIELD_ACTION)).thenReturn(CHECKPOINT_ACTION_INITSTATE);
+        Mockito.when(mockCheckpointTuple.getValueByField(CHECKPOINT_FIELD_ACTION)).thenReturn(INITSTATE);
         Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(0));
         Mockito.doNothing().when(mockOutputCollector).ack(mockCheckpointTuple);
         executor.execute(mockCheckpointTuple);
@@ -99,7 +102,7 @@ public class StatefulBoltExecutorTest {
         Mockito.when(mockTuple.getSourceStreamId()).thenReturn("default");
         executor.execute(mockTuple);
         Mockito.when(mockCheckpointTuple.getSourceStreamId()).thenReturn(CheckpointSpout.CHECKPOINT_STREAM_ID);
-        Mockito.when(mockCheckpointTuple.getStringByField(CHECKPOINT_FIELD_ACTION)).thenReturn(CHECKPOINT_ACTION_ROLLBACK);
+        Mockito.when(mockCheckpointTuple.getValueByField(CHECKPOINT_FIELD_ACTION)).thenReturn(ROLLBACK);
         Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(0));
         Mockito.doNothing().when(mockOutputCollector).ack(mockCheckpointTuple);
         executor.execute(mockCheckpointTuple);
@@ -111,7 +114,7 @@ public class StatefulBoltExecutorTest {
         Mockito.when(mockTuple.getSourceStreamId()).thenReturn("default");
         executor.execute(mockTuple);
         Mockito.when(mockCheckpointTuple.getSourceStreamId()).thenReturn(CheckpointSpout.CHECKPOINT_STREAM_ID);
-        Mockito.when(mockCheckpointTuple.getStringByField(CHECKPOINT_FIELD_ACTION)).thenReturn(CHECKPOINT_ACTION_COMMIT);
+        Mockito.when(mockCheckpointTuple.getValueByField(CHECKPOINT_FIELD_ACTION)).thenReturn(COMMIT);
         Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(0));
         Mockito.doNothing().when(mockOutputCollector).ack(mockCheckpointTuple);
         executor.execute(mockCheckpointTuple);