@Before public void setUp() throws Exception { mockBolt = Mockito.mock(IStatefulBolt.class); executor = new StatefulBoltExecutor<>(mockBolt); mockTopologyContext = Mockito.mock(TopologyContext.class); mockOutputCollector = Mockito.mock(OutputCollector.class); mockState = Mockito.mock(KeyValueState.class); Mockito.when(mockTopologyContext.getThisComponentId()).thenReturn("test"); Mockito.when(mockTopologyContext.getThisTaskId()).thenReturn(1); GlobalStreamId globalStreamId = new GlobalStreamId("test", CheckpointSpout.CHECKPOINT_STREAM_ID); Map<GlobalStreamId, Grouping> thisSources = Collections.singletonMap(globalStreamId, mock(Grouping.class)); Mockito.when(mockTopologyContext.getThisSources()).thenReturn(thisSources); Mockito.when(mockTopologyContext.getComponentTasks(Mockito.any())).thenReturn(Collections.singletonList(1)); mockTuple = Mockito.mock(Tuple.class); mockCheckpointTuple = Mockito.mock(Tuple.class); executor.prepare(mockStormConf, mockTopologyContext, mockOutputCollector, mockState); }
bolt.preCommit(txid); state.commit(txid); ack(preparedTuples); } else if (action == ROLLBACK) { bolt.preRollback(); state.rollback(); fail(preparedTuples); fail(collector.ackedTuples()); } else if (action == INITSTATE) { if (!boltInitialized) { LOG.debug("{} pending tuples to process", pendingTuples.size()); for (Tuple tuple : pendingTuples) { doExecute(tuple);
@Override public void declareOutputFields(OutputFieldsDeclarer declarer) { bolt.declareOutputFields(declarer); declareCheckpointStream(declarer); }
/** * 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. * <p> * The framework provides at-least once guarantee for the state updates. Bolts (both stateful and non-stateful) in a stateful topology * are expected to anchor the tuples while emitting and ack the input tuples once its processed. * </p> * * @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); }
@Override public void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector) { // get the last successfully committed state from state store String namespace = context.getThisComponentId() + "-" + context.getThisTaskId(); prepare(topoConf, context, collector, StateFactory.getState(namespace, topoConf, context)); }
void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector, State state) { init(context, collector); this.collector = new AckTrackingOutputCollector(collector); bolt.prepare(topoConf, context, this.collector); this.state = state; }
@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); } }
@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)); }
/** * Define a new bolt in this topology. This defines a stateful windowed bolt, intended for stateful windowing operations. The {@link * IStatefulWindowedBolt#execute(TupleWindow)} method is triggered for each window interval with the list of current events in the * window. During initialization of this bolt {@link IStatefulWindowedBolt#initState(State)} is invoked 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 windowed 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. * @param <T> the type of the state (e.g. {@link org.apache.storm.state.KeyValueState}) * @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, IStatefulWindowedBolt<T> bolt, Number parallelism_hint) throws IllegalArgumentException { hasStatefulBolt = true; IStatefulBolt<T> executor; if (bolt.isPersistent()) { executor = new PersistentWindowedBoltExecutor<>(bolt); } else { executor = new StatefulWindowedBoltExecutor<T>(bolt); } return setBolt(id, new StatefulBoltExecutor<T>(executor), parallelism_hint); }
@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)); }
void prepare(Map stormConf, TopologyContext context, OutputCollector collector, State state) { init(context, collector); this.collector = new AckTrackingOutputCollector(collector); bolt.prepare(stormConf, context, this.collector); this.state = state; }
@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); } }
@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(); }
bolt.preCommit(txid); state.commit(txid); ack(preparedTuples); } else if (action == ROLLBACK) { bolt.preRollback(); state.rollback(); fail(preparedTuples); fail(collector.ackedTuples()); } else if (action == INITSTATE) { if (!boltInitialized) { LOG.debug("{} pending tuples to process", pendingTuples.size()); for (Tuple tuple : pendingTuples) { doExecute(tuple);
/** * 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. * <p> * The framework provides at-least once guarantee for the state updates. Bolts (both stateful and non-stateful) in a stateful topology * are expected to anchor the tuples while emitting and ack the input tuples once its processed. * </p> * @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); }
@Override public void declareOutputFields(OutputFieldsDeclarer declarer) { bolt.declareOutputFields(declarer); declareCheckpointStream(declarer); }
@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)); }
/** * Define a new bolt in this topology. This defines a stateful windowed bolt, intended for stateful * windowing operations. The {@link IStatefulWindowedBolt#execute(TupleWindow)} method is triggered * for each window interval with the list of current events in the window. During initialization of * this bolt {@link IStatefulWindowedBolt#initState(State)} is invoked 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 windowed 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. * @param <T> the type of the state (e.g. {@link org.apache.storm.state.KeyValueState}) * @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, IStatefulWindowedBolt<T> bolt, Number parallelism_hint) throws IllegalArgumentException { hasStatefulBolt = true; return setBolt(id, new StatefulBoltExecutor<T>(new StatefulWindowedBoltExecutor<T>(bolt)), parallelism_hint); }
@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 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(); }