@Override public void onActivation(Supplier<Iterator<Tuple>> eventsIt, Supplier<Iterator<Tuple>> newEventsIt, Supplier<Iterator<Tuple>> expiredIt, Long timestamp) { /* * Here we don't set the tuples in windowedOutputCollector's context and emit un-anchored. * The checkpoint tuple will trigger a checkpoint in the receiver with the emitted tuples. */ boltExecute(eventsIt, newEventsIt, expiredIt, timestamp); state.clearIteratorPins(); } };
void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector, KeyValueState<Long, WindowPartition<Tuple>> windowState, KeyValueState<String, Deque<Long>> partitionState, KeyValueState<String, Optional<?>> windowSystemState) { outputCollector = collector; this.windowSystemState = windowSystemState; state = new WindowState<>(windowState, partitionState, windowSystemState, this::getState, statefulWindowedBolt.maxEventsInMemory()); doPrepare(topoConf, context, new NoAckOutputCollector(collector), state, true); restoreWindowSystemState(); }
@Override protected void validate(Map<String, Object> topoConf, BaseWindowedBolt.Count windowLengthCount, BaseWindowedBolt.Duration windowLengthDuration, BaseWindowedBolt.Count slidingIntervalCount, BaseWindowedBolt.Duration slidingIntervalDuration) { if (windowLengthCount == null && windowLengthDuration == null) { throw new IllegalArgumentException("Window length is not specified"); } int interval = getCheckpointIntervalMillis(topoConf); int timeout = getTopologyTimeoutMillis(topoConf); if (interval > timeout) { throw new IllegalArgumentException(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL + interval + " is more than " + Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS + " value " + timeout); } }
@Override public void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector) { List<String> registrations = (List<String>) topoConf.getOrDefault(Config.TOPOLOGY_STATE_KRYO_REGISTER, new ArrayList<>()); registrations.add(ConcurrentLinkedQueue.class.getName()); registrations.add(LinkedList.class.getName()); registrations.add(AtomicInteger.class.getName()); registrations.add(EventImpl.class.getName()); registrations.add(WindowPartition.class.getName()); registrations.add(DefaultEvictionContext.class.getName()); topoConf.put(Config.TOPOLOGY_STATE_KRYO_REGISTER, registrations); prepare(topoConf, context, collector, getWindowState(topoConf, context), getPartitionState(topoConf, context), getWindowSystemState(topoConf, context)); }
@Test public void testActivation() throws Exception { Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(), Mockito.anyLong())).thenReturn(true); executor.initState(null); executor.waterMarkEventGenerator = mockWaterMarkEventGenerator; mockTuples.forEach(t -> executor.execute(t)); executor.getWindowManager().add(new WaterMarkEvent<>(activationTs)); executor.prePrepare(0);
@Test public void testCacheEviction() { Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(), Mockito.anyLong())).thenReturn(true); executor.initState(null); executor.waterMarkEventGenerator = mockWaterMarkEventGenerator; int tupleCount = 20000; List<Tuple> mockTuples = getMockTuples(tupleCount); mockTuples.forEach(t -> executor.execute(t)); executor.getWindowManager().add(new WaterMarkEvent<>(activationTs));
@Test public void testExecuteTuple() throws Exception { Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(), Mockito.anyLong())).thenReturn(true); Tuple mockTuple = Mockito.mock(Tuple.class); executor.initState(null); executor.waterMarkEventGenerator = mockWaterMarkEventGenerator; executor.execute(mockTuple); // should be ack-ed once Mockito.verify(mockOutputCollector, Mockito.times(1)).ack(mockTuple); }
/** * 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); }
@Test public void testExecuteLatetuple() throws Exception { Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(), Mockito.anyLong())).thenReturn(false); Tuple mockTuple = Mockito.mock(Tuple.class); executor.initState(null); executor.waterMarkEventGenerator = mockWaterMarkEventGenerator; executor.execute(mockTuple); // ack-ed once Mockito.verify(mockOutputCollector, Mockito.times(1)).ack(mockTuple); // late tuple emitted ArgumentCaptor<String> stringCaptor = ArgumentCaptor.forClass(String.class); Mockito.verify(mockOutputCollector, Mockito.times(1)) .emit(stringCaptor.capture(), anchorCaptor.capture(), valuesCaptor.capture()); Assert.assertEquals(LATE_STREAM, stringCaptor.getValue()); Assert.assertEquals(Collections.singletonList(mockTuple), anchorCaptor.getValue()); Assert.assertEquals(new Values(mockTuple), valuesCaptor.getValue()); }