You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@storm.apache.org by arunmahadevan <gi...@git.apache.org> on 2017/07/17 16:03:24 UTC

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

GitHub user arunmahadevan opened a pull request:

    https://github.com/apache/storm/pull/2218

    STORM-2614: Enhance stateful windowing to persist the window state

    Right now the tuples in window are stored in memory. This limits the usage to windows
    that fit in memory. Also the source tuples cannot be acked until the window expiry.
    By persisting the window transparently in the state backend and caching/iterating them as needed,
    we could support larger windows and also support windowed bolts with user/application state.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/arunmahadevan/storm STORM-2614

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/storm/pull/2218.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #2218
    
----
commit 376d3120a5075d9687f63a587dca5e040a5aa6eb
Author: Arun Mahadevan <ar...@apache.org>
Date:   2017-06-28T16:13:26Z

    STORM-2614: Enhance stateful windowing to persist the window state
    
    Right now the tuples in window are stored in memory. This limits the usage to windows
    that fit in memory. Also the source tuples cannot be acked until the window expiry.
    By persisting the window transparently in the state backend and caching/iterating them as needed,
    we could support larger windows and also support windowed bolts with user/application state.

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130273647
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/WindowPartitionCache.java ---
    @@ -0,0 +1,142 @@
    +/**
    + * 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 java.util.concurrent.ConcurrentMap;
    +
    +/**
    + * A loading cache abstraction for caching {@link PersistentWindowedBoltExecutor.WindowPartition}.
    + *
    + * @param <K> the key type
    + * @param <V> the value type
    + */
    +public interface WindowPartitionCache<K, V> {
    +
    +    /**
    +     * Get value from the cache or load the value.
    +     *
    +     * @param key the key
    +     * @return the value
    +     */
    +    V get(K key);
    +
    +    /**
    +     * Get value from the cache or load the value pinning it
    +     * so that the entry will never get evicted.
    +     *
    +     * @param key the key
    +     * @return the value
    +     */
    +    V getPinned(K key);
    --- End diff --
    
    makes sense.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r131219494
  
    --- Diff: storm-client/test/jvm/org/apache/storm/topology/SimpleWindowPartitionCacheTest.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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.utils.Utils;
    +import org.apache.storm.windowing.persistence.SimpleWindowPartitionCache;
    +import org.apache.storm.windowing.persistence.WindowPartitionCache;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.concurrent.FutureTask;
    +
    +/**
    + * Unit tests for {@link SimpleWindowPartitionCache}
    + */
    +public class SimpleWindowPartitionCacheTest {
    +
    +    @Before
    +    public void setUp() throws Exception {
    +    }
    +
    +    @Test(expected = IllegalArgumentException.class)
    +    public void testBuildInvalid1() throws Exception {
    +        SimpleWindowPartitionCache.<Integer, Integer>newBuilder()
    +            .maximumSize(0)
    +            .build(null);
    +    }
    +
    +    @Test(expected = IllegalArgumentException.class)
    +    public void testBuildInvalid2() throws Exception {
    +        SimpleWindowPartitionCache.<Integer, Integer>newBuilder()
    +            .maximumSize(-1)
    +            .build(null);
    +    }
    +
    +    @Test(expected = NullPointerException.class)
    +    public void testBuildInvalid3() throws Exception {
    +        SimpleWindowPartitionCache.<Integer, Integer>newBuilder()
    +            .maximumSize(1)
    +            .build(null);
    +    }
    +
    +    @Test
    +    public void testBuildOk() throws Exception {
    +        SimpleWindowPartitionCache.<Integer, Integer>newBuilder()
    +            .maximumSize(1)
    +            .removalListener((key, val, removalCause) -> {
    +            })
    +            .build(key -> key);
    +    }
    +
    +    @Test
    +    public void testGet() throws Exception {
    +        List<Integer> removed = new ArrayList<>();
    +        List<Integer> loaded = new ArrayList<>();
    +        SimpleWindowPartitionCache<Integer, Integer> cache =
    +            SimpleWindowPartitionCache.<Integer, Integer>newBuilder()
    +                .maximumSize(2)
    +                .removalListener((key, val, removalCause) -> removed.add(key))
    +                .build(key -> {
    +                    loaded.add(key);
    +                    return key;
    +                });
    +
    +        cache.get(1);
    +        cache.get(2);
    +        cache.get(3);
    +        Assert.assertEquals(Arrays.asList(1, 2, 3), loaded);
    +        Assert.assertEquals(Collections.singletonList(2), removed);
    --- End diff --
    
    Since 2 is the largest un-pinned entry before 3 is loaded. The cache evicts the largest unpinned entry.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r131219546
  
    --- Diff: storm-client/test/jvm/org/apache/storm/windowing/persistence/WindowStateTest.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.windowing.persistence;
    +
    +import org.apache.storm.state.KeyValueState;
    +import org.apache.storm.tuple.Tuple;
    +import org.apache.storm.windowing.Event;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.mockito.ArgumentCaptor;
    +import org.mockito.Captor;
    +import org.mockito.Mock;
    +import org.mockito.Mockito;
    +import org.mockito.MockitoAnnotations;
    +import org.mockito.invocation.InvocationOnMock;
    +import org.mockito.runners.MockitoJUnitRunner;
    +import org.mockito.stubbing.Answer;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.function.Consumer;
    +import java.util.function.Supplier;
    +
    +import static org.mockito.AdditionalAnswers.returnsArgAt;
    +
    +/**
    + * Unit tests for {@link WindowState}
    + */
    +@RunWith(MockitoJUnitRunner.class)
    +public class WindowStateTest {
    +
    +    @Mock
    +    private KeyValueState<Long, WindowState.WindowPartition<Integer>> windowState;
    +    @Mock
    +    private KeyValueState<String, Deque<Long>> partitionIdsState;
    +    @Mock
    +    private KeyValueState<String, Optional<?>> systemState;
    +    @Mock
    +    private Supplier<Map<String, Optional<?>>> supplier;
    +    @Captor
    +    private ArgumentCaptor<Long> longCaptor;
    +    @Captor
    +    private ArgumentCaptor<WindowState.WindowPartition<Integer>> windowValuesCaptor;
    +
    +    private static final int MAX_EVENTS_PER_PARTITION = 1000;
    +
    +    @Before
    +    public void setUp() throws Exception {
    +        MockitoAnnotations.initMocks(this);
    +    }
    +
    +    @Test
    +    public void testAdd() throws Exception {
    +        Mockito.when(partitionIdsState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(windowState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +
    +        WindowState<Integer> ws = getWindowState(10 * MAX_EVENTS_PER_PARTITION);
    +
    +        long partitions = 15;
    +        long numEvents = partitions * MAX_EVENTS_PER_PARTITION;
    +        for (int i = 0; i < numEvents; i++) {
    +            ws.add(getEvent(i));
    +        }
    +        // 5 events evicted to window state
    --- End diff --
    
    yes


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by satishd <gi...@git.apache.org>.
Github user satishd commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130010224
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/SimpleWindowPartitionCache.java ---
    @@ -0,0 +1,191 @@
    +/**
    + * 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 java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.concurrent.ConcurrentMap;
    +import java.util.concurrent.ConcurrentSkipListMap;
    +import java.util.concurrent.locks.ReentrantLock;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * A simple implementation that evicts the largest un-pinned entry from the cache. This works well
    + * for caching window partitions since the access pattern is mostly sequential scans.
    + */
    +public class SimpleWindowPartitionCache<K, V> implements WindowPartitionCache<K, V> {
    --- End diff --
    
    It is good to have unit tests, including with multiple threads.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129652806
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutor.java ---
    @@ -0,0 +1,563 @@
    +/**
    + * 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 java.util.AbstractCollection;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentLinkedQueue;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.locks.ReentrantLock;
    +import java.util.function.Supplier;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.state.KeyValueState;
    +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.windowing.DefaultEvictionContext;
    +import org.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.EventImpl;
    +import org.apache.storm.windowing.WindowLifecycleListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static java.util.Collections.emptyIterator;
    +import static org.apache.storm.topology.WindowPartitionCache.CacheLoader;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalCause;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalListener;
    +
    +/**
    + * Wraps a {@link IStatefulWindowedBolt} and handles the execution. Uses state and the underlying
    + * checkpointing mechanisms to save the tuples in window to state. The tuples are also kept in-memory
    + * by transparently caching the window partitions and checkpointing them as needed.
    + */
    +public class PersistentWindowedBoltExecutor<T extends State> extends WindowedBoltExecutor implements IStatefulBolt<T> {
    +    private static final Logger LOG = LoggerFactory.getLogger(PersistentWindowedBoltExecutor.class);
    +    private final IStatefulWindowedBolt<T> statefulWindowedBolt;
    +    private transient TopologyContext topologyContext;
    +    private transient OutputCollector outputCollector;
    +    private transient WindowState<Tuple> state;
    +    private transient boolean stateInitialized;
    +    private transient boolean prePrepared;
    +
    +    public PersistentWindowedBoltExecutor(IStatefulWindowedBolt<T> bolt) {
    +        super(bolt);
    +        statefulWindowedBolt = bolt;
    +    }
    +
    +    @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));
    +    }
    +
    +    // package access for unit tests
    +    void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                 KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                 KeyValueState<String, Deque<Long>> partitionState,
    +                 KeyValueState<String, Optional<?>> windowSystemState) {
    +        init(topoConf, context, collector, windowState, partitionState, windowSystemState);
    +        doPrepare(topoConf, context, new NoAckOutputCollector(collector), state, true);
    +        Map<String, Optional<?>> wstate = new HashMap<>();
    +        windowSystemState.forEach(s -> wstate.put(s.getKey(), s.getValue()));
    +        restoreState(wstate);
    +    }
    +
    +    @Override
    +    protected void start() {
    +        if (stateInitialized) {
    +            super.start();
    +        } else {
    +            LOG.debug("Will invoke start after state is initialized");
    +        }
    +    }
    +
    +    @Override
    +    public void execute(Tuple input) {
    +        if (!stateInitialized) {
    +            throw new IllegalStateException("execute invoked before initState with input tuple " + input);
    +        }
    +        super.execute(input);
    +        // StatefulBoltExecutor does the actual ack when the state is saved.
    +        outputCollector.ack(input);
    +    }
    +
    +    @Override
    +    public void initState(T state) {
    +        if (stateInitialized) {
    +            LOG.warn("State is already initialized. Ignoring initState");
    +        } else {
    +            statefulWindowedBolt.initState(state);
    +            stateInitialized = true;
    +            start();
    +        }
    +    }
    +
    +    @Override
    +    public void prePrepare(long txid) {
    +        if (stateInitialized) {
    +            LOG.debug("Prepare streamState, txid {}", txid);
    +            state.prepareCommit(txid);
    +            prePrepared = true;
    +        } else {
    +            LOG.warn("Cannot prepare before initState");
    +        }
    +    }
    +
    +    @Override
    +    public void preCommit(long txid) {
    +        if (prePrepared) {
    +            LOG.debug("Commit streamState, txid {}", txid);
    +            state.commit(txid);
    +        } else {
    +            LOG.debug("Ignoring preCommit and not committing streamState.");
    +        }
    +    }
    +
    +    @Override
    +    public void preRollback() {
    +        LOG.debug("Rollback streamState, stateInitialized {}", stateInitialized);
    +        state.rollback();
    +    }
    +
    +    @Override
    +    protected WindowLifecycleListener<Tuple> newWindowLifecycleListener() {
    +        return new WindowLifecycleListener<Tuple>() {
    +            @Override
    +            public void onExpiry(List<Tuple> events) {
    +                /*
    +                 * NO-OP: the events are ack-ed in execute
    +                 */
    +            }
    +
    +            @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();
    +            }
    +        };
    +    }
    +
    +    private void init(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                      KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                      KeyValueState<String, Deque<Long>> partitionState,
    +                      KeyValueState<String, Optional<?>> windowSystemState) {
    +        topologyContext = context;
    +        outputCollector = collector;
    +        state = new WindowState<>(windowState, partitionState, windowSystemState, this::getState,
    +            statefulWindowedBolt.maxEventsInMemory());
    +    }
    +
    +    private KeyValueState<Long, WindowPartition<Tuple>> getWindowState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window";
    +        return (KeyValueState<Long, WindowPartition<Tuple>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Deque<Long>> getPartitionState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-partitions";
    +        return (KeyValueState<String, Deque<Long>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Optional<?>> getWindowSystemState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-systemstate";
    +        return (KeyValueState<String, Optional<?>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    // a wrapper around the window related states that are checkpointed
    +    private static class WindowState<T> extends AbstractCollection<Event<T>> {
    +        // number of events per window-partition
    +        private static final int PARTITION_SZ = 1000;
    +        private static final int MIN_PARTITIONS = 10;
    +        private static final String PARTITION_IDS_KEY = "pk";
    +        private final KeyValueState<String, Deque<Long>> partitionIds;
    +        private final KeyValueState<Long, WindowPartition<T>> windowPartitions;
    +        private final KeyValueState<String, Optional<?>> windowSystemState;
    +        // ordered partition keys
    +        private Deque<Long> pids;
    +        private volatile long latestPartitionId;
    +        private WindowPartitionCache<Long, WindowPartition<T>> cache;
    +        private Supplier<Map<String, Optional<?>>> windowSystemStateSupplier;
    +        private final ReentrantLock partitionIdsLock = new ReentrantLock(true);
    +        private final ReentrantLock windowPartitionsLock = new ReentrantLock(true);
    +        private final long maxEventsInMemory;
    +        private WindowPartition<T> latestPartition;
    +        private Set<Long> iteratorPins = new HashSet<>();
    +
    +        WindowState(KeyValueState<Long, WindowPartition<T>> windowPartitions,
    +                    KeyValueState<String, Deque<Long>> partitionIds,
    +                    KeyValueState<String, Optional<?>> windowSystemState,
    +                    Supplier<Map<String, Optional<?>>> windowSystemStateSupplier,
    +                    long maxEventsInMemory) {
    +            this.windowPartitions = windowPartitions;
    +            this.partitionIds = partitionIds;
    +            this.windowSystemState = windowSystemState;
    +            this.windowSystemStateSupplier = windowSystemStateSupplier;
    +            this.maxEventsInMemory = Math.max(PARTITION_SZ * MIN_PARTITIONS, maxEventsInMemory);
    +            initCache();
    +            initPartitions();
    +        }
    +
    +        @Override
    +        public boolean add(Event<T> event) {
    +            if (latestPartition.size() >= PARTITION_SZ) {
    +                cache.unpin(latestPartition.getId());
    +                latestPartition = getPinnedPartition(getNextPartitionId());
    +            }
    +            latestPartition.add(event);
    +            return true;
    +        }
    +
    +        @Override
    +        public Iterator<Event<T>> iterator() {
    +
    +            return new Iterator<Event<T>>() {
    +                private Iterator<Long> ids = getIds();
    +                private Iterator<Event<T>> current = emptyIterator();
    +                private Iterator<Event<T>> removeFrom;
    +                private WindowPartition<T> curPartition;
    +
    +                private Iterator<Long> getIds() {
    +                    try {
    +                        partitionIdsLock.lock();
    +                        LOG.debug("Iterator pids: {}", pids);
    +                        return new ArrayList<>(pids).iterator();
    +                    } finally {
    +                        partitionIdsLock.unlock();
    +                    }
    +                }
    +
    +                @Override
    +                public void remove() {
    +                    if (removeFrom == null) {
    +                        throw new IllegalStateException("No calls to next() since last call to remove()");
    +                    }
    +                    removeFrom.remove();
    +                    removeFrom = null;
    +                }
    +
    +                @Override
    +                public boolean hasNext() {
    +                    boolean curHasNext = current.hasNext();
    +                    while (!curHasNext && ids.hasNext()) {
    +                        if (curPartition != null) {
    +                            unpin(curPartition.getId());
    +                        }
    +                        curPartition = getPinnedPartition(ids.next());
    +                        iteratorPins.add(curPartition.getId());
    +                        if (curPartition != null) {
    +                            current = curPartition.iterator();
    +                            curHasNext = current.hasNext();
    +                        }
    +                    }
    +                    // un-pin the last partition
    +                    if (!curHasNext && curPartition != null) {
    +                        unpin(curPartition.getId());
    +                        curPartition = null;
    +                    }
    +                    return curHasNext;
    +                }
    +
    +                @Override
    +                public Event<T> next() {
    +                    if (!hasNext()) {
    +                        throw new NoSuchElementException();
    +                    }
    +                    removeFrom = current;
    +                    return current.next();
    +                }
    +
    +                private void unpin(long id) {
    +                    cache.unpin(id);
    +                    iteratorPins.remove(id);
    +                }
    +            };
    +        }
    +
    +        void clearIteratorPins() {
    +            LOG.debug("clearIteratorPins '{}'", iteratorPins);
    +            Iterator<Long> it = iteratorPins.iterator();
    +            while (it.hasNext()) {
    +                cache.unpin(it.next());
    +                it.remove();
    +            }
    +        }
    +
    +        @Override
    +        public int size() {
    +            throw new UnsupportedOperationException();
    +        }
    +
    +        void prepareCommit(long txid) {
    +            flush();
    +            partitionIds.prepareCommit(txid);
    +            windowPartitions.prepareCommit(txid);
    +            windowSystemState.prepareCommit(txid);
    +        }
    +
    +        void commit(long txid) {
    +            partitionIds.commit(txid);
    +            windowPartitions.commit(txid);
    +            windowSystemState.commit(txid);
    +        }
    +
    +        void rollback() {
    +            partitionIds.rollback();
    +            windowPartitions.rollback();
    +            windowSystemState.rollback();
    +        }
    +
    +        private void initPartitions() {
    +            pids = partitionIds.get(PARTITION_IDS_KEY, new LinkedList<>());
    +            if (pids.isEmpty()) {
    +                pids.add(0L);
    +                partitionIds.put(PARTITION_IDS_KEY, pids);
    +            } else {
    +                latestPartitionId = pids.peekLast();
    +            }
    +            latestPartition = cache.getPinned(latestPartitionId);
    +        }
    +
    +        private void initCache() {
    +            long size = maxEventsInMemory / PARTITION_SZ;
    +            LOG.info("maxEventsInMemory: {}, partition size: {}, number of partitions: {}",
    +                maxEventsInMemory, PARTITION_SZ, size);
    +            cache = SimpleWindowPartitionCache.<Long, WindowPartition<T>>newBuilder()
    +                .maximumSize(size)
    +                .removalListener(new RemovalListener<Long, WindowPartition<T>>() {
    +                    @Override
    +                    public void onRemoval(Long pid, WindowPartition<T> p, RemovalCause removalCause) {
    +                        LOG.debug("onRemoval for id '{}', WindowPartition '{}'", pid, p);
    +                        if (pid == null || p == null) {
    --- End diff --
    
    Is this possible?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129763520
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutor.java ---
    @@ -0,0 +1,563 @@
    +/**
    + * 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 java.util.AbstractCollection;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentLinkedQueue;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.locks.ReentrantLock;
    +import java.util.function.Supplier;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.state.KeyValueState;
    +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.windowing.DefaultEvictionContext;
    +import org.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.EventImpl;
    +import org.apache.storm.windowing.WindowLifecycleListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static java.util.Collections.emptyIterator;
    +import static org.apache.storm.topology.WindowPartitionCache.CacheLoader;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalCause;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalListener;
    +
    +/**
    + * Wraps a {@link IStatefulWindowedBolt} and handles the execution. Uses state and the underlying
    + * checkpointing mechanisms to save the tuples in window to state. The tuples are also kept in-memory
    + * by transparently caching the window partitions and checkpointing them as needed.
    + */
    +public class PersistentWindowedBoltExecutor<T extends State> extends WindowedBoltExecutor implements IStatefulBolt<T> {
    +    private static final Logger LOG = LoggerFactory.getLogger(PersistentWindowedBoltExecutor.class);
    +    private final IStatefulWindowedBolt<T> statefulWindowedBolt;
    +    private transient TopologyContext topologyContext;
    +    private transient OutputCollector outputCollector;
    +    private transient WindowState<Tuple> state;
    +    private transient boolean stateInitialized;
    +    private transient boolean prePrepared;
    +
    +    public PersistentWindowedBoltExecutor(IStatefulWindowedBolt<T> bolt) {
    +        super(bolt);
    +        statefulWindowedBolt = bolt;
    +    }
    +
    +    @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));
    +    }
    +
    +    // package access for unit tests
    +    void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                 KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                 KeyValueState<String, Deque<Long>> partitionState,
    +                 KeyValueState<String, Optional<?>> windowSystemState) {
    +        init(topoConf, context, collector, windowState, partitionState, windowSystemState);
    +        doPrepare(topoConf, context, new NoAckOutputCollector(collector), state, true);
    +        Map<String, Optional<?>> wstate = new HashMap<>();
    +        windowSystemState.forEach(s -> wstate.put(s.getKey(), s.getValue()));
    +        restoreState(wstate);
    +    }
    +
    +    @Override
    +    protected void start() {
    +        if (stateInitialized) {
    +            super.start();
    +        } else {
    +            LOG.debug("Will invoke start after state is initialized");
    +        }
    +    }
    +
    +    @Override
    +    public void execute(Tuple input) {
    +        if (!stateInitialized) {
    +            throw new IllegalStateException("execute invoked before initState with input tuple " + input);
    +        }
    +        super.execute(input);
    +        // StatefulBoltExecutor does the actual ack when the state is saved.
    +        outputCollector.ack(input);
    +    }
    +
    +    @Override
    +    public void initState(T state) {
    +        if (stateInitialized) {
    +            LOG.warn("State is already initialized. Ignoring initState");
    +        } else {
    +            statefulWindowedBolt.initState(state);
    +            stateInitialized = true;
    +            start();
    +        }
    +    }
    +
    +    @Override
    +    public void prePrepare(long txid) {
    +        if (stateInitialized) {
    +            LOG.debug("Prepare streamState, txid {}", txid);
    +            state.prepareCommit(txid);
    +            prePrepared = true;
    +        } else {
    +            LOG.warn("Cannot prepare before initState");
    +        }
    +    }
    +
    +    @Override
    +    public void preCommit(long txid) {
    +        if (prePrepared) {
    +            LOG.debug("Commit streamState, txid {}", txid);
    +            state.commit(txid);
    +        } else {
    +            LOG.debug("Ignoring preCommit and not committing streamState.");
    +        }
    +    }
    +
    +    @Override
    +    public void preRollback() {
    +        LOG.debug("Rollback streamState, stateInitialized {}", stateInitialized);
    +        state.rollback();
    +    }
    +
    +    @Override
    +    protected WindowLifecycleListener<Tuple> newWindowLifecycleListener() {
    +        return new WindowLifecycleListener<Tuple>() {
    +            @Override
    +            public void onExpiry(List<Tuple> events) {
    +                /*
    +                 * NO-OP: the events are ack-ed in execute
    +                 */
    +            }
    +
    +            @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();
    +            }
    +        };
    +    }
    +
    +    private void init(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                      KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                      KeyValueState<String, Deque<Long>> partitionState,
    +                      KeyValueState<String, Optional<?>> windowSystemState) {
    +        topologyContext = context;
    +        outputCollector = collector;
    +        state = new WindowState<>(windowState, partitionState, windowSystemState, this::getState,
    +            statefulWindowedBolt.maxEventsInMemory());
    +    }
    +
    +    private KeyValueState<Long, WindowPartition<Tuple>> getWindowState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window";
    +        return (KeyValueState<Long, WindowPartition<Tuple>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Deque<Long>> getPartitionState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-partitions";
    +        return (KeyValueState<String, Deque<Long>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Optional<?>> getWindowSystemState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-systemstate";
    +        return (KeyValueState<String, Optional<?>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    // a wrapper around the window related states that are checkpointed
    +    private static class WindowState<T> extends AbstractCollection<Event<T>> {
    +        // number of events per window-partition
    +        private static final int PARTITION_SZ = 1000;
    --- End diff --
    
    yes


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #2218: STORM-2614: Enhance stateful windowing to persist the win...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2218
  
    @arunmahadevan
    +1 Nice work.
    Could you squash the commits into one? You can merge it by yourself, or please mention me if you want me to merge.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130273697
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutor.java ---
    @@ -0,0 +1,596 @@
    +/**
    + * 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 java.util.AbstractCollection;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Objects;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentLinkedQueue;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.locks.ReentrantLock;
    +import java.util.function.Supplier;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.state.KeyValueState;
    +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.topology.base.BaseWindowedBolt;
    +import org.apache.storm.tuple.Tuple;
    +import org.apache.storm.windowing.DefaultEvictionContext;
    +import org.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.EventImpl;
    +import org.apache.storm.windowing.WindowLifecycleListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static java.util.Collections.emptyIterator;
    +import static org.apache.storm.topology.WindowPartitionCache.CacheLoader;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalCause;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalListener;
    +
    +/**
    + * Wraps a {@link IStatefulWindowedBolt} and handles the execution. Uses state and the underlying
    + * checkpointing mechanisms to save the tuples in window to state. The tuples are also kept in-memory
    + * by transparently caching the window partitions and checkpointing them as needed.
    + */
    +public class PersistentWindowedBoltExecutor<T extends State> extends WindowedBoltExecutor implements IStatefulBolt<T> {
    +    private static final Logger LOG = LoggerFactory.getLogger(PersistentWindowedBoltExecutor.class);
    +    private final IStatefulWindowedBolt<T> statefulWindowedBolt;
    +    private transient TopologyContext topologyContext;
    +    private transient OutputCollector outputCollector;
    +    private transient WindowState<Tuple> state;
    +    private transient boolean stateInitialized;
    +    private transient boolean prePrepared;
    +
    +    public PersistentWindowedBoltExecutor(IStatefulWindowedBolt<T> bolt) {
    +        super(bolt);
    +        statefulWindowedBolt = bolt;
    +    }
    +
    +    @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));
    +    }
    +
    +    @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);
    +        }
    +    }
    +
    +    private int getCheckpointIntervalMillis(Map<String, Object> topoConf) {
    +        int checkpointInterval = Integer.MAX_VALUE;
    +        if (topoConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL) != null) {
    +            checkpointInterval = ((Number) topoConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL)).intValue();
    +        }
    +        return checkpointInterval;
    +    }
    +
    +    // package access for unit tests
    +    void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                 KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                 KeyValueState<String, Deque<Long>> partitionState,
    +                 KeyValueState<String, Optional<?>> windowSystemState) {
    +        init(topoConf, context, collector, windowState, partitionState, windowSystemState);
    +        doPrepare(topoConf, context, new NoAckOutputCollector(collector), state, true);
    +        Map<String, Optional<?>> wstate = new HashMap<>();
    +        windowSystemState.forEach(s -> wstate.put(s.getKey(), s.getValue()));
    +        restoreState(wstate);
    +    }
    +
    +    @Override
    +    protected void start() {
    +        if (stateInitialized) {
    +            super.start();
    +        } else {
    +            LOG.debug("Will invoke start after state is initialized");
    +        }
    +    }
    +
    +    @Override
    +    public void execute(Tuple input) {
    +        if (!stateInitialized) {
    +            throw new IllegalStateException("execute invoked before initState with input tuple " + input);
    +        }
    +        super.execute(input);
    +        // StatefulBoltExecutor does the actual ack when the state is saved.
    +        outputCollector.ack(input);
    +    }
    +
    +    @Override
    +    public void initState(T state) {
    +        if (stateInitialized) {
    +            String msg = "initState invoked when the state is already initialized";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        } else {
    +            statefulWindowedBolt.initState(state);
    +            stateInitialized = true;
    +            start();
    +        }
    +    }
    +
    +    @Override
    +    public void prePrepare(long txid) {
    +        if (stateInitialized) {
    +            LOG.debug("Prepare streamState, txid {}", txid);
    +            state.prepareCommit(txid);
    +            prePrepared = true;
    +        } else {
    +            String msg = "Cannot prepare before initState";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        }
    +    }
    +
    +    @Override
    +    public void preCommit(long txid) {
    +        // preCommit can be invoked during recovery before the state is initialized
    +        if (prePrepared || !stateInitialized) {
    +            LOG.debug("Commit streamState, txid {}", txid);
    +            state.commit(txid);
    +        } else {
    +            String msg = "preCommit before prePrepare in initialized state";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        }
    +    }
    +
    +    @Override
    +    public void preRollback() {
    +        LOG.debug("Rollback streamState, stateInitialized {}", stateInitialized);
    +        state.rollback();
    +    }
    +
    +    @Override
    +    protected WindowLifecycleListener<Tuple> newWindowLifecycleListener() {
    +        return new WindowLifecycleListener<Tuple>() {
    +            @Override
    +            public void onExpiry(List<Tuple> events) {
    +                /*
    +                 * NO-OP: the events are ack-ed in execute
    +                 */
    +            }
    +
    +            @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();
    +            }
    +        };
    +    }
    +
    +    private void init(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                      KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                      KeyValueState<String, Deque<Long>> partitionState,
    +                      KeyValueState<String, Optional<?>> windowSystemState) {
    +        topologyContext = context;
    +        outputCollector = collector;
    +        state = new WindowState<>(windowState, partitionState, windowSystemState, this::getState,
    +            statefulWindowedBolt.maxEventsInMemory());
    +    }
    +
    +    private KeyValueState<Long, WindowPartition<Tuple>> getWindowState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window";
    +        return (KeyValueState<Long, WindowPartition<Tuple>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Deque<Long>> getPartitionState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-partitions";
    +        return (KeyValueState<String, Deque<Long>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Optional<?>> getWindowSystemState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-systemstate";
    +        return (KeyValueState<String, Optional<?>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    // a wrapper around the window related states that are checkpointed
    +    private static class WindowState<T> extends AbstractCollection<Event<T>> {
    +        // number of events per window-partition
    +        private static final int MAX_PARTITION_EVENTS = 1000;
    +        private static final int MIN_PARTITIONS = 10;
    +        private static final String PARTITION_IDS_KEY = "pk";
    +        private final KeyValueState<String, Deque<Long>> partitionIdsState;
    +        private final KeyValueState<Long, WindowPartition<T>> windowPartitionsState;
    +        private final KeyValueState<String, Optional<?>> windowSystemState;
    +        // ordered partition keys
    +        private Deque<Long> partitionIds;
    +        private volatile long latestPartitionId;
    +        private WindowPartitionCache<Long, WindowPartition<T>> cache;
    +        private Supplier<Map<String, Optional<?>>> windowSystemStateSupplier;
    +        private final ReentrantLock partitionIdsLock = new ReentrantLock(true);
    +        private final ReentrantLock windowPartitionsLock = new ReentrantLock(true);
    +        private final long maxEventsInMemory;
    +        private WindowPartition<T> latestPartition;
    +        private Set<Long> iteratorPins = new HashSet<>();
    +
    +        WindowState(KeyValueState<Long, WindowPartition<T>> windowPartitionsState,
    +                    KeyValueState<String, Deque<Long>> partitionIdsState,
    +                    KeyValueState<String, Optional<?>> windowSystemState,
    +                    Supplier<Map<String, Optional<?>>> windowSystemStateSupplier,
    +                    long maxEventsInMemory) {
    +            this.windowPartitionsState = windowPartitionsState;
    +            this.partitionIdsState = partitionIdsState;
    +            this.windowSystemState = windowSystemState;
    +            this.windowSystemStateSupplier = windowSystemStateSupplier;
    +            this.maxEventsInMemory = Math.max(MAX_PARTITION_EVENTS * MIN_PARTITIONS, maxEventsInMemory);
    +            initCache();
    +            initPartitions();
    +        }
    +
    +        @Override
    +        public boolean add(Event<T> event) {
    +            if (latestPartition.size() >= MAX_PARTITION_EVENTS) {
    +                cache.unpin(latestPartition.getId());
    +                latestPartition = getPinnedPartition(getNextPartitionId());
    +            }
    +            latestPartition.add(event);
    +            return true;
    +        }
    +
    +        @Override
    +        public Iterator<Event<T>> iterator() {
    +
    +            return new Iterator<Event<T>>() {
    +                private Iterator<Long> ids = getIds();
    +                private Iterator<Event<T>> current = emptyIterator();
    +                private Iterator<Event<T>> removeFrom;
    +                private WindowPartition<T> curPartition;
    +
    +                private Iterator<Long> getIds() {
    +                    try {
    +                        partitionIdsLock.lock();
    +                        LOG.debug("Iterator partitionIds: {}", partitionIds);
    +                        return new ArrayList<>(partitionIds).iterator();
    +                    } finally {
    +                        partitionIdsLock.unlock();
    +                    }
    +                }
    +
    +                @Override
    +                public void remove() {
    +                    if (removeFrom == null) {
    +                        throw new IllegalStateException("No calls to next() since last call to remove()");
    +                    }
    +                    removeFrom.remove();
    +                    removeFrom = null;
    +                }
    +
    +                @Override
    +                public boolean hasNext() {
    +                    boolean curHasNext = current.hasNext();
    +                    while (!curHasNext && ids.hasNext()) {
    +                        if (curPartition != null) {
    +                            unpin(curPartition.getId());
    +                        }
    +                        curPartition = getPinnedPartition(ids.next());
    +                        if (curPartition != null) {
    +                            iteratorPins.add(curPartition.getId());
    +                            current = curPartition.iterator();
    +                            curHasNext = current.hasNext();
    +                        }
    +                    }
    +                    // un-pin the last partition
    +                    if (!curHasNext && curPartition != null) {
    +                        unpin(curPartition.getId());
    +                        curPartition = null;
    +                    }
    +                    return curHasNext;
    +                }
    +
    +                @Override
    +                public Event<T> next() {
    +                    if (!hasNext()) {
    +                        throw new NoSuchElementException();
    +                    }
    +                    removeFrom = current;
    +                    return current.next();
    +                }
    +
    +                private void unpin(long id) {
    +                    cache.unpin(id);
    +                    iteratorPins.remove(id);
    +                }
    +            };
    +        }
    +
    +        void clearIteratorPins() {
    +            LOG.debug("clearIteratorPins '{}'", iteratorPins);
    +            Iterator<Long> it = iteratorPins.iterator();
    +            while (it.hasNext()) {
    +                cache.unpin(it.next());
    +                it.remove();
    +            }
    +        }
    +
    +        @Override
    +        public int size() {
    +            throw new UnsupportedOperationException();
    +        }
    +
    +        void prepareCommit(long txid) {
    +            flush();
    +            partitionIdsState.prepareCommit(txid);
    +            windowPartitionsState.prepareCommit(txid);
    +            windowSystemState.prepareCommit(txid);
    +        }
    +
    +        void commit(long txid) {
    +            partitionIdsState.commit(txid);
    +            windowPartitionsState.commit(txid);
    +            windowSystemState.commit(txid);
    +        }
    +
    +        void rollback() {
    +            partitionIdsState.rollback();
    +            windowPartitionsState.rollback();
    +            windowSystemState.rollback();
    +        }
    +
    +        private void initPartitions() {
    +            partitionIds = partitionIdsState.get(PARTITION_IDS_KEY, new LinkedList<>());
    --- End diff --
    
    Thanks for noticing, yes this needs to be handled.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130273727
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutor.java ---
    @@ -0,0 +1,596 @@
    +/**
    + * 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 java.util.AbstractCollection;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Objects;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentLinkedQueue;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.locks.ReentrantLock;
    +import java.util.function.Supplier;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.state.KeyValueState;
    +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.topology.base.BaseWindowedBolt;
    +import org.apache.storm.tuple.Tuple;
    +import org.apache.storm.windowing.DefaultEvictionContext;
    +import org.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.EventImpl;
    +import org.apache.storm.windowing.WindowLifecycleListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static java.util.Collections.emptyIterator;
    +import static org.apache.storm.topology.WindowPartitionCache.CacheLoader;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalCause;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalListener;
    +
    +/**
    + * Wraps a {@link IStatefulWindowedBolt} and handles the execution. Uses state and the underlying
    + * checkpointing mechanisms to save the tuples in window to state. The tuples are also kept in-memory
    + * by transparently caching the window partitions and checkpointing them as needed.
    + */
    +public class PersistentWindowedBoltExecutor<T extends State> extends WindowedBoltExecutor implements IStatefulBolt<T> {
    +    private static final Logger LOG = LoggerFactory.getLogger(PersistentWindowedBoltExecutor.class);
    +    private final IStatefulWindowedBolt<T> statefulWindowedBolt;
    +    private transient TopologyContext topologyContext;
    +    private transient OutputCollector outputCollector;
    +    private transient WindowState<Tuple> state;
    +    private transient boolean stateInitialized;
    +    private transient boolean prePrepared;
    +
    +    public PersistentWindowedBoltExecutor(IStatefulWindowedBolt<T> bolt) {
    +        super(bolt);
    +        statefulWindowedBolt = bolt;
    +    }
    +
    +    @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));
    +    }
    +
    +    @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);
    +        }
    +    }
    +
    +    private int getCheckpointIntervalMillis(Map<String, Object> topoConf) {
    +        int checkpointInterval = Integer.MAX_VALUE;
    +        if (topoConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL) != null) {
    +            checkpointInterval = ((Number) topoConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL)).intValue();
    +        }
    +        return checkpointInterval;
    +    }
    +
    +    // package access for unit tests
    +    void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                 KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                 KeyValueState<String, Deque<Long>> partitionState,
    +                 KeyValueState<String, Optional<?>> windowSystemState) {
    +        init(topoConf, context, collector, windowState, partitionState, windowSystemState);
    +        doPrepare(topoConf, context, new NoAckOutputCollector(collector), state, true);
    +        Map<String, Optional<?>> wstate = new HashMap<>();
    +        windowSystemState.forEach(s -> wstate.put(s.getKey(), s.getValue()));
    +        restoreState(wstate);
    +    }
    +
    +    @Override
    +    protected void start() {
    +        if (stateInitialized) {
    +            super.start();
    +        } else {
    +            LOG.debug("Will invoke start after state is initialized");
    +        }
    +    }
    +
    +    @Override
    +    public void execute(Tuple input) {
    +        if (!stateInitialized) {
    +            throw new IllegalStateException("execute invoked before initState with input tuple " + input);
    +        }
    +        super.execute(input);
    +        // StatefulBoltExecutor does the actual ack when the state is saved.
    +        outputCollector.ack(input);
    +    }
    +
    +    @Override
    +    public void initState(T state) {
    +        if (stateInitialized) {
    +            String msg = "initState invoked when the state is already initialized";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        } else {
    +            statefulWindowedBolt.initState(state);
    +            stateInitialized = true;
    +            start();
    +        }
    +    }
    +
    +    @Override
    +    public void prePrepare(long txid) {
    +        if (stateInitialized) {
    +            LOG.debug("Prepare streamState, txid {}", txid);
    +            state.prepareCommit(txid);
    +            prePrepared = true;
    +        } else {
    +            String msg = "Cannot prepare before initState";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        }
    +    }
    +
    +    @Override
    +    public void preCommit(long txid) {
    +        // preCommit can be invoked during recovery before the state is initialized
    +        if (prePrepared || !stateInitialized) {
    +            LOG.debug("Commit streamState, txid {}", txid);
    +            state.commit(txid);
    +        } else {
    +            String msg = "preCommit before prePrepare in initialized state";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        }
    +    }
    +
    +    @Override
    +    public void preRollback() {
    +        LOG.debug("Rollback streamState, stateInitialized {}", stateInitialized);
    +        state.rollback();
    +    }
    +
    +    @Override
    +    protected WindowLifecycleListener<Tuple> newWindowLifecycleListener() {
    +        return new WindowLifecycleListener<Tuple>() {
    +            @Override
    +            public void onExpiry(List<Tuple> events) {
    +                /*
    +                 * NO-OP: the events are ack-ed in execute
    +                 */
    +            }
    +
    +            @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();
    +            }
    +        };
    +    }
    +
    +    private void init(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                      KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                      KeyValueState<String, Deque<Long>> partitionState,
    +                      KeyValueState<String, Optional<?>> windowSystemState) {
    +        topologyContext = context;
    +        outputCollector = collector;
    +        state = new WindowState<>(windowState, partitionState, windowSystemState, this::getState,
    +            statefulWindowedBolt.maxEventsInMemory());
    +    }
    +
    +    private KeyValueState<Long, WindowPartition<Tuple>> getWindowState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window";
    +        return (KeyValueState<Long, WindowPartition<Tuple>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Deque<Long>> getPartitionState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-partitions";
    +        return (KeyValueState<String, Deque<Long>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Optional<?>> getWindowSystemState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-systemstate";
    +        return (KeyValueState<String, Optional<?>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    // a wrapper around the window related states that are checkpointed
    +    private static class WindowState<T> extends AbstractCollection<Event<T>> {
    +        // number of events per window-partition
    +        private static final int MAX_PARTITION_EVENTS = 1000;
    +        private static final int MIN_PARTITIONS = 10;
    +        private static final String PARTITION_IDS_KEY = "pk";
    +        private final KeyValueState<String, Deque<Long>> partitionIdsState;
    +        private final KeyValueState<Long, WindowPartition<T>> windowPartitionsState;
    +        private final KeyValueState<String, Optional<?>> windowSystemState;
    +        // ordered partition keys
    +        private Deque<Long> partitionIds;
    +        private volatile long latestPartitionId;
    +        private WindowPartitionCache<Long, WindowPartition<T>> cache;
    +        private Supplier<Map<String, Optional<?>>> windowSystemStateSupplier;
    +        private final ReentrantLock partitionIdsLock = new ReentrantLock(true);
    +        private final ReentrantLock windowPartitionsLock = new ReentrantLock(true);
    +        private final long maxEventsInMemory;
    +        private WindowPartition<T> latestPartition;
    +        private Set<Long> iteratorPins = new HashSet<>();
    +
    +        WindowState(KeyValueState<Long, WindowPartition<T>> windowPartitionsState,
    +                    KeyValueState<String, Deque<Long>> partitionIdsState,
    +                    KeyValueState<String, Optional<?>> windowSystemState,
    +                    Supplier<Map<String, Optional<?>>> windowSystemStateSupplier,
    +                    long maxEventsInMemory) {
    +            this.windowPartitionsState = windowPartitionsState;
    +            this.partitionIdsState = partitionIdsState;
    +            this.windowSystemState = windowSystemState;
    +            this.windowSystemStateSupplier = windowSystemStateSupplier;
    +            this.maxEventsInMemory = Math.max(MAX_PARTITION_EVENTS * MIN_PARTITIONS, maxEventsInMemory);
    +            initCache();
    +            initPartitions();
    +        }
    +
    +        @Override
    +        public boolean add(Event<T> event) {
    +            if (latestPartition.size() >= MAX_PARTITION_EVENTS) {
    +                cache.unpin(latestPartition.getId());
    +                latestPartition = getPinnedPartition(getNextPartitionId());
    +            }
    +            latestPartition.add(event);
    +            return true;
    +        }
    +
    +        @Override
    +        public Iterator<Event<T>> iterator() {
    +
    +            return new Iterator<Event<T>>() {
    +                private Iterator<Long> ids = getIds();
    +                private Iterator<Event<T>> current = emptyIterator();
    +                private Iterator<Event<T>> removeFrom;
    +                private WindowPartition<T> curPartition;
    +
    +                private Iterator<Long> getIds() {
    +                    try {
    +                        partitionIdsLock.lock();
    +                        LOG.debug("Iterator partitionIds: {}", partitionIds);
    +                        return new ArrayList<>(partitionIds).iterator();
    +                    } finally {
    +                        partitionIdsLock.unlock();
    +                    }
    +                }
    +
    +                @Override
    +                public void remove() {
    +                    if (removeFrom == null) {
    +                        throw new IllegalStateException("No calls to next() since last call to remove()");
    +                    }
    +                    removeFrom.remove();
    +                    removeFrom = null;
    +                }
    +
    +                @Override
    +                public boolean hasNext() {
    +                    boolean curHasNext = current.hasNext();
    +                    while (!curHasNext && ids.hasNext()) {
    +                        if (curPartition != null) {
    +                            unpin(curPartition.getId());
    +                        }
    +                        curPartition = getPinnedPartition(ids.next());
    +                        if (curPartition != null) {
    +                            iteratorPins.add(curPartition.getId());
    +                            current = curPartition.iterator();
    +                            curHasNext = current.hasNext();
    +                        }
    +                    }
    +                    // un-pin the last partition
    +                    if (!curHasNext && curPartition != null) {
    +                        unpin(curPartition.getId());
    +                        curPartition = null;
    +                    }
    +                    return curHasNext;
    +                }
    +
    +                @Override
    +                public Event<T> next() {
    +                    if (!hasNext()) {
    +                        throw new NoSuchElementException();
    +                    }
    +                    removeFrom = current;
    +                    return current.next();
    +                }
    +
    +                private void unpin(long id) {
    +                    cache.unpin(id);
    +                    iteratorPins.remove(id);
    +                }
    +            };
    +        }
    +
    +        void clearIteratorPins() {
    +            LOG.debug("clearIteratorPins '{}'", iteratorPins);
    +            Iterator<Long> it = iteratorPins.iterator();
    +            while (it.hasNext()) {
    +                cache.unpin(it.next());
    +                it.remove();
    +            }
    +        }
    +
    +        @Override
    +        public int size() {
    +            throw new UnsupportedOperationException();
    +        }
    +
    +        void prepareCommit(long txid) {
    +            flush();
    +            partitionIdsState.prepareCommit(txid);
    +            windowPartitionsState.prepareCommit(txid);
    +            windowSystemState.prepareCommit(txid);
    +        }
    +
    +        void commit(long txid) {
    +            partitionIdsState.commit(txid);
    +            windowPartitionsState.commit(txid);
    +            windowSystemState.commit(txid);
    +        }
    +
    +        void rollback() {
    +            partitionIdsState.rollback();
    +            windowPartitionsState.rollback();
    +            windowSystemState.rollback();
    +        }
    +
    +        private void initPartitions() {
    +            partitionIds = partitionIdsState.get(PARTITION_IDS_KEY, new LinkedList<>());
    +            if (partitionIds.isEmpty()) {
    +                partitionIds.add(0L);
    +                partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    +            }
    +            latestPartitionId = partitionIds.peekLast();
    +            latestPartition = cache.getPinned(latestPartitionId);
    +        }
    +
    +        private void initCache() {
    +            long size = maxEventsInMemory / MAX_PARTITION_EVENTS;
    +            LOG.info("maxEventsInMemory: {}, partition size: {}, number of partitions: {}",
    +                maxEventsInMemory, MAX_PARTITION_EVENTS, size);
    +            cache = SimpleWindowPartitionCache.<Long, WindowPartition<T>>newBuilder()
    +                .maximumSize(size)
    +                .removalListener(new RemovalListener<Long, WindowPartition<T>>() {
    +                    @Override
    +                    public void onRemoval(Long pid, WindowPartition<T> p, RemovalCause removalCause) {
    +                        Objects.requireNonNull(pid, "Null partition id");
    +                        Objects.requireNonNull(p, "Null window partition");
    +                        LOG.debug("onRemoval for id '{}', WindowPartition '{}'", pid, p);
    +                        try {
    +                            windowPartitionsLock.lock();
    +                            if (p.isEmpty() && pid != latestPartitionId) {
    +                                // if the empty partition was not invalidated by flush, but evicted from cache
    +                                if (removalCause != RemovalCause.EXPLICIT) {
    +                                    deletePartition(pid);
    +                                    windowPartitionsState.delete(pid);
    +                                }
    +                            } else if (p.isModified()) {
    +                                windowPartitionsState.put(pid, p);
    +                            } else {
    +                                LOG.debug("WindowPartition '{}' is not modified", pid);
    +                            }
    +                        } finally {
    +                            windowPartitionsLock.unlock();
    +                        }
    +                    }
    +                }).build(new CacheLoader<Long, WindowPartition<T>>() {
    +                    @Override
    +                   public WindowPartition<T> load(Long id) {
    +                        LOG.debug("Load partition: {}", id);
    +                        // load from state
    +                        try {
    +                            windowPartitionsLock.lock();
    +                            return windowPartitionsState.get(id, new WindowPartition<>(id));
    +                        } finally {
    +                            windowPartitionsLock.unlock();
    +                        }
    +                    }
    +                    }
    +                );
    +        }
    +
    +        private void deletePartition(long pid) {
    +            LOG.debug("Delete partition: {}", pid);
    +            try {
    +                partitionIdsLock.lock();
    +                partitionIds.remove(pid);
    +                partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    +            } finally {
    +                partitionIdsLock.unlock();
    +            }
    +        }
    +
    +        private long getNextPartitionId() {
    +            try {
    +                partitionIdsLock.lock();
    +                partitionIds.add(++latestPartitionId);
    +                partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    +            } finally {
    +                partitionIdsLock.unlock();
    +            }
    +            return latestPartitionId;
    +        }
    +
    +        private WindowPartition<T> getPinnedPartition(long id) {
    +            return cache.getPinned(id);
    +        }
    +
    +        private void flush() {
    +            LOG.debug("Flushing modified partitions");
    +            cache.asMap().forEach((pid, p) -> {
    +                Long pidToInvalidate = null;
    +                try {
    +                    windowPartitionsLock.lock();
    +                    if (p.isEmpty() && pid != latestPartitionId) {
    +                        LOG.debug("Invalidating empty partition {}", pid);
    +                        deletePartition(pid);
    +                        windowPartitionsState.delete(pid);
    +                        pidToInvalidate = pid;
    +                    } else if (p.isModified()) {
    +                        LOG.debug("Updating modified partition {}", pid);
    +                        p.clearModified();
    +                        windowPartitionsState.put(pid, p);
    +                    }
    +                } finally {
    +                    windowPartitionsLock.unlock();
    +                }
    +                // invalidate after releasing the lock
    +                if (pidToInvalidate != null) {
    +                    cache.invalidate(pidToInvalidate);
    +                }
    +            });
    +            windowSystemStateSupplier.get().forEach(windowSystemState::put);
    +        }
    +    }
    +
    +    // the window partition that holds the events
    +    public static class WindowPartition<T> implements Iterable<Event<T>> {
    --- End diff --
    
    Will refactor.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130228020
  
    --- Diff: storm-client/test/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutorTest.java ---
    @@ -0,0 +1,261 @@
    +/**
    + * 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.Config;
    +import org.apache.storm.generated.GlobalStreamId;
    +import org.apache.storm.state.KeyValueState;
    +import org.apache.storm.streams.Pair;
    +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.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.TimestampExtractor;
    +import org.apache.storm.windowing.TupleWindow;
    +import org.apache.storm.windowing.WaterMarkEvent;
    +import org.apache.storm.windowing.WaterMarkEventGenerator;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.mockito.ArgumentCaptor;
    +import org.mockito.Captor;
    +import org.mockito.Mockito;
    +import org.mockito.MockitoAnnotations;
    +import org.mockito.invocation.InvocationOnMock;
    +import org.mockito.runners.MockitoJUnitRunner;
    +import org.mockito.stubbing.Answer;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.stream.Collectors;
    +import java.util.stream.LongStream;
    +
    +import static org.mockito.AdditionalAnswers.returnsArgAt;
    +
    +/**
    + * Unit tests for {@link PersistentWindowedBoltExecutor}
    + */
    +@RunWith(MockitoJUnitRunner.class)
    +public class PersistentWindowedBoltExecutorTest {
    +    public static final String LATE_STREAM = "late_stream";
    +    private PersistentWindowedBoltExecutor<KeyValueState<String, String>> executor;
    +    private IStatefulWindowedBolt<KeyValueState<String, String>> mockBolt;
    +    private OutputCollector mockOutputCollector;
    +    private TopologyContext mockTopologyContext;
    +    private Map<String, Object> mockStormConf = new HashMap<>();
    +    private TimestampExtractor mockTimestampExtractor;
    +    private WaterMarkEventGenerator mockWaterMarkEventGenerator;
    +    private KeyValueState mockPartitionState;
    +    private KeyValueState mockWindowState;
    +    private KeyValueState mockSystemState;
    +
    +    private static final String PARTITION_KEY = "pk";
    +    private static final String EVICTION_STATE_KEY = "es";
    +    private static final String TRIGGER_STATE_KEY = "ts";
    +    private long tupleTs;
    +
    +    @Captor
    +    private ArgumentCaptor<Tuple> tupleCaptor;
    +    @Captor
    +    private ArgumentCaptor<Collection<Tuple>> anchorCaptor;
    +    @Captor
    +    private ArgumentCaptor<String> stringCaptor;
    +    @Captor
    +    private ArgumentCaptor<Long> longCaptor;
    +    @Captor
    +    private ArgumentCaptor<Values> valuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<TupleWindow> tupleWindowCaptor;
    +    @Captor
    +    private ArgumentCaptor<Deque<Long>> partitionValuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<PersistentWindowedBoltExecutor.WindowPartition<Event<Tuple>>> windowValuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<Object> systemValuesCaptor;
    +
    +    @Before
    +    public void setUp() throws Exception {
    +        MockitoAnnotations.initMocks(this);
    +        mockBolt = Mockito.mock(IStatefulWindowedBolt.class);
    +        mockWaterMarkEventGenerator = Mockito.mock(WaterMarkEventGenerator.class);
    +        mockTimestampExtractor = Mockito.mock(TimestampExtractor.class);
    +        tupleTs = System.currentTimeMillis();
    +        Mockito.when(mockTimestampExtractor.extractTimestamp(Mockito.any())).thenReturn(tupleTs);
    +        Mockito.when(mockBolt.getTimestampExtractor()).thenReturn(mockTimestampExtractor);
    +        Mockito.when(mockBolt.isPersistent()).thenReturn(true);
    +        mockPartitionState = Mockito.mock(KeyValueState.class);
    +        mockWindowState = Mockito.mock(KeyValueState.class);
    +        mockSystemState = Mockito.mock(KeyValueState.class);
    +        mockTopologyContext = Mockito.mock(TopologyContext.class);
    +        Mockito.when(mockTopologyContext.getThisStreams()).thenReturn(Collections.singleton(LATE_STREAM));
    +        mockOutputCollector = Mockito.mock(OutputCollector.class);
    +        executor = new PersistentWindowedBoltExecutor<>(mockBolt);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_WINDOW_LENGTH_COUNT, 5);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_SLIDING_INTERVAL_COUNT, 5);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_LATE_TUPLE_STREAM, LATE_STREAM);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_WATERMARK_EVENT_INTERVAL_MS, 100_000);
    +        mockStormConf.put(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS, 30);
    +        mockStormConf.put(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL, 1000);
    +        Mockito.when(mockPartitionState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(mockWindowState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(mockSystemState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        executor.prepare(mockStormConf, mockTopologyContext, mockOutputCollector,
    +            mockWindowState, mockPartitionState, mockSystemState);
    +    }
    +
    +    @Test
    +    public void testExecuteTuple() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), 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);
    +    }
    +
    +    @Test
    +    public void testExecuteLatetuple() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), 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
    +        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());
    +    }
    +
    +    @Test
    +    public void testActivation() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), Mockito.anyLong())).thenReturn(true);
    +        executor.initState(null);
    +        executor.waterMarkEventGenerator = mockWaterMarkEventGenerator;
    +
    +        List<Tuple> mockTuples = getMockTuples(5);
    +        mockTuples.forEach(t -> executor.execute(t));
    +        // all tuples acked
    +        Mockito.verify(mockOutputCollector, Mockito.times(5)).ack(tupleCaptor.capture());
    +        Assert.assertArrayEquals(mockTuples.toArray(), tupleCaptor.getAllValues().toArray());
    +
    +        Mockito.doAnswer(new Answer<Void>() {
    +            @Override
    +            public Void answer(InvocationOnMock invocation) throws Throwable {
    +                TupleWindow window = (TupleWindow) invocation.getArguments()[0];
    +                // iterate the tuples
    +                Assert.assertEquals(5, window.get().size());
    +                // iterating multiple times should produce same events
    +                Assert.assertEquals(5, window.get().size());
    +                Assert.assertEquals(5, window.get().size());
    +                return null;
    +            }
    +        }).when(mockBolt).execute(Mockito.any());
    +        // trigger the window
    +        long activationTs = tupleTs + 1000;
    +        executor.getWindowManager().add(new WaterMarkEvent<>(activationTs));
    +        executor.prePrepare(0);
    +
    +        // partition ids
    +        Mockito.verify(mockPartitionState, Mockito.times(1)).put(stringCaptor.capture(), partitionValuesCaptor.capture());
    +        Assert.assertEquals(PARTITION_KEY, stringCaptor.getAllValues().get(0));
    +        Assert.assertEquals(Collections.singletonList(0L), partitionValuesCaptor.getAllValues().get(0));
    +
    +        // window partitions
    +        Mockito.verify(mockWindowState, Mockito.times(1)).put(longCaptor.capture(), windowValuesCaptor.capture());
    +        Assert.assertEquals(0L, (long) longCaptor.getAllValues().get(0));
    +        Assert.assertEquals(5, windowValuesCaptor.getAllValues().get(0).size());
    +        List<Event<?>> tuples = windowValuesCaptor.getAllValues().get(0)
    +            .getEvents().stream().map(Event::get).collect(Collectors.toList());
    +        Assert.assertArrayEquals(mockTuples.toArray(), tuples.toArray());
    +
    +        // window system state
    +        Mockito.verify(mockSystemState, Mockito.times(2)).put(stringCaptor.capture(), systemValuesCaptor.capture());
    +        Assert.assertEquals(EVICTION_STATE_KEY, stringCaptor.getAllValues().get(1));
    +        Assert.assertEquals(Optional.of(Pair.of(5L, 5L)), systemValuesCaptor.getAllValues().get(0));
    +        Assert.assertEquals(TRIGGER_STATE_KEY, stringCaptor.getAllValues().get(2));
    +        Assert.assertEquals(Optional.of(tupleTs), systemValuesCaptor.getAllValues().get(1));
    +    }
    +
    +    @Test
    +    public void testCacheEviction() {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), Mockito.anyLong())).thenReturn(true);
    +        executor.initState(null);
    +        executor.waterMarkEventGenerator = mockWaterMarkEventGenerator;
    +        List<Tuple> mockTuples = getMockTuples(20000);
    +        mockTuples.forEach(t -> executor.execute(t));
    +
    +        Mockito.verify(mockWindowState, Mockito.times(10)).put(longCaptor.capture(), windowValuesCaptor.capture());
    +        Assert.assertEquals(10, longCaptor.getAllValues().size());
    +        Assert.assertEquals(10, windowValuesCaptor.getAllValues().size());
    +        // number of evicted events
    +        Assert.assertEquals(10_000, windowValuesCaptor.getAllValues().stream()
    +            .mapToInt(x -> x.size()).sum());
    +
    +        Map<Long, PersistentWindowedBoltExecutor.WindowPartition<Event<Tuple>>> partitionMap = new HashMap<>();
    +        windowValuesCaptor.getAllValues().forEach(v -> partitionMap.put(v.getId(), v));
    +
    +        Mockito.verify(mockPartitionState, Mockito.times(20)).put(stringCaptor.capture(), partitionValuesCaptor.capture());
    +        Assert.assertEquals(LongStream.range(0, 20).boxed().collect(Collectors.toList()), partitionValuesCaptor.getAllValues().get(19));
    --- End diff --
    
    I can't tell why 0...20 is the expected value here


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #2218: STORM-2614: Enhance stateful windowing to persist the win...

Posted by roshannaik <gi...@git.apache.org>.
Github user roshannaik commented on the issue:

    https://github.com/apache/storm/pull/2218
  
    @arunmahadevan  ... could you  put that info on the design in the JIRA itself. easier for future reference.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129763692
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/SimpleWindowPartitionCache.java ---
    @@ -0,0 +1,191 @@
    +/**
    + * 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 java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.concurrent.ConcurrentMap;
    +import java.util.concurrent.ConcurrentSkipListMap;
    +import java.util.concurrent.locks.ReentrantLock;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * A simple implementation that evicts the largest un-pinned entry from the cache. This works well
    + * for caching window partitions since the access pattern is mostly sequential scans.
    + */
    +public class SimpleWindowPartitionCache<K, V> implements WindowPartitionCache<K, V> {
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleWindowPartitionCache.class);
    +
    +    private final ConcurrentSkipListMap<K, V> map = new ConcurrentSkipListMap<>();
    +    private final Map<K, Long> pinned = new HashMap<>();
    +    private final long maximumSize;
    +    private final RemovalListener<K, V> removalListener;
    +    private final CacheLoader<K, V> cacheLoader;
    +    private final ReentrantLock lock = new ReentrantLock(true);
    +    private int size;
    +
    +    @Override
    +    public V get(K key) {
    +        return getOrLoad(key, false);
    +    }
    +
    +    @Override
    +    public V getPinned(K key) {
    +        return getOrLoad(key, true);
    +    }
    +
    +    @Override
    +    public boolean unpin(K key) {
    +        LOG.debug("unpin '{}'", key);
    +        boolean res = false;
    +        try {
    +            lock.lock();
    +            if (pinned.compute(key, (k, v) -> v == null ? 0 : v - 1) <= 0) {
    +                pinned.remove(key);
    +                res = true;
    +            }
    +        } finally {
    +            lock.unlock();
    +        }
    +        LOG.debug("pinned '{}'", pinned);
    +        return res;
    +    }
    +
    +    @Override
    +    public ConcurrentMap<K, V> asMap() {
    +        return map;
    +    }
    +
    +    @Override
    +    public void invalidate(K key) {
    +        try {
    +            lock.lock();
    +            if (isPinned(key)) {
    +                LOG.debug("Entry '{}' is pinned, skipping invalidation", key);
    +            } else {
    +                LOG.debug("Invalidating entry '{}'", key);
    +                V val = map.remove(key);
    +                if (val != null) {
    +                    --size;
    +                    pinned.remove(key);
    +                    removalListener.onRemoval(key, val, RemovalCause.EXPLICIT);
    +                }
    +            }
    +        } finally {
    +            lock.unlock();
    +        }
    +    }
    +
    +    // Get or load from the cache optionally pinning the entry
    +    // so that it wont get evicted from the cache
    +    private V getOrLoad(K key, boolean shouldPin) {
    +        V val;
    +        if (shouldPin) {
    +            try {
    +                lock.lock();
    +                val = load(key);
    +                pin(key);
    +            } finally {
    +                lock.unlock();
    +            }
    +        } else {
    +            val = map.get(key);
    +            if (val == null) {
    +                try {
    +                    lock.lock();
    +                    val = load(key);
    +                } finally {
    +                    lock.unlock();
    +                }
    +            }
    +        }
    +
    +        return val;
    +    }
    +
    +    private V load(K key) {
    +        V val = map.get(key);
    +        if (val == null) {
    +            val = cacheLoader.load(key);
    +            if (val == null) {
    +                throw new NullPointerException("Null value");
    +            }
    +            ensureCapacity();
    +            map.put(key, val);
    +            ++size;
    +        }
    +        return val;
    +    }
    +
    +    private void ensureCapacity() {
    +        if (size >= maximumSize) {
    +            Iterator<Map.Entry<K, V>> it = map.descendingMap().entrySet().iterator();
    +            while (it.hasNext()) {
    +                Map.Entry<K, V> next = it.next();
    +                if (!isPinned(next.getKey())) {
    +                    it.remove();
    +                    removalListener.onRemoval(next.getKey(), next.getValue(), RemovalCause.REPLACED);
    +                    --size;
    +                    break;
    +                }
    +            }
    +        }
    +    }
    +
    +    private void pin(K key) {
    +        LOG.debug("pin '{}'", key);
    +        pinned.compute(key, (k, v) -> v == null ? 1L : v + 1);
    +        LOG.debug("pinned '{}'", pinned);
    +    }
    +
    +    private boolean isPinned(K key) {
    +        return pinned.getOrDefault(key, 0L) > 0;
    --- End diff --
    
    containsKey should work here since we are removing the entry in `unpin` if the value <= 0, but prefer to leave it as is to be defensive against future code refactoring.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129109566
  
    --- Diff: docs/Windowing.md ---
    @@ -266,3 +266,105 @@ tuples can be received within the timeout period.
     An example toplogy `SlidingWindowTopology` shows how to use the apis to compute a sliding window sum and a tumbling window 
     average.
     
    +## Stateful windowing
    +The default windowing implementation in storm stores the tuples in memory until they are processed and expired from the 
    +window. This limits the use cases to windows that
    +fit entirely in memory. Also the source tuples cannot be ack-ed until the window expiry requiring large message timeouts
    +(topology.message.timeout.secs should be larger than the window length + sliding interval). This also puts extra loads 
    +due to the complex acking and anchoring requirements.
    + 
    +To address the above limitations and to support larger window sizes, storm provides stateful windowing support via `IStatefulWindowedBolt`. 
    +User bolts should typically extend `BaseStatefulWindowedBolt` for the windowing operations with the framework automatically 
    +managing the state of the window in the background.
    +
    +If the sources provide a monotonically increasing identifier as a part of the message, the framework can use this
    +to periodically checkpoint the last expired and evaluated message ids, to avoid duplicate window evaluations in case of 
    +failures or restarts. During recovery, the tuples with message ids lower than last expired id are discarded and tuples with 
    +message id between the last expired and last evaluated message ids are fed into the system without activating any triggers. 
    +The tuples beyond the last evaluated message ids are processed as usual. This can be enabled by setting
    +the `messageIdField` as shown below,
    +
    +```java
    +topologyBuilder.setBolt("mybolt",
    +                   new MyStatefulWindowedBolt()
    +                   .withWindow(...) // windowing configuarations
    +                   .withMessageIdField("msgid"), // a monotonically increasing 'long' field in the tuple
    +                   parallelism)
    +               .shuffleGrouping("spout");
    +```
    +
    +However, this option is feasible only if the sources can provide a monotonically increasing identifier in the tuple and the same is maintained
    +while re-emitting the messages in case of failures. Here the tuples in window are still held in memory.
    --- End diff --
    
    What is the last sentence here saying?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #2218: STORM-2614: Enhance stateful windowing to persist the win...

Posted by satishd <gi...@git.apache.org>.
Github user satishd commented on the issue:

    https://github.com/apache/storm/pull/2218
  
    @HeartSaVioR I will go through this PR in next 1-2 days and put my comments. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130226994
  
    --- Diff: storm-client/src/jvm/org/apache/storm/windowing/WindowManager.java ---
    @@ -53,28 +61,34 @@
         public static final int EXPIRE_EVENTS_THRESHOLD = 100;
     
         private final WindowLifecycleListener<T> windowLifecycleListener;
    -    private final ConcurrentLinkedQueue<Event<T>> queue;
    +    private final Collection<Event<T>> queue;
         private final List<T> expiredEvents;
         private final Set<Event<T>> prevWindowEvents;
         private final AtomicInteger eventsSinceLastExpiry;
         private final ReentrantLock lock;
    -    private EvictionPolicy<T> evictionPolicy;
    -    private TriggerPolicy<T> triggerPolicy;
    -
    +    private final boolean stateful;
    +    private EvictionPolicy<T, ?> evictionPolicy;
    +    private TriggerPolicy<T, ?> triggerPolicy;
         public WindowManager(WindowLifecycleListener<T> lifecycleListener) {
    +        this(lifecycleListener, new ConcurrentLinkedQueue<>(), false);
    +    }
    +
    +    public WindowManager(WindowLifecycleListener<T> lifecycleListener, Collection<Event<T>> queue, boolean stateful) {
    --- End diff --
    
    Could you add a note here and in the API in WindowedBoltExecutor that this Collection has to be thread safe?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130273736
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutor.java ---
    @@ -0,0 +1,596 @@
    +/**
    + * 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 java.util.AbstractCollection;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Objects;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentLinkedQueue;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.locks.ReentrantLock;
    +import java.util.function.Supplier;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.state.KeyValueState;
    +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.topology.base.BaseWindowedBolt;
    +import org.apache.storm.tuple.Tuple;
    +import org.apache.storm.windowing.DefaultEvictionContext;
    +import org.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.EventImpl;
    +import org.apache.storm.windowing.WindowLifecycleListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static java.util.Collections.emptyIterator;
    +import static org.apache.storm.topology.WindowPartitionCache.CacheLoader;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalCause;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalListener;
    +
    +/**
    + * Wraps a {@link IStatefulWindowedBolt} and handles the execution. Uses state and the underlying
    + * checkpointing mechanisms to save the tuples in window to state. The tuples are also kept in-memory
    + * by transparently caching the window partitions and checkpointing them as needed.
    + */
    +public class PersistentWindowedBoltExecutor<T extends State> extends WindowedBoltExecutor implements IStatefulBolt<T> {
    +    private static final Logger LOG = LoggerFactory.getLogger(PersistentWindowedBoltExecutor.class);
    +    private final IStatefulWindowedBolt<T> statefulWindowedBolt;
    +    private transient TopologyContext topologyContext;
    +    private transient OutputCollector outputCollector;
    +    private transient WindowState<Tuple> state;
    +    private transient boolean stateInitialized;
    +    private transient boolean prePrepared;
    +
    +    public PersistentWindowedBoltExecutor(IStatefulWindowedBolt<T> bolt) {
    +        super(bolt);
    +        statefulWindowedBolt = bolt;
    +    }
    +
    +    @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));
    +    }
    +
    +    @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);
    +        }
    +    }
    +
    +    private int getCheckpointIntervalMillis(Map<String, Object> topoConf) {
    +        int checkpointInterval = Integer.MAX_VALUE;
    +        if (topoConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL) != null) {
    +            checkpointInterval = ((Number) topoConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL)).intValue();
    +        }
    +        return checkpointInterval;
    +    }
    +
    +    // package access for unit tests
    +    void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                 KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                 KeyValueState<String, Deque<Long>> partitionState,
    +                 KeyValueState<String, Optional<?>> windowSystemState) {
    +        init(topoConf, context, collector, windowState, partitionState, windowSystemState);
    +        doPrepare(topoConf, context, new NoAckOutputCollector(collector), state, true);
    +        Map<String, Optional<?>> wstate = new HashMap<>();
    +        windowSystemState.forEach(s -> wstate.put(s.getKey(), s.getValue()));
    +        restoreState(wstate);
    +    }
    +
    +    @Override
    +    protected void start() {
    +        if (stateInitialized) {
    +            super.start();
    +        } else {
    +            LOG.debug("Will invoke start after state is initialized");
    +        }
    +    }
    +
    +    @Override
    +    public void execute(Tuple input) {
    +        if (!stateInitialized) {
    +            throw new IllegalStateException("execute invoked before initState with input tuple " + input);
    +        }
    +        super.execute(input);
    +        // StatefulBoltExecutor does the actual ack when the state is saved.
    +        outputCollector.ack(input);
    +    }
    +
    +    @Override
    +    public void initState(T state) {
    +        if (stateInitialized) {
    +            String msg = "initState invoked when the state is already initialized";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        } else {
    +            statefulWindowedBolt.initState(state);
    +            stateInitialized = true;
    +            start();
    +        }
    +    }
    +
    +    @Override
    +    public void prePrepare(long txid) {
    +        if (stateInitialized) {
    +            LOG.debug("Prepare streamState, txid {}", txid);
    +            state.prepareCommit(txid);
    +            prePrepared = true;
    +        } else {
    +            String msg = "Cannot prepare before initState";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        }
    +    }
    +
    +    @Override
    +    public void preCommit(long txid) {
    +        // preCommit can be invoked during recovery before the state is initialized
    +        if (prePrepared || !stateInitialized) {
    +            LOG.debug("Commit streamState, txid {}", txid);
    +            state.commit(txid);
    +        } else {
    +            String msg = "preCommit before prePrepare in initialized state";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        }
    +    }
    +
    +    @Override
    +    public void preRollback() {
    +        LOG.debug("Rollback streamState, stateInitialized {}", stateInitialized);
    +        state.rollback();
    +    }
    +
    +    @Override
    +    protected WindowLifecycleListener<Tuple> newWindowLifecycleListener() {
    +        return new WindowLifecycleListener<Tuple>() {
    +            @Override
    +            public void onExpiry(List<Tuple> events) {
    +                /*
    +                 * NO-OP: the events are ack-ed in execute
    +                 */
    +            }
    +
    +            @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();
    +            }
    +        };
    +    }
    +
    +    private void init(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                      KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                      KeyValueState<String, Deque<Long>> partitionState,
    +                      KeyValueState<String, Optional<?>> windowSystemState) {
    +        topologyContext = context;
    +        outputCollector = collector;
    +        state = new WindowState<>(windowState, partitionState, windowSystemState, this::getState,
    +            statefulWindowedBolt.maxEventsInMemory());
    +    }
    +
    +    private KeyValueState<Long, WindowPartition<Tuple>> getWindowState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window";
    +        return (KeyValueState<Long, WindowPartition<Tuple>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Deque<Long>> getPartitionState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-partitions";
    +        return (KeyValueState<String, Deque<Long>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Optional<?>> getWindowSystemState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-systemstate";
    +        return (KeyValueState<String, Optional<?>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    // a wrapper around the window related states that are checkpointed
    +    private static class WindowState<T> extends AbstractCollection<Event<T>> {
    +        // number of events per window-partition
    +        private static final int MAX_PARTITION_EVENTS = 1000;
    +        private static final int MIN_PARTITIONS = 10;
    +        private static final String PARTITION_IDS_KEY = "pk";
    +        private final KeyValueState<String, Deque<Long>> partitionIdsState;
    +        private final KeyValueState<Long, WindowPartition<T>> windowPartitionsState;
    +        private final KeyValueState<String, Optional<?>> windowSystemState;
    +        // ordered partition keys
    +        private Deque<Long> partitionIds;
    +        private volatile long latestPartitionId;
    +        private WindowPartitionCache<Long, WindowPartition<T>> cache;
    +        private Supplier<Map<String, Optional<?>>> windowSystemStateSupplier;
    +        private final ReentrantLock partitionIdsLock = new ReentrantLock(true);
    +        private final ReentrantLock windowPartitionsLock = new ReentrantLock(true);
    +        private final long maxEventsInMemory;
    +        private WindowPartition<T> latestPartition;
    +        private Set<Long> iteratorPins = new HashSet<>();
    +
    +        WindowState(KeyValueState<Long, WindowPartition<T>> windowPartitionsState,
    +                    KeyValueState<String, Deque<Long>> partitionIdsState,
    +                    KeyValueState<String, Optional<?>> windowSystemState,
    +                    Supplier<Map<String, Optional<?>>> windowSystemStateSupplier,
    +                    long maxEventsInMemory) {
    +            this.windowPartitionsState = windowPartitionsState;
    +            this.partitionIdsState = partitionIdsState;
    +            this.windowSystemState = windowSystemState;
    +            this.windowSystemStateSupplier = windowSystemStateSupplier;
    +            this.maxEventsInMemory = Math.max(MAX_PARTITION_EVENTS * MIN_PARTITIONS, maxEventsInMemory);
    +            initCache();
    +            initPartitions();
    +        }
    +
    +        @Override
    +        public boolean add(Event<T> event) {
    +            if (latestPartition.size() >= MAX_PARTITION_EVENTS) {
    +                cache.unpin(latestPartition.getId());
    +                latestPartition = getPinnedPartition(getNextPartitionId());
    +            }
    +            latestPartition.add(event);
    +            return true;
    +        }
    +
    +        @Override
    +        public Iterator<Event<T>> iterator() {
    +
    +            return new Iterator<Event<T>>() {
    +                private Iterator<Long> ids = getIds();
    +                private Iterator<Event<T>> current = emptyIterator();
    +                private Iterator<Event<T>> removeFrom;
    +                private WindowPartition<T> curPartition;
    +
    +                private Iterator<Long> getIds() {
    +                    try {
    +                        partitionIdsLock.lock();
    +                        LOG.debug("Iterator partitionIds: {}", partitionIds);
    +                        return new ArrayList<>(partitionIds).iterator();
    +                    } finally {
    +                        partitionIdsLock.unlock();
    +                    }
    +                }
    +
    +                @Override
    +                public void remove() {
    +                    if (removeFrom == null) {
    +                        throw new IllegalStateException("No calls to next() since last call to remove()");
    +                    }
    +                    removeFrom.remove();
    +                    removeFrom = null;
    +                }
    +
    +                @Override
    +                public boolean hasNext() {
    +                    boolean curHasNext = current.hasNext();
    +                    while (!curHasNext && ids.hasNext()) {
    +                        if (curPartition != null) {
    +                            unpin(curPartition.getId());
    +                        }
    +                        curPartition = getPinnedPartition(ids.next());
    +                        if (curPartition != null) {
    +                            iteratorPins.add(curPartition.getId());
    +                            current = curPartition.iterator();
    +                            curHasNext = current.hasNext();
    +                        }
    +                    }
    +                    // un-pin the last partition
    +                    if (!curHasNext && curPartition != null) {
    +                        unpin(curPartition.getId());
    +                        curPartition = null;
    +                    }
    +                    return curHasNext;
    +                }
    +
    +                @Override
    +                public Event<T> next() {
    +                    if (!hasNext()) {
    +                        throw new NoSuchElementException();
    +                    }
    +                    removeFrom = current;
    +                    return current.next();
    +                }
    +
    +                private void unpin(long id) {
    +                    cache.unpin(id);
    +                    iteratorPins.remove(id);
    +                }
    +            };
    +        }
    +
    +        void clearIteratorPins() {
    +            LOG.debug("clearIteratorPins '{}'", iteratorPins);
    +            Iterator<Long> it = iteratorPins.iterator();
    +            while (it.hasNext()) {
    +                cache.unpin(it.next());
    +                it.remove();
    +            }
    +        }
    +
    +        @Override
    +        public int size() {
    +            throw new UnsupportedOperationException();
    +        }
    +
    +        void prepareCommit(long txid) {
    +            flush();
    +            partitionIdsState.prepareCommit(txid);
    +            windowPartitionsState.prepareCommit(txid);
    +            windowSystemState.prepareCommit(txid);
    +        }
    +
    +        void commit(long txid) {
    +            partitionIdsState.commit(txid);
    +            windowPartitionsState.commit(txid);
    +            windowSystemState.commit(txid);
    +        }
    +
    +        void rollback() {
    +            partitionIdsState.rollback();
    +            windowPartitionsState.rollback();
    +            windowSystemState.rollback();
    +        }
    +
    +        private void initPartitions() {
    +            partitionIds = partitionIdsState.get(PARTITION_IDS_KEY, new LinkedList<>());
    +            if (partitionIds.isEmpty()) {
    +                partitionIds.add(0L);
    +                partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    +            }
    +            latestPartitionId = partitionIds.peekLast();
    +            latestPartition = cache.getPinned(latestPartitionId);
    +        }
    +
    +        private void initCache() {
    +            long size = maxEventsInMemory / MAX_PARTITION_EVENTS;
    +            LOG.info("maxEventsInMemory: {}, partition size: {}, number of partitions: {}",
    +                maxEventsInMemory, MAX_PARTITION_EVENTS, size);
    +            cache = SimpleWindowPartitionCache.<Long, WindowPartition<T>>newBuilder()
    +                .maximumSize(size)
    +                .removalListener(new RemovalListener<Long, WindowPartition<T>>() {
    +                    @Override
    +                    public void onRemoval(Long pid, WindowPartition<T> p, RemovalCause removalCause) {
    +                        Objects.requireNonNull(pid, "Null partition id");
    +                        Objects.requireNonNull(p, "Null window partition");
    +                        LOG.debug("onRemoval for id '{}', WindowPartition '{}'", pid, p);
    +                        try {
    +                            windowPartitionsLock.lock();
    +                            if (p.isEmpty() && pid != latestPartitionId) {
    +                                // if the empty partition was not invalidated by flush, but evicted from cache
    +                                if (removalCause != RemovalCause.EXPLICIT) {
    +                                    deletePartition(pid);
    +                                    windowPartitionsState.delete(pid);
    +                                }
    +                            } else if (p.isModified()) {
    +                                windowPartitionsState.put(pid, p);
    +                            } else {
    +                                LOG.debug("WindowPartition '{}' is not modified", pid);
    +                            }
    +                        } finally {
    +                            windowPartitionsLock.unlock();
    +                        }
    +                    }
    +                }).build(new CacheLoader<Long, WindowPartition<T>>() {
    +                    @Override
    +                   public WindowPartition<T> load(Long id) {
    +                        LOG.debug("Load partition: {}", id);
    +                        // load from state
    +                        try {
    +                            windowPartitionsLock.lock();
    +                            return windowPartitionsState.get(id, new WindowPartition<>(id));
    +                        } finally {
    +                            windowPartitionsLock.unlock();
    +                        }
    +                    }
    +                    }
    +                );
    +        }
    +
    +        private void deletePartition(long pid) {
    +            LOG.debug("Delete partition: {}", pid);
    +            try {
    +                partitionIdsLock.lock();
    +                partitionIds.remove(pid);
    +                partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    +            } finally {
    +                partitionIdsLock.unlock();
    +            }
    +        }
    +
    +        private long getNextPartitionId() {
    +            try {
    +                partitionIdsLock.lock();
    +                partitionIds.add(++latestPartitionId);
    +                partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    +            } finally {
    +                partitionIdsLock.unlock();
    +            }
    +            return latestPartitionId;
    +        }
    +
    +        private WindowPartition<T> getPinnedPartition(long id) {
    +            return cache.getPinned(id);
    +        }
    +
    +        private void flush() {
    +            LOG.debug("Flushing modified partitions");
    +            cache.asMap().forEach((pid, p) -> {
    +                Long pidToInvalidate = null;
    +                try {
    +                    windowPartitionsLock.lock();
    +                    if (p.isEmpty() && pid != latestPartitionId) {
    +                        LOG.debug("Invalidating empty partition {}", pid);
    +                        deletePartition(pid);
    +                        windowPartitionsState.delete(pid);
    +                        pidToInvalidate = pid;
    +                    } else if (p.isModified()) {
    +                        LOG.debug("Updating modified partition {}", pid);
    +                        p.clearModified();
    +                        windowPartitionsState.put(pid, p);
    +                    }
    +                } finally {
    +                    windowPartitionsLock.unlock();
    +                }
    +                // invalidate after releasing the lock
    +                if (pidToInvalidate != null) {
    +                    cache.invalidate(pidToInvalidate);
    +                }
    +            });
    +            windowSystemStateSupplier.get().forEach(windowSystemState::put);
    +        }
    +    }
    +
    +    // the window partition that holds the events
    +    public static class WindowPartition<T> implements Iterable<Event<T>> {
    +        private final ConcurrentLinkedQueue<Event<T>> events = new ConcurrentLinkedQueue<>();
    +        private final AtomicInteger size = new AtomicInteger();
    --- End diff --
    
    because the `ConcurrentLinkedQueue.size` is not a constant time operation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r128156078
  
    --- Diff: docs/Windowing.md ---
    @@ -266,3 +266,105 @@ tuples can be received within the timeout period.
     An example toplogy `SlidingWindowTopology` shows how to use the apis to compute a sliding window sum and a tumbling window 
     average.
     
    +## Stateful windowing
    +The default windowing implementation in storm stores the tuples in memory until they are processed and expired from the 
    +window. This limits the use cases to windows that
    +fit entirely in memory. Also the source tuples cannot be ack-ed until the window expiry requiring large message timeouts
    +(topology.message.timeout.secs should be larger than the window length + sliding interval). This also puts extra loads 
    +due to the complex acking and anchoring requirements.
    + 
    +To address the above limitations and to support larger window sizes, storm provides stateful windowing support via `IStatefulWindowedBolt`. 
    +User bolts should typically extend `BaseStatefulWindowedBolt` for the windowing operations with the framework automatically 
    +managing the state of the window in the background.
    +
    +If the sources provide a monotonically increasing identifier as a part of the message, the framework can use this
    +to periodically checkpoint the last expired and evaluated message ids, to avoid duplicate window evaluations in case of 
    +failures or restarts. During recovery, the tuples with message ids lower than last expired id are discarded and tuples with 
    +message id between the last expired and last evaluated message ids are fed into the system without activating any triggers. 
    +The tuples beyond the last evaluated message ids are processed as usual. This can be enabled by setting
    +the `messageIdField` as shown below,
    +
    +```java
    +topologyBuilder.setBolt("mybolt",
    +                   new MyStatefulWindowedBolt()
    +                   .withWindow(...) // windowing configuarations
    +                   .withMessageIdField("msgid"), // a monotonically increasing 'long' field in the tuple
    +                   parallelism)
    +               .shuffleGrouping("spout");
    +```
    +
    +However, this option is feasible only if the sources can provide a monotonically increasing identifier in the tuple and the same is maintained
    +while re-emitting the messages in case of failures. Here the tuples in window are still held in memory.
    +
    +For more details take a look at the sample topology in storm starter `StatefulWindowingTopology` which will help you get started.
    +
    +### Window checkpointing
    +
    +With window checkpointing, the monotonically increasing id is no-longer required since the framework transparently saves the state of the window periodically into the configured state backend.
    +The state that is saved includes the tuples in the window, any system state that is required to recover the state of processing
    +and also the user state.
    +
    +```java
    +topologyBuilder.setBolt("mybolt",
    +                   new MyStatefulPersistentWindowedBolt()
    +                   .withWindow(...) // windowing configuarations
    +                   .withPersistence() // persist the window state
    +                   .withMaxEventsInMemory(25000), // max number of events to be cached in memory
    +                    parallelism)
    +               .shuffleGrouping("spout");
    +
    +```
    +
    +The `withPersistence` instructs the framework to transparently save the tuples in window along with
    +any associated system and user state to the state backend. The `withMaxEventsInMemory` is an optional 
    +configuration that specifies the maximum number of tuples that may be kept in memory. The tuples are transparently loaded from 
    +the state backend as required and the most recently used tuples are retained in memory (backed by a LRU cache).
    --- End diff --
    
    By 'used' I meant accessed. This could be the latest partition where the events are being added or the partitions that were accessed recently during window activation. Yes, its true that the older partitions needs to be accessed before the recent ones while iterating the window. Right now we invalidate any empty partitions during checkpointing. I will see how we can further optimize to keep the older partitions (queue head) cached.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by satishd <gi...@git.apache.org>.
Github user satishd commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r128992641
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/IStatefulWindowedBolt.java ---
    @@ -15,12 +15,33 @@
      * 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 windowed bolt abstraction for supporting windowing operation with state
    + * A windowed bolt abstraction for supporting windowing operation with state.
      */
     public interface IStatefulWindowedBolt<T extends State> extends IStatefulComponent<T>, IWindowedBolt {
    +    /**
    +     * If the stateful windowed bolt should have its windows persisted in state and maintain a subset
    +     * (recent events) in memory.
    --- End diff --
    
    this maybe frequent events instead of recent events. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130227512
  
    --- Diff: storm-client/test/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutorTest.java ---
    @@ -0,0 +1,261 @@
    +/**
    + * 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.Config;
    +import org.apache.storm.generated.GlobalStreamId;
    +import org.apache.storm.state.KeyValueState;
    +import org.apache.storm.streams.Pair;
    +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.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.TimestampExtractor;
    +import org.apache.storm.windowing.TupleWindow;
    +import org.apache.storm.windowing.WaterMarkEvent;
    +import org.apache.storm.windowing.WaterMarkEventGenerator;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.mockito.ArgumentCaptor;
    +import org.mockito.Captor;
    +import org.mockito.Mockito;
    +import org.mockito.MockitoAnnotations;
    +import org.mockito.invocation.InvocationOnMock;
    +import org.mockito.runners.MockitoJUnitRunner;
    +import org.mockito.stubbing.Answer;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.stream.Collectors;
    +import java.util.stream.LongStream;
    +
    +import static org.mockito.AdditionalAnswers.returnsArgAt;
    +
    +/**
    + * Unit tests for {@link PersistentWindowedBoltExecutor}
    + */
    +@RunWith(MockitoJUnitRunner.class)
    +public class PersistentWindowedBoltExecutorTest {
    +    public static final String LATE_STREAM = "late_stream";
    +    private PersistentWindowedBoltExecutor<KeyValueState<String, String>> executor;
    +    private IStatefulWindowedBolt<KeyValueState<String, String>> mockBolt;
    +    private OutputCollector mockOutputCollector;
    +    private TopologyContext mockTopologyContext;
    +    private Map<String, Object> mockStormConf = new HashMap<>();
    +    private TimestampExtractor mockTimestampExtractor;
    +    private WaterMarkEventGenerator mockWaterMarkEventGenerator;
    +    private KeyValueState mockPartitionState;
    +    private KeyValueState mockWindowState;
    +    private KeyValueState mockSystemState;
    +
    +    private static final String PARTITION_KEY = "pk";
    +    private static final String EVICTION_STATE_KEY = "es";
    +    private static final String TRIGGER_STATE_KEY = "ts";
    +    private long tupleTs;
    +
    +    @Captor
    +    private ArgumentCaptor<Tuple> tupleCaptor;
    +    @Captor
    +    private ArgumentCaptor<Collection<Tuple>> anchorCaptor;
    +    @Captor
    +    private ArgumentCaptor<String> stringCaptor;
    +    @Captor
    +    private ArgumentCaptor<Long> longCaptor;
    +    @Captor
    +    private ArgumentCaptor<Values> valuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<TupleWindow> tupleWindowCaptor;
    +    @Captor
    +    private ArgumentCaptor<Deque<Long>> partitionValuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<PersistentWindowedBoltExecutor.WindowPartition<Event<Tuple>>> windowValuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<Object> systemValuesCaptor;
    +
    +    @Before
    +    public void setUp() throws Exception {
    +        MockitoAnnotations.initMocks(this);
    +        mockBolt = Mockito.mock(IStatefulWindowedBolt.class);
    +        mockWaterMarkEventGenerator = Mockito.mock(WaterMarkEventGenerator.class);
    +        mockTimestampExtractor = Mockito.mock(TimestampExtractor.class);
    +        tupleTs = System.currentTimeMillis();
    +        Mockito.when(mockTimestampExtractor.extractTimestamp(Mockito.any())).thenReturn(tupleTs);
    +        Mockito.when(mockBolt.getTimestampExtractor()).thenReturn(mockTimestampExtractor);
    +        Mockito.when(mockBolt.isPersistent()).thenReturn(true);
    +        mockPartitionState = Mockito.mock(KeyValueState.class);
    +        mockWindowState = Mockito.mock(KeyValueState.class);
    +        mockSystemState = Mockito.mock(KeyValueState.class);
    +        mockTopologyContext = Mockito.mock(TopologyContext.class);
    +        Mockito.when(mockTopologyContext.getThisStreams()).thenReturn(Collections.singleton(LATE_STREAM));
    +        mockOutputCollector = Mockito.mock(OutputCollector.class);
    +        executor = new PersistentWindowedBoltExecutor<>(mockBolt);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_WINDOW_LENGTH_COUNT, 5);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_SLIDING_INTERVAL_COUNT, 5);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_LATE_TUPLE_STREAM, LATE_STREAM);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_WATERMARK_EVENT_INTERVAL_MS, 100_000);
    +        mockStormConf.put(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS, 30);
    +        mockStormConf.put(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL, 1000);
    +        Mockito.when(mockPartitionState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(mockWindowState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(mockSystemState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        executor.prepare(mockStormConf, mockTopologyContext, mockOutputCollector,
    +            mockWindowState, mockPartitionState, mockSystemState);
    +    }
    +
    +    @Test
    +    public void testExecuteTuple() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), 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);
    +    }
    +
    +    @Test
    +    public void testExecuteLatetuple() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), 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
    +        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());
    +    }
    +
    +    @Test
    +    public void testActivation() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), Mockito.anyLong())).thenReturn(true);
    +        executor.initState(null);
    +        executor.waterMarkEventGenerator = mockWaterMarkEventGenerator;
    +
    +        List<Tuple> mockTuples = getMockTuples(5);
    +        mockTuples.forEach(t -> executor.execute(t));
    +        // all tuples acked
    +        Mockito.verify(mockOutputCollector, Mockito.times(5)).ack(tupleCaptor.capture());
    +        Assert.assertArrayEquals(mockTuples.toArray(), tupleCaptor.getAllValues().toArray());
    +
    +        Mockito.doAnswer(new Answer<Void>() {
    +            @Override
    +            public Void answer(InvocationOnMock invocation) throws Throwable {
    +                TupleWindow window = (TupleWindow) invocation.getArguments()[0];
    +                // iterate the tuples
    +                Assert.assertEquals(5, window.get().size());
    +                // iterating multiple times should produce same events
    +                Assert.assertEquals(5, window.get().size());
    +                Assert.assertEquals(5, window.get().size());
    +                return null;
    +            }
    +        }).when(mockBolt).execute(Mockito.any());
    +        // trigger the window
    +        long activationTs = tupleTs + 1000;
    +        executor.getWindowManager().add(new WaterMarkEvent<>(activationTs));
    +        executor.prePrepare(0);
    +
    +        // partition ids
    +        Mockito.verify(mockPartitionState, Mockito.times(1)).put(stringCaptor.capture(), partitionValuesCaptor.capture());
    +        Assert.assertEquals(PARTITION_KEY, stringCaptor.getAllValues().get(0));
    +        Assert.assertEquals(Collections.singletonList(0L), partitionValuesCaptor.getAllValues().get(0));
    +
    +        // window partitions
    +        Mockito.verify(mockWindowState, Mockito.times(1)).put(longCaptor.capture(), windowValuesCaptor.capture());
    +        Assert.assertEquals(0L, (long) longCaptor.getAllValues().get(0));
    --- End diff --
    
    Nit: Consider naming these values


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130225709
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutor.java ---
    @@ -0,0 +1,596 @@
    +/**
    + * 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 java.util.AbstractCollection;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Objects;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentLinkedQueue;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.locks.ReentrantLock;
    +import java.util.function.Supplier;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.state.KeyValueState;
    +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.topology.base.BaseWindowedBolt;
    +import org.apache.storm.tuple.Tuple;
    +import org.apache.storm.windowing.DefaultEvictionContext;
    +import org.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.EventImpl;
    +import org.apache.storm.windowing.WindowLifecycleListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static java.util.Collections.emptyIterator;
    +import static org.apache.storm.topology.WindowPartitionCache.CacheLoader;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalCause;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalListener;
    +
    +/**
    + * Wraps a {@link IStatefulWindowedBolt} and handles the execution. Uses state and the underlying
    + * checkpointing mechanisms to save the tuples in window to state. The tuples are also kept in-memory
    + * by transparently caching the window partitions and checkpointing them as needed.
    + */
    +public class PersistentWindowedBoltExecutor<T extends State> extends WindowedBoltExecutor implements IStatefulBolt<T> {
    +    private static final Logger LOG = LoggerFactory.getLogger(PersistentWindowedBoltExecutor.class);
    +    private final IStatefulWindowedBolt<T> statefulWindowedBolt;
    +    private transient TopologyContext topologyContext;
    +    private transient OutputCollector outputCollector;
    +    private transient WindowState<Tuple> state;
    +    private transient boolean stateInitialized;
    +    private transient boolean prePrepared;
    +
    +    public PersistentWindowedBoltExecutor(IStatefulWindowedBolt<T> bolt) {
    +        super(bolt);
    +        statefulWindowedBolt = bolt;
    +    }
    +
    +    @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));
    +    }
    +
    +    @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);
    +        }
    +    }
    +
    +    private int getCheckpointIntervalMillis(Map<String, Object> topoConf) {
    +        int checkpointInterval = Integer.MAX_VALUE;
    +        if (topoConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL) != null) {
    +            checkpointInterval = ((Number) topoConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL)).intValue();
    +        }
    +        return checkpointInterval;
    +    }
    +
    +    // package access for unit tests
    +    void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                 KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                 KeyValueState<String, Deque<Long>> partitionState,
    +                 KeyValueState<String, Optional<?>> windowSystemState) {
    +        init(topoConf, context, collector, windowState, partitionState, windowSystemState);
    +        doPrepare(topoConf, context, new NoAckOutputCollector(collector), state, true);
    +        Map<String, Optional<?>> wstate = new HashMap<>();
    +        windowSystemState.forEach(s -> wstate.put(s.getKey(), s.getValue()));
    +        restoreState(wstate);
    +    }
    +
    +    @Override
    +    protected void start() {
    +        if (stateInitialized) {
    +            super.start();
    +        } else {
    +            LOG.debug("Will invoke start after state is initialized");
    +        }
    +    }
    +
    +    @Override
    +    public void execute(Tuple input) {
    +        if (!stateInitialized) {
    +            throw new IllegalStateException("execute invoked before initState with input tuple " + input);
    +        }
    +        super.execute(input);
    +        // StatefulBoltExecutor does the actual ack when the state is saved.
    +        outputCollector.ack(input);
    +    }
    +
    +    @Override
    +    public void initState(T state) {
    +        if (stateInitialized) {
    +            String msg = "initState invoked when the state is already initialized";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        } else {
    +            statefulWindowedBolt.initState(state);
    +            stateInitialized = true;
    +            start();
    +        }
    +    }
    +
    +    @Override
    +    public void prePrepare(long txid) {
    +        if (stateInitialized) {
    +            LOG.debug("Prepare streamState, txid {}", txid);
    +            state.prepareCommit(txid);
    +            prePrepared = true;
    +        } else {
    +            String msg = "Cannot prepare before initState";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        }
    +    }
    +
    +    @Override
    +    public void preCommit(long txid) {
    +        // preCommit can be invoked during recovery before the state is initialized
    +        if (prePrepared || !stateInitialized) {
    +            LOG.debug("Commit streamState, txid {}", txid);
    +            state.commit(txid);
    +        } else {
    +            String msg = "preCommit before prePrepare in initialized state";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        }
    +    }
    +
    +    @Override
    +    public void preRollback() {
    +        LOG.debug("Rollback streamState, stateInitialized {}", stateInitialized);
    +        state.rollback();
    +    }
    +
    +    @Override
    +    protected WindowLifecycleListener<Tuple> newWindowLifecycleListener() {
    +        return new WindowLifecycleListener<Tuple>() {
    +            @Override
    +            public void onExpiry(List<Tuple> events) {
    +                /*
    +                 * NO-OP: the events are ack-ed in execute
    +                 */
    +            }
    +
    +            @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();
    +            }
    +        };
    +    }
    +
    +    private void init(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                      KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                      KeyValueState<String, Deque<Long>> partitionState,
    +                      KeyValueState<String, Optional<?>> windowSystemState) {
    +        topologyContext = context;
    +        outputCollector = collector;
    +        state = new WindowState<>(windowState, partitionState, windowSystemState, this::getState,
    +            statefulWindowedBolt.maxEventsInMemory());
    +    }
    +
    +    private KeyValueState<Long, WindowPartition<Tuple>> getWindowState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window";
    +        return (KeyValueState<Long, WindowPartition<Tuple>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Deque<Long>> getPartitionState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-partitions";
    +        return (KeyValueState<String, Deque<Long>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Optional<?>> getWindowSystemState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-systemstate";
    +        return (KeyValueState<String, Optional<?>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    // a wrapper around the window related states that are checkpointed
    +    private static class WindowState<T> extends AbstractCollection<Event<T>> {
    +        // number of events per window-partition
    +        private static final int MAX_PARTITION_EVENTS = 1000;
    +        private static final int MIN_PARTITIONS = 10;
    +        private static final String PARTITION_IDS_KEY = "pk";
    +        private final KeyValueState<String, Deque<Long>> partitionIdsState;
    +        private final KeyValueState<Long, WindowPartition<T>> windowPartitionsState;
    +        private final KeyValueState<String, Optional<?>> windowSystemState;
    +        // ordered partition keys
    +        private Deque<Long> partitionIds;
    +        private volatile long latestPartitionId;
    +        private WindowPartitionCache<Long, WindowPartition<T>> cache;
    +        private Supplier<Map<String, Optional<?>>> windowSystemStateSupplier;
    +        private final ReentrantLock partitionIdsLock = new ReentrantLock(true);
    +        private final ReentrantLock windowPartitionsLock = new ReentrantLock(true);
    +        private final long maxEventsInMemory;
    +        private WindowPartition<T> latestPartition;
    +        private Set<Long> iteratorPins = new HashSet<>();
    +
    +        WindowState(KeyValueState<Long, WindowPartition<T>> windowPartitionsState,
    +                    KeyValueState<String, Deque<Long>> partitionIdsState,
    +                    KeyValueState<String, Optional<?>> windowSystemState,
    +                    Supplier<Map<String, Optional<?>>> windowSystemStateSupplier,
    +                    long maxEventsInMemory) {
    +            this.windowPartitionsState = windowPartitionsState;
    +            this.partitionIdsState = partitionIdsState;
    +            this.windowSystemState = windowSystemState;
    +            this.windowSystemStateSupplier = windowSystemStateSupplier;
    +            this.maxEventsInMemory = Math.max(MAX_PARTITION_EVENTS * MIN_PARTITIONS, maxEventsInMemory);
    +            initCache();
    +            initPartitions();
    +        }
    +
    +        @Override
    +        public boolean add(Event<T> event) {
    +            if (latestPartition.size() >= MAX_PARTITION_EVENTS) {
    +                cache.unpin(latestPartition.getId());
    +                latestPartition = getPinnedPartition(getNextPartitionId());
    +            }
    +            latestPartition.add(event);
    +            return true;
    +        }
    +
    +        @Override
    +        public Iterator<Event<T>> iterator() {
    +
    +            return new Iterator<Event<T>>() {
    +                private Iterator<Long> ids = getIds();
    +                private Iterator<Event<T>> current = emptyIterator();
    +                private Iterator<Event<T>> removeFrom;
    +                private WindowPartition<T> curPartition;
    +
    +                private Iterator<Long> getIds() {
    +                    try {
    +                        partitionIdsLock.lock();
    +                        LOG.debug("Iterator partitionIds: {}", partitionIds);
    +                        return new ArrayList<>(partitionIds).iterator();
    +                    } finally {
    +                        partitionIdsLock.unlock();
    +                    }
    +                }
    +
    +                @Override
    +                public void remove() {
    +                    if (removeFrom == null) {
    +                        throw new IllegalStateException("No calls to next() since last call to remove()");
    +                    }
    +                    removeFrom.remove();
    +                    removeFrom = null;
    +                }
    +
    +                @Override
    +                public boolean hasNext() {
    +                    boolean curHasNext = current.hasNext();
    +                    while (!curHasNext && ids.hasNext()) {
    +                        if (curPartition != null) {
    +                            unpin(curPartition.getId());
    +                        }
    +                        curPartition = getPinnedPartition(ids.next());
    +                        if (curPartition != null) {
    +                            iteratorPins.add(curPartition.getId());
    +                            current = curPartition.iterator();
    +                            curHasNext = current.hasNext();
    +                        }
    +                    }
    +                    // un-pin the last partition
    +                    if (!curHasNext && curPartition != null) {
    +                        unpin(curPartition.getId());
    +                        curPartition = null;
    +                    }
    +                    return curHasNext;
    +                }
    +
    +                @Override
    +                public Event<T> next() {
    +                    if (!hasNext()) {
    +                        throw new NoSuchElementException();
    +                    }
    +                    removeFrom = current;
    +                    return current.next();
    +                }
    +
    +                private void unpin(long id) {
    +                    cache.unpin(id);
    +                    iteratorPins.remove(id);
    +                }
    +            };
    +        }
    +
    +        void clearIteratorPins() {
    +            LOG.debug("clearIteratorPins '{}'", iteratorPins);
    +            Iterator<Long> it = iteratorPins.iterator();
    +            while (it.hasNext()) {
    +                cache.unpin(it.next());
    +                it.remove();
    +            }
    +        }
    +
    +        @Override
    +        public int size() {
    +            throw new UnsupportedOperationException();
    +        }
    +
    +        void prepareCommit(long txid) {
    +            flush();
    +            partitionIdsState.prepareCommit(txid);
    +            windowPartitionsState.prepareCommit(txid);
    +            windowSystemState.prepareCommit(txid);
    +        }
    +
    +        void commit(long txid) {
    +            partitionIdsState.commit(txid);
    +            windowPartitionsState.commit(txid);
    +            windowSystemState.commit(txid);
    +        }
    +
    +        void rollback() {
    +            partitionIdsState.rollback();
    +            windowPartitionsState.rollback();
    +            windowSystemState.rollback();
    +        }
    +
    +        private void initPartitions() {
    +            partitionIds = partitionIdsState.get(PARTITION_IDS_KEY, new LinkedList<>());
    +            if (partitionIds.isEmpty()) {
    +                partitionIds.add(0L);
    +                partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    +            }
    +            latestPartitionId = partitionIds.peekLast();
    +            latestPartition = cache.getPinned(latestPartitionId);
    +        }
    +
    +        private void initCache() {
    +            long size = maxEventsInMemory / MAX_PARTITION_EVENTS;
    +            LOG.info("maxEventsInMemory: {}, partition size: {}, number of partitions: {}",
    +                maxEventsInMemory, MAX_PARTITION_EVENTS, size);
    +            cache = SimpleWindowPartitionCache.<Long, WindowPartition<T>>newBuilder()
    +                .maximumSize(size)
    +                .removalListener(new RemovalListener<Long, WindowPartition<T>>() {
    +                    @Override
    +                    public void onRemoval(Long pid, WindowPartition<T> p, RemovalCause removalCause) {
    +                        Objects.requireNonNull(pid, "Null partition id");
    +                        Objects.requireNonNull(p, "Null window partition");
    +                        LOG.debug("onRemoval for id '{}', WindowPartition '{}'", pid, p);
    +                        try {
    +                            windowPartitionsLock.lock();
    +                            if (p.isEmpty() && pid != latestPartitionId) {
    +                                // if the empty partition was not invalidated by flush, but evicted from cache
    +                                if (removalCause != RemovalCause.EXPLICIT) {
    +                                    deletePartition(pid);
    +                                    windowPartitionsState.delete(pid);
    +                                }
    +                            } else if (p.isModified()) {
    +                                windowPartitionsState.put(pid, p);
    +                            } else {
    +                                LOG.debug("WindowPartition '{}' is not modified", pid);
    +                            }
    +                        } finally {
    +                            windowPartitionsLock.unlock();
    +                        }
    +                    }
    +                }).build(new CacheLoader<Long, WindowPartition<T>>() {
    +                    @Override
    +                   public WindowPartition<T> load(Long id) {
    +                        LOG.debug("Load partition: {}", id);
    +                        // load from state
    +                        try {
    +                            windowPartitionsLock.lock();
    +                            return windowPartitionsState.get(id, new WindowPartition<>(id));
    +                        } finally {
    +                            windowPartitionsLock.unlock();
    +                        }
    +                    }
    +                    }
    +                );
    +        }
    +
    +        private void deletePartition(long pid) {
    +            LOG.debug("Delete partition: {}", pid);
    +            try {
    +                partitionIdsLock.lock();
    +                partitionIds.remove(pid);
    +                partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    +            } finally {
    +                partitionIdsLock.unlock();
    +            }
    +        }
    +
    +        private long getNextPartitionId() {
    +            try {
    +                partitionIdsLock.lock();
    +                partitionIds.add(++latestPartitionId);
    +                partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    +            } finally {
    +                partitionIdsLock.unlock();
    +            }
    +            return latestPartitionId;
    +        }
    +
    +        private WindowPartition<T> getPinnedPartition(long id) {
    +            return cache.getPinned(id);
    +        }
    +
    +        private void flush() {
    +            LOG.debug("Flushing modified partitions");
    +            cache.asMap().forEach((pid, p) -> {
    +                Long pidToInvalidate = null;
    +                try {
    +                    windowPartitionsLock.lock();
    +                    if (p.isEmpty() && pid != latestPartitionId) {
    +                        LOG.debug("Invalidating empty partition {}", pid);
    +                        deletePartition(pid);
    +                        windowPartitionsState.delete(pid);
    +                        pidToInvalidate = pid;
    +                    } else if (p.isModified()) {
    +                        LOG.debug("Updating modified partition {}", pid);
    +                        p.clearModified();
    +                        windowPartitionsState.put(pid, p);
    +                    }
    +                } finally {
    +                    windowPartitionsLock.unlock();
    +                }
    +                // invalidate after releasing the lock
    +                if (pidToInvalidate != null) {
    +                    cache.invalidate(pidToInvalidate);
    +                }
    +            });
    +            windowSystemStateSupplier.get().forEach(windowSystemState::put);
    +        }
    +    }
    +
    +    // the window partition that holds the events
    +    public static class WindowPartition<T> implements Iterable<Event<T>> {
    +        private final ConcurrentLinkedQueue<Event<T>> events = new ConcurrentLinkedQueue<>();
    +        private final AtomicInteger size = new AtomicInteger();
    +        private final long id;
    +        private transient volatile boolean modified;
    +
    +        public WindowPartition(long id) {
    +            this.id = id;
    +        }
    +
    +        void add(Event<T> event) {
    +            events.add(event);
    +            size.incrementAndGet();
    +            setModified();
    +        }
    +
    +        boolean isModified() {
    +            return modified;
    +        }
    +
    +        void setModified() {
    +            if (!modified) {
    +                modified = true;
    +            }
    +        }
    +
    +        void clearModified() {
    +            if (modified) {
    --- End diff --
    
    This doesn't seem necessary?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130273719
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutor.java ---
    @@ -0,0 +1,596 @@
    +/**
    + * 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 java.util.AbstractCollection;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Objects;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentLinkedQueue;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.locks.ReentrantLock;
    +import java.util.function.Supplier;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.state.KeyValueState;
    +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.topology.base.BaseWindowedBolt;
    +import org.apache.storm.tuple.Tuple;
    +import org.apache.storm.windowing.DefaultEvictionContext;
    +import org.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.EventImpl;
    +import org.apache.storm.windowing.WindowLifecycleListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static java.util.Collections.emptyIterator;
    +import static org.apache.storm.topology.WindowPartitionCache.CacheLoader;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalCause;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalListener;
    +
    +/**
    + * Wraps a {@link IStatefulWindowedBolt} and handles the execution. Uses state and the underlying
    + * checkpointing mechanisms to save the tuples in window to state. The tuples are also kept in-memory
    + * by transparently caching the window partitions and checkpointing them as needed.
    + */
    +public class PersistentWindowedBoltExecutor<T extends State> extends WindowedBoltExecutor implements IStatefulBolt<T> {
    +    private static final Logger LOG = LoggerFactory.getLogger(PersistentWindowedBoltExecutor.class);
    +    private final IStatefulWindowedBolt<T> statefulWindowedBolt;
    +    private transient TopologyContext topologyContext;
    +    private transient OutputCollector outputCollector;
    +    private transient WindowState<Tuple> state;
    +    private transient boolean stateInitialized;
    +    private transient boolean prePrepared;
    +
    +    public PersistentWindowedBoltExecutor(IStatefulWindowedBolt<T> bolt) {
    +        super(bolt);
    +        statefulWindowedBolt = bolt;
    +    }
    +
    +    @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));
    +    }
    +
    +    @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);
    +        }
    +    }
    +
    +    private int getCheckpointIntervalMillis(Map<String, Object> topoConf) {
    +        int checkpointInterval = Integer.MAX_VALUE;
    +        if (topoConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL) != null) {
    +            checkpointInterval = ((Number) topoConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL)).intValue();
    +        }
    +        return checkpointInterval;
    +    }
    +
    +    // package access for unit tests
    +    void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                 KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                 KeyValueState<String, Deque<Long>> partitionState,
    +                 KeyValueState<String, Optional<?>> windowSystemState) {
    +        init(topoConf, context, collector, windowState, partitionState, windowSystemState);
    +        doPrepare(topoConf, context, new NoAckOutputCollector(collector), state, true);
    +        Map<String, Optional<?>> wstate = new HashMap<>();
    +        windowSystemState.forEach(s -> wstate.put(s.getKey(), s.getValue()));
    +        restoreState(wstate);
    +    }
    +
    +    @Override
    +    protected void start() {
    +        if (stateInitialized) {
    +            super.start();
    +        } else {
    +            LOG.debug("Will invoke start after state is initialized");
    +        }
    +    }
    +
    +    @Override
    +    public void execute(Tuple input) {
    +        if (!stateInitialized) {
    +            throw new IllegalStateException("execute invoked before initState with input tuple " + input);
    +        }
    +        super.execute(input);
    +        // StatefulBoltExecutor does the actual ack when the state is saved.
    +        outputCollector.ack(input);
    +    }
    +
    +    @Override
    +    public void initState(T state) {
    +        if (stateInitialized) {
    +            String msg = "initState invoked when the state is already initialized";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        } else {
    +            statefulWindowedBolt.initState(state);
    +            stateInitialized = true;
    +            start();
    +        }
    +    }
    +
    +    @Override
    +    public void prePrepare(long txid) {
    +        if (stateInitialized) {
    +            LOG.debug("Prepare streamState, txid {}", txid);
    +            state.prepareCommit(txid);
    +            prePrepared = true;
    +        } else {
    +            String msg = "Cannot prepare before initState";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        }
    +    }
    +
    +    @Override
    +    public void preCommit(long txid) {
    +        // preCommit can be invoked during recovery before the state is initialized
    +        if (prePrepared || !stateInitialized) {
    +            LOG.debug("Commit streamState, txid {}", txid);
    +            state.commit(txid);
    +        } else {
    +            String msg = "preCommit before prePrepare in initialized state";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        }
    +    }
    +
    +    @Override
    +    public void preRollback() {
    +        LOG.debug("Rollback streamState, stateInitialized {}", stateInitialized);
    +        state.rollback();
    +    }
    +
    +    @Override
    +    protected WindowLifecycleListener<Tuple> newWindowLifecycleListener() {
    +        return new WindowLifecycleListener<Tuple>() {
    +            @Override
    +            public void onExpiry(List<Tuple> events) {
    +                /*
    +                 * NO-OP: the events are ack-ed in execute
    +                 */
    +            }
    +
    +            @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();
    +            }
    +        };
    +    }
    +
    +    private void init(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                      KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                      KeyValueState<String, Deque<Long>> partitionState,
    +                      KeyValueState<String, Optional<?>> windowSystemState) {
    +        topologyContext = context;
    +        outputCollector = collector;
    +        state = new WindowState<>(windowState, partitionState, windowSystemState, this::getState,
    +            statefulWindowedBolt.maxEventsInMemory());
    +    }
    +
    +    private KeyValueState<Long, WindowPartition<Tuple>> getWindowState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window";
    +        return (KeyValueState<Long, WindowPartition<Tuple>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Deque<Long>> getPartitionState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-partitions";
    +        return (KeyValueState<String, Deque<Long>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Optional<?>> getWindowSystemState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-systemstate";
    +        return (KeyValueState<String, Optional<?>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    // a wrapper around the window related states that are checkpointed
    +    private static class WindowState<T> extends AbstractCollection<Event<T>> {
    +        // number of events per window-partition
    +        private static final int MAX_PARTITION_EVENTS = 1000;
    +        private static final int MIN_PARTITIONS = 10;
    +        private static final String PARTITION_IDS_KEY = "pk";
    +        private final KeyValueState<String, Deque<Long>> partitionIdsState;
    +        private final KeyValueState<Long, WindowPartition<T>> windowPartitionsState;
    +        private final KeyValueState<String, Optional<?>> windowSystemState;
    +        // ordered partition keys
    +        private Deque<Long> partitionIds;
    +        private volatile long latestPartitionId;
    +        private WindowPartitionCache<Long, WindowPartition<T>> cache;
    +        private Supplier<Map<String, Optional<?>>> windowSystemStateSupplier;
    +        private final ReentrantLock partitionIdsLock = new ReentrantLock(true);
    +        private final ReentrantLock windowPartitionsLock = new ReentrantLock(true);
    +        private final long maxEventsInMemory;
    +        private WindowPartition<T> latestPartition;
    +        private Set<Long> iteratorPins = new HashSet<>();
    +
    +        WindowState(KeyValueState<Long, WindowPartition<T>> windowPartitionsState,
    +                    KeyValueState<String, Deque<Long>> partitionIdsState,
    +                    KeyValueState<String, Optional<?>> windowSystemState,
    +                    Supplier<Map<String, Optional<?>>> windowSystemStateSupplier,
    +                    long maxEventsInMemory) {
    +            this.windowPartitionsState = windowPartitionsState;
    +            this.partitionIdsState = partitionIdsState;
    +            this.windowSystemState = windowSystemState;
    +            this.windowSystemStateSupplier = windowSystemStateSupplier;
    +            this.maxEventsInMemory = Math.max(MAX_PARTITION_EVENTS * MIN_PARTITIONS, maxEventsInMemory);
    +            initCache();
    +            initPartitions();
    +        }
    +
    +        @Override
    +        public boolean add(Event<T> event) {
    +            if (latestPartition.size() >= MAX_PARTITION_EVENTS) {
    +                cache.unpin(latestPartition.getId());
    +                latestPartition = getPinnedPartition(getNextPartitionId());
    +            }
    +            latestPartition.add(event);
    +            return true;
    +        }
    +
    +        @Override
    +        public Iterator<Event<T>> iterator() {
    +
    +            return new Iterator<Event<T>>() {
    +                private Iterator<Long> ids = getIds();
    +                private Iterator<Event<T>> current = emptyIterator();
    +                private Iterator<Event<T>> removeFrom;
    +                private WindowPartition<T> curPartition;
    +
    +                private Iterator<Long> getIds() {
    +                    try {
    +                        partitionIdsLock.lock();
    +                        LOG.debug("Iterator partitionIds: {}", partitionIds);
    +                        return new ArrayList<>(partitionIds).iterator();
    +                    } finally {
    +                        partitionIdsLock.unlock();
    +                    }
    +                }
    +
    +                @Override
    +                public void remove() {
    +                    if (removeFrom == null) {
    +                        throw new IllegalStateException("No calls to next() since last call to remove()");
    +                    }
    +                    removeFrom.remove();
    +                    removeFrom = null;
    +                }
    +
    +                @Override
    +                public boolean hasNext() {
    +                    boolean curHasNext = current.hasNext();
    +                    while (!curHasNext && ids.hasNext()) {
    +                        if (curPartition != null) {
    +                            unpin(curPartition.getId());
    +                        }
    +                        curPartition = getPinnedPartition(ids.next());
    +                        if (curPartition != null) {
    +                            iteratorPins.add(curPartition.getId());
    +                            current = curPartition.iterator();
    +                            curHasNext = current.hasNext();
    +                        }
    +                    }
    +                    // un-pin the last partition
    +                    if (!curHasNext && curPartition != null) {
    +                        unpin(curPartition.getId());
    +                        curPartition = null;
    +                    }
    +                    return curHasNext;
    +                }
    +
    +                @Override
    +                public Event<T> next() {
    +                    if (!hasNext()) {
    +                        throw new NoSuchElementException();
    +                    }
    +                    removeFrom = current;
    +                    return current.next();
    +                }
    +
    +                private void unpin(long id) {
    +                    cache.unpin(id);
    +                    iteratorPins.remove(id);
    +                }
    +            };
    +        }
    +
    +        void clearIteratorPins() {
    +            LOG.debug("clearIteratorPins '{}'", iteratorPins);
    +            Iterator<Long> it = iteratorPins.iterator();
    +            while (it.hasNext()) {
    +                cache.unpin(it.next());
    +                it.remove();
    +            }
    +        }
    +
    +        @Override
    +        public int size() {
    +            throw new UnsupportedOperationException();
    +        }
    +
    +        void prepareCommit(long txid) {
    +            flush();
    +            partitionIdsState.prepareCommit(txid);
    +            windowPartitionsState.prepareCommit(txid);
    +            windowSystemState.prepareCommit(txid);
    +        }
    +
    +        void commit(long txid) {
    +            partitionIdsState.commit(txid);
    +            windowPartitionsState.commit(txid);
    +            windowSystemState.commit(txid);
    +        }
    +
    +        void rollback() {
    +            partitionIdsState.rollback();
    +            windowPartitionsState.rollback();
    +            windowSystemState.rollback();
    +        }
    +
    +        private void initPartitions() {
    +            partitionIds = partitionIdsState.get(PARTITION_IDS_KEY, new LinkedList<>());
    +            if (partitionIds.isEmpty()) {
    +                partitionIds.add(0L);
    +                partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    +            }
    +            latestPartitionId = partitionIds.peekLast();
    +            latestPartition = cache.getPinned(latestPartitionId);
    +        }
    +
    +        private void initCache() {
    +            long size = maxEventsInMemory / MAX_PARTITION_EVENTS;
    +            LOG.info("maxEventsInMemory: {}, partition size: {}, number of partitions: {}",
    +                maxEventsInMemory, MAX_PARTITION_EVENTS, size);
    +            cache = SimpleWindowPartitionCache.<Long, WindowPartition<T>>newBuilder()
    +                .maximumSize(size)
    +                .removalListener(new RemovalListener<Long, WindowPartition<T>>() {
    +                    @Override
    +                    public void onRemoval(Long pid, WindowPartition<T> p, RemovalCause removalCause) {
    +                        Objects.requireNonNull(pid, "Null partition id");
    +                        Objects.requireNonNull(p, "Null window partition");
    +                        LOG.debug("onRemoval for id '{}', WindowPartition '{}'", pid, p);
    +                        try {
    +                            windowPartitionsLock.lock();
    +                            if (p.isEmpty() && pid != latestPartitionId) {
    +                                // if the empty partition was not invalidated by flush, but evicted from cache
    +                                if (removalCause != RemovalCause.EXPLICIT) {
    +                                    deletePartition(pid);
    +                                    windowPartitionsState.delete(pid);
    +                                }
    +                            } else if (p.isModified()) {
    +                                windowPartitionsState.put(pid, p);
    +                            } else {
    +                                LOG.debug("WindowPartition '{}' is not modified", pid);
    +                            }
    +                        } finally {
    +                            windowPartitionsLock.unlock();
    +                        }
    +                    }
    +                }).build(new CacheLoader<Long, WindowPartition<T>>() {
    +                    @Override
    +                   public WindowPartition<T> load(Long id) {
    +                        LOG.debug("Load partition: {}", id);
    +                        // load from state
    +                        try {
    +                            windowPartitionsLock.lock();
    +                            return windowPartitionsState.get(id, new WindowPartition<>(id));
    +                        } finally {
    +                            windowPartitionsLock.unlock();
    +                        }
    +                    }
    +                    }
    +                );
    +        }
    +
    +        private void deletePartition(long pid) {
    +            LOG.debug("Delete partition: {}", pid);
    +            try {
    +                partitionIdsLock.lock();
    +                partitionIds.remove(pid);
    +                partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    --- End diff --
    
    this is to save the initial list of partition ids `[0]` to the state.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r128289791
  
    --- Diff: docs/Windowing.md ---
    @@ -266,3 +266,105 @@ tuples can be received within the timeout period.
     An example toplogy `SlidingWindowTopology` shows how to use the apis to compute a sliding window sum and a tumbling window 
     average.
     
    +## Stateful windowing
    +The default windowing implementation in storm stores the tuples in memory until they are processed and expired from the 
    +window. This limits the use cases to windows that
    +fit entirely in memory. Also the source tuples cannot be ack-ed until the window expiry requiring large message timeouts
    +(topology.message.timeout.secs should be larger than the window length + sliding interval). This also puts extra loads 
    +due to the complex acking and anchoring requirements.
    + 
    +To address the above limitations and to support larger window sizes, storm provides stateful windowing support via `IStatefulWindowedBolt`. 
    +User bolts should typically extend `BaseStatefulWindowedBolt` for the windowing operations with the framework automatically 
    +managing the state of the window in the background.
    +
    +If the sources provide a monotonically increasing identifier as a part of the message, the framework can use this
    +to periodically checkpoint the last expired and evaluated message ids, to avoid duplicate window evaluations in case of 
    +failures or restarts. During recovery, the tuples with message ids lower than last expired id are discarded and tuples with 
    +message id between the last expired and last evaluated message ids are fed into the system without activating any triggers. 
    +The tuples beyond the last evaluated message ids are processed as usual. This can be enabled by setting
    +the `messageIdField` as shown below,
    +
    +```java
    +topologyBuilder.setBolt("mybolt",
    +                   new MyStatefulWindowedBolt()
    +                   .withWindow(...) // windowing configuarations
    +                   .withMessageIdField("msgid"), // a monotonically increasing 'long' field in the tuple
    +                   parallelism)
    +               .shuffleGrouping("spout");
    +```
    +
    +However, this option is feasible only if the sources can provide a monotonically increasing identifier in the tuple and the same is maintained
    +while re-emitting the messages in case of failures. Here the tuples in window are still held in memory.
    +
    +For more details take a look at the sample topology in storm starter `StatefulWindowingTopology` which will help you get started.
    +
    +### Window checkpointing
    +
    +With window checkpointing, the monotonically increasing id is no-longer required since the framework transparently saves the state of the window periodically into the configured state backend.
    +The state that is saved includes the tuples in the window, any system state that is required to recover the state of processing
    +and also the user state.
    +
    +```java
    +topologyBuilder.setBolt("mybolt",
    +                   new MyStatefulPersistentWindowedBolt()
    +                   .withWindow(...) // windowing configuarations
    +                   .withPersistence() // persist the window state
    +                   .withMaxEventsInMemory(25000), // max number of events to be cached in memory
    +                    parallelism)
    +               .shuffleGrouping("spout");
    +
    +```
    +
    +The `withPersistence` instructs the framework to transparently save the tuples in window along with
    +any associated system and user state to the state backend. The `withMaxEventsInMemory` is an optional 
    +configuration that specifies the maximum number of tuples that may be kept in memory. The tuples are transparently loaded from 
    +the state backend as required and the most recently used tuples are retained in memory (backed by a LRU cache).
    --- End diff --
    
    @HeartSaVioR, replaced Guava with Caffeine, which has better cache hit rate with sequential access and is based on LFU. I ran some tests with sequential queue scans (one of the typical operations in windowing) and it gave much better results. Caffeine is also api compatible with guava and is already used elsewhere in storm.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #2218: STORM-2614: Enhance stateful windowing to persist the win...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on the issue:

    https://github.com/apache/storm/pull/2218
  
    @srdo, addressed your comments, let me know if I missed something.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130273679
  
    --- Diff: docs/Windowing.md ---
    @@ -266,3 +266,108 @@ tuples can be received within the timeout period.
     An example toplogy `SlidingWindowTopology` shows how to use the apis to compute a sliding window sum and a tumbling window 
     average.
     
    +## Stateful windowing
    +The default windowing implementation in storm stores the tuples in memory until they are processed and expired from the 
    +window. This limits the use cases to windows that
    +fit entirely in memory. Also the source tuples cannot be ack-ed until the window expiry requiring large message timeouts
    +(topology.message.timeout.secs should be larger than the window length + sliding interval). This also puts extra loads 
    +due to the complex acking and anchoring requirements.
    + 
    +To address the above limitations and to support larger window sizes, storm provides stateful windowing support via `IStatefulWindowedBolt`. 
    +User bolts should typically extend `BaseStatefulWindowedBolt` for the windowing operations with the framework automatically 
    +managing the state of the window in the background.
    +
    +If the sources provide a monotonically increasing identifier as a part of the message, the framework can use this
    +to periodically checkpoint the last expired and evaluated message ids, to avoid duplicate window evaluations in case of 
    +failures or restarts. During recovery, the tuples with message ids lower than last expired id are discarded and tuples with 
    +message id between the last expired and last evaluated message ids are fed into the system without activating any previously
    +activated windows.
    +The tuples beyond the last evaluated message ids are processed as usual. This can be enabled by setting
    +the `messageIdField` as shown below,
    +
    +```java
    +topologyBuilder.setBolt("mybolt",
    +                   new MyStatefulWindowedBolt()
    +                   .withWindow(...) // windowing configuarations
    +                   .withMessageIdField("msgid"), // a monotonically increasing 'long' field in the tuple
    +                   parallelism)
    +               .shuffleGrouping("spout");
    +```
    +
    +However, this option is feasible only if the sources can provide a monotonically increasing identifier in the tuple and the same is maintained
    +while re-emitting the messages in case of failures. With this option the tuples are still buffered in memory until processed
    +and expired from the window.
    +
    +For more details take a look at the sample topology in storm-starter [StatefulWindowingTopology](../examples/storm-starter/src/jvm/org/apache/storm/starter/StatefulWindowingTopology.java) which will help you get started.
    +
    +### Window checkpointing
    +
    +With window checkpointing, the monotonically increasing id is no longer required since the framework transparently saves the state of the window periodically into the configured state backend.
    +The state that is saved includes the tuples in the window, any system state that is required to recover the state of processing
    +and also the user state.
    +
    +```java
    +topologyBuilder.setBolt("mybolt",
    +                   new MyStatefulPersistentWindowedBolt()
    +                   .withWindow(...) // windowing configuarations
    +                   .withPersistence() // persist the window state
    +                   .withMaxEventsInMemory(25000), // max number of events to be cached in memory
    +                    parallelism)
    +               .shuffleGrouping("spout");
    +
    +```
    +
    +The `withPersistence` instructs the framework to transparently save the tuples in window along with
    +any associated system and user state to the state backend. The `withMaxEventsInMemory` is an optional 
    +configuration that specifies the maximum number of tuples that may be kept in memory. The tuples are transparently loaded from 
    +the state backend as required and the ones that are most likely to be used again are retained in memory.
    +
    +The state backend can be configured by setting the topology state provider config,
    +
    +```java
    +// use redis for state persistence
    +conf.put(Config.TOPOLOGY_STATE_PROVIDER, "org.apache.storm.redis.state.RedisKeyValueStateProvider");
    +
    +```
    +Currently storm supports Redis and HBase as state backends and uses the underlying state-checkpointing
    +framework for saving the window state. For more details on state checkpointing see [State-checkpointing.md](State-checkpointing.md)
    +
    +Here is an example of a persistent windowed bolt that uses the window checkpointing to save its state. The `initState`
    +is invoked with the last saved state (user state) at initialization time. The execute method is invoked based on the configured
    +windowing parameters and the tuples in the active window can be accessed via an `iterator` as shown below.
    +
    +```java
    +public class MyStatefulPersistentWindowedBolt extends BaseStatefulWindowedBolt<K, V> {
    +  private KeyValueState<K, V> state;
    +  
    +  @Override
    +  public void initState(KeyValueState<K, V> state) {
    +    this.state = state;
    +   // ...
    +   // restore the state from the last saved state.
    +   // ...
    +  }
    +  
    +  @Override
    +  public void execute(TupleWindow window) {      
    +    // iterate over tuples in the current window
    +    Iterator<Tuple> it = window.getIter();
    +    while (it.hasNext()) {
    +        // compute some result based on the tuples in window
    +    }
    +    
    +    // possibly update any state to be maintained across windows
    +    state.put(STATE_KEY, updatedValue);
    +    
    +    // emit the results downstream
    +    collector.emit(new Values(result));
    +  }
    +}
    +```
    +
    +Note: In case of persistent windowed bolts, use `TupleWindow.getIter` to retrieve an iterator over the 
    +events in the window. If the number of tuples in windows is huge, invoking `TupleWindow.get` would
    +try to load all the tuples into memory and may throw an OOM exception.
    --- End diff --
    
    will do.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130226013
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/SimpleWindowPartitionCache.java ---
    @@ -0,0 +1,191 @@
    +/**
    + * 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 java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.concurrent.ConcurrentMap;
    +import java.util.concurrent.ConcurrentSkipListMap;
    +import java.util.concurrent.locks.ReentrantLock;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * A simple implementation that evicts the largest un-pinned entry from the cache. This works well
    + * for caching window partitions since the access pattern is mostly sequential scans.
    + */
    +public class SimpleWindowPartitionCache<K, V> implements WindowPartitionCache<K, V> {
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleWindowPartitionCache.class);
    +
    +    private final ConcurrentSkipListMap<K, V> map = new ConcurrentSkipListMap<>();
    +    private final Map<K, Long> pinned = new HashMap<>();
    +    private final long maximumSize;
    +    private final RemovalListener<K, V> removalListener;
    +    private final CacheLoader<K, V> cacheLoader;
    +    private final ReentrantLock lock = new ReentrantLock(true);
    +    private int size;
    --- End diff --
    
    This seems to follow the size of map. Why not use map.size()?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r131123972
  
    --- Diff: storm-client/test/jvm/org/apache/storm/windowing/persistence/WindowStateTest.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.windowing.persistence;
    +
    +import org.apache.storm.state.KeyValueState;
    +import org.apache.storm.tuple.Tuple;
    +import org.apache.storm.windowing.Event;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.mockito.ArgumentCaptor;
    +import org.mockito.Captor;
    +import org.mockito.Mock;
    +import org.mockito.Mockito;
    +import org.mockito.MockitoAnnotations;
    +import org.mockito.invocation.InvocationOnMock;
    +import org.mockito.runners.MockitoJUnitRunner;
    +import org.mockito.stubbing.Answer;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.function.Consumer;
    +import java.util.function.Supplier;
    +
    +import static org.mockito.AdditionalAnswers.returnsArgAt;
    +
    +/**
    + * Unit tests for {@link WindowState}
    + */
    +@RunWith(MockitoJUnitRunner.class)
    +public class WindowStateTest {
    +
    +    @Mock
    +    private KeyValueState<Long, WindowState.WindowPartition<Integer>> windowState;
    +    @Mock
    +    private KeyValueState<String, Deque<Long>> partitionIdsState;
    +    @Mock
    +    private KeyValueState<String, Optional<?>> systemState;
    +    @Mock
    +    private Supplier<Map<String, Optional<?>>> supplier;
    +    @Captor
    +    private ArgumentCaptor<Long> longCaptor;
    +    @Captor
    +    private ArgumentCaptor<WindowState.WindowPartition<Integer>> windowValuesCaptor;
    +
    +    private static final int MAX_EVENTS_PER_PARTITION = 1000;
    --- End diff --
    
    Isn't this the same as the constant in WindowState? If so I think you should refer to that one instead of duplicating it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130273929
  
    --- Diff: storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java ---
    @@ -2363,7 +2363,7 @@ private CommonTopoInfo getCommonTopoInfo(String topoId, String operation) throws
                 ret.launchTimeSecs = 0;
             }
             ret.assignment = state.assignmentInfo(topoId, null);
    -        ret.beats = Utils.OR(heartbeatsCache.get().get(topoId), Collections.emptyMap());
    +        ret.beats = Utils.<Map<List<Integer>, Map<String, Object>>>OR(heartbeatsCache.get().get(topoId), Collections.emptyMap());
    --- End diff --
    
    I am getting the below error when I do a `mvn clean install` in the storm-server module without the type hints. Will check again.
    
    ```
    [ERROR] COMPILATION ERROR :
    [INFO] -------------------------------------------------------------
    [ERROR] /Users/aiyer/github/storm/storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java:[2366,29] incompatible types: inferred type does not conform to upper bound(s)
        inferred: java.util.Map<? extends java.lang.Object,? extends java.lang.Object>
        upper bound(s): java.util.Map<java.util.List<java.lang.Integer>,java.util.Map<java.lang.String,java.lang.Object>>,java.lang.Object
    
    $ java -version
    java version "1.8.0_51"
    Java(TM) SE Runtime Environment (build 1.8.0_51-b16)
    Java HotSpot(TM) 64-Bit Server VM (build 25.51-b03, mixed mode)
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130433567
  
    --- Diff: storm-client/src/jvm/org/apache/storm/windowing/WatermarkCountEvictionPolicy.java ---
    @@ -17,20 +17,28 @@
      */
     package org.apache.storm.windowing;
     
    +import org.apache.storm.streams.Pair;
    +
    +import java.util.concurrent.atomic.AtomicLong;
    +
     /**
      * An eviction policy that tracks count based on watermark ts and
      * evicts events up to the watermark based on a threshold count.
      *
      * @param <T> the type of event tracked by this policy.
      */
    -public class WatermarkCountEvictionPolicy<T> extends CountEvictionPolicy<T> {
    +public class WatermarkCountEvictionPolicy<T> implements EvictionPolicy<T, Pair<Long, Long>> {
    --- End diff --
    
    Okay


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130228532
  
    --- Diff: storm-client/src/jvm/org/apache/storm/windowing/WindowManager.java ---
    @@ -111,14 +125,86 @@ public void add(Event<T> windowEvent) {
                 LOG.debug("Got watermark event with ts {}", windowEvent.getTimestamp());
             }
             track(windowEvent);
    -        compactWindow();
    +        if (!stateful) {
    +            compactWindow();
    +        }
         }
     
         /**
          * The callback invoked by the trigger policy.
          */
         @Override
         public boolean onTrigger() {
    +        return stateful ? doOnTriggerStateful() : doOnTrigger();
    +    }
    +
    +    private static class IteratorStatus {
    +        private boolean valid = true;
    +
    +        void invalidate() {
    +            valid = false;
    +        }
    +
    +        boolean isValid() {
    +            return valid;
    +        }
    +    }
    +
    +    private static<T> Iterator<T> expiringIterator(Iterator<T> inner, IteratorStatus status) {
    +        return new Iterator<T>() {
    +            @Override
    +            public boolean hasNext() {
    +                if (status.isValid()) {
    +                    return inner.hasNext();
    +                }
    +                throw new IllegalStateException("Stale iterator");
    +            }
    +
    +            @Override
    +            public T next() {
    +                if (status.isValid()) {
    +                    return inner.next();
    +                }
    +                throw new IllegalStateException("Stale iterator");
    +            }
    +        };
    +    }
    +
    +    private boolean doOnTriggerStateful() {
    +        Supplier<Iterator<T>> scanEventsStateful = this::scanEventsStateful;
    +        Iterator<T> it = scanEventsStateful.get();
    --- End diff --
    
    Why isn't it necessary to lock while accessing this iterator?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130273816
  
    --- Diff: storm-client/src/jvm/org/apache/storm/windowing/WatermarkCountEvictionPolicy.java ---
    @@ -17,20 +17,28 @@
      */
     package org.apache.storm.windowing;
     
    +import org.apache.storm.streams.Pair;
    +
    +import java.util.concurrent.atomic.AtomicLong;
    +
     /**
      * An eviction policy that tracks count based on watermark ts and
      * evicts events up to the watermark based on a threshold count.
      *
      * @param <T> the type of event tracked by this policy.
      */
    -public class WatermarkCountEvictionPolicy<T> extends CountEvictionPolicy<T> {
    +public class WatermarkCountEvictionPolicy<T> implements EvictionPolicy<T, Pair<Long, Long>> {
    --- End diff --
    
    Theres some code duplication, but the logic is not the same and delegation might not be correct here. Also it doesn't look very straightforward without a lot of refactoring.  Would like to keep it this way for now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129113343
  
    --- Diff: examples/storm-starter/src/jvm/org/apache/storm/starter/PersistentWindowingTopology.java ---
    @@ -0,0 +1,176 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.storm.starter;
    +
    +import static org.apache.storm.topology.base.BaseWindowedBolt.Duration;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.concurrent.TimeUnit;
    +import org.apache.storm.Config;
    +import org.apache.storm.StormSubmitter;
    +import org.apache.storm.starter.spout.RandomIntegerSpout;
    +import org.apache.storm.state.KeyValueState;
    +import org.apache.storm.streams.Pair;
    +import org.apache.storm.task.OutputCollector;
    +import org.apache.storm.task.TopologyContext;
    +import org.apache.storm.topology.OutputFieldsDeclarer;
    +import org.apache.storm.topology.TopologyBuilder;
    +import org.apache.storm.topology.base.BaseStatefulWindowedBolt;
    +import org.apache.storm.tuple.Fields;
    +import org.apache.storm.tuple.Tuple;
    +import org.apache.storm.tuple.Values;
    +import org.apache.storm.windowing.TupleWindow;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * An example that demonstrates the usage of {@link org.apache.storm.topology.IStatefulWindowedBolt} with window
    + * persistence.
    + * <p>
    + * The framework automatically checkpoints the tuples in window along with the bolt's state and restores the same
    + * during restarts.
    + * </p>
    + *
    + * <p>
    + * This topology uses 'redis' for state persistence, so you should also start a redis instance before deploying.
    + * If you are running in local mode you can just start a redis server locally which will be used for storing the state. The default
    + * RedisKeyValueStateProvider parameters can be overridden by setting "topology.state.provider.config", for e.g.
    + * <pre>
    + * {
    + *   "jedisPoolConfig": {
    + *     "host": "redis-server-host",
    + *     "port": 6379,
    + *     "timeout": 2000,
    + *     "database": 0,
    + *     "password": "xyz"
    + *   }
    + * }
    + * </pre>
    + * </p>
    + */
    +public class PersistentWindowingTopology {
    +    private static final Logger LOG = LoggerFactory.getLogger(PersistentWindowingTopology.class);
    +
    +    // wrapper to hold global and window averages
    +    private static class Averages {
    +        private final double global;
    +        private final double window;
    +
    +        Averages(double global, double window) {
    +            this.global = global;
    +            this.window = window;
    +        }
    +
    +        @Override
    +        public String toString() {
    +            return "Averages{" + "global=" + String.format("%.2f", global) + ", window=" + String.format("%.2f", window) + '}';
    +        }
    +    }
    +
    +    /**
    +     * A bolt that uses stateful persistence to store the windows along with the state (global avg).
    +     */
    +    private static class AvgBolt extends BaseStatefulWindowedBolt<KeyValueState<String, Pair<Long, Long>>> {
    +        private static final String STATE_KEY = "avg";
    +
    +        private OutputCollector collector;
    +        private KeyValueState<String, Pair<Long, Long>> state;
    +        private Pair<Long, Long> globalAvg;
    +
    +        @Override
    +        public void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector) {
    +            this.collector = collector;
    +        }
    +
    +        @Override
    +        public void initState(KeyValueState<String, Pair<Long, Long>> state) {
    +            this.state = state;
    +            globalAvg = state.get(STATE_KEY, Pair.of(0L, 0L));
    +            LOG.info("initState with global avg [" + (double) globalAvg._1 / globalAvg._2 + "]");
    --- End diff --
    
    Nit: Consider using the getters instead of _1 and _2. It's a little weird that those fields are public when they also have getters IMO.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129643650
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutor.java ---
    @@ -0,0 +1,563 @@
    +/**
    + * 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 java.util.AbstractCollection;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentLinkedQueue;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.locks.ReentrantLock;
    +import java.util.function.Supplier;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.state.KeyValueState;
    +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.windowing.DefaultEvictionContext;
    +import org.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.EventImpl;
    +import org.apache.storm.windowing.WindowLifecycleListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static java.util.Collections.emptyIterator;
    +import static org.apache.storm.topology.WindowPartitionCache.CacheLoader;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalCause;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalListener;
    +
    +/**
    + * Wraps a {@link IStatefulWindowedBolt} and handles the execution. Uses state and the underlying
    + * checkpointing mechanisms to save the tuples in window to state. The tuples are also kept in-memory
    + * by transparently caching the window partitions and checkpointing them as needed.
    + */
    +public class PersistentWindowedBoltExecutor<T extends State> extends WindowedBoltExecutor implements IStatefulBolt<T> {
    +    private static final Logger LOG = LoggerFactory.getLogger(PersistentWindowedBoltExecutor.class);
    +    private final IStatefulWindowedBolt<T> statefulWindowedBolt;
    +    private transient TopologyContext topologyContext;
    +    private transient OutputCollector outputCollector;
    +    private transient WindowState<Tuple> state;
    +    private transient boolean stateInitialized;
    +    private transient boolean prePrepared;
    +
    +    public PersistentWindowedBoltExecutor(IStatefulWindowedBolt<T> bolt) {
    +        super(bolt);
    +        statefulWindowedBolt = bolt;
    +    }
    +
    +    @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));
    +    }
    +
    +    // package access for unit tests
    +    void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                 KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                 KeyValueState<String, Deque<Long>> partitionState,
    +                 KeyValueState<String, Optional<?>> windowSystemState) {
    +        init(topoConf, context, collector, windowState, partitionState, windowSystemState);
    +        doPrepare(topoConf, context, new NoAckOutputCollector(collector), state, true);
    +        Map<String, Optional<?>> wstate = new HashMap<>();
    +        windowSystemState.forEach(s -> wstate.put(s.getKey(), s.getValue()));
    +        restoreState(wstate);
    +    }
    +
    +    @Override
    +    protected void start() {
    +        if (stateInitialized) {
    +            super.start();
    +        } else {
    +            LOG.debug("Will invoke start after state is initialized");
    +        }
    +    }
    +
    +    @Override
    +    public void execute(Tuple input) {
    +        if (!stateInitialized) {
    +            throw new IllegalStateException("execute invoked before initState with input tuple " + input);
    +        }
    +        super.execute(input);
    +        // StatefulBoltExecutor does the actual ack when the state is saved.
    +        outputCollector.ack(input);
    +    }
    +
    +    @Override
    +    public void initState(T state) {
    +        if (stateInitialized) {
    +            LOG.warn("State is already initialized. Ignoring initState");
    +        } else {
    +            statefulWindowedBolt.initState(state);
    +            stateInitialized = true;
    +            start();
    +        }
    +    }
    +
    +    @Override
    +    public void prePrepare(long txid) {
    +        if (stateInitialized) {
    +            LOG.debug("Prepare streamState, txid {}", txid);
    +            state.prepareCommit(txid);
    +            prePrepared = true;
    +        } else {
    +            LOG.warn("Cannot prepare before initState");
    +        }
    +    }
    +
    +    @Override
    +    public void preCommit(long txid) {
    +        if (prePrepared) {
    +            LOG.debug("Commit streamState, txid {}", txid);
    +            state.commit(txid);
    +        } else {
    +            LOG.debug("Ignoring preCommit and not committing streamState.");
    +        }
    +    }
    +
    +    @Override
    +    public void preRollback() {
    +        LOG.debug("Rollback streamState, stateInitialized {}", stateInitialized);
    +        state.rollback();
    +    }
    +
    +    @Override
    +    protected WindowLifecycleListener<Tuple> newWindowLifecycleListener() {
    +        return new WindowLifecycleListener<Tuple>() {
    +            @Override
    +            public void onExpiry(List<Tuple> events) {
    +                /*
    +                 * NO-OP: the events are ack-ed in execute
    +                 */
    +            }
    +
    +            @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();
    +            }
    +        };
    +    }
    +
    +    private void init(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                      KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                      KeyValueState<String, Deque<Long>> partitionState,
    +                      KeyValueState<String, Optional<?>> windowSystemState) {
    +        topologyContext = context;
    +        outputCollector = collector;
    +        state = new WindowState<>(windowState, partitionState, windowSystemState, this::getState,
    +            statefulWindowedBolt.maxEventsInMemory());
    +    }
    +
    +    private KeyValueState<Long, WindowPartition<Tuple>> getWindowState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window";
    +        return (KeyValueState<Long, WindowPartition<Tuple>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Deque<Long>> getPartitionState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-partitions";
    +        return (KeyValueState<String, Deque<Long>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Optional<?>> getWindowSystemState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-systemstate";
    +        return (KeyValueState<String, Optional<?>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    // a wrapper around the window related states that are checkpointed
    +    private static class WindowState<T> extends AbstractCollection<Event<T>> {
    +        // number of events per window-partition
    +        private static final int PARTITION_SZ = 1000;
    +        private static final int MIN_PARTITIONS = 10;
    +        private static final String PARTITION_IDS_KEY = "pk";
    +        private final KeyValueState<String, Deque<Long>> partitionIds;
    +        private final KeyValueState<Long, WindowPartition<T>> windowPartitions;
    +        private final KeyValueState<String, Optional<?>> windowSystemState;
    +        // ordered partition keys
    +        private Deque<Long> pids;
    +        private volatile long latestPartitionId;
    +        private WindowPartitionCache<Long, WindowPartition<T>> cache;
    +        private Supplier<Map<String, Optional<?>>> windowSystemStateSupplier;
    +        private final ReentrantLock partitionIdsLock = new ReentrantLock(true);
    +        private final ReentrantLock windowPartitionsLock = new ReentrantLock(true);
    +        private final long maxEventsInMemory;
    +        private WindowPartition<T> latestPartition;
    +        private Set<Long> iteratorPins = new HashSet<>();
    +
    +        WindowState(KeyValueState<Long, WindowPartition<T>> windowPartitions,
    +                    KeyValueState<String, Deque<Long>> partitionIds,
    +                    KeyValueState<String, Optional<?>> windowSystemState,
    +                    Supplier<Map<String, Optional<?>>> windowSystemStateSupplier,
    +                    long maxEventsInMemory) {
    +            this.windowPartitions = windowPartitions;
    +            this.partitionIds = partitionIds;
    +            this.windowSystemState = windowSystemState;
    +            this.windowSystemStateSupplier = windowSystemStateSupplier;
    +            this.maxEventsInMemory = Math.max(PARTITION_SZ * MIN_PARTITIONS, maxEventsInMemory);
    +            initCache();
    +            initPartitions();
    +        }
    +
    +        @Override
    +        public boolean add(Event<T> event) {
    +            if (latestPartition.size() >= PARTITION_SZ) {
    +                cache.unpin(latestPartition.getId());
    +                latestPartition = getPinnedPartition(getNextPartitionId());
    +            }
    +            latestPartition.add(event);
    +            return true;
    +        }
    +
    +        @Override
    +        public Iterator<Event<T>> iterator() {
    +
    +            return new Iterator<Event<T>>() {
    +                private Iterator<Long> ids = getIds();
    +                private Iterator<Event<T>> current = emptyIterator();
    +                private Iterator<Event<T>> removeFrom;
    +                private WindowPartition<T> curPartition;
    +
    +                private Iterator<Long> getIds() {
    +                    try {
    +                        partitionIdsLock.lock();
    +                        LOG.debug("Iterator pids: {}", pids);
    --- End diff --
    
    Nit: Expand to partition ids, pids may be confusing for users


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r127958478
  
    --- Diff: docs/Windowing.md ---
    @@ -266,3 +266,105 @@ tuples can be received within the timeout period.
     An example toplogy `SlidingWindowTopology` shows how to use the apis to compute a sliding window sum and a tumbling window 
     average.
     
    +## Stateful windowing
    +The default windowing implementation in storm stores the tuples in memory until they are processed and expired from the 
    +window. This limits the use cases to windows that
    +fit entirely in memory. Also the source tuples cannot be ack-ed until the window expiry requiring large message timeouts
    +(topology.message.timeout.secs should be larger than the window length + sliding interval). This also puts extra loads 
    +due to the complex acking and anchoring requirements.
    + 
    +To address the above limitations and to support larger window sizes, storm provides stateful windowing support via `IStatefulWindowedBolt`. 
    +User bolts should typically extend `BaseStatefulWindowedBolt` for the windowing operations with the framework automatically 
    +managing the state of the window in the background.
    +
    +If the sources provide a monotonically increasing identifier as a part of the message, the framework can use this
    +to periodically checkpoint the last expired and evaluated message ids, to avoid duplicate window evaluations in case of 
    +failures or restarts. During recovery, the tuples with message ids lower than last expired id are discarded and tuples with 
    +message id between the last expired and last evaluated message ids are fed into the system without activating any triggers. 
    +The tuples beyond the last evaluated message ids are processed as usual. This can be enabled by setting
    +the `messageIdField` as shown below,
    +
    +```java
    +topologyBuilder.setBolt("mybolt",
    +                   new MyStatefulWindowedBolt()
    +                   .withWindow(...) // windowing configuarations
    +                   .withMessageIdField("msgid"), // a monotonically increasing 'long' field in the tuple
    +                   parallelism)
    +               .shuffleGrouping("spout");
    +```
    +
    +However, this option is feasible only if the sources can provide a monotonically increasing identifier in the tuple and the same is maintained
    +while re-emitting the messages in case of failures. Here the tuples in window are still held in memory.
    +
    +For more details take a look at the sample topology in storm starter `StatefulWindowingTopology` which will help you get started.
    +
    +### Window checkpointing
    +
    +With window checkpointing, the monotonically increasing id is no-longer required since the framework transparently saves the state of the window periodically into the configured state backend.
    +The state that is saved includes the tuples in the window, any system state that is required to recover the state of processing
    +and also the user state.
    +
    +```java
    +topologyBuilder.setBolt("mybolt",
    +                   new MyStatefulPersistentWindowedBolt()
    +                   .withWindow(...) // windowing configuarations
    +                   .withPersistence() // persist the window state
    +                   .withMaxEventsInMemory(25000), // max number of events to be cached in memory
    +                    parallelism)
    +               .shuffleGrouping("spout");
    +
    +```
    +
    +The `withPersistence` instructs the framework to transparently save the tuples in window along with
    +any associated system and user state to the state backend. The `withMaxEventsInMemory` is an optional 
    +configuration that specifies the maximum number of tuples that may be kept in memory. The tuples are transparently loaded from 
    +the state backend as required and the most recently used tuples are retained in memory (backed by a LRU cache).
    --- End diff --
    
    I feel the definition of 'used' in this sentence needs to be cleared. Yes I think caching tuples in memory would be the right way to do, but just not clear we can apply LRU in this case, especially older tuples are occasionally having more chance to be fired.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by satishd <gi...@git.apache.org>.
Github user satishd commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129803794
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/WindowPartitionCache.java ---
    @@ -0,0 +1,142 @@
    +/**
    + * 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 java.util.concurrent.ConcurrentMap;
    +
    +/**
    + * A loading cache abstraction for caching {@link PersistentWindowedBoltExecutor.WindowPartition}.
    + *
    + * @param <K> the key type
    + * @param <V> the value type
    + */
    +public interface WindowPartitionCache<K, V> {
    +
    +    /**
    +     * Get value from the cache or load the value.
    +     *
    +     * @param key the key
    +     * @return the value
    +     */
    +    V get(K key);
    +
    +    /**
    +     * Get value from the cache or load the value pinning it
    +     * so that the entry will never get evicted.
    +     *
    +     * @param key the key
    +     * @return the value
    +     */
    +    V getPinned(K key);
    --- End diff --
    
    This method name can be changed to `pinAndGet` as `getPinned` was implies getting the value if it is pinned.  


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130273607
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/base/BaseStatefulWindowedBolt.java ---
    @@ -151,6 +158,38 @@
             return this;
         }
     
    +    /**
    +     * If set, the stateful windowed bolt would use the backend state for window persistence and
    +     * only keep a sub-set of events in memory as specified by {@link #withMaxEventsInMemory(long)}.
    +     */
    +    public BaseStatefulWindowedBolt<T> withPersistence() {
    +        persistent = true;
    +        return this;
    +    }
    +
    +    /**
    +     * The maximum number of window events to keep in memory. This is meaningful only if
    +     * {@link #withPersistence()} is also set. As the number of events in memory grows close
    +     * to the maximum, the events that are less likely to be used again are evicted and persisted.
    +     * The default value for this is {@code 1,000,000}.
    +     *
    +     * @param maxEventsInMemory the maximum number of window events to keep in memory
    +     */
    +    public BaseStatefulWindowedBolt<T> withMaxEventsInMemory(long maxEventsInMemory) {
    --- End diff --
    
    Yes, it would be a good option to provide in future. Will raise a followup JIRA.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130225936
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutor.java ---
    @@ -0,0 +1,596 @@
    +/**
    + * 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 java.util.AbstractCollection;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Objects;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentLinkedQueue;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.locks.ReentrantLock;
    +import java.util.function.Supplier;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.state.KeyValueState;
    +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.topology.base.BaseWindowedBolt;
    +import org.apache.storm.tuple.Tuple;
    +import org.apache.storm.windowing.DefaultEvictionContext;
    +import org.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.EventImpl;
    +import org.apache.storm.windowing.WindowLifecycleListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static java.util.Collections.emptyIterator;
    +import static org.apache.storm.topology.WindowPartitionCache.CacheLoader;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalCause;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalListener;
    +
    +/**
    + * Wraps a {@link IStatefulWindowedBolt} and handles the execution. Uses state and the underlying
    + * checkpointing mechanisms to save the tuples in window to state. The tuples are also kept in-memory
    + * by transparently caching the window partitions and checkpointing them as needed.
    + */
    +public class PersistentWindowedBoltExecutor<T extends State> extends WindowedBoltExecutor implements IStatefulBolt<T> {
    +    private static final Logger LOG = LoggerFactory.getLogger(PersistentWindowedBoltExecutor.class);
    +    private final IStatefulWindowedBolt<T> statefulWindowedBolt;
    +    private transient TopologyContext topologyContext;
    +    private transient OutputCollector outputCollector;
    +    private transient WindowState<Tuple> state;
    +    private transient boolean stateInitialized;
    +    private transient boolean prePrepared;
    +
    +    public PersistentWindowedBoltExecutor(IStatefulWindowedBolt<T> bolt) {
    +        super(bolt);
    +        statefulWindowedBolt = bolt;
    +    }
    +
    +    @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));
    +    }
    +
    +    @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);
    +        }
    +    }
    +
    +    private int getCheckpointIntervalMillis(Map<String, Object> topoConf) {
    +        int checkpointInterval = Integer.MAX_VALUE;
    +        if (topoConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL) != null) {
    +            checkpointInterval = ((Number) topoConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL)).intValue();
    +        }
    +        return checkpointInterval;
    +    }
    +
    +    // package access for unit tests
    +    void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                 KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                 KeyValueState<String, Deque<Long>> partitionState,
    +                 KeyValueState<String, Optional<?>> windowSystemState) {
    +        init(topoConf, context, collector, windowState, partitionState, windowSystemState);
    +        doPrepare(topoConf, context, new NoAckOutputCollector(collector), state, true);
    +        Map<String, Optional<?>> wstate = new HashMap<>();
    +        windowSystemState.forEach(s -> wstate.put(s.getKey(), s.getValue()));
    +        restoreState(wstate);
    +    }
    +
    +    @Override
    +    protected void start() {
    +        if (stateInitialized) {
    +            super.start();
    +        } else {
    +            LOG.debug("Will invoke start after state is initialized");
    +        }
    +    }
    +
    +    @Override
    +    public void execute(Tuple input) {
    +        if (!stateInitialized) {
    +            throw new IllegalStateException("execute invoked before initState with input tuple " + input);
    +        }
    +        super.execute(input);
    +        // StatefulBoltExecutor does the actual ack when the state is saved.
    +        outputCollector.ack(input);
    +    }
    +
    +    @Override
    +    public void initState(T state) {
    +        if (stateInitialized) {
    +            String msg = "initState invoked when the state is already initialized";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        } else {
    +            statefulWindowedBolt.initState(state);
    +            stateInitialized = true;
    +            start();
    +        }
    +    }
    +
    +    @Override
    +    public void prePrepare(long txid) {
    +        if (stateInitialized) {
    +            LOG.debug("Prepare streamState, txid {}", txid);
    +            state.prepareCommit(txid);
    +            prePrepared = true;
    +        } else {
    +            String msg = "Cannot prepare before initState";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        }
    +    }
    +
    +    @Override
    +    public void preCommit(long txid) {
    +        // preCommit can be invoked during recovery before the state is initialized
    +        if (prePrepared || !stateInitialized) {
    +            LOG.debug("Commit streamState, txid {}", txid);
    +            state.commit(txid);
    +        } else {
    +            String msg = "preCommit before prePrepare in initialized state";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        }
    +    }
    +
    +    @Override
    +    public void preRollback() {
    +        LOG.debug("Rollback streamState, stateInitialized {}", stateInitialized);
    +        state.rollback();
    +    }
    +
    +    @Override
    +    protected WindowLifecycleListener<Tuple> newWindowLifecycleListener() {
    +        return new WindowLifecycleListener<Tuple>() {
    +            @Override
    +            public void onExpiry(List<Tuple> events) {
    +                /*
    +                 * NO-OP: the events are ack-ed in execute
    +                 */
    +            }
    +
    +            @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();
    +            }
    +        };
    +    }
    +
    +    private void init(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                      KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                      KeyValueState<String, Deque<Long>> partitionState,
    +                      KeyValueState<String, Optional<?>> windowSystemState) {
    +        topologyContext = context;
    +        outputCollector = collector;
    +        state = new WindowState<>(windowState, partitionState, windowSystemState, this::getState,
    +            statefulWindowedBolt.maxEventsInMemory());
    +    }
    +
    +    private KeyValueState<Long, WindowPartition<Tuple>> getWindowState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window";
    +        return (KeyValueState<Long, WindowPartition<Tuple>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Deque<Long>> getPartitionState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-partitions";
    +        return (KeyValueState<String, Deque<Long>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Optional<?>> getWindowSystemState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-systemstate";
    +        return (KeyValueState<String, Optional<?>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    // a wrapper around the window related states that are checkpointed
    +    private static class WindowState<T> extends AbstractCollection<Event<T>> {
    +        // number of events per window-partition
    +        private static final int MAX_PARTITION_EVENTS = 1000;
    +        private static final int MIN_PARTITIONS = 10;
    +        private static final String PARTITION_IDS_KEY = "pk";
    +        private final KeyValueState<String, Deque<Long>> partitionIdsState;
    +        private final KeyValueState<Long, WindowPartition<T>> windowPartitionsState;
    +        private final KeyValueState<String, Optional<?>> windowSystemState;
    +        // ordered partition keys
    +        private Deque<Long> partitionIds;
    +        private volatile long latestPartitionId;
    +        private WindowPartitionCache<Long, WindowPartition<T>> cache;
    +        private Supplier<Map<String, Optional<?>>> windowSystemStateSupplier;
    +        private final ReentrantLock partitionIdsLock = new ReentrantLock(true);
    +        private final ReentrantLock windowPartitionsLock = new ReentrantLock(true);
    +        private final long maxEventsInMemory;
    +        private WindowPartition<T> latestPartition;
    +        private Set<Long> iteratorPins = new HashSet<>();
    +
    +        WindowState(KeyValueState<Long, WindowPartition<T>> windowPartitionsState,
    +                    KeyValueState<String, Deque<Long>> partitionIdsState,
    +                    KeyValueState<String, Optional<?>> windowSystemState,
    +                    Supplier<Map<String, Optional<?>>> windowSystemStateSupplier,
    +                    long maxEventsInMemory) {
    +            this.windowPartitionsState = windowPartitionsState;
    +            this.partitionIdsState = partitionIdsState;
    +            this.windowSystemState = windowSystemState;
    +            this.windowSystemStateSupplier = windowSystemStateSupplier;
    +            this.maxEventsInMemory = Math.max(MAX_PARTITION_EVENTS * MIN_PARTITIONS, maxEventsInMemory);
    +            initCache();
    +            initPartitions();
    +        }
    +
    +        @Override
    +        public boolean add(Event<T> event) {
    +            if (latestPartition.size() >= MAX_PARTITION_EVENTS) {
    +                cache.unpin(latestPartition.getId());
    +                latestPartition = getPinnedPartition(getNextPartitionId());
    +            }
    +            latestPartition.add(event);
    +            return true;
    +        }
    +
    +        @Override
    +        public Iterator<Event<T>> iterator() {
    +
    +            return new Iterator<Event<T>>() {
    +                private Iterator<Long> ids = getIds();
    +                private Iterator<Event<T>> current = emptyIterator();
    +                private Iterator<Event<T>> removeFrom;
    +                private WindowPartition<T> curPartition;
    +
    +                private Iterator<Long> getIds() {
    +                    try {
    +                        partitionIdsLock.lock();
    +                        LOG.debug("Iterator partitionIds: {}", partitionIds);
    +                        return new ArrayList<>(partitionIds).iterator();
    +                    } finally {
    +                        partitionIdsLock.unlock();
    +                    }
    +                }
    +
    +                @Override
    +                public void remove() {
    +                    if (removeFrom == null) {
    +                        throw new IllegalStateException("No calls to next() since last call to remove()");
    +                    }
    +                    removeFrom.remove();
    +                    removeFrom = null;
    +                }
    +
    +                @Override
    +                public boolean hasNext() {
    +                    boolean curHasNext = current.hasNext();
    +                    while (!curHasNext && ids.hasNext()) {
    +                        if (curPartition != null) {
    +                            unpin(curPartition.getId());
    +                        }
    +                        curPartition = getPinnedPartition(ids.next());
    +                        if (curPartition != null) {
    +                            iteratorPins.add(curPartition.getId());
    +                            current = curPartition.iterator();
    +                            curHasNext = current.hasNext();
    +                        }
    +                    }
    +                    // un-pin the last partition
    +                    if (!curHasNext && curPartition != null) {
    +                        unpin(curPartition.getId());
    +                        curPartition = null;
    +                    }
    +                    return curHasNext;
    +                }
    +
    +                @Override
    +                public Event<T> next() {
    +                    if (!hasNext()) {
    +                        throw new NoSuchElementException();
    +                    }
    +                    removeFrom = current;
    +                    return current.next();
    +                }
    +
    +                private void unpin(long id) {
    +                    cache.unpin(id);
    +                    iteratorPins.remove(id);
    +                }
    +            };
    +        }
    +
    +        void clearIteratorPins() {
    +            LOG.debug("clearIteratorPins '{}'", iteratorPins);
    +            Iterator<Long> it = iteratorPins.iterator();
    +            while (it.hasNext()) {
    +                cache.unpin(it.next());
    +                it.remove();
    +            }
    +        }
    +
    +        @Override
    +        public int size() {
    +            throw new UnsupportedOperationException();
    +        }
    +
    +        void prepareCommit(long txid) {
    +            flush();
    +            partitionIdsState.prepareCommit(txid);
    +            windowPartitionsState.prepareCommit(txid);
    +            windowSystemState.prepareCommit(txid);
    +        }
    +
    +        void commit(long txid) {
    +            partitionIdsState.commit(txid);
    +            windowPartitionsState.commit(txid);
    +            windowSystemState.commit(txid);
    +        }
    +
    +        void rollback() {
    +            partitionIdsState.rollback();
    +            windowPartitionsState.rollback();
    +            windowSystemState.rollback();
    +        }
    +
    +        private void initPartitions() {
    +            partitionIds = partitionIdsState.get(PARTITION_IDS_KEY, new LinkedList<>());
    --- End diff --
    
    I'm not familiar with the State API but how does storing this into a local variable interact with rollbacks? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r131219624
  
    --- Diff: storm-client/src/jvm/org/apache/storm/windowing/persistence/WindowState.java ---
    @@ -0,0 +1,422 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.windowing.persistence;
    +
    +import com.google.common.collect.ImmutableMap;
    +import java.util.AbstractCollection;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Objects;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentLinkedQueue;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.locks.ReentrantLock;
    +import java.util.function.Supplier;
    +import org.apache.storm.state.KeyValueState;
    +import org.apache.storm.windowing.Event;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * A wrapper around the window related states that are checkpointed.
    + */
    +public class WindowState<T> extends AbstractCollection<Event<T>> {
    +    private static final Logger LOG = LoggerFactory.getLogger(WindowState.class);
    +
    +    // number of events per window-partition
    +    public static final int MAX_PARTITION_EVENTS = 1000;
    +    public static final int MIN_PARTITIONS = 10;
    +    private static final String PARTITION_IDS_KEY = "pk";
    +    private final KeyValueState<String, Deque<Long>> partitionIdsState;
    +    private final KeyValueState<Long, WindowPartition<T>> windowPartitionsState;
    +    private final KeyValueState<String, Optional<?>> windowSystemState;
    +    // ordered partition keys
    +    private volatile Deque<Long> partitionIds;
    +    private volatile long latestPartitionId;
    +    private volatile WindowPartition<T> latestPartition;
    +    private volatile WindowPartitionCache<Long, WindowPartition<T>> cache;
    +    private Supplier<Map<String, Optional<?>>> windowSystemStateSupplier;
    +    private final ReentrantLock partitionIdsLock = new ReentrantLock(true);
    +    private final WindowPartitionLock windowPartitionsLock = new WindowPartitionLock();
    +    private final long maxEventsInMemory;
    +    private Set<Long> iteratorPins = new HashSet<>();
    +
    +    public WindowState(KeyValueState<Long, WindowPartition<T>> windowPartitionsState,
    +                KeyValueState<String, Deque<Long>> partitionIdsState,
    +                KeyValueState<String, Optional<?>> windowSystemState,
    +                Supplier<Map<String, Optional<?>>> windowSystemStateSupplier,
    +                long maxEventsInMemory) {
    +        this.windowPartitionsState = windowPartitionsState;
    +        this.partitionIdsState = partitionIdsState;
    +        this.windowSystemState = windowSystemState;
    +        this.windowSystemStateSupplier = windowSystemStateSupplier;
    +        this.maxEventsInMemory = Math.max(MAX_PARTITION_EVENTS * MIN_PARTITIONS, maxEventsInMemory);
    +        init();
    +    }
    +
    +    @Override
    +    public boolean add(Event<T> event) {
    +        if (latestPartition.size() >= MAX_PARTITION_EVENTS) {
    +            cache.unpin(latestPartition.getId());
    +            latestPartition = getPinnedPartition(getNextPartitionId());
    +        }
    +        latestPartition.add(event);
    +        return true;
    +    }
    +
    +    @Override
    +    public Iterator<Event<T>> iterator() {
    +
    +        return new Iterator<Event<T>>() {
    +            private Iterator<Long> ids = getIds();
    +            private Iterator<Event<T>> current = Collections.emptyIterator();
    +            private Iterator<Event<T>> removeFrom;
    +            private WindowPartition<T> curPartition;
    +
    +            private Iterator<Long> getIds() {
    +                try {
    +                    partitionIdsLock.lock();
    +                    LOG.debug("Iterator partitionIds: {}", partitionIds);
    +                    return new ArrayList<>(partitionIds).iterator();
    +                } finally {
    +                    partitionIdsLock.unlock();
    +                }
    +            }
    +
    +            @Override
    +            public void remove() {
    +                if (removeFrom == null) {
    +                    throw new IllegalStateException("No calls to next() since last call to remove()");
    +                }
    +                removeFrom.remove();
    +                removeFrom = null;
    +            }
    +
    +            @Override
    +            public boolean hasNext() {
    +                boolean curHasNext = current.hasNext();
    +                while (!curHasNext && ids.hasNext()) {
    +                    if (curPartition != null) {
    +                        unpin(curPartition.getId());
    +                    }
    +                    curPartition = getPinnedPartition(ids.next());
    +                    if (curPartition != null) {
    +                        iteratorPins.add(curPartition.getId());
    +                        current = curPartition.iterator();
    +                        curHasNext = current.hasNext();
    +                    }
    +                }
    +                // un-pin the last partition
    +                if (!curHasNext && curPartition != null) {
    +                    unpin(curPartition.getId());
    +                    curPartition = null;
    +                }
    +                return curHasNext;
    +            }
    +
    +            @Override
    +            public Event<T> next() {
    +                if (!hasNext()) {
    +                    throw new NoSuchElementException();
    +                }
    +                removeFrom = current;
    +                return current.next();
    +            }
    +
    +            private void unpin(long id) {
    +                cache.unpin(id);
    +                iteratorPins.remove(id);
    +            }
    +        };
    +    }
    +
    +    public void clearIteratorPins() {
    +        LOG.debug("clearIteratorPins '{}'", iteratorPins);
    +        Iterator<Long> it = iteratorPins.iterator();
    +        while (it.hasNext()) {
    +            cache.unpin(it.next());
    +            it.remove();
    +        }
    +    }
    +
    +    @Override
    +    public int size() {
    +        throw new UnsupportedOperationException();
    +    }
    +
    +    /**
    +     * Prepares the {@link WindowState} for commit.
    +     *
    +     * @param txid the transaction id
    +     */
    +    public void prepareCommit(long txid) {
    +        flush();
    +        partitionIdsState.prepareCommit(txid);
    +        windowPartitionsState.prepareCommit(txid);
    +        windowSystemState.prepareCommit(txid);
    +    }
    +
    +    /**
    +     * Commits the {@link WindowState}.
    +     *
    +     * @param txid the transaction id
    +     */
    +    public void commit(long txid) {
    +        partitionIdsState.commit(txid);
    +        windowPartitionsState.commit(txid);
    +        windowSystemState.commit(txid);
    +    }
    +
    +    /**
    +     * Rolls back the {@link WindowState}.
    +     *
    +     * @param reInit if the members should be synced with the values from the state.
    +     */
    +    public void rollback(boolean reInit) {
    +        partitionIdsState.rollback();
    +        windowPartitionsState.rollback();
    +        windowSystemState.rollback();
    +        // re-init cache and partitions
    +        if (reInit) {
    +            init();
    +        }
    +    }
    +
    +    private void init() {
    +        initCache();
    +        initPartitions();
    +    }
    +
    +    private void initPartitions() {
    +        partitionIds = partitionIdsState.get(PARTITION_IDS_KEY, new LinkedList<>());
    +        if (partitionIds.isEmpty()) {
    +            partitionIds.add(0L);
    +            partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    +        }
    +        latestPartitionId = partitionIds.peekLast();
    +        latestPartition = cache.pinAndGet(latestPartitionId);
    +    }
    +
    +    private void initCache() {
    +        long size = maxEventsInMemory / MAX_PARTITION_EVENTS;
    +        LOG.info("maxEventsInMemory: {}, partition size: {}, number of partitions: {}",
    +            maxEventsInMemory, MAX_PARTITION_EVENTS, size);
    +        cache = SimpleWindowPartitionCache.<Long, WindowPartition<T>>newBuilder()
    +            .maximumSize(size)
    +            .removalListener(new WindowPartitionCache.RemovalListener<Long, WindowPartition<T>>() {
    +                @Override
    +                public void onRemoval(Long pid, WindowPartition<T> p, WindowPartitionCache.RemovalCause removalCause) {
    +                    Objects.requireNonNull(pid, "Null partition id");
    +                    Objects.requireNonNull(p, "Null window partition");
    +                    LOG.debug("onRemoval for id '{}', WindowPartition '{}'", pid, p);
    +                    try {
    +                        windowPartitionsLock.lock(pid);
    +                        if (p.isEmpty() && pid != latestPartitionId) {
    +                            // if the empty partition was not invalidated by flush, but evicted from cache
    +                            if (removalCause != WindowPartitionCache.RemovalCause.EXPLICIT) {
    +                                deletePartition(pid);
    +                                windowPartitionsState.delete(pid);
    +                            }
    +                        } else if (p.isModified()) {
    +                            windowPartitionsState.put(pid, p);
    +                        } else {
    +                            LOG.debug("WindowPartition '{}' is not modified", pid);
    +                        }
    +                    } finally {
    +                        windowPartitionsLock.unlock(pid);
    +                    }
    +                }
    +            }).build(new WindowPartitionCache.CacheLoader<Long, WindowPartition<T>>() {
    +                @Override
    +                public WindowPartition<T> load(Long id) {
    +                    LOG.debug("Load partition: {}", id);
    +                    // load from state
    +                    try {
    +                        windowPartitionsLock.lock(id);
    +                        return windowPartitionsState.get(id, new WindowPartition<>(id));
    +                    } finally {
    +                        windowPartitionsLock.unlock(id);
    +                    }
    +                }
    +            });
    +    }
    +
    +    private void deletePartition(long pid) {
    +        LOG.debug("Delete partition: {}", pid);
    +        try {
    +            partitionIdsLock.lock();
    +            partitionIds.remove(pid);
    +            partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    +        } finally {
    +            partitionIdsLock.unlock();
    +        }
    +    }
    +
    +    private long getNextPartitionId() {
    +        try {
    +            partitionIdsLock.lock();
    +            partitionIds.add(++latestPartitionId);
    +            partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    +        } finally {
    +            partitionIdsLock.unlock();
    +        }
    +        return latestPartitionId;
    +    }
    +
    +    private WindowPartition<T> getPinnedPartition(long id) {
    +        return cache.pinAndGet(id);
    +    }
    +
    +    private void flush() {
    +        LOG.debug("Flushing modified partitions");
    +        cache.asMap().forEach((pid, p) -> {
    +            Long pidToInvalidate = null;
    +            try {
    +                windowPartitionsLock.lock(pid);
    +                if (p.isEmpty() && pid != latestPartitionId) {
    +                    LOG.debug("Invalidating empty partition {}", pid);
    +                    deletePartition(pid);
    +                    windowPartitionsState.delete(pid);
    +                    pidToInvalidate = pid;
    +                } else if (p.isModified()) {
    +                    LOG.debug("Updating modified partition {}", pid);
    +                    p.clearModified();
    +                    windowPartitionsState.put(pid, p);
    +                }
    +            } finally {
    +                windowPartitionsLock.unlock(pid);
    +            }
    +            // invalidate after releasing the lock
    +            if (pidToInvalidate != null) {
    +                cache.invalidate(pidToInvalidate);
    --- End diff --
    
    it will be removed when (1) that partition gets unpinned and the entry is evicted from the cache due to loading other entries or (2) the next time the flush happens and the partition is empty. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r131120403
  
    --- Diff: storm-client/test/jvm/org/apache/storm/topology/SimpleWindowPartitionCacheTest.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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.utils.Utils;
    +import org.apache.storm.windowing.persistence.SimpleWindowPartitionCache;
    +import org.apache.storm.windowing.persistence.WindowPartitionCache;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.concurrent.FutureTask;
    +
    +/**
    + * Unit tests for {@link SimpleWindowPartitionCache}
    + */
    +public class SimpleWindowPartitionCacheTest {
    +
    +    @Before
    +    public void setUp() throws Exception {
    +    }
    +
    +    @Test(expected = IllegalArgumentException.class)
    +    public void testBuildInvalid1() throws Exception {
    +        SimpleWindowPartitionCache.<Integer, Integer>newBuilder()
    +            .maximumSize(0)
    +            .build(null);
    +    }
    +
    +    @Test(expected = IllegalArgumentException.class)
    +    public void testBuildInvalid2() throws Exception {
    +        SimpleWindowPartitionCache.<Integer, Integer>newBuilder()
    +            .maximumSize(-1)
    +            .build(null);
    +    }
    +
    +    @Test(expected = NullPointerException.class)
    +    public void testBuildInvalid3() throws Exception {
    +        SimpleWindowPartitionCache.<Integer, Integer>newBuilder()
    +            .maximumSize(1)
    +            .build(null);
    +    }
    +
    +    @Test
    +    public void testBuildOk() throws Exception {
    +        SimpleWindowPartitionCache.<Integer, Integer>newBuilder()
    +            .maximumSize(1)
    +            .removalListener((key, val, removalCause) -> {
    +            })
    +            .build(key -> key);
    +    }
    +
    +    @Test
    +    public void testGet() throws Exception {
    +        List<Integer> removed = new ArrayList<>();
    +        List<Integer> loaded = new ArrayList<>();
    +        SimpleWindowPartitionCache<Integer, Integer> cache =
    +            SimpleWindowPartitionCache.<Integer, Integer>newBuilder()
    +                .maximumSize(2)
    +                .removalListener((key, val, removalCause) -> removed.add(key))
    +                .build(key -> {
    +                    loaded.add(key);
    +                    return key;
    +                });
    +
    +        cache.get(1);
    +        cache.get(2);
    +        cache.get(3);
    +        Assert.assertEquals(Arrays.asList(1, 2, 3), loaded);
    +        Assert.assertEquals(Collections.singletonList(2), removed);
    +    }
    +
    +    @Test(expected = NullPointerException.class)
    +    public void testGetNull() throws Exception {
    +        SimpleWindowPartitionCache<Integer, Integer> cache =
    +            SimpleWindowPartitionCache.<Integer, Integer>newBuilder()
    +                .maximumSize(2)
    +                .build(key -> null);
    +
    +        cache.get(1);
    +    }
    +
    +    @Test
    +    public void testEvictNoRemovalListener() throws Exception {
    +        SimpleWindowPartitionCache<Integer, Integer> cache =
    +            SimpleWindowPartitionCache.<Integer, Integer>newBuilder()
    +                .maximumSize(1)
    +                .build(key -> {
    +                    return key;
    +                });
    +        cache.get(1);
    +        cache.get(2);
    +        Assert.assertEquals(Collections.singletonMap(2, 2), cache.asMap());
    +        cache.invalidate(2);
    +        Assert.assertEquals(Collections.emptyMap(), cache.asMap());
    +    }
    +
    +    @Test
    +    public void testPinAndGet() throws Exception {
    +        List<Integer> removed = new ArrayList<>();
    +        List<Integer> loaded = new ArrayList<>();
    +        SimpleWindowPartitionCache<Integer, Integer> cache =
    +            SimpleWindowPartitionCache.<Integer, Integer>newBuilder()
    +                .maximumSize(1)
    +                .removalListener(new WindowPartitionCache.RemovalListener<Integer, Integer>() {
    +                    @Override
    +                    public void onRemoval(Integer key, Integer val, WindowPartitionCache.RemovalCause removalCause) {
    +                        removed.add(key);
    +                    }
    +                })
    +                .build(new WindowPartitionCache.CacheLoader<Integer, Integer>() {
    +                    @Override
    +                    public Integer load(Integer key) {
    +                        loaded.add(key);
    +                        return key;
    +                    }
    +                });
    +
    +        cache.get(1);
    +        cache.pinAndGet(2);
    +        cache.get(3);
    +        Assert.assertEquals(Arrays.asList(1, 2, 3), loaded);
    +        Assert.assertEquals(Collections.singletonList(1), removed);
    +    }
    +
    +    @Test
    +    public void testInvalidate() throws Exception {
    +        List<Integer> removed = new ArrayList<>();
    +        List<Integer> loaded = new ArrayList<>();
    +        SimpleWindowPartitionCache<Integer, Integer> cache =
    +            SimpleWindowPartitionCache.<Integer, Integer>newBuilder()
    +                .maximumSize(1)
    +                .removalListener((key, val, removalCause) -> removed.add(key))
    +                .build(key -> {
    +                    loaded.add(key);
    +                    return key;
    +                });
    +
    +        cache.pinAndGet(1);
    +        cache.invalidate(1);
    +        Assert.assertEquals(Collections.singletonList(1), loaded);
    +        Assert.assertEquals(Collections.emptyList(), removed);
    +        Assert.assertEquals(cache.asMap(), Collections.singletonMap(1, 1));
    +
    +        cache.unpin(1);
    +        cache.invalidate(1);
    +        Assert.assertTrue(cache.asMap().isEmpty());
    +    }
    +
    +
    +    @Test
    +    public void testConcurrentGet() throws Exception {
    +        List<Integer> loaded = new ArrayList<>();
    +        SimpleWindowPartitionCache<Integer, Object> cache =
    +            SimpleWindowPartitionCache.<Integer, Object>newBuilder()
    +                .maximumSize(1)
    +                .build(key -> {
    +                    Utils.sleep(1000);
    +                    loaded.add(key);
    +                    return new Object();
    +                });
    +
    +        FutureTask<Object> ft1 = new FutureTask<>(() -> cache.pinAndGet(1));
    +        FutureTask<Object> ft2 = new FutureTask<>(() -> cache.pinAndGet(1));
    +        Thread t1 = new Thread(ft1);
    +        Thread t2 = new Thread(ft2);
    +        t1.start();
    +        t2.start();
    +        t1.join();
    --- End diff --
    
    Nit: Put a large timeout on these so the tests don't hang if we introduce a deadlock at some point


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129763373
  
    --- Diff: docs/Windowing.md ---
    @@ -266,3 +266,105 @@ tuples can be received within the timeout period.
     An example toplogy `SlidingWindowTopology` shows how to use the apis to compute a sliding window sum and a tumbling window 
     average.
     
    +## Stateful windowing
    +The default windowing implementation in storm stores the tuples in memory until they are processed and expired from the 
    +window. This limits the use cases to windows that
    +fit entirely in memory. Also the source tuples cannot be ack-ed until the window expiry requiring large message timeouts
    +(topology.message.timeout.secs should be larger than the window length + sliding interval). This also puts extra loads 
    +due to the complex acking and anchoring requirements.
    + 
    +To address the above limitations and to support larger window sizes, storm provides stateful windowing support via `IStatefulWindowedBolt`. 
    +User bolts should typically extend `BaseStatefulWindowedBolt` for the windowing operations with the framework automatically 
    +managing the state of the window in the background.
    +
    +If the sources provide a monotonically increasing identifier as a part of the message, the framework can use this
    +to periodically checkpoint the last expired and evaluated message ids, to avoid duplicate window evaluations in case of 
    +failures or restarts. During recovery, the tuples with message ids lower than last expired id are discarded and tuples with 
    +message id between the last expired and last evaluated message ids are fed into the system without activating any triggers. 
    +The tuples beyond the last evaluated message ids are processed as usual. This can be enabled by setting
    +the `messageIdField` as shown below,
    +
    +```java
    +topologyBuilder.setBolt("mybolt",
    +                   new MyStatefulWindowedBolt()
    +                   .withWindow(...) // windowing configuarations
    +                   .withMessageIdField("msgid"), // a monotonically increasing 'long' field in the tuple
    +                   parallelism)
    +               .shuffleGrouping("spout");
    +```
    +
    +However, this option is feasible only if the sources can provide a monotonically increasing identifier in the tuple and the same is maintained
    +while re-emitting the messages in case of failures. Here the tuples in window are still held in memory.
    +
    +For more details take a look at the sample topology in storm starter `StatefulWindowingTopology` which will help you get started.
    --- End diff --
    
    yes, will link to the file


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130273840
  
    --- Diff: storm-client/test/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutorTest.java ---
    @@ -0,0 +1,261 @@
    +/**
    + * 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.Config;
    +import org.apache.storm.generated.GlobalStreamId;
    +import org.apache.storm.state.KeyValueState;
    +import org.apache.storm.streams.Pair;
    +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.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.TimestampExtractor;
    +import org.apache.storm.windowing.TupleWindow;
    +import org.apache.storm.windowing.WaterMarkEvent;
    +import org.apache.storm.windowing.WaterMarkEventGenerator;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.mockito.ArgumentCaptor;
    +import org.mockito.Captor;
    +import org.mockito.Mockito;
    +import org.mockito.MockitoAnnotations;
    +import org.mockito.invocation.InvocationOnMock;
    +import org.mockito.runners.MockitoJUnitRunner;
    +import org.mockito.stubbing.Answer;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.stream.Collectors;
    +import java.util.stream.LongStream;
    +
    +import static org.mockito.AdditionalAnswers.returnsArgAt;
    +
    +/**
    + * Unit tests for {@link PersistentWindowedBoltExecutor}
    + */
    +@RunWith(MockitoJUnitRunner.class)
    +public class PersistentWindowedBoltExecutorTest {
    +    public static final String LATE_STREAM = "late_stream";
    +    private PersistentWindowedBoltExecutor<KeyValueState<String, String>> executor;
    +    private IStatefulWindowedBolt<KeyValueState<String, String>> mockBolt;
    +    private OutputCollector mockOutputCollector;
    +    private TopologyContext mockTopologyContext;
    +    private Map<String, Object> mockStormConf = new HashMap<>();
    +    private TimestampExtractor mockTimestampExtractor;
    +    private WaterMarkEventGenerator mockWaterMarkEventGenerator;
    +    private KeyValueState mockPartitionState;
    --- End diff --
    
    thanks for the input, yes `@Mock` can be used.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r131125567
  
    --- Diff: storm-client/test/jvm/org/apache/storm/windowing/persistence/WindowStateTest.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.windowing.persistence;
    +
    +import org.apache.storm.state.KeyValueState;
    +import org.apache.storm.tuple.Tuple;
    +import org.apache.storm.windowing.Event;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.mockito.ArgumentCaptor;
    +import org.mockito.Captor;
    +import org.mockito.Mock;
    +import org.mockito.Mockito;
    +import org.mockito.MockitoAnnotations;
    +import org.mockito.invocation.InvocationOnMock;
    +import org.mockito.runners.MockitoJUnitRunner;
    +import org.mockito.stubbing.Answer;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.function.Consumer;
    +import java.util.function.Supplier;
    +
    +import static org.mockito.AdditionalAnswers.returnsArgAt;
    +
    +/**
    + * Unit tests for {@link WindowState}
    + */
    +@RunWith(MockitoJUnitRunner.class)
    +public class WindowStateTest {
    +
    +    @Mock
    +    private KeyValueState<Long, WindowState.WindowPartition<Integer>> windowState;
    +    @Mock
    +    private KeyValueState<String, Deque<Long>> partitionIdsState;
    +    @Mock
    +    private KeyValueState<String, Optional<?>> systemState;
    +    @Mock
    +    private Supplier<Map<String, Optional<?>>> supplier;
    +    @Captor
    +    private ArgumentCaptor<Long> longCaptor;
    +    @Captor
    +    private ArgumentCaptor<WindowState.WindowPartition<Integer>> windowValuesCaptor;
    +
    +    private static final int MAX_EVENTS_PER_PARTITION = 1000;
    +
    +    @Before
    +    public void setUp() throws Exception {
    +        MockitoAnnotations.initMocks(this);
    +    }
    +
    +    @Test
    +    public void testAdd() throws Exception {
    +        Mockito.when(partitionIdsState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(windowState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +
    +        WindowState<Integer> ws = getWindowState(10 * MAX_EVENTS_PER_PARTITION);
    +
    +        long partitions = 15;
    +        long numEvents = partitions * MAX_EVENTS_PER_PARTITION;
    +        for (int i = 0; i < numEvents; i++) {
    +            ws.add(getEvent(i));
    +        }
    +        // 5 events evicted to window state
    +        Mockito.verify(windowState, Mockito.times(5)).put(longCaptor.capture(), windowValuesCaptor.capture());
    +        Assert.assertEquals(5, longCaptor.getAllValues().size());
    +        // each evicted partition has MAX_EVENTS_PER_PARTITION
    +        windowValuesCaptor.getAllValues().forEach(wp -> {
    +            Assert.assertEquals(MAX_EVENTS_PER_PARTITION, wp.size());
    +        });
    +        // last partition is not evicted
    +        Assert.assertFalse(longCaptor.getAllValues().contains(partitions - 1));
    +    }
    +
    +    @Test
    +    public void testIterator() throws Exception {
    +        Map<Long, WindowState.WindowPartition<Event<Tuple>>> partitionMap = new HashMap<>();
    +        Mockito.when(partitionIdsState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(windowState.get(Mockito.any(), Mockito.any())).then(new Answer<Object>() {
    +            @Override
    +            public Object answer(InvocationOnMock invocation) throws Throwable {
    +                Object[] args = invocation.getArguments();
    +                WindowState.WindowPartition<Event<Tuple>> evicted = partitionMap.get(args[0]);
    +                return evicted != null ? evicted : args[1];
    +            }
    +        });
    +
    +        Mockito.doAnswer(new Answer<Void>() {
    +            @Override
    +            public Void answer(InvocationOnMock invocation) throws Throwable {
    +                Object[] args = invocation.getArguments();
    +                partitionMap.put((long)args[0], (WindowState.WindowPartition<Event<Tuple>>)args[1]);
    +                return null;
    +            }
    +        }).when(windowState).put(Mockito.any(), Mockito.any());
    +
    +        Mockito.doAnswer(new Answer<Void>() {
    +            @Override
    +            public Void answer(InvocationOnMock invocation) throws Throwable {
    +                Object[] args = invocation.getArguments();
    +                partitionMap.remove(args[0]);
    +                return null;
    +            }
    +        }).when(windowState).delete(Mockito.anyLong());
    +
    +        Mockito.when(supplier.get()).thenReturn(Collections.emptyMap());
    +
    +        WindowState<Integer> ws = getWindowState(10 * MAX_EVENTS_PER_PARTITION);
    +
    +        long partitions = 15;
    +
    +        long numEvents = partitions * MAX_EVENTS_PER_PARTITION;
    +        List<Event<Integer>> expected = new ArrayList<>();
    +        for (int i = 0; i < numEvents; i++) {
    +            Event<Integer> event = getEvent(i);
    +            expected.add(event);
    +            ws.add(event);
    +        }
    +
    +        Assert.assertEquals(5, partitionMap.size());
    +        Iterator<Event<Integer>> it = ws.iterator();
    +        List<Event<Integer>> actual = new ArrayList<>();
    +        it.forEachRemaining(actual::add);
    +        Assert.assertEquals(expected, actual);
    +
    +        // iterate again
    +        it = ws.iterator();
    +        actual.clear();
    +        it.forEachRemaining(actual::add);
    +        Assert.assertEquals(expected, actual);
    +
    +        // remove
    +        it = ws.iterator();
    +        while (it.hasNext()) {
    +            it.next();
    +            it.remove();
    +        }
    +
    +        it = ws.iterator();
    +        actual.clear();
    +        it.forEachRemaining(actual::add);
    +        Assert.assertEquals(Collections.emptyList(), actual);
    +    }
    +
    +    @Test
    +    public void testIteratorPartitionNotEvicted() throws Exception {
    +        Map<Long, WindowState.WindowPartition<Event<Tuple>>> partitionMap = new HashMap<>();
    +        Mockito.when(partitionIdsState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(windowState.get(Mockito.any(), Mockito.any())).then(new Answer<Object>() {
    +            @Override
    +            public Object answer(InvocationOnMock invocation) throws Throwable {
    +                Object[] args = invocation.getArguments();
    +                WindowState.WindowPartition<Event<Tuple>> evicted = partitionMap.get(args[0]);
    +                return evicted != null ? evicted : args[1];
    +            }
    +        });
    +
    +        Mockito.doAnswer(new Answer<Void>() {
    +            @Override
    +            public Void answer(InvocationOnMock invocation) throws Throwable {
    +                Object[] args = invocation.getArguments();
    +                partitionMap.put((long)args[0], (WindowState.WindowPartition<Event<Tuple>>)args[1]);
    +                return null;
    +            }
    +        }).when(windowState).put(Mockito.any(), Mockito.any());
    +
    +        Mockito.when(supplier.get()).thenReturn(Collections.emptyMap());
    +
    +        WindowState<Integer> ws = getWindowState(10 * MAX_EVENTS_PER_PARTITION);
    +
    +        long partitions = 10;
    +
    +        long numEvents = partitions * MAX_EVENTS_PER_PARTITION;
    +        List<Event<Integer>> expected = new ArrayList<>();
    +        for (int i = 0; i < numEvents; i++) {
    +            Event<Integer> event = getEvent(i);
    +            expected.add(event);
    +            ws.add(event);
    +        }
    +
    +        Iterator<Event<Integer>> it = ws.iterator();
    +        for(int i=0; i<9500; i++) {
    --- End diff --
    
    I think it would be helpful to put a note here about this causing the partition to be pinned. At a glance it's not obvious why it should work this way.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129763588
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutor.java ---
    @@ -0,0 +1,563 @@
    +/**
    + * 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 java.util.AbstractCollection;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentLinkedQueue;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.locks.ReentrantLock;
    +import java.util.function.Supplier;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.state.KeyValueState;
    +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.windowing.DefaultEvictionContext;
    +import org.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.EventImpl;
    +import org.apache.storm.windowing.WindowLifecycleListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static java.util.Collections.emptyIterator;
    +import static org.apache.storm.topology.WindowPartitionCache.CacheLoader;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalCause;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalListener;
    +
    +/**
    + * Wraps a {@link IStatefulWindowedBolt} and handles the execution. Uses state and the underlying
    + * checkpointing mechanisms to save the tuples in window to state. The tuples are also kept in-memory
    + * by transparently caching the window partitions and checkpointing them as needed.
    + */
    +public class PersistentWindowedBoltExecutor<T extends State> extends WindowedBoltExecutor implements IStatefulBolt<T> {
    +    private static final Logger LOG = LoggerFactory.getLogger(PersistentWindowedBoltExecutor.class);
    +    private final IStatefulWindowedBolt<T> statefulWindowedBolt;
    +    private transient TopologyContext topologyContext;
    +    private transient OutputCollector outputCollector;
    +    private transient WindowState<Tuple> state;
    +    private transient boolean stateInitialized;
    +    private transient boolean prePrepared;
    +
    +    public PersistentWindowedBoltExecutor(IStatefulWindowedBolt<T> bolt) {
    +        super(bolt);
    +        statefulWindowedBolt = bolt;
    +    }
    +
    +    @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));
    +    }
    +
    +    // package access for unit tests
    +    void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                 KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                 KeyValueState<String, Deque<Long>> partitionState,
    +                 KeyValueState<String, Optional<?>> windowSystemState) {
    +        init(topoConf, context, collector, windowState, partitionState, windowSystemState);
    +        doPrepare(topoConf, context, new NoAckOutputCollector(collector), state, true);
    +        Map<String, Optional<?>> wstate = new HashMap<>();
    +        windowSystemState.forEach(s -> wstate.put(s.getKey(), s.getValue()));
    +        restoreState(wstate);
    +    }
    +
    +    @Override
    +    protected void start() {
    +        if (stateInitialized) {
    +            super.start();
    +        } else {
    +            LOG.debug("Will invoke start after state is initialized");
    +        }
    +    }
    +
    +    @Override
    +    public void execute(Tuple input) {
    +        if (!stateInitialized) {
    +            throw new IllegalStateException("execute invoked before initState with input tuple " + input);
    +        }
    +        super.execute(input);
    +        // StatefulBoltExecutor does the actual ack when the state is saved.
    +        outputCollector.ack(input);
    +    }
    +
    +    @Override
    +    public void initState(T state) {
    +        if (stateInitialized) {
    +            LOG.warn("State is already initialized. Ignoring initState");
    +        } else {
    +            statefulWindowedBolt.initState(state);
    +            stateInitialized = true;
    +            start();
    +        }
    +    }
    +
    +    @Override
    +    public void prePrepare(long txid) {
    +        if (stateInitialized) {
    +            LOG.debug("Prepare streamState, txid {}", txid);
    +            state.prepareCommit(txid);
    +            prePrepared = true;
    +        } else {
    +            LOG.warn("Cannot prepare before initState");
    +        }
    +    }
    +
    +    @Override
    +    public void preCommit(long txid) {
    +        if (prePrepared) {
    +            LOG.debug("Commit streamState, txid {}", txid);
    +            state.commit(txid);
    +        } else {
    +            LOG.debug("Ignoring preCommit and not committing streamState.");
    +        }
    +    }
    +
    +    @Override
    +    public void preRollback() {
    +        LOG.debug("Rollback streamState, stateInitialized {}", stateInitialized);
    +        state.rollback();
    +    }
    +
    +    @Override
    +    protected WindowLifecycleListener<Tuple> newWindowLifecycleListener() {
    +        return new WindowLifecycleListener<Tuple>() {
    +            @Override
    +            public void onExpiry(List<Tuple> events) {
    +                /*
    +                 * NO-OP: the events are ack-ed in execute
    +                 */
    +            }
    +
    +            @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();
    +            }
    +        };
    +    }
    +
    +    private void init(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                      KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                      KeyValueState<String, Deque<Long>> partitionState,
    +                      KeyValueState<String, Optional<?>> windowSystemState) {
    +        topologyContext = context;
    +        outputCollector = collector;
    +        state = new WindowState<>(windowState, partitionState, windowSystemState, this::getState,
    +            statefulWindowedBolt.maxEventsInMemory());
    +    }
    +
    +    private KeyValueState<Long, WindowPartition<Tuple>> getWindowState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window";
    +        return (KeyValueState<Long, WindowPartition<Tuple>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Deque<Long>> getPartitionState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-partitions";
    +        return (KeyValueState<String, Deque<Long>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Optional<?>> getWindowSystemState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-systemstate";
    +        return (KeyValueState<String, Optional<?>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    // a wrapper around the window related states that are checkpointed
    +    private static class WindowState<T> extends AbstractCollection<Event<T>> {
    +        // number of events per window-partition
    +        private static final int PARTITION_SZ = 1000;
    +        private static final int MIN_PARTITIONS = 10;
    +        private static final String PARTITION_IDS_KEY = "pk";
    +        private final KeyValueState<String, Deque<Long>> partitionIds;
    +        private final KeyValueState<Long, WindowPartition<T>> windowPartitions;
    +        private final KeyValueState<String, Optional<?>> windowSystemState;
    +        // ordered partition keys
    +        private Deque<Long> pids;
    +        private volatile long latestPartitionId;
    +        private WindowPartitionCache<Long, WindowPartition<T>> cache;
    +        private Supplier<Map<String, Optional<?>>> windowSystemStateSupplier;
    +        private final ReentrantLock partitionIdsLock = new ReentrantLock(true);
    +        private final ReentrantLock windowPartitionsLock = new ReentrantLock(true);
    +        private final long maxEventsInMemory;
    +        private WindowPartition<T> latestPartition;
    +        private Set<Long> iteratorPins = new HashSet<>();
    +
    +        WindowState(KeyValueState<Long, WindowPartition<T>> windowPartitions,
    +                    KeyValueState<String, Deque<Long>> partitionIds,
    +                    KeyValueState<String, Optional<?>> windowSystemState,
    +                    Supplier<Map<String, Optional<?>>> windowSystemStateSupplier,
    +                    long maxEventsInMemory) {
    +            this.windowPartitions = windowPartitions;
    +            this.partitionIds = partitionIds;
    +            this.windowSystemState = windowSystemState;
    +            this.windowSystemStateSupplier = windowSystemStateSupplier;
    +            this.maxEventsInMemory = Math.max(PARTITION_SZ * MIN_PARTITIONS, maxEventsInMemory);
    +            initCache();
    +            initPartitions();
    +        }
    +
    +        @Override
    +        public boolean add(Event<T> event) {
    +            if (latestPartition.size() >= PARTITION_SZ) {
    +                cache.unpin(latestPartition.getId());
    +                latestPartition = getPinnedPartition(getNextPartitionId());
    +            }
    +            latestPartition.add(event);
    +            return true;
    +        }
    +
    +        @Override
    +        public Iterator<Event<T>> iterator() {
    +
    +            return new Iterator<Event<T>>() {
    +                private Iterator<Long> ids = getIds();
    +                private Iterator<Event<T>> current = emptyIterator();
    +                private Iterator<Event<T>> removeFrom;
    +                private WindowPartition<T> curPartition;
    +
    +                private Iterator<Long> getIds() {
    +                    try {
    +                        partitionIdsLock.lock();
    +                        LOG.debug("Iterator pids: {}", pids);
    +                        return new ArrayList<>(pids).iterator();
    +                    } finally {
    +                        partitionIdsLock.unlock();
    +                    }
    +                }
    +
    +                @Override
    +                public void remove() {
    +                    if (removeFrom == null) {
    +                        throw new IllegalStateException("No calls to next() since last call to remove()");
    +                    }
    +                    removeFrom.remove();
    +                    removeFrom = null;
    +                }
    +
    +                @Override
    +                public boolean hasNext() {
    +                    boolean curHasNext = current.hasNext();
    +                    while (!curHasNext && ids.hasNext()) {
    +                        if (curPartition != null) {
    +                            unpin(curPartition.getId());
    +                        }
    +                        curPartition = getPinnedPartition(ids.next());
    +                        iteratorPins.add(curPartition.getId());
    +                        if (curPartition != null) {
    +                            current = curPartition.iterator();
    +                            curHasNext = current.hasNext();
    +                        }
    +                    }
    +                    // un-pin the last partition
    +                    if (!curHasNext && curPartition != null) {
    +                        unpin(curPartition.getId());
    +                        curPartition = null;
    +                    }
    +                    return curHasNext;
    +                }
    +
    +                @Override
    +                public Event<T> next() {
    +                    if (!hasNext()) {
    +                        throw new NoSuchElementException();
    +                    }
    +                    removeFrom = current;
    +                    return current.next();
    +                }
    +
    +                private void unpin(long id) {
    +                    cache.unpin(id);
    +                    iteratorPins.remove(id);
    +                }
    +            };
    +        }
    +
    +        void clearIteratorPins() {
    +            LOG.debug("clearIteratorPins '{}'", iteratorPins);
    +            Iterator<Long> it = iteratorPins.iterator();
    +            while (it.hasNext()) {
    +                cache.unpin(it.next());
    +                it.remove();
    +            }
    +        }
    +
    +        @Override
    +        public int size() {
    +            throw new UnsupportedOperationException();
    +        }
    +
    +        void prepareCommit(long txid) {
    +            flush();
    +            partitionIds.prepareCommit(txid);
    +            windowPartitions.prepareCommit(txid);
    +            windowSystemState.prepareCommit(txid);
    +        }
    +
    +        void commit(long txid) {
    +            partitionIds.commit(txid);
    +            windowPartitions.commit(txid);
    +            windowSystemState.commit(txid);
    +        }
    +
    +        void rollback() {
    +            partitionIds.rollback();
    +            windowPartitions.rollback();
    +            windowSystemState.rollback();
    +        }
    +
    +        private void initPartitions() {
    +            pids = partitionIds.get(PARTITION_IDS_KEY, new LinkedList<>());
    +            if (pids.isEmpty()) {
    +                pids.add(0L);
    +                partitionIds.put(PARTITION_IDS_KEY, pids);
    +            } else {
    +                latestPartitionId = pids.peekLast();
    +            }
    +            latestPartition = cache.getPinned(latestPartitionId);
    +        }
    +
    +        private void initCache() {
    +            long size = maxEventsInMemory / PARTITION_SZ;
    +            LOG.info("maxEventsInMemory: {}, partition size: {}, number of partitions: {}",
    +                maxEventsInMemory, PARTITION_SZ, size);
    +            cache = SimpleWindowPartitionCache.<Long, WindowPartition<T>>newBuilder()
    +                .maximumSize(size)
    +                .removalListener(new RemovalListener<Long, WindowPartition<T>>() {
    +                    @Override
    +                    public void onRemoval(Long pid, WindowPartition<T> p, RemovalCause removalCause) {
    +                        LOG.debug("onRemoval for id '{}', WindowPartition '{}'", pid, p);
    +                        if (pid == null || p == null) {
    --- End diff --
    
    not possible now, will change it to asserts.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129140797
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutor.java ---
    @@ -0,0 +1,563 @@
    +/**
    + * 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 java.util.AbstractCollection;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentLinkedQueue;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.locks.ReentrantLock;
    +import java.util.function.Supplier;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.state.KeyValueState;
    +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.windowing.DefaultEvictionContext;
    +import org.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.EventImpl;
    +import org.apache.storm.windowing.WindowLifecycleListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static java.util.Collections.emptyIterator;
    +import static org.apache.storm.topology.WindowPartitionCache.CacheLoader;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalCause;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalListener;
    +
    +/**
    + * Wraps a {@link IStatefulWindowedBolt} and handles the execution. Uses state and the underlying
    + * checkpointing mechanisms to save the tuples in window to state. The tuples are also kept in-memory
    + * by transparently caching the window partitions and checkpointing them as needed.
    + */
    +public class PersistentWindowedBoltExecutor<T extends State> extends WindowedBoltExecutor implements IStatefulBolt<T> {
    +    private static final Logger LOG = LoggerFactory.getLogger(PersistentWindowedBoltExecutor.class);
    +    private final IStatefulWindowedBolt<T> statefulWindowedBolt;
    +    private transient TopologyContext topologyContext;
    +    private transient OutputCollector outputCollector;
    +    private transient WindowState<Tuple> state;
    +    private transient boolean stateInitialized;
    +    private transient boolean prePrepared;
    +
    +    public PersistentWindowedBoltExecutor(IStatefulWindowedBolt<T> bolt) {
    +        super(bolt);
    +        statefulWindowedBolt = bolt;
    +    }
    +
    +    @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));
    +    }
    +
    +    // package access for unit tests
    +    void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                 KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                 KeyValueState<String, Deque<Long>> partitionState,
    +                 KeyValueState<String, Optional<?>> windowSystemState) {
    +        init(topoConf, context, collector, windowState, partitionState, windowSystemState);
    +        doPrepare(topoConf, context, new NoAckOutputCollector(collector), state, true);
    +        Map<String, Optional<?>> wstate = new HashMap<>();
    +        windowSystemState.forEach(s -> wstate.put(s.getKey(), s.getValue()));
    +        restoreState(wstate);
    +    }
    +
    +    @Override
    +    protected void start() {
    +        if (stateInitialized) {
    +            super.start();
    +        } else {
    +            LOG.debug("Will invoke start after state is initialized");
    +        }
    +    }
    +
    +    @Override
    +    public void execute(Tuple input) {
    +        if (!stateInitialized) {
    +            throw new IllegalStateException("execute invoked before initState with input tuple " + input);
    +        }
    +        super.execute(input);
    +        // StatefulBoltExecutor does the actual ack when the state is saved.
    +        outputCollector.ack(input);
    +    }
    +
    +    @Override
    +    public void initState(T state) {
    +        if (stateInitialized) {
    +            LOG.warn("State is already initialized. Ignoring initState");
    +        } else {
    +            statefulWindowedBolt.initState(state);
    +            stateInitialized = true;
    +            start();
    +        }
    +    }
    +
    +    @Override
    +    public void prePrepare(long txid) {
    +        if (stateInitialized) {
    +            LOG.debug("Prepare streamState, txid {}", txid);
    +            state.prepareCommit(txid);
    +            prePrepared = true;
    +        } else {
    +            LOG.warn("Cannot prepare before initState");
    +        }
    +    }
    +
    +    @Override
    +    public void preCommit(long txid) {
    +        if (prePrepared) {
    +            LOG.debug("Commit streamState, txid {}", txid);
    +            state.commit(txid);
    +        } else {
    +            LOG.debug("Ignoring preCommit and not committing streamState.");
    --- End diff --
    
    When can this happen?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130274003
  
    --- Diff: storm-client/src/jvm/org/apache/storm/windowing/WindowManager.java ---
    @@ -111,14 +125,86 @@ public void add(Event<T> windowEvent) {
                 LOG.debug("Got watermark event with ts {}", windowEvent.getTimestamp());
             }
             track(windowEvent);
    -        compactWindow();
    +        if (!stateful) {
    +            compactWindow();
    +        }
         }
     
         /**
          * The callback invoked by the trigger policy.
          */
         @Override
         public boolean onTrigger() {
    +        return stateful ? doOnTriggerStateful() : doOnTrigger();
    +    }
    +
    +    private static class IteratorStatus {
    +        private boolean valid = true;
    +
    +        void invalidate() {
    +            valid = false;
    +        }
    +
    +        boolean isValid() {
    +            return valid;
    +        }
    +    }
    +
    +    private static<T> Iterator<T> expiringIterator(Iterator<T> inner, IteratorStatus status) {
    +        return new Iterator<T>() {
    +            @Override
    +            public boolean hasNext() {
    +                if (status.isValid()) {
    +                    return inner.hasNext();
    +                }
    +                throw new IllegalStateException("Stale iterator");
    +            }
    +
    +            @Override
    +            public T next() {
    +                if (status.isValid()) {
    +                    return inner.next();
    +                }
    +                throw new IllegalStateException("Stale iterator");
    +            }
    +        };
    +    }
    +
    +    private boolean doOnTriggerStateful() {
    +        Supplier<Iterator<T>> scanEventsStateful = this::scanEventsStateful;
    +        Iterator<T> it = scanEventsStateful.get();
    +        boolean hasEvents = it.hasNext();
    +        if (hasEvents) {
    +            final IteratorStatus status = new IteratorStatus();
    +            LOG.debug("invoking windowLifecycleListener onActivation with iterator");
    +            // reuse the retrieved iterator
    +            Supplier<Iterator<T>> wrapper = new Supplier<Iterator<T>>() {
    +                Iterator<T> initial = it;
    +                @Override
    +                public Iterator<T> get() {
    +                    if (status.isValid()) {
    --- End diff --
    
    The iterator is invalidated after returning from the activation callback. Basically bolts are not supposed to hold a reference to the iterator obtained via `Window.getIter` after it returns from `execute` (say to iterate the values later). The window events can expire and some internal state is cleared so its not feasible and not very meaningful to do this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #2218: STORM-2614: Enhance stateful windowing to persist the win...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on the issue:

    https://github.com/apache/storm/pull/2218
  
    @srdo , thanks for taking time to do an in-depth review. Have addressed your latest comments.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #2218: STORM-2614: Enhance stateful windowing to persist the win...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on the issue:

    https://github.com/apache/storm/pull/2218
  
    After some more tests (with Caffeine for caching) I found some instances where the current partition (where entries are added or being removed) could get evicted causing issues in serialization. So I added a simple caching with support for pinning that evicts the latest un-pinned partition and based on tests found it to be better and fits the current use case well. @HeartSaVioR , you may want to re-visit the changes.
    
    @roshannaik , added the high level design notes to the JIRA as well.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129109171
  
    --- Diff: docs/Windowing.md ---
    @@ -266,3 +266,105 @@ tuples can be received within the timeout period.
     An example toplogy `SlidingWindowTopology` shows how to use the apis to compute a sliding window sum and a tumbling window 
     average.
     
    +## Stateful windowing
    +The default windowing implementation in storm stores the tuples in memory until they are processed and expired from the 
    +window. This limits the use cases to windows that
    +fit entirely in memory. Also the source tuples cannot be ack-ed until the window expiry requiring large message timeouts
    +(topology.message.timeout.secs should be larger than the window length + sliding interval). This also puts extra loads 
    +due to the complex acking and anchoring requirements.
    + 
    +To address the above limitations and to support larger window sizes, storm provides stateful windowing support via `IStatefulWindowedBolt`. 
    +User bolts should typically extend `BaseStatefulWindowedBolt` for the windowing operations with the framework automatically 
    +managing the state of the window in the background.
    +
    +If the sources provide a monotonically increasing identifier as a part of the message, the framework can use this
    +to periodically checkpoint the last expired and evaluated message ids, to avoid duplicate window evaluations in case of 
    +failures or restarts. During recovery, the tuples with message ids lower than last expired id are discarded and tuples with 
    +message id between the last expired and last evaluated message ids are fed into the system without activating any triggers. 
    --- End diff --
    
    Nit: I'm assuming the triggers referred to here are the TriggerPolicy instances? I don't think triggers are mentioned elsewhere in this doc, could we put a bit about them in?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by satishd <gi...@git.apache.org>.
Github user satishd commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r128993342
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutor.java ---
    @@ -0,0 +1,563 @@
    +/**
    + * 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 java.util.AbstractCollection;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentLinkedQueue;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.locks.ReentrantLock;
    +import java.util.function.Supplier;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.state.KeyValueState;
    +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.windowing.DefaultEvictionContext;
    +import org.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.EventImpl;
    +import org.apache.storm.windowing.WindowLifecycleListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static java.util.Collections.emptyIterator;
    +import static org.apache.storm.topology.WindowPartitionCache.CacheLoader;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalCause;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalListener;
    +
    +/**
    + * Wraps a {@link IStatefulWindowedBolt} and handles the execution. Uses state and the underlying
    + * checkpointing mechanisms to save the tuples in window to state. The tuples are also kept in-memory
    + * by transparently caching the window partitions and checkpointing them as needed.
    + */
    +public class PersistentWindowedBoltExecutor<T extends State> extends WindowedBoltExecutor implements IStatefulBolt<T> {
    +    private static final Logger LOG = LoggerFactory.getLogger(PersistentWindowedBoltExecutor.class);
    +    private final IStatefulWindowedBolt<T> statefulWindowedBolt;
    +    private transient TopologyContext topologyContext;
    --- End diff --
    
    nit: may want to remove this instance as it is never used.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129655963
  
    --- Diff: storm-client/src/jvm/org/apache/storm/windowing/TimeTriggerPolicy.java ---
    @@ -30,20 +30,20 @@
     /**
      * Invokes {@link TriggerHandler#onTrigger()} after the duration.
      */
    -public class TimeTriggerPolicy<T> implements TriggerPolicy<T> {
    +public class TimeTriggerPolicy<T> implements TriggerPolicy<T, Object> {
    --- End diff --
    
    Nit: Can you use Void instead?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129651528
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutor.java ---
    @@ -0,0 +1,563 @@
    +/**
    + * 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 java.util.AbstractCollection;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentLinkedQueue;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.locks.ReentrantLock;
    +import java.util.function.Supplier;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.state.KeyValueState;
    +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.windowing.DefaultEvictionContext;
    +import org.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.EventImpl;
    +import org.apache.storm.windowing.WindowLifecycleListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static java.util.Collections.emptyIterator;
    +import static org.apache.storm.topology.WindowPartitionCache.CacheLoader;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalCause;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalListener;
    +
    +/**
    + * Wraps a {@link IStatefulWindowedBolt} and handles the execution. Uses state and the underlying
    + * checkpointing mechanisms to save the tuples in window to state. The tuples are also kept in-memory
    + * by transparently caching the window partitions and checkpointing them as needed.
    + */
    +public class PersistentWindowedBoltExecutor<T extends State> extends WindowedBoltExecutor implements IStatefulBolt<T> {
    +    private static final Logger LOG = LoggerFactory.getLogger(PersistentWindowedBoltExecutor.class);
    +    private final IStatefulWindowedBolt<T> statefulWindowedBolt;
    +    private transient TopologyContext topologyContext;
    +    private transient OutputCollector outputCollector;
    +    private transient WindowState<Tuple> state;
    +    private transient boolean stateInitialized;
    +    private transient boolean prePrepared;
    +
    +    public PersistentWindowedBoltExecutor(IStatefulWindowedBolt<T> bolt) {
    +        super(bolt);
    +        statefulWindowedBolt = bolt;
    +    }
    +
    +    @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));
    +    }
    +
    +    // package access for unit tests
    +    void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                 KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                 KeyValueState<String, Deque<Long>> partitionState,
    +                 KeyValueState<String, Optional<?>> windowSystemState) {
    +        init(topoConf, context, collector, windowState, partitionState, windowSystemState);
    +        doPrepare(topoConf, context, new NoAckOutputCollector(collector), state, true);
    +        Map<String, Optional<?>> wstate = new HashMap<>();
    +        windowSystemState.forEach(s -> wstate.put(s.getKey(), s.getValue()));
    +        restoreState(wstate);
    +    }
    +
    +    @Override
    +    protected void start() {
    +        if (stateInitialized) {
    +            super.start();
    +        } else {
    +            LOG.debug("Will invoke start after state is initialized");
    +        }
    +    }
    +
    +    @Override
    +    public void execute(Tuple input) {
    +        if (!stateInitialized) {
    +            throw new IllegalStateException("execute invoked before initState with input tuple " + input);
    +        }
    +        super.execute(input);
    +        // StatefulBoltExecutor does the actual ack when the state is saved.
    +        outputCollector.ack(input);
    +    }
    +
    +    @Override
    +    public void initState(T state) {
    +        if (stateInitialized) {
    +            LOG.warn("State is already initialized. Ignoring initState");
    +        } else {
    +            statefulWindowedBolt.initState(state);
    +            stateInitialized = true;
    +            start();
    +        }
    +    }
    +
    +    @Override
    +    public void prePrepare(long txid) {
    +        if (stateInitialized) {
    +            LOG.debug("Prepare streamState, txid {}", txid);
    +            state.prepareCommit(txid);
    +            prePrepared = true;
    +        } else {
    +            LOG.warn("Cannot prepare before initState");
    +        }
    +    }
    +
    +    @Override
    +    public void preCommit(long txid) {
    +        if (prePrepared) {
    +            LOG.debug("Commit streamState, txid {}", txid);
    +            state.commit(txid);
    +        } else {
    +            LOG.debug("Ignoring preCommit and not committing streamState.");
    +        }
    +    }
    +
    +    @Override
    +    public void preRollback() {
    +        LOG.debug("Rollback streamState, stateInitialized {}", stateInitialized);
    +        state.rollback();
    +    }
    +
    +    @Override
    +    protected WindowLifecycleListener<Tuple> newWindowLifecycleListener() {
    +        return new WindowLifecycleListener<Tuple>() {
    +            @Override
    +            public void onExpiry(List<Tuple> events) {
    +                /*
    +                 * NO-OP: the events are ack-ed in execute
    +                 */
    +            }
    +
    +            @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();
    +            }
    +        };
    +    }
    +
    +    private void init(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                      KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                      KeyValueState<String, Deque<Long>> partitionState,
    +                      KeyValueState<String, Optional<?>> windowSystemState) {
    +        topologyContext = context;
    +        outputCollector = collector;
    +        state = new WindowState<>(windowState, partitionState, windowSystemState, this::getState,
    +            statefulWindowedBolt.maxEventsInMemory());
    +    }
    +
    +    private KeyValueState<Long, WindowPartition<Tuple>> getWindowState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window";
    +        return (KeyValueState<Long, WindowPartition<Tuple>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Deque<Long>> getPartitionState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-partitions";
    +        return (KeyValueState<String, Deque<Long>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Optional<?>> getWindowSystemState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-systemstate";
    +        return (KeyValueState<String, Optional<?>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    // a wrapper around the window related states that are checkpointed
    +    private static class WindowState<T> extends AbstractCollection<Event<T>> {
    +        // number of events per window-partition
    +        private static final int PARTITION_SZ = 1000;
    +        private static final int MIN_PARTITIONS = 10;
    +        private static final String PARTITION_IDS_KEY = "pk";
    +        private final KeyValueState<String, Deque<Long>> partitionIds;
    +        private final KeyValueState<Long, WindowPartition<T>> windowPartitions;
    +        private final KeyValueState<String, Optional<?>> windowSystemState;
    +        // ordered partition keys
    +        private Deque<Long> pids;
    +        private volatile long latestPartitionId;
    +        private WindowPartitionCache<Long, WindowPartition<T>> cache;
    +        private Supplier<Map<String, Optional<?>>> windowSystemStateSupplier;
    +        private final ReentrantLock partitionIdsLock = new ReentrantLock(true);
    +        private final ReentrantLock windowPartitionsLock = new ReentrantLock(true);
    +        private final long maxEventsInMemory;
    +        private WindowPartition<T> latestPartition;
    +        private Set<Long> iteratorPins = new HashSet<>();
    +
    +        WindowState(KeyValueState<Long, WindowPartition<T>> windowPartitions,
    +                    KeyValueState<String, Deque<Long>> partitionIds,
    +                    KeyValueState<String, Optional<?>> windowSystemState,
    +                    Supplier<Map<String, Optional<?>>> windowSystemStateSupplier,
    +                    long maxEventsInMemory) {
    +            this.windowPartitions = windowPartitions;
    +            this.partitionIds = partitionIds;
    +            this.windowSystemState = windowSystemState;
    +            this.windowSystemStateSupplier = windowSystemStateSupplier;
    +            this.maxEventsInMemory = Math.max(PARTITION_SZ * MIN_PARTITIONS, maxEventsInMemory);
    +            initCache();
    +            initPartitions();
    +        }
    +
    +        @Override
    +        public boolean add(Event<T> event) {
    +            if (latestPartition.size() >= PARTITION_SZ) {
    +                cache.unpin(latestPartition.getId());
    +                latestPartition = getPinnedPartition(getNextPartitionId());
    +            }
    +            latestPartition.add(event);
    +            return true;
    +        }
    +
    +        @Override
    +        public Iterator<Event<T>> iterator() {
    +
    +            return new Iterator<Event<T>>() {
    +                private Iterator<Long> ids = getIds();
    +                private Iterator<Event<T>> current = emptyIterator();
    +                private Iterator<Event<T>> removeFrom;
    +                private WindowPartition<T> curPartition;
    +
    +                private Iterator<Long> getIds() {
    +                    try {
    +                        partitionIdsLock.lock();
    +                        LOG.debug("Iterator pids: {}", pids);
    +                        return new ArrayList<>(pids).iterator();
    +                    } finally {
    +                        partitionIdsLock.unlock();
    +                    }
    +                }
    +
    +                @Override
    +                public void remove() {
    +                    if (removeFrom == null) {
    +                        throw new IllegalStateException("No calls to next() since last call to remove()");
    +                    }
    +                    removeFrom.remove();
    +                    removeFrom = null;
    +                }
    +
    +                @Override
    +                public boolean hasNext() {
    +                    boolean curHasNext = current.hasNext();
    +                    while (!curHasNext && ids.hasNext()) {
    +                        if (curPartition != null) {
    +                            unpin(curPartition.getId());
    +                        }
    +                        curPartition = getPinnedPartition(ids.next());
    +                        iteratorPins.add(curPartition.getId());
    +                        if (curPartition != null) {
    +                            current = curPartition.iterator();
    +                            curHasNext = current.hasNext();
    +                        }
    +                    }
    +                    // un-pin the last partition
    +                    if (!curHasNext && curPartition != null) {
    +                        unpin(curPartition.getId());
    +                        curPartition = null;
    +                    }
    +                    return curHasNext;
    +                }
    +
    +                @Override
    +                public Event<T> next() {
    +                    if (!hasNext()) {
    +                        throw new NoSuchElementException();
    +                    }
    +                    removeFrom = current;
    +                    return current.next();
    +                }
    +
    +                private void unpin(long id) {
    +                    cache.unpin(id);
    +                    iteratorPins.remove(id);
    +                }
    +            };
    +        }
    +
    +        void clearIteratorPins() {
    +            LOG.debug("clearIteratorPins '{}'", iteratorPins);
    +            Iterator<Long> it = iteratorPins.iterator();
    +            while (it.hasNext()) {
    +                cache.unpin(it.next());
    +                it.remove();
    +            }
    +        }
    +
    +        @Override
    +        public int size() {
    +            throw new UnsupportedOperationException();
    +        }
    +
    +        void prepareCommit(long txid) {
    +            flush();
    +            partitionIds.prepareCommit(txid);
    +            windowPartitions.prepareCommit(txid);
    +            windowSystemState.prepareCommit(txid);
    +        }
    +
    +        void commit(long txid) {
    +            partitionIds.commit(txid);
    +            windowPartitions.commit(txid);
    +            windowSystemState.commit(txid);
    +        }
    +
    +        void rollback() {
    +            partitionIds.rollback();
    +            windowPartitions.rollback();
    +            windowSystemState.rollback();
    +        }
    +
    +        private void initPartitions() {
    +            pids = partitionIds.get(PARTITION_IDS_KEY, new LinkedList<>());
    +            if (pids.isEmpty()) {
    +                pids.add(0L);
    +                partitionIds.put(PARTITION_IDS_KEY, pids);
    +            } else {
    +                latestPartitionId = pids.peekLast();
    --- End diff --
    
    Shouldn't this be set if pids was empty?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130228086
  
    --- Diff: storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java ---
    @@ -2363,7 +2363,7 @@ private CommonTopoInfo getCommonTopoInfo(String topoId, String operation) throws
                 ret.launchTimeSecs = 0;
             }
             ret.assignment = state.assignmentInfo(topoId, null);
    -        ret.beats = Utils.OR(heartbeatsCache.get().get(topoId), Collections.emptyMap());
    +        ret.beats = Utils.<Map<List<Integer>, Map<String, Object>>>OR(heartbeatsCache.get().get(topoId), Collections.emptyMap());
    --- End diff --
    
    Why is this necessary?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r128436267
  
    --- Diff: docs/Windowing.md ---
    @@ -266,3 +266,105 @@ tuples can be received within the timeout period.
     An example toplogy `SlidingWindowTopology` shows how to use the apis to compute a sliding window sum and a tumbling window 
     average.
     
    +## Stateful windowing
    +The default windowing implementation in storm stores the tuples in memory until they are processed and expired from the 
    +window. This limits the use cases to windows that
    +fit entirely in memory. Also the source tuples cannot be ack-ed until the window expiry requiring large message timeouts
    +(topology.message.timeout.secs should be larger than the window length + sliding interval). This also puts extra loads 
    +due to the complex acking and anchoring requirements.
    + 
    +To address the above limitations and to support larger window sizes, storm provides stateful windowing support via `IStatefulWindowedBolt`. 
    +User bolts should typically extend `BaseStatefulWindowedBolt` for the windowing operations with the framework automatically 
    +managing the state of the window in the background.
    +
    +If the sources provide a monotonically increasing identifier as a part of the message, the framework can use this
    +to periodically checkpoint the last expired and evaluated message ids, to avoid duplicate window evaluations in case of 
    +failures or restarts. During recovery, the tuples with message ids lower than last expired id are discarded and tuples with 
    +message id between the last expired and last evaluated message ids are fed into the system without activating any triggers. 
    +The tuples beyond the last evaluated message ids are processed as usual. This can be enabled by setting
    +the `messageIdField` as shown below,
    +
    +```java
    +topologyBuilder.setBolt("mybolt",
    +                   new MyStatefulWindowedBolt()
    +                   .withWindow(...) // windowing configuarations
    +                   .withMessageIdField("msgid"), // a monotonically increasing 'long' field in the tuple
    +                   parallelism)
    +               .shuffleGrouping("spout");
    +```
    +
    +However, this option is feasible only if the sources can provide a monotonically increasing identifier in the tuple and the same is maintained
    +while re-emitting the messages in case of failures. Here the tuples in window are still held in memory.
    +
    +For more details take a look at the sample topology in storm starter `StatefulWindowingTopology` which will help you get started.
    +
    +### Window checkpointing
    +
    +With window checkpointing, the monotonically increasing id is no-longer required since the framework transparently saves the state of the window periodically into the configured state backend.
    +The state that is saved includes the tuples in the window, any system state that is required to recover the state of processing
    +and also the user state.
    +
    +```java
    +topologyBuilder.setBolt("mybolt",
    +                   new MyStatefulPersistentWindowedBolt()
    +                   .withWindow(...) // windowing configuarations
    +                   .withPersistence() // persist the window state
    +                   .withMaxEventsInMemory(25000), // max number of events to be cached in memory
    +                    parallelism)
    +               .shuffleGrouping("spout");
    +
    +```
    +
    +The `withPersistence` instructs the framework to transparently save the tuples in window along with
    +any associated system and user state to the state backend. The `withMaxEventsInMemory` is an optional 
    +configuration that specifies the maximum number of tuples that may be kept in memory. The tuples are transparently loaded from 
    +the state backend as required and the most recently used tuples are retained in memory (backed by a LRU cache).
    --- End diff --
    
    @arunmahadevan 
    Great news to hear that we've much better results. 
    Actually I forgot to leave comment that we can use Caffeine and less depend on Guava, so the replacement is also preferred.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130273640
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutor.java ---
    @@ -0,0 +1,596 @@
    +/**
    + * 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 java.util.AbstractCollection;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Objects;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentLinkedQueue;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.locks.ReentrantLock;
    +import java.util.function.Supplier;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.state.KeyValueState;
    +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.topology.base.BaseWindowedBolt;
    +import org.apache.storm.tuple.Tuple;
    +import org.apache.storm.windowing.DefaultEvictionContext;
    +import org.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.EventImpl;
    +import org.apache.storm.windowing.WindowLifecycleListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static java.util.Collections.emptyIterator;
    +import static org.apache.storm.topology.WindowPartitionCache.CacheLoader;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalCause;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalListener;
    +
    +/**
    + * Wraps a {@link IStatefulWindowedBolt} and handles the execution. Uses state and the underlying
    + * checkpointing mechanisms to save the tuples in window to state. The tuples are also kept in-memory
    + * by transparently caching the window partitions and checkpointing them as needed.
    + */
    +public class PersistentWindowedBoltExecutor<T extends State> extends WindowedBoltExecutor implements IStatefulBolt<T> {
    +    private static final Logger LOG = LoggerFactory.getLogger(PersistentWindowedBoltExecutor.class);
    +    private final IStatefulWindowedBolt<T> statefulWindowedBolt;
    +    private transient TopologyContext topologyContext;
    +    private transient OutputCollector outputCollector;
    +    private transient WindowState<Tuple> state;
    +    private transient boolean stateInitialized;
    +    private transient boolean prePrepared;
    +
    +    public PersistentWindowedBoltExecutor(IStatefulWindowedBolt<T> bolt) {
    +        super(bolt);
    +        statefulWindowedBolt = bolt;
    +    }
    +
    +    @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));
    +    }
    +
    +    @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);
    +        }
    +    }
    +
    +    private int getCheckpointIntervalMillis(Map<String, Object> topoConf) {
    +        int checkpointInterval = Integer.MAX_VALUE;
    +        if (topoConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL) != null) {
    +            checkpointInterval = ((Number) topoConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL)).intValue();
    +        }
    +        return checkpointInterval;
    +    }
    +
    +    // package access for unit tests
    +    void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                 KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                 KeyValueState<String, Deque<Long>> partitionState,
    +                 KeyValueState<String, Optional<?>> windowSystemState) {
    +        init(topoConf, context, collector, windowState, partitionState, windowSystemState);
    +        doPrepare(topoConf, context, new NoAckOutputCollector(collector), state, true);
    +        Map<String, Optional<?>> wstate = new HashMap<>();
    +        windowSystemState.forEach(s -> wstate.put(s.getKey(), s.getValue()));
    +        restoreState(wstate);
    +    }
    +
    +    @Override
    +    protected void start() {
    +        if (stateInitialized) {
    +            super.start();
    +        } else {
    +            LOG.debug("Will invoke start after state is initialized");
    +        }
    +    }
    +
    +    @Override
    +    public void execute(Tuple input) {
    +        if (!stateInitialized) {
    +            throw new IllegalStateException("execute invoked before initState with input tuple " + input);
    +        }
    +        super.execute(input);
    +        // StatefulBoltExecutor does the actual ack when the state is saved.
    +        outputCollector.ack(input);
    +    }
    +
    +    @Override
    +    public void initState(T state) {
    +        if (stateInitialized) {
    +            String msg = "initState invoked when the state is already initialized";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        } else {
    +            statefulWindowedBolt.initState(state);
    +            stateInitialized = true;
    +            start();
    +        }
    +    }
    +
    +    @Override
    +    public void prePrepare(long txid) {
    +        if (stateInitialized) {
    +            LOG.debug("Prepare streamState, txid {}", txid);
    +            state.prepareCommit(txid);
    +            prePrepared = true;
    +        } else {
    +            String msg = "Cannot prepare before initState";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        }
    +    }
    +
    +    @Override
    +    public void preCommit(long txid) {
    +        // preCommit can be invoked during recovery before the state is initialized
    +        if (prePrepared || !stateInitialized) {
    +            LOG.debug("Commit streamState, txid {}", txid);
    +            state.commit(txid);
    +        } else {
    +            String msg = "preCommit before prePrepare in initialized state";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        }
    +    }
    +
    +    @Override
    +    public void preRollback() {
    +        LOG.debug("Rollback streamState, stateInitialized {}", stateInitialized);
    +        state.rollback();
    +    }
    +
    +    @Override
    +    protected WindowLifecycleListener<Tuple> newWindowLifecycleListener() {
    +        return new WindowLifecycleListener<Tuple>() {
    +            @Override
    +            public void onExpiry(List<Tuple> events) {
    +                /*
    +                 * NO-OP: the events are ack-ed in execute
    +                 */
    +            }
    +
    +            @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();
    +            }
    +        };
    +    }
    +
    +    private void init(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                      KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                      KeyValueState<String, Deque<Long>> partitionState,
    +                      KeyValueState<String, Optional<?>> windowSystemState) {
    +        topologyContext = context;
    +        outputCollector = collector;
    +        state = new WindowState<>(windowState, partitionState, windowSystemState, this::getState,
    +            statefulWindowedBolt.maxEventsInMemory());
    +    }
    +
    +    private KeyValueState<Long, WindowPartition<Tuple>> getWindowState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window";
    +        return (KeyValueState<Long, WindowPartition<Tuple>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Deque<Long>> getPartitionState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-partitions";
    +        return (KeyValueState<String, Deque<Long>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Optional<?>> getWindowSystemState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-systemstate";
    +        return (KeyValueState<String, Optional<?>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    // a wrapper around the window related states that are checkpointed
    +    private static class WindowState<T> extends AbstractCollection<Event<T>> {
    --- End diff --
    
    can be refactored.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by satishd <gi...@git.apache.org>.
Github user satishd commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129804787
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/SimpleWindowPartitionCache.java ---
    @@ -0,0 +1,191 @@
    +/**
    + * 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 java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.concurrent.ConcurrentMap;
    +import java.util.concurrent.ConcurrentSkipListMap;
    +import java.util.concurrent.locks.ReentrantLock;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * A simple implementation that evicts the largest un-pinned entry from the cache. This works well
    + * for caching window partitions since the access pattern is mostly sequential scans.
    + */
    +public class SimpleWindowPartitionCache<K, V> implements WindowPartitionCache<K, V> {
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleWindowPartitionCache.class);
    +
    +    private final ConcurrentSkipListMap<K, V> map = new ConcurrentSkipListMap<>();
    +    private final Map<K, Long> pinned = new HashMap<>();
    +    private final long maximumSize;
    +    private final RemovalListener<K, V> removalListener;
    +    private final CacheLoader<K, V> cacheLoader;
    +    private final ReentrantLock lock = new ReentrantLock(true);
    +    private int size;
    +
    +    @Override
    +    public V get(K key) {
    +        return getOrLoad(key, false);
    +    }
    +
    +    @Override
    +    public V getPinned(K key) {
    +        return getOrLoad(key, true);
    +    }
    +
    +    @Override
    +    public boolean unpin(K key) {
    +        LOG.debug("unpin '{}'", key);
    +        boolean res = false;
    +        try {
    +            lock.lock();
    +            if (pinned.compute(key, (k, v) -> v == null ? 0 : v - 1) <= 0) {
    +                pinned.remove(key);
    +                res = true;
    +            }
    +        } finally {
    +            lock.unlock();
    +        }
    +        LOG.debug("pinned '{}'", pinned);
    +        return res;
    +    }
    +
    +    @Override
    +    public ConcurrentMap<K, V> asMap() {
    +        return map;
    +    }
    +
    +    @Override
    +    public void invalidate(K key) {
    +        try {
    +            lock.lock();
    +            if (isPinned(key)) {
    +                LOG.debug("Entry '{}' is pinned, skipping invalidation", key);
    +            } else {
    +                LOG.debug("Invalidating entry '{}'", key);
    +                V val = map.remove(key);
    +                if (val != null) {
    +                    --size;
    +                    pinned.remove(key);
    +                    removalListener.onRemoval(key, val, RemovalCause.EXPLICIT);
    +                }
    +            }
    +        } finally {
    +            lock.unlock();
    +        }
    +    }
    +
    +    // Get or load from the cache optionally pinning the entry
    +    // so that it wont get evicted from the cache
    +    private V getOrLoad(K key, boolean shouldPin) {
    +        V val;
    +        if (shouldPin) {
    +            try {
    +                lock.lock();
    +                val = load(key);
    +                pin(key);
    +            } finally {
    +                lock.unlock();
    +            }
    +        } else {
    +            val = map.get(key);
    --- End diff --
    
    Why are we checking here as this is already checked in `load(key)`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #2218: STORM-2614: Enhance stateful windowing to persist the win...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on the issue:

    https://github.com/apache/storm/pull/2218
  
    Thanks for the reviews, squashed and merged to master.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130273915
  
    --- Diff: storm-client/test/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutorTest.java ---
    @@ -0,0 +1,261 @@
    +/**
    + * 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.Config;
    +import org.apache.storm.generated.GlobalStreamId;
    +import org.apache.storm.state.KeyValueState;
    +import org.apache.storm.streams.Pair;
    +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.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.TimestampExtractor;
    +import org.apache.storm.windowing.TupleWindow;
    +import org.apache.storm.windowing.WaterMarkEvent;
    +import org.apache.storm.windowing.WaterMarkEventGenerator;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.mockito.ArgumentCaptor;
    +import org.mockito.Captor;
    +import org.mockito.Mockito;
    +import org.mockito.MockitoAnnotations;
    +import org.mockito.invocation.InvocationOnMock;
    +import org.mockito.runners.MockitoJUnitRunner;
    +import org.mockito.stubbing.Answer;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.stream.Collectors;
    +import java.util.stream.LongStream;
    +
    +import static org.mockito.AdditionalAnswers.returnsArgAt;
    +
    +/**
    + * Unit tests for {@link PersistentWindowedBoltExecutor}
    + */
    +@RunWith(MockitoJUnitRunner.class)
    +public class PersistentWindowedBoltExecutorTest {
    +    public static final String LATE_STREAM = "late_stream";
    +    private PersistentWindowedBoltExecutor<KeyValueState<String, String>> executor;
    +    private IStatefulWindowedBolt<KeyValueState<String, String>> mockBolt;
    +    private OutputCollector mockOutputCollector;
    +    private TopologyContext mockTopologyContext;
    +    private Map<String, Object> mockStormConf = new HashMap<>();
    +    private TimestampExtractor mockTimestampExtractor;
    +    private WaterMarkEventGenerator mockWaterMarkEventGenerator;
    +    private KeyValueState mockPartitionState;
    +    private KeyValueState mockWindowState;
    +    private KeyValueState mockSystemState;
    +
    +    private static final String PARTITION_KEY = "pk";
    +    private static final String EVICTION_STATE_KEY = "es";
    +    private static final String TRIGGER_STATE_KEY = "ts";
    +    private long tupleTs;
    +
    +    @Captor
    +    private ArgumentCaptor<Tuple> tupleCaptor;
    +    @Captor
    +    private ArgumentCaptor<Collection<Tuple>> anchorCaptor;
    +    @Captor
    +    private ArgumentCaptor<String> stringCaptor;
    +    @Captor
    +    private ArgumentCaptor<Long> longCaptor;
    +    @Captor
    +    private ArgumentCaptor<Values> valuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<TupleWindow> tupleWindowCaptor;
    +    @Captor
    +    private ArgumentCaptor<Deque<Long>> partitionValuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<PersistentWindowedBoltExecutor.WindowPartition<Event<Tuple>>> windowValuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<Object> systemValuesCaptor;
    +
    +    @Before
    +    public void setUp() throws Exception {
    +        MockitoAnnotations.initMocks(this);
    +        mockBolt = Mockito.mock(IStatefulWindowedBolt.class);
    +        mockWaterMarkEventGenerator = Mockito.mock(WaterMarkEventGenerator.class);
    +        mockTimestampExtractor = Mockito.mock(TimestampExtractor.class);
    +        tupleTs = System.currentTimeMillis();
    +        Mockito.when(mockTimestampExtractor.extractTimestamp(Mockito.any())).thenReturn(tupleTs);
    +        Mockito.when(mockBolt.getTimestampExtractor()).thenReturn(mockTimestampExtractor);
    +        Mockito.when(mockBolt.isPersistent()).thenReturn(true);
    +        mockPartitionState = Mockito.mock(KeyValueState.class);
    +        mockWindowState = Mockito.mock(KeyValueState.class);
    +        mockSystemState = Mockito.mock(KeyValueState.class);
    +        mockTopologyContext = Mockito.mock(TopologyContext.class);
    +        Mockito.when(mockTopologyContext.getThisStreams()).thenReturn(Collections.singleton(LATE_STREAM));
    +        mockOutputCollector = Mockito.mock(OutputCollector.class);
    +        executor = new PersistentWindowedBoltExecutor<>(mockBolt);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_WINDOW_LENGTH_COUNT, 5);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_SLIDING_INTERVAL_COUNT, 5);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_LATE_TUPLE_STREAM, LATE_STREAM);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_WATERMARK_EVENT_INTERVAL_MS, 100_000);
    +        mockStormConf.put(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS, 30);
    +        mockStormConf.put(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL, 1000);
    +        Mockito.when(mockPartitionState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(mockWindowState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(mockSystemState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        executor.prepare(mockStormConf, mockTopologyContext, mockOutputCollector,
    +            mockWindowState, mockPartitionState, mockSystemState);
    +    }
    +
    +    @Test
    +    public void testExecuteTuple() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), 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);
    +    }
    +
    +    @Test
    +    public void testExecuteLatetuple() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), 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
    +        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());
    +    }
    +
    +    @Test
    +    public void testActivation() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), Mockito.anyLong())).thenReturn(true);
    +        executor.initState(null);
    +        executor.waterMarkEventGenerator = mockWaterMarkEventGenerator;
    +
    +        List<Tuple> mockTuples = getMockTuples(5);
    +        mockTuples.forEach(t -> executor.execute(t));
    +        // all tuples acked
    +        Mockito.verify(mockOutputCollector, Mockito.times(5)).ack(tupleCaptor.capture());
    +        Assert.assertArrayEquals(mockTuples.toArray(), tupleCaptor.getAllValues().toArray());
    +
    +        Mockito.doAnswer(new Answer<Void>() {
    +            @Override
    +            public Void answer(InvocationOnMock invocation) throws Throwable {
    +                TupleWindow window = (TupleWindow) invocation.getArguments()[0];
    +                // iterate the tuples
    +                Assert.assertEquals(5, window.get().size());
    +                // iterating multiple times should produce same events
    +                Assert.assertEquals(5, window.get().size());
    +                Assert.assertEquals(5, window.get().size());
    +                return null;
    +            }
    +        }).when(mockBolt).execute(Mockito.any());
    +        // trigger the window
    +        long activationTs = tupleTs + 1000;
    +        executor.getWindowManager().add(new WaterMarkEvent<>(activationTs));
    +        executor.prePrepare(0);
    +
    +        // partition ids
    +        Mockito.verify(mockPartitionState, Mockito.times(1)).put(stringCaptor.capture(), partitionValuesCaptor.capture());
    +        Assert.assertEquals(PARTITION_KEY, stringCaptor.getAllValues().get(0));
    +        Assert.assertEquals(Collections.singletonList(0L), partitionValuesCaptor.getAllValues().get(0));
    +
    +        // window partitions
    +        Mockito.verify(mockWindowState, Mockito.times(1)).put(longCaptor.capture(), windowValuesCaptor.capture());
    +        Assert.assertEquals(0L, (long) longCaptor.getAllValues().get(0));
    +        Assert.assertEquals(5, windowValuesCaptor.getAllValues().get(0).size());
    +        List<Event<?>> tuples = windowValuesCaptor.getAllValues().get(0)
    +            .getEvents().stream().map(Event::get).collect(Collectors.toList());
    +        Assert.assertArrayEquals(mockTuples.toArray(), tuples.toArray());
    +
    +        // window system state
    +        Mockito.verify(mockSystemState, Mockito.times(2)).put(stringCaptor.capture(), systemValuesCaptor.capture());
    +        Assert.assertEquals(EVICTION_STATE_KEY, stringCaptor.getAllValues().get(1));
    +        Assert.assertEquals(Optional.of(Pair.of(5L, 5L)), systemValuesCaptor.getAllValues().get(0));
    +        Assert.assertEquals(TRIGGER_STATE_KEY, stringCaptor.getAllValues().get(2));
    +        Assert.assertEquals(Optional.of(tupleTs), systemValuesCaptor.getAllValues().get(1));
    +    }
    +
    +    @Test
    +    public void testCacheEviction() {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), Mockito.anyLong())).thenReturn(true);
    +        executor.initState(null);
    +        executor.waterMarkEventGenerator = mockWaterMarkEventGenerator;
    +        List<Tuple> mockTuples = getMockTuples(20000);
    +        mockTuples.forEach(t -> executor.execute(t));
    +
    +        Mockito.verify(mockWindowState, Mockito.times(10)).put(longCaptor.capture(), windowValuesCaptor.capture());
    +        Assert.assertEquals(10, longCaptor.getAllValues().size());
    +        Assert.assertEquals(10, windowValuesCaptor.getAllValues().size());
    +        // number of evicted events
    +        Assert.assertEquals(10_000, windowValuesCaptor.getAllValues().stream()
    +            .mapToInt(x -> x.size()).sum());
    +
    +        Map<Long, PersistentWindowedBoltExecutor.WindowPartition<Event<Tuple>>> partitionMap = new HashMap<>();
    +        windowValuesCaptor.getAllValues().forEach(v -> partitionMap.put(v.getId(), v));
    +
    +        Mockito.verify(mockPartitionState, Mockito.times(20)).put(stringCaptor.capture(), partitionValuesCaptor.capture());
    +        Assert.assertEquals(LongStream.range(0, 20).boxed().collect(Collectors.toList()), partitionValuesCaptor.getAllValues().get(19));
    +
    +        Mockito.when(mockWindowState.get(Mockito.any(), Mockito.any())).then(new Answer<Object>() {
    +            @Override
    +            public Object answer(InvocationOnMock invocation) throws Throwable {
    +                Object[] args = invocation.getArguments();
    +                PersistentWindowedBoltExecutor.WindowPartition<Event<Tuple>> evicted = partitionMap.get(args[0]);
    +                return evicted != null ? evicted : args[1];
    +            }
    +        });
    +
    +        Mockito.doAnswer(new Answer<Void>() {
    +            @Override
    +            public Void answer(InvocationOnMock invocation) throws Throwable {
    +                Object[] args = invocation.getArguments();
    +                partitionMap.put((long)args[0], (PersistentWindowedBoltExecutor.WindowPartition<Event<Tuple>>)args[1]);
    +                return null;
    +            }
    +        }).when(mockWindowState).put(Mockito.any(), Mockito.any());
    +
    +        // trigger the window
    +        long activationTs = tupleTs + 1000;
    +        executor.getWindowManager().add(new WaterMarkEvent<>(activationTs));
    +
    +        Mockito.verify(mockBolt, Mockito.times(4000)).execute(Mockito.any());
    --- End diff --
    
    20,000 events and a count window of 5. Will address by naming the constants.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130528788
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutor.java ---
    @@ -0,0 +1,596 @@
    +/**
    + * 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 java.util.AbstractCollection;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Objects;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentLinkedQueue;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.locks.ReentrantLock;
    +import java.util.function.Supplier;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.state.KeyValueState;
    +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.topology.base.BaseWindowedBolt;
    +import org.apache.storm.tuple.Tuple;
    +import org.apache.storm.windowing.DefaultEvictionContext;
    +import org.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.EventImpl;
    +import org.apache.storm.windowing.WindowLifecycleListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static java.util.Collections.emptyIterator;
    +import static org.apache.storm.topology.WindowPartitionCache.CacheLoader;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalCause;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalListener;
    +
    +/**
    + * Wraps a {@link IStatefulWindowedBolt} and handles the execution. Uses state and the underlying
    + * checkpointing mechanisms to save the tuples in window to state. The tuples are also kept in-memory
    + * by transparently caching the window partitions and checkpointing them as needed.
    + */
    +public class PersistentWindowedBoltExecutor<T extends State> extends WindowedBoltExecutor implements IStatefulBolt<T> {
    +    private static final Logger LOG = LoggerFactory.getLogger(PersistentWindowedBoltExecutor.class);
    +    private final IStatefulWindowedBolt<T> statefulWindowedBolt;
    +    private transient TopologyContext topologyContext;
    +    private transient OutputCollector outputCollector;
    +    private transient WindowState<Tuple> state;
    +    private transient boolean stateInitialized;
    +    private transient boolean prePrepared;
    +
    +    public PersistentWindowedBoltExecutor(IStatefulWindowedBolt<T> bolt) {
    +        super(bolt);
    +        statefulWindowedBolt = bolt;
    +    }
    +
    +    @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));
    +    }
    +
    +    @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);
    +        }
    +    }
    +
    +    private int getCheckpointIntervalMillis(Map<String, Object> topoConf) {
    +        int checkpointInterval = Integer.MAX_VALUE;
    +        if (topoConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL) != null) {
    +            checkpointInterval = ((Number) topoConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL)).intValue();
    +        }
    +        return checkpointInterval;
    +    }
    +
    +    // package access for unit tests
    +    void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                 KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                 KeyValueState<String, Deque<Long>> partitionState,
    +                 KeyValueState<String, Optional<?>> windowSystemState) {
    +        init(topoConf, context, collector, windowState, partitionState, windowSystemState);
    +        doPrepare(topoConf, context, new NoAckOutputCollector(collector), state, true);
    +        Map<String, Optional<?>> wstate = new HashMap<>();
    +        windowSystemState.forEach(s -> wstate.put(s.getKey(), s.getValue()));
    +        restoreState(wstate);
    +    }
    +
    +    @Override
    +    protected void start() {
    +        if (stateInitialized) {
    +            super.start();
    +        } else {
    +            LOG.debug("Will invoke start after state is initialized");
    +        }
    +    }
    +
    +    @Override
    +    public void execute(Tuple input) {
    +        if (!stateInitialized) {
    +            throw new IllegalStateException("execute invoked before initState with input tuple " + input);
    +        }
    +        super.execute(input);
    +        // StatefulBoltExecutor does the actual ack when the state is saved.
    +        outputCollector.ack(input);
    +    }
    +
    +    @Override
    +    public void initState(T state) {
    +        if (stateInitialized) {
    +            String msg = "initState invoked when the state is already initialized";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        } else {
    +            statefulWindowedBolt.initState(state);
    +            stateInitialized = true;
    +            start();
    +        }
    +    }
    +
    +    @Override
    +    public void prePrepare(long txid) {
    +        if (stateInitialized) {
    +            LOG.debug("Prepare streamState, txid {}", txid);
    +            state.prepareCommit(txid);
    +            prePrepared = true;
    +        } else {
    +            String msg = "Cannot prepare before initState";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        }
    +    }
    +
    +    @Override
    +    public void preCommit(long txid) {
    +        // preCommit can be invoked during recovery before the state is initialized
    +        if (prePrepared || !stateInitialized) {
    +            LOG.debug("Commit streamState, txid {}", txid);
    +            state.commit(txid);
    +        } else {
    +            String msg = "preCommit before prePrepare in initialized state";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        }
    +    }
    +
    +    @Override
    +    public void preRollback() {
    +        LOG.debug("Rollback streamState, stateInitialized {}", stateInitialized);
    +        state.rollback();
    +    }
    +
    +    @Override
    +    protected WindowLifecycleListener<Tuple> newWindowLifecycleListener() {
    +        return new WindowLifecycleListener<Tuple>() {
    +            @Override
    +            public void onExpiry(List<Tuple> events) {
    +                /*
    +                 * NO-OP: the events are ack-ed in execute
    +                 */
    +            }
    +
    +            @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();
    +            }
    +        };
    +    }
    +
    +    private void init(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                      KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                      KeyValueState<String, Deque<Long>> partitionState,
    +                      KeyValueState<String, Optional<?>> windowSystemState) {
    +        topologyContext = context;
    +        outputCollector = collector;
    +        state = new WindowState<>(windowState, partitionState, windowSystemState, this::getState,
    +            statefulWindowedBolt.maxEventsInMemory());
    +    }
    +
    +    private KeyValueState<Long, WindowPartition<Tuple>> getWindowState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window";
    +        return (KeyValueState<Long, WindowPartition<Tuple>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Deque<Long>> getPartitionState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-partitions";
    +        return (KeyValueState<String, Deque<Long>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Optional<?>> getWindowSystemState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-systemstate";
    +        return (KeyValueState<String, Optional<?>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    // a wrapper around the window related states that are checkpointed
    +    private static class WindowState<T> extends AbstractCollection<Event<T>> {
    +        // number of events per window-partition
    +        private static final int MAX_PARTITION_EVENTS = 1000;
    +        private static final int MIN_PARTITIONS = 10;
    +        private static final String PARTITION_IDS_KEY = "pk";
    +        private final KeyValueState<String, Deque<Long>> partitionIdsState;
    +        private final KeyValueState<Long, WindowPartition<T>> windowPartitionsState;
    +        private final KeyValueState<String, Optional<?>> windowSystemState;
    +        // ordered partition keys
    +        private Deque<Long> partitionIds;
    +        private volatile long latestPartitionId;
    +        private WindowPartitionCache<Long, WindowPartition<T>> cache;
    +        private Supplier<Map<String, Optional<?>>> windowSystemStateSupplier;
    +        private final ReentrantLock partitionIdsLock = new ReentrantLock(true);
    +        private final ReentrantLock windowPartitionsLock = new ReentrantLock(true);
    +        private final long maxEventsInMemory;
    +        private WindowPartition<T> latestPartition;
    +        private Set<Long> iteratorPins = new HashSet<>();
    +
    +        WindowState(KeyValueState<Long, WindowPartition<T>> windowPartitionsState,
    +                    KeyValueState<String, Deque<Long>> partitionIdsState,
    +                    KeyValueState<String, Optional<?>> windowSystemState,
    +                    Supplier<Map<String, Optional<?>>> windowSystemStateSupplier,
    +                    long maxEventsInMemory) {
    +            this.windowPartitionsState = windowPartitionsState;
    +            this.partitionIdsState = partitionIdsState;
    +            this.windowSystemState = windowSystemState;
    +            this.windowSystemStateSupplier = windowSystemStateSupplier;
    +            this.maxEventsInMemory = Math.max(MAX_PARTITION_EVENTS * MIN_PARTITIONS, maxEventsInMemory);
    +            initCache();
    +            initPartitions();
    +        }
    +
    +        @Override
    +        public boolean add(Event<T> event) {
    +            if (latestPartition.size() >= MAX_PARTITION_EVENTS) {
    +                cache.unpin(latestPartition.getId());
    +                latestPartition = getPinnedPartition(getNextPartitionId());
    +            }
    +            latestPartition.add(event);
    +            return true;
    +        }
    +
    +        @Override
    +        public Iterator<Event<T>> iterator() {
    +
    +            return new Iterator<Event<T>>() {
    +                private Iterator<Long> ids = getIds();
    +                private Iterator<Event<T>> current = emptyIterator();
    +                private Iterator<Event<T>> removeFrom;
    +                private WindowPartition<T> curPartition;
    +
    +                private Iterator<Long> getIds() {
    +                    try {
    +                        partitionIdsLock.lock();
    +                        LOG.debug("Iterator partitionIds: {}", partitionIds);
    +                        return new ArrayList<>(partitionIds).iterator();
    +                    } finally {
    +                        partitionIdsLock.unlock();
    +                    }
    +                }
    +
    +                @Override
    +                public void remove() {
    +                    if (removeFrom == null) {
    +                        throw new IllegalStateException("No calls to next() since last call to remove()");
    +                    }
    +                    removeFrom.remove();
    +                    removeFrom = null;
    +                }
    +
    +                @Override
    +                public boolean hasNext() {
    +                    boolean curHasNext = current.hasNext();
    +                    while (!curHasNext && ids.hasNext()) {
    +                        if (curPartition != null) {
    +                            unpin(curPartition.getId());
    +                        }
    +                        curPartition = getPinnedPartition(ids.next());
    +                        if (curPartition != null) {
    +                            iteratorPins.add(curPartition.getId());
    +                            current = curPartition.iterator();
    +                            curHasNext = current.hasNext();
    +                        }
    +                    }
    +                    // un-pin the last partition
    +                    if (!curHasNext && curPartition != null) {
    +                        unpin(curPartition.getId());
    +                        curPartition = null;
    +                    }
    +                    return curHasNext;
    +                }
    +
    +                @Override
    +                public Event<T> next() {
    +                    if (!hasNext()) {
    +                        throw new NoSuchElementException();
    +                    }
    +                    removeFrom = current;
    +                    return current.next();
    +                }
    +
    +                private void unpin(long id) {
    +                    cache.unpin(id);
    +                    iteratorPins.remove(id);
    +                }
    +            };
    +        }
    +
    +        void clearIteratorPins() {
    +            LOG.debug("clearIteratorPins '{}'", iteratorPins);
    +            Iterator<Long> it = iteratorPins.iterator();
    +            while (it.hasNext()) {
    +                cache.unpin(it.next());
    +                it.remove();
    +            }
    +        }
    +
    +        @Override
    +        public int size() {
    +            throw new UnsupportedOperationException();
    +        }
    +
    +        void prepareCommit(long txid) {
    +            flush();
    +            partitionIdsState.prepareCommit(txid);
    +            windowPartitionsState.prepareCommit(txid);
    +            windowSystemState.prepareCommit(txid);
    +        }
    +
    +        void commit(long txid) {
    +            partitionIdsState.commit(txid);
    +            windowPartitionsState.commit(txid);
    +            windowSystemState.commit(txid);
    +        }
    +
    +        void rollback() {
    +            partitionIdsState.rollback();
    +            windowPartitionsState.rollback();
    +            windowSystemState.rollback();
    +        }
    +
    +        private void initPartitions() {
    +            partitionIds = partitionIdsState.get(PARTITION_IDS_KEY, new LinkedList<>());
    +            if (partitionIds.isEmpty()) {
    +                partitionIds.add(0L);
    +                partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    +            }
    +            latestPartitionId = partitionIds.peekLast();
    +            latestPartition = cache.getPinned(latestPartitionId);
    +        }
    +
    +        private void initCache() {
    +            long size = maxEventsInMemory / MAX_PARTITION_EVENTS;
    +            LOG.info("maxEventsInMemory: {}, partition size: {}, number of partitions: {}",
    +                maxEventsInMemory, MAX_PARTITION_EVENTS, size);
    +            cache = SimpleWindowPartitionCache.<Long, WindowPartition<T>>newBuilder()
    +                .maximumSize(size)
    +                .removalListener(new RemovalListener<Long, WindowPartition<T>>() {
    +                    @Override
    +                    public void onRemoval(Long pid, WindowPartition<T> p, RemovalCause removalCause) {
    +                        Objects.requireNonNull(pid, "Null partition id");
    +                        Objects.requireNonNull(p, "Null window partition");
    +                        LOG.debug("onRemoval for id '{}', WindowPartition '{}'", pid, p);
    +                        try {
    +                            windowPartitionsLock.lock();
    +                            if (p.isEmpty() && pid != latestPartitionId) {
    +                                // if the empty partition was not invalidated by flush, but evicted from cache
    +                                if (removalCause != RemovalCause.EXPLICIT) {
    +                                    deletePartition(pid);
    +                                    windowPartitionsState.delete(pid);
    +                                }
    +                            } else if (p.isModified()) {
    +                                windowPartitionsState.put(pid, p);
    +                            } else {
    +                                LOG.debug("WindowPartition '{}' is not modified", pid);
    +                            }
    +                        } finally {
    +                            windowPartitionsLock.unlock();
    +                        }
    +                    }
    +                }).build(new CacheLoader<Long, WindowPartition<T>>() {
    +                    @Override
    +                   public WindowPartition<T> load(Long id) {
    +                        LOG.debug("Load partition: {}", id);
    +                        // load from state
    +                        try {
    +                            windowPartitionsLock.lock();
    +                            return windowPartitionsState.get(id, new WindowPartition<>(id));
    +                        } finally {
    +                            windowPartitionsLock.unlock();
    +                        }
    +                    }
    +                    }
    +                );
    +        }
    +
    +        private void deletePartition(long pid) {
    +            LOG.debug("Delete partition: {}", pid);
    +            try {
    +                partitionIdsLock.lock();
    +                partitionIds.remove(pid);
    +                partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    --- End diff --
    
    Ok I wrongly assumed the comment was for `initPartitions`, so discard my previous comment. 
    Here we first update the `partitionIds` member variable. The put is to save the value into the kv-state. The partitionIdsState state has a single entry something like `"key" -> List of ids`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r131219950
  
    --- Diff: storm-client/test/jvm/org/apache/storm/topology/SimpleWindowPartitionCacheTest.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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.utils.Utils;
    +import org.apache.storm.windowing.persistence.SimpleWindowPartitionCache;
    +import org.apache.storm.windowing.persistence.WindowPartitionCache;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.concurrent.FutureTask;
    +
    +/**
    + * Unit tests for {@link SimpleWindowPartitionCache}
    + */
    +public class SimpleWindowPartitionCacheTest {
    +
    +    @Before
    +    public void setUp() throws Exception {
    +    }
    +
    +    @Test(expected = IllegalArgumentException.class)
    +    public void testBuildInvalid1() throws Exception {
    +        SimpleWindowPartitionCache.<Integer, Integer>newBuilder()
    +            .maximumSize(0)
    +            .build(null);
    +    }
    +
    +    @Test(expected = IllegalArgumentException.class)
    +    public void testBuildInvalid2() throws Exception {
    +        SimpleWindowPartitionCache.<Integer, Integer>newBuilder()
    +            .maximumSize(-1)
    +            .build(null);
    +    }
    +
    +    @Test(expected = NullPointerException.class)
    +    public void testBuildInvalid3() throws Exception {
    +        SimpleWindowPartitionCache.<Integer, Integer>newBuilder()
    +            .maximumSize(1)
    +            .build(null);
    +    }
    +
    +    @Test
    +    public void testBuildOk() throws Exception {
    +        SimpleWindowPartitionCache.<Integer, Integer>newBuilder()
    +            .maximumSize(1)
    +            .removalListener((key, val, removalCause) -> {
    +            })
    +            .build(key -> key);
    +    }
    +
    +    @Test
    +    public void testGet() throws Exception {
    +        List<Integer> removed = new ArrayList<>();
    +        List<Integer> loaded = new ArrayList<>();
    +        SimpleWindowPartitionCache<Integer, Integer> cache =
    +            SimpleWindowPartitionCache.<Integer, Integer>newBuilder()
    +                .maximumSize(2)
    +                .removalListener((key, val, removalCause) -> removed.add(key))
    +                .build(key -> {
    +                    loaded.add(key);
    +                    return key;
    +                });
    +
    +        cache.get(1);
    +        cache.get(2);
    +        cache.get(3);
    +        Assert.assertEquals(Arrays.asList(1, 2, 3), loaded);
    +        Assert.assertEquals(Collections.singletonList(2), removed);
    +    }
    +
    +    @Test(expected = NullPointerException.class)
    +    public void testGetNull() throws Exception {
    +        SimpleWindowPartitionCache<Integer, Integer> cache =
    +            SimpleWindowPartitionCache.<Integer, Integer>newBuilder()
    +                .maximumSize(2)
    +                .build(key -> null);
    +
    +        cache.get(1);
    +    }
    +
    +    @Test
    +    public void testEvictNoRemovalListener() throws Exception {
    +        SimpleWindowPartitionCache<Integer, Integer> cache =
    +            SimpleWindowPartitionCache.<Integer, Integer>newBuilder()
    +                .maximumSize(1)
    +                .build(key -> {
    +                    return key;
    +                });
    +        cache.get(1);
    +        cache.get(2);
    +        Assert.assertEquals(Collections.singletonMap(2, 2), cache.asMap());
    +        cache.invalidate(2);
    +        Assert.assertEquals(Collections.emptyMap(), cache.asMap());
    +    }
    +
    +    @Test
    +    public void testPinAndGet() throws Exception {
    +        List<Integer> removed = new ArrayList<>();
    +        List<Integer> loaded = new ArrayList<>();
    +        SimpleWindowPartitionCache<Integer, Integer> cache =
    +            SimpleWindowPartitionCache.<Integer, Integer>newBuilder()
    +                .maximumSize(1)
    +                .removalListener(new WindowPartitionCache.RemovalListener<Integer, Integer>() {
    +                    @Override
    +                    public void onRemoval(Integer key, Integer val, WindowPartitionCache.RemovalCause removalCause) {
    +                        removed.add(key);
    +                    }
    +                })
    +                .build(new WindowPartitionCache.CacheLoader<Integer, Integer>() {
    +                    @Override
    +                    public Integer load(Integer key) {
    +                        loaded.add(key);
    +                        return key;
    +                    }
    +                });
    +
    +        cache.get(1);
    +        cache.pinAndGet(2);
    +        cache.get(3);
    +        Assert.assertEquals(Arrays.asList(1, 2, 3), loaded);
    +        Assert.assertEquals(Collections.singletonList(1), removed);
    +    }
    +
    +    @Test
    +    public void testInvalidate() throws Exception {
    +        List<Integer> removed = new ArrayList<>();
    +        List<Integer> loaded = new ArrayList<>();
    +        SimpleWindowPartitionCache<Integer, Integer> cache =
    +            SimpleWindowPartitionCache.<Integer, Integer>newBuilder()
    +                .maximumSize(1)
    +                .removalListener((key, val, removalCause) -> removed.add(key))
    +                .build(key -> {
    +                    loaded.add(key);
    +                    return key;
    +                });
    +
    +        cache.pinAndGet(1);
    +        cache.invalidate(1);
    +        Assert.assertEquals(Collections.singletonList(1), loaded);
    +        Assert.assertEquals(Collections.emptyList(), removed);
    +        Assert.assertEquals(cache.asMap(), Collections.singletonMap(1, 1));
    +
    +        cache.unpin(1);
    +        cache.invalidate(1);
    +        Assert.assertTrue(cache.asMap().isEmpty());
    +    }
    +
    +
    +    @Test
    +    public void testConcurrentGet() throws Exception {
    +        List<Integer> loaded = new ArrayList<>();
    +        SimpleWindowPartitionCache<Integer, Object> cache =
    +            SimpleWindowPartitionCache.<Integer, Object>newBuilder()
    +                .maximumSize(1)
    +                .build(key -> {
    +                    Utils.sleep(1000);
    +                    loaded.add(key);
    +                    return new Object();
    +                });
    +
    +        FutureTask<Object> ft1 = new FutureTask<>(() -> cache.pinAndGet(1));
    +        FutureTask<Object> ft2 = new FutureTask<>(() -> cache.pinAndGet(1));
    +        Thread t1 = new Thread(ft1);
    +        Thread t2 = new Thread(ft2);
    +        t1.start();
    +        t2.start();
    +        t1.join();
    --- End diff --
    
    Added


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130227685
  
    --- Diff: storm-client/test/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutorTest.java ---
    @@ -0,0 +1,261 @@
    +/**
    + * 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.Config;
    +import org.apache.storm.generated.GlobalStreamId;
    +import org.apache.storm.state.KeyValueState;
    +import org.apache.storm.streams.Pair;
    +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.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.TimestampExtractor;
    +import org.apache.storm.windowing.TupleWindow;
    +import org.apache.storm.windowing.WaterMarkEvent;
    +import org.apache.storm.windowing.WaterMarkEventGenerator;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.mockito.ArgumentCaptor;
    +import org.mockito.Captor;
    +import org.mockito.Mockito;
    +import org.mockito.MockitoAnnotations;
    +import org.mockito.invocation.InvocationOnMock;
    +import org.mockito.runners.MockitoJUnitRunner;
    +import org.mockito.stubbing.Answer;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.stream.Collectors;
    +import java.util.stream.LongStream;
    +
    +import static org.mockito.AdditionalAnswers.returnsArgAt;
    +
    +/**
    + * Unit tests for {@link PersistentWindowedBoltExecutor}
    + */
    +@RunWith(MockitoJUnitRunner.class)
    +public class PersistentWindowedBoltExecutorTest {
    +    public static final String LATE_STREAM = "late_stream";
    +    private PersistentWindowedBoltExecutor<KeyValueState<String, String>> executor;
    +    private IStatefulWindowedBolt<KeyValueState<String, String>> mockBolt;
    +    private OutputCollector mockOutputCollector;
    +    private TopologyContext mockTopologyContext;
    +    private Map<String, Object> mockStormConf = new HashMap<>();
    +    private TimestampExtractor mockTimestampExtractor;
    +    private WaterMarkEventGenerator mockWaterMarkEventGenerator;
    +    private KeyValueState mockPartitionState;
    +    private KeyValueState mockWindowState;
    +    private KeyValueState mockSystemState;
    +
    +    private static final String PARTITION_KEY = "pk";
    +    private static final String EVICTION_STATE_KEY = "es";
    +    private static final String TRIGGER_STATE_KEY = "ts";
    +    private long tupleTs;
    +
    +    @Captor
    +    private ArgumentCaptor<Tuple> tupleCaptor;
    +    @Captor
    +    private ArgumentCaptor<Collection<Tuple>> anchorCaptor;
    +    @Captor
    +    private ArgumentCaptor<String> stringCaptor;
    +    @Captor
    +    private ArgumentCaptor<Long> longCaptor;
    +    @Captor
    +    private ArgumentCaptor<Values> valuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<TupleWindow> tupleWindowCaptor;
    +    @Captor
    +    private ArgumentCaptor<Deque<Long>> partitionValuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<PersistentWindowedBoltExecutor.WindowPartition<Event<Tuple>>> windowValuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<Object> systemValuesCaptor;
    +
    +    @Before
    +    public void setUp() throws Exception {
    +        MockitoAnnotations.initMocks(this);
    +        mockBolt = Mockito.mock(IStatefulWindowedBolt.class);
    +        mockWaterMarkEventGenerator = Mockito.mock(WaterMarkEventGenerator.class);
    +        mockTimestampExtractor = Mockito.mock(TimestampExtractor.class);
    +        tupleTs = System.currentTimeMillis();
    +        Mockito.when(mockTimestampExtractor.extractTimestamp(Mockito.any())).thenReturn(tupleTs);
    +        Mockito.when(mockBolt.getTimestampExtractor()).thenReturn(mockTimestampExtractor);
    +        Mockito.when(mockBolt.isPersistent()).thenReturn(true);
    +        mockPartitionState = Mockito.mock(KeyValueState.class);
    +        mockWindowState = Mockito.mock(KeyValueState.class);
    +        mockSystemState = Mockito.mock(KeyValueState.class);
    +        mockTopologyContext = Mockito.mock(TopologyContext.class);
    +        Mockito.when(mockTopologyContext.getThisStreams()).thenReturn(Collections.singleton(LATE_STREAM));
    +        mockOutputCollector = Mockito.mock(OutputCollector.class);
    +        executor = new PersistentWindowedBoltExecutor<>(mockBolt);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_WINDOW_LENGTH_COUNT, 5);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_SLIDING_INTERVAL_COUNT, 5);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_LATE_TUPLE_STREAM, LATE_STREAM);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_WATERMARK_EVENT_INTERVAL_MS, 100_000);
    +        mockStormConf.put(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS, 30);
    +        mockStormConf.put(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL, 1000);
    +        Mockito.when(mockPartitionState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(mockWindowState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(mockSystemState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        executor.prepare(mockStormConf, mockTopologyContext, mockOutputCollector,
    +            mockWindowState, mockPartitionState, mockSystemState);
    +    }
    +
    +    @Test
    +    public void testExecuteTuple() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), 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);
    +    }
    +
    +    @Test
    +    public void testExecuteLatetuple() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), 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
    +        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());
    +    }
    +
    +    @Test
    +    public void testActivation() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), Mockito.anyLong())).thenReturn(true);
    +        executor.initState(null);
    +        executor.waterMarkEventGenerator = mockWaterMarkEventGenerator;
    +
    +        List<Tuple> mockTuples = getMockTuples(5);
    +        mockTuples.forEach(t -> executor.execute(t));
    +        // all tuples acked
    +        Mockito.verify(mockOutputCollector, Mockito.times(5)).ack(tupleCaptor.capture());
    +        Assert.assertArrayEquals(mockTuples.toArray(), tupleCaptor.getAllValues().toArray());
    +
    +        Mockito.doAnswer(new Answer<Void>() {
    +            @Override
    +            public Void answer(InvocationOnMock invocation) throws Throwable {
    +                TupleWindow window = (TupleWindow) invocation.getArguments()[0];
    +                // iterate the tuples
    +                Assert.assertEquals(5, window.get().size());
    +                // iterating multiple times should produce same events
    +                Assert.assertEquals(5, window.get().size());
    +                Assert.assertEquals(5, window.get().size());
    +                return null;
    +            }
    +        }).when(mockBolt).execute(Mockito.any());
    +        // trigger the window
    +        long activationTs = tupleTs + 1000;
    +        executor.getWindowManager().add(new WaterMarkEvent<>(activationTs));
    +        executor.prePrepare(0);
    +
    +        // partition ids
    +        Mockito.verify(mockPartitionState, Mockito.times(1)).put(stringCaptor.capture(), partitionValuesCaptor.capture());
    +        Assert.assertEquals(PARTITION_KEY, stringCaptor.getAllValues().get(0));
    +        Assert.assertEquals(Collections.singletonList(0L), partitionValuesCaptor.getAllValues().get(0));
    +
    +        // window partitions
    +        Mockito.verify(mockWindowState, Mockito.times(1)).put(longCaptor.capture(), windowValuesCaptor.capture());
    +        Assert.assertEquals(0L, (long) longCaptor.getAllValues().get(0));
    +        Assert.assertEquals(5, windowValuesCaptor.getAllValues().get(0).size());
    +        List<Event<?>> tuples = windowValuesCaptor.getAllValues().get(0)
    +            .getEvents().stream().map(Event::get).collect(Collectors.toList());
    +        Assert.assertArrayEquals(mockTuples.toArray(), tuples.toArray());
    +
    +        // window system state
    +        Mockito.verify(mockSystemState, Mockito.times(2)).put(stringCaptor.capture(), systemValuesCaptor.capture());
    +        Assert.assertEquals(EVICTION_STATE_KEY, stringCaptor.getAllValues().get(1));
    +        Assert.assertEquals(Optional.of(Pair.of(5L, 5L)), systemValuesCaptor.getAllValues().get(0));
    +        Assert.assertEquals(TRIGGER_STATE_KEY, stringCaptor.getAllValues().get(2));
    +        Assert.assertEquals(Optional.of(tupleTs), systemValuesCaptor.getAllValues().get(1));
    +    }
    +
    +    @Test
    +    public void testCacheEviction() {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), Mockito.anyLong())).thenReturn(true);
    +        executor.initState(null);
    +        executor.waterMarkEventGenerator = mockWaterMarkEventGenerator;
    +        List<Tuple> mockTuples = getMockTuples(20000);
    +        mockTuples.forEach(t -> executor.execute(t));
    +
    +        Mockito.verify(mockWindowState, Mockito.times(10)).put(longCaptor.capture(), windowValuesCaptor.capture());
    +        Assert.assertEquals(10, longCaptor.getAllValues().size());
    --- End diff --
    
    You already know this is the case because of the verify


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129647014
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/SimpleWindowPartitionCache.java ---
    @@ -0,0 +1,191 @@
    +/**
    + * 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 java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.concurrent.ConcurrentMap;
    +import java.util.concurrent.ConcurrentSkipListMap;
    +import java.util.concurrent.locks.ReentrantLock;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * A simple implementation that evicts the largest un-pinned entry from the cache. This works well
    + * for caching window partitions since the access pattern is mostly sequential scans.
    + */
    +public class SimpleWindowPartitionCache<K, V> implements WindowPartitionCache<K, V> {
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleWindowPartitionCache.class);
    +
    +    private final ConcurrentSkipListMap<K, V> map = new ConcurrentSkipListMap<>();
    +    private final Map<K, Long> pinned = new HashMap<>();
    +    private final long maximumSize;
    +    private final RemovalListener<K, V> removalListener;
    +    private final CacheLoader<K, V> cacheLoader;
    +    private final ReentrantLock lock = new ReentrantLock(true);
    +    private int size;
    +
    +    @Override
    +    public V get(K key) {
    +        return getOrLoad(key, false);
    +    }
    +
    +    @Override
    +    public V getPinned(K key) {
    +        return getOrLoad(key, true);
    +    }
    +
    +    @Override
    +    public boolean unpin(K key) {
    +        LOG.debug("unpin '{}'", key);
    +        boolean res = false;
    +        try {
    +            lock.lock();
    +            if (pinned.compute(key, (k, v) -> v == null ? 0 : v - 1) <= 0) {
    +                pinned.remove(key);
    +                res = true;
    +            }
    +        } finally {
    +            lock.unlock();
    +        }
    +        LOG.debug("pinned '{}'", pinned);
    +        return res;
    +    }
    +
    +    @Override
    +    public ConcurrentMap<K, V> asMap() {
    +        return map;
    +    }
    +
    +    @Override
    +    public void invalidate(K key) {
    +        try {
    +            lock.lock();
    +            if (isPinned(key)) {
    +                LOG.debug("Entry '{}' is pinned, skipping invalidation", key);
    +            } else {
    +                LOG.debug("Invalidating entry '{}'", key);
    +                V val = map.remove(key);
    +                if (val != null) {
    +                    --size;
    +                    pinned.remove(key);
    +                    removalListener.onRemoval(key, val, RemovalCause.EXPLICIT);
    +                }
    +            }
    +        } finally {
    +            lock.unlock();
    +        }
    +    }
    +
    +    // Get or load from the cache optionally pinning the entry
    +    // so that it wont get evicted from the cache
    +    private V getOrLoad(K key, boolean shouldPin) {
    +        V val;
    +        if (shouldPin) {
    +            try {
    +                lock.lock();
    +                val = load(key);
    +                pin(key);
    +            } finally {
    +                lock.unlock();
    +            }
    +        } else {
    +            val = map.get(key);
    +            if (val == null) {
    +                try {
    +                    lock.lock();
    +                    val = load(key);
    +                } finally {
    +                    lock.unlock();
    +                }
    +            }
    +        }
    +
    +        return val;
    +    }
    +
    +    private V load(K key) {
    +        V val = map.get(key);
    +        if (val == null) {
    +            val = cacheLoader.load(key);
    +            if (val == null) {
    +                throw new NullPointerException("Null value");
    --- End diff --
    
    Nit: Consider adding the key to the message for easier debugging


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129139943
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutor.java ---
    @@ -0,0 +1,563 @@
    +/**
    + * 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 java.util.AbstractCollection;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentLinkedQueue;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.locks.ReentrantLock;
    +import java.util.function.Supplier;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.state.KeyValueState;
    +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.windowing.DefaultEvictionContext;
    +import org.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.EventImpl;
    +import org.apache.storm.windowing.WindowLifecycleListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static java.util.Collections.emptyIterator;
    +import static org.apache.storm.topology.WindowPartitionCache.CacheLoader;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalCause;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalListener;
    +
    +/**
    + * Wraps a {@link IStatefulWindowedBolt} and handles the execution. Uses state and the underlying
    + * checkpointing mechanisms to save the tuples in window to state. The tuples are also kept in-memory
    + * by transparently caching the window partitions and checkpointing them as needed.
    + */
    +public class PersistentWindowedBoltExecutor<T extends State> extends WindowedBoltExecutor implements IStatefulBolt<T> {
    +    private static final Logger LOG = LoggerFactory.getLogger(PersistentWindowedBoltExecutor.class);
    +    private final IStatefulWindowedBolt<T> statefulWindowedBolt;
    +    private transient TopologyContext topologyContext;
    +    private transient OutputCollector outputCollector;
    +    private transient WindowState<Tuple> state;
    +    private transient boolean stateInitialized;
    +    private transient boolean prePrepared;
    +
    +    public PersistentWindowedBoltExecutor(IStatefulWindowedBolt<T> bolt) {
    +        super(bolt);
    +        statefulWindowedBolt = bolt;
    +    }
    +
    +    @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));
    +    }
    +
    +    // package access for unit tests
    +    void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                 KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                 KeyValueState<String, Deque<Long>> partitionState,
    +                 KeyValueState<String, Optional<?>> windowSystemState) {
    +        init(topoConf, context, collector, windowState, partitionState, windowSystemState);
    +        doPrepare(topoConf, context, new NoAckOutputCollector(collector), state, true);
    +        Map<String, Optional<?>> wstate = new HashMap<>();
    +        windowSystemState.forEach(s -> wstate.put(s.getKey(), s.getValue()));
    +        restoreState(wstate);
    +    }
    +
    +    @Override
    +    protected void start() {
    +        if (stateInitialized) {
    +            super.start();
    +        } else {
    +            LOG.debug("Will invoke start after state is initialized");
    +        }
    +    }
    +
    +    @Override
    +    public void execute(Tuple input) {
    +        if (!stateInitialized) {
    +            throw new IllegalStateException("execute invoked before initState with input tuple " + input);
    +        }
    +        super.execute(input);
    +        // StatefulBoltExecutor does the actual ack when the state is saved.
    +        outputCollector.ack(input);
    +    }
    +
    +    @Override
    +    public void initState(T state) {
    +        if (stateInitialized) {
    +            LOG.warn("State is already initialized. Ignoring initState");
    --- End diff --
    
    Is this something that can happen during normal operation? If not isn't it better to error out, so we can catch and fix the bug? The same goes for the other methods that make this check.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129652453
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutor.java ---
    @@ -0,0 +1,563 @@
    +/**
    + * 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 java.util.AbstractCollection;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentLinkedQueue;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.locks.ReentrantLock;
    +import java.util.function.Supplier;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.state.KeyValueState;
    +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.windowing.DefaultEvictionContext;
    +import org.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.EventImpl;
    +import org.apache.storm.windowing.WindowLifecycleListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static java.util.Collections.emptyIterator;
    +import static org.apache.storm.topology.WindowPartitionCache.CacheLoader;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalCause;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalListener;
    +
    +/**
    + * Wraps a {@link IStatefulWindowedBolt} and handles the execution. Uses state and the underlying
    + * checkpointing mechanisms to save the tuples in window to state. The tuples are also kept in-memory
    + * by transparently caching the window partitions and checkpointing them as needed.
    + */
    +public class PersistentWindowedBoltExecutor<T extends State> extends WindowedBoltExecutor implements IStatefulBolt<T> {
    +    private static final Logger LOG = LoggerFactory.getLogger(PersistentWindowedBoltExecutor.class);
    +    private final IStatefulWindowedBolt<T> statefulWindowedBolt;
    +    private transient TopologyContext topologyContext;
    +    private transient OutputCollector outputCollector;
    +    private transient WindowState<Tuple> state;
    +    private transient boolean stateInitialized;
    +    private transient boolean prePrepared;
    +
    +    public PersistentWindowedBoltExecutor(IStatefulWindowedBolt<T> bolt) {
    +        super(bolt);
    +        statefulWindowedBolt = bolt;
    +    }
    +
    +    @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));
    +    }
    +
    +    // package access for unit tests
    +    void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                 KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                 KeyValueState<String, Deque<Long>> partitionState,
    +                 KeyValueState<String, Optional<?>> windowSystemState) {
    +        init(topoConf, context, collector, windowState, partitionState, windowSystemState);
    +        doPrepare(topoConf, context, new NoAckOutputCollector(collector), state, true);
    +        Map<String, Optional<?>> wstate = new HashMap<>();
    +        windowSystemState.forEach(s -> wstate.put(s.getKey(), s.getValue()));
    +        restoreState(wstate);
    +    }
    +
    +    @Override
    +    protected void start() {
    +        if (stateInitialized) {
    +            super.start();
    +        } else {
    +            LOG.debug("Will invoke start after state is initialized");
    +        }
    +    }
    +
    +    @Override
    +    public void execute(Tuple input) {
    +        if (!stateInitialized) {
    +            throw new IllegalStateException("execute invoked before initState with input tuple " + input);
    +        }
    +        super.execute(input);
    +        // StatefulBoltExecutor does the actual ack when the state is saved.
    +        outputCollector.ack(input);
    +    }
    +
    +    @Override
    +    public void initState(T state) {
    +        if (stateInitialized) {
    +            LOG.warn("State is already initialized. Ignoring initState");
    +        } else {
    +            statefulWindowedBolt.initState(state);
    +            stateInitialized = true;
    +            start();
    +        }
    +    }
    +
    +    @Override
    +    public void prePrepare(long txid) {
    +        if (stateInitialized) {
    +            LOG.debug("Prepare streamState, txid {}", txid);
    +            state.prepareCommit(txid);
    +            prePrepared = true;
    +        } else {
    +            LOG.warn("Cannot prepare before initState");
    +        }
    +    }
    +
    +    @Override
    +    public void preCommit(long txid) {
    +        if (prePrepared) {
    +            LOG.debug("Commit streamState, txid {}", txid);
    +            state.commit(txid);
    +        } else {
    +            LOG.debug("Ignoring preCommit and not committing streamState.");
    +        }
    +    }
    +
    +    @Override
    +    public void preRollback() {
    +        LOG.debug("Rollback streamState, stateInitialized {}", stateInitialized);
    +        state.rollback();
    +    }
    +
    +    @Override
    +    protected WindowLifecycleListener<Tuple> newWindowLifecycleListener() {
    +        return new WindowLifecycleListener<Tuple>() {
    +            @Override
    +            public void onExpiry(List<Tuple> events) {
    +                /*
    +                 * NO-OP: the events are ack-ed in execute
    +                 */
    +            }
    +
    +            @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();
    +            }
    +        };
    +    }
    +
    +    private void init(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                      KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                      KeyValueState<String, Deque<Long>> partitionState,
    +                      KeyValueState<String, Optional<?>> windowSystemState) {
    +        topologyContext = context;
    +        outputCollector = collector;
    +        state = new WindowState<>(windowState, partitionState, windowSystemState, this::getState,
    +            statefulWindowedBolt.maxEventsInMemory());
    +    }
    +
    +    private KeyValueState<Long, WindowPartition<Tuple>> getWindowState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window";
    +        return (KeyValueState<Long, WindowPartition<Tuple>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Deque<Long>> getPartitionState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-partitions";
    +        return (KeyValueState<String, Deque<Long>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Optional<?>> getWindowSystemState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-systemstate";
    +        return (KeyValueState<String, Optional<?>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    // a wrapper around the window related states that are checkpointed
    +    private static class WindowState<T> extends AbstractCollection<Event<T>> {
    +        // number of events per window-partition
    +        private static final int PARTITION_SZ = 1000;
    --- End diff --
    
    Nit: Rename to MAX_PARTITION_EVENTS or something similar, the current name doesn't mention what unit the size is


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130227794
  
    --- Diff: storm-client/test/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutorTest.java ---
    @@ -0,0 +1,261 @@
    +/**
    + * 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.Config;
    +import org.apache.storm.generated.GlobalStreamId;
    +import org.apache.storm.state.KeyValueState;
    +import org.apache.storm.streams.Pair;
    +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.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.TimestampExtractor;
    +import org.apache.storm.windowing.TupleWindow;
    +import org.apache.storm.windowing.WaterMarkEvent;
    +import org.apache.storm.windowing.WaterMarkEventGenerator;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.mockito.ArgumentCaptor;
    +import org.mockito.Captor;
    +import org.mockito.Mockito;
    +import org.mockito.MockitoAnnotations;
    +import org.mockito.invocation.InvocationOnMock;
    +import org.mockito.runners.MockitoJUnitRunner;
    +import org.mockito.stubbing.Answer;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.stream.Collectors;
    +import java.util.stream.LongStream;
    +
    +import static org.mockito.AdditionalAnswers.returnsArgAt;
    +
    +/**
    + * Unit tests for {@link PersistentWindowedBoltExecutor}
    + */
    +@RunWith(MockitoJUnitRunner.class)
    +public class PersistentWindowedBoltExecutorTest {
    +    public static final String LATE_STREAM = "late_stream";
    +    private PersistentWindowedBoltExecutor<KeyValueState<String, String>> executor;
    +    private IStatefulWindowedBolt<KeyValueState<String, String>> mockBolt;
    +    private OutputCollector mockOutputCollector;
    +    private TopologyContext mockTopologyContext;
    +    private Map<String, Object> mockStormConf = new HashMap<>();
    +    private TimestampExtractor mockTimestampExtractor;
    +    private WaterMarkEventGenerator mockWaterMarkEventGenerator;
    +    private KeyValueState mockPartitionState;
    +    private KeyValueState mockWindowState;
    +    private KeyValueState mockSystemState;
    +
    +    private static final String PARTITION_KEY = "pk";
    +    private static final String EVICTION_STATE_KEY = "es";
    +    private static final String TRIGGER_STATE_KEY = "ts";
    +    private long tupleTs;
    +
    +    @Captor
    +    private ArgumentCaptor<Tuple> tupleCaptor;
    +    @Captor
    +    private ArgumentCaptor<Collection<Tuple>> anchorCaptor;
    +    @Captor
    +    private ArgumentCaptor<String> stringCaptor;
    +    @Captor
    +    private ArgumentCaptor<Long> longCaptor;
    +    @Captor
    +    private ArgumentCaptor<Values> valuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<TupleWindow> tupleWindowCaptor;
    +    @Captor
    +    private ArgumentCaptor<Deque<Long>> partitionValuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<PersistentWindowedBoltExecutor.WindowPartition<Event<Tuple>>> windowValuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<Object> systemValuesCaptor;
    +
    +    @Before
    +    public void setUp() throws Exception {
    +        MockitoAnnotations.initMocks(this);
    +        mockBolt = Mockito.mock(IStatefulWindowedBolt.class);
    +        mockWaterMarkEventGenerator = Mockito.mock(WaterMarkEventGenerator.class);
    +        mockTimestampExtractor = Mockito.mock(TimestampExtractor.class);
    +        tupleTs = System.currentTimeMillis();
    +        Mockito.when(mockTimestampExtractor.extractTimestamp(Mockito.any())).thenReturn(tupleTs);
    +        Mockito.when(mockBolt.getTimestampExtractor()).thenReturn(mockTimestampExtractor);
    +        Mockito.when(mockBolt.isPersistent()).thenReturn(true);
    +        mockPartitionState = Mockito.mock(KeyValueState.class);
    +        mockWindowState = Mockito.mock(KeyValueState.class);
    +        mockSystemState = Mockito.mock(KeyValueState.class);
    +        mockTopologyContext = Mockito.mock(TopologyContext.class);
    +        Mockito.when(mockTopologyContext.getThisStreams()).thenReturn(Collections.singleton(LATE_STREAM));
    +        mockOutputCollector = Mockito.mock(OutputCollector.class);
    +        executor = new PersistentWindowedBoltExecutor<>(mockBolt);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_WINDOW_LENGTH_COUNT, 5);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_SLIDING_INTERVAL_COUNT, 5);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_LATE_TUPLE_STREAM, LATE_STREAM);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_WATERMARK_EVENT_INTERVAL_MS, 100_000);
    +        mockStormConf.put(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS, 30);
    +        mockStormConf.put(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL, 1000);
    +        Mockito.when(mockPartitionState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(mockWindowState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(mockSystemState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        executor.prepare(mockStormConf, mockTopologyContext, mockOutputCollector,
    +            mockWindowState, mockPartitionState, mockSystemState);
    +    }
    +
    +    @Test
    +    public void testExecuteTuple() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), 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);
    +    }
    +
    +    @Test
    +    public void testExecuteLatetuple() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), 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
    +        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());
    +    }
    +
    +    @Test
    +    public void testActivation() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), Mockito.anyLong())).thenReturn(true);
    +        executor.initState(null);
    +        executor.waterMarkEventGenerator = mockWaterMarkEventGenerator;
    +
    +        List<Tuple> mockTuples = getMockTuples(5);
    +        mockTuples.forEach(t -> executor.execute(t));
    +        // all tuples acked
    +        Mockito.verify(mockOutputCollector, Mockito.times(5)).ack(tupleCaptor.capture());
    +        Assert.assertArrayEquals(mockTuples.toArray(), tupleCaptor.getAllValues().toArray());
    +
    +        Mockito.doAnswer(new Answer<Void>() {
    +            @Override
    +            public Void answer(InvocationOnMock invocation) throws Throwable {
    +                TupleWindow window = (TupleWindow) invocation.getArguments()[0];
    +                // iterate the tuples
    +                Assert.assertEquals(5, window.get().size());
    +                // iterating multiple times should produce same events
    +                Assert.assertEquals(5, window.get().size());
    +                Assert.assertEquals(5, window.get().size());
    +                return null;
    +            }
    +        }).when(mockBolt).execute(Mockito.any());
    +        // trigger the window
    +        long activationTs = tupleTs + 1000;
    +        executor.getWindowManager().add(new WaterMarkEvent<>(activationTs));
    +        executor.prePrepare(0);
    +
    +        // partition ids
    +        Mockito.verify(mockPartitionState, Mockito.times(1)).put(stringCaptor.capture(), partitionValuesCaptor.capture());
    +        Assert.assertEquals(PARTITION_KEY, stringCaptor.getAllValues().get(0));
    +        Assert.assertEquals(Collections.singletonList(0L), partitionValuesCaptor.getAllValues().get(0));
    +
    +        // window partitions
    +        Mockito.verify(mockWindowState, Mockito.times(1)).put(longCaptor.capture(), windowValuesCaptor.capture());
    +        Assert.assertEquals(0L, (long) longCaptor.getAllValues().get(0));
    +        Assert.assertEquals(5, windowValuesCaptor.getAllValues().get(0).size());
    +        List<Event<?>> tuples = windowValuesCaptor.getAllValues().get(0)
    +            .getEvents().stream().map(Event::get).collect(Collectors.toList());
    +        Assert.assertArrayEquals(mockTuples.toArray(), tuples.toArray());
    +
    +        // window system state
    +        Mockito.verify(mockSystemState, Mockito.times(2)).put(stringCaptor.capture(), systemValuesCaptor.capture());
    +        Assert.assertEquals(EVICTION_STATE_KEY, stringCaptor.getAllValues().get(1));
    +        Assert.assertEquals(Optional.of(Pair.of(5L, 5L)), systemValuesCaptor.getAllValues().get(0));
    +        Assert.assertEquals(TRIGGER_STATE_KEY, stringCaptor.getAllValues().get(2));
    +        Assert.assertEquals(Optional.of(tupleTs), systemValuesCaptor.getAllValues().get(1));
    +    }
    +
    +    @Test
    +    public void testCacheEviction() {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), Mockito.anyLong())).thenReturn(true);
    +        executor.initState(null);
    +        executor.waterMarkEventGenerator = mockWaterMarkEventGenerator;
    +        List<Tuple> mockTuples = getMockTuples(20000);
    +        mockTuples.forEach(t -> executor.execute(t));
    +
    +        Mockito.verify(mockWindowState, Mockito.times(10)).put(longCaptor.capture(), windowValuesCaptor.capture());
    +        Assert.assertEquals(10, longCaptor.getAllValues().size());
    +        Assert.assertEquals(10, windowValuesCaptor.getAllValues().size());
    +        // number of evicted events
    +        Assert.assertEquals(10_000, windowValuesCaptor.getAllValues().stream()
    +            .mapToInt(x -> x.size()).sum());
    +
    +        Map<Long, PersistentWindowedBoltExecutor.WindowPartition<Event<Tuple>>> partitionMap = new HashMap<>();
    +        windowValuesCaptor.getAllValues().forEach(v -> partitionMap.put(v.getId(), v));
    +
    +        Mockito.verify(mockPartitionState, Mockito.times(20)).put(stringCaptor.capture(), partitionValuesCaptor.capture());
    --- End diff --
    
    Also refer to the constants here. 20 is pretty magical at first glance


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130228711
  
    --- Diff: storm-client/src/jvm/org/apache/storm/windowing/TupleWindowIterImpl.java ---
    @@ -0,0 +1,80 @@
    +/**
    + * 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.windowing;
    +
    +import com.google.common.collect.Iterators;
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.function.Supplier;
    +import org.apache.storm.tuple.Tuple;
    +
    +/**
    + * An iterator based implementation over the events in a window.
    + */
    +public class TupleWindowIterImpl implements TupleWindow {
    +    private final Supplier<Iterator<Tuple>> tuplesIt;
    +    private final Supplier<Iterator<Tuple>> newTuplesIt;
    +    private final Supplier<Iterator<Tuple>> expiredTuplesIt;
    +    private final Long startTimestamp;
    +    private final Long endTimestamp;
    +
    +    public TupleWindowIterImpl(Supplier<Iterator<Tuple>> tuplesIt,
    +                               Supplier<Iterator<Tuple>> newTuplesIt,
    +                               Supplier<Iterator<Tuple>> expiredTuplesIt,
    +                               Long startTimestamp, Long endTimestamp) {
    +        this.tuplesIt = tuplesIt;
    +        this.newTuplesIt = newTuplesIt;
    +        this.expiredTuplesIt = expiredTuplesIt;
    +        this.startTimestamp = startTimestamp;
    +        this.endTimestamp = endTimestamp;
    +    }
    +
    +    @Override
    +    public List<Tuple> get() {
    --- End diff --
    
    Doesn't this put the entire window in memory?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #2218: STORM-2614: Enhance stateful windowing to persist the win...

Posted by roshannaik <gi...@git.apache.org>.
Github user roshannaik commented on the issue:

    https://github.com/apache/storm/pull/2218
  
    Sounds like a checkpoint mechanism limited to windowed bolts. Could you provide a design document for this ?  


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129111624
  
    --- Diff: examples/storm-starter/src/jvm/org/apache/storm/starter/PersistentWindowingTopology.java ---
    @@ -0,0 +1,176 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.storm.starter;
    +
    +import static org.apache.storm.topology.base.BaseWindowedBolt.Duration;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.concurrent.TimeUnit;
    +import org.apache.storm.Config;
    +import org.apache.storm.StormSubmitter;
    +import org.apache.storm.starter.spout.RandomIntegerSpout;
    +import org.apache.storm.state.KeyValueState;
    +import org.apache.storm.streams.Pair;
    +import org.apache.storm.task.OutputCollector;
    +import org.apache.storm.task.TopologyContext;
    +import org.apache.storm.topology.OutputFieldsDeclarer;
    +import org.apache.storm.topology.TopologyBuilder;
    +import org.apache.storm.topology.base.BaseStatefulWindowedBolt;
    +import org.apache.storm.tuple.Fields;
    +import org.apache.storm.tuple.Tuple;
    +import org.apache.storm.tuple.Values;
    +import org.apache.storm.windowing.TupleWindow;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * An example that demonstrates the usage of {@link org.apache.storm.topology.IStatefulWindowedBolt} with window
    + * persistence.
    + * <p>
    + * The framework automatically checkpoints the tuples in window along with the bolt's state and restores the same
    + * during restarts.
    + * </p>
    + *
    + * <p>
    + * This topology uses 'redis' for state persistence, so you should also start a redis instance before deploying.
    + * If you are running in local mode you can just start a redis server locally which will be used for storing the state. The default
    + * RedisKeyValueStateProvider parameters can be overridden by setting "topology.state.provider.config", for e.g.
    --- End diff --
    
    Nit: Link to https://github.com/apache/storm/blob/master/storm-client/src/jvm/org/apache/storm/Config.java#L239 instead


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #2218: STORM-2614: Enhance stateful windowing to persist the win...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on the issue:

    https://github.com/apache/storm/pull/2218
  
    @ben-manes , thanks for the insights. Right now the custom logic is kind of straightforward. May be I will attempt to provide an alternate implementation on top of Caffeine which could be useful in future.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129665587
  
    --- Diff: storm-client/src/jvm/org/apache/storm/windowing/TupleWindowIterImpl.java ---
    @@ -0,0 +1,80 @@
    +/**
    + * 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.windowing;
    +
    +import com.google.common.collect.Iterators;
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.function.Supplier;
    +import org.apache.storm.tuple.Tuple;
    +
    +/**
    + * An iterator based implementation over the events in a window.
    + */
    +public class TupleWindowIterImpl implements TupleWindow {
    +    private final Supplier<Iterator<Tuple>> tuplesIt;
    +    private final Supplier<Iterator<Tuple>> newTuplesIt;
    +    private final Supplier<Iterator<Tuple>> expiredTuplesIt;
    +    private final Long startTimestamp;
    +    private final Long endTimestamp;
    +
    +    public TupleWindowIterImpl(Supplier<Iterator<Tuple>> tuplesIt,
    +                               Supplier<Iterator<Tuple>> newTuplesIt,
    +                               Supplier<Iterator<Tuple>> expiredTuplesIt,
    +                               Long startTimestamp, Long endTimestamp) {
    +        this.tuplesIt = tuplesIt;
    +        this.newTuplesIt = newTuplesIt;
    +        this.expiredTuplesIt = expiredTuplesIt;
    +        this.startTimestamp = startTimestamp;
    +        this.endTimestamp = endTimestamp;
    +    }
    +
    +    @Override
    +    public List<Tuple> get() {
    +        List<Tuple> tuples = new ArrayList<>();
    +        tuplesIt.get().forEachRemaining(t -> tuples.add(t));
    +        return tuples;
    +    }
    +
    +    @Override
    +    public List<Tuple> getNew() {
    +        throw new UnsupportedOperationException("Not implemented");
    --- End diff --
    
    Why is this unsupported, it seems like they would be available in newTuplesIt?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130226080
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/StatefulBoltExecutor.java ---
    @@ -123,6 +123,11 @@ protected void handleCheckpoint(Tuple checkpointTuple, Action action, long txid)
                     }
                     pendingTuples.clear();
                 } else {
    +                /*
    +                 * If a worker crashes, all worker's state are rolled back and an initState message is sent across the topology so that
    --- End diff --
    
    Nit: all workers' states (or "the states of all workers" if you prefer)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129763661
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/SimpleWindowPartitionCache.java ---
    @@ -0,0 +1,191 @@
    +/**
    + * 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 java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.concurrent.ConcurrentMap;
    +import java.util.concurrent.ConcurrentSkipListMap;
    +import java.util.concurrent.locks.ReentrantLock;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * A simple implementation that evicts the largest un-pinned entry from the cache. This works well
    + * for caching window partitions since the access pattern is mostly sequential scans.
    + */
    +public class SimpleWindowPartitionCache<K, V> implements WindowPartitionCache<K, V> {
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleWindowPartitionCache.class);
    +
    +    private final ConcurrentSkipListMap<K, V> map = new ConcurrentSkipListMap<>();
    +    private final Map<K, Long> pinned = new HashMap<>();
    +    private final long maximumSize;
    +    private final RemovalListener<K, V> removalListener;
    +    private final CacheLoader<K, V> cacheLoader;
    +    private final ReentrantLock lock = new ReentrantLock(true);
    +    private int size;
    +
    +    @Override
    +    public V get(K key) {
    +        return getOrLoad(key, false);
    +    }
    +
    +    @Override
    +    public V getPinned(K key) {
    +        return getOrLoad(key, true);
    +    }
    +
    +    @Override
    +    public boolean unpin(K key) {
    +        LOG.debug("unpin '{}'", key);
    +        boolean res = false;
    +        try {
    +            lock.lock();
    +            if (pinned.compute(key, (k, v) -> v == null ? 0 : v - 1) <= 0) {
    +                pinned.remove(key);
    +                res = true;
    +            }
    +        } finally {
    +            lock.unlock();
    +        }
    +        LOG.debug("pinned '{}'", pinned);
    +        return res;
    +    }
    +
    +    @Override
    +    public ConcurrentMap<K, V> asMap() {
    +        return map;
    +    }
    +
    +    @Override
    +    public void invalidate(K key) {
    +        try {
    +            lock.lock();
    +            if (isPinned(key)) {
    +                LOG.debug("Entry '{}' is pinned, skipping invalidation", key);
    +            } else {
    +                LOG.debug("Invalidating entry '{}'", key);
    +                V val = map.remove(key);
    +                if (val != null) {
    +                    --size;
    +                    pinned.remove(key);
    +                    removalListener.onRemoval(key, val, RemovalCause.EXPLICIT);
    +                }
    +            }
    +        } finally {
    +            lock.unlock();
    +        }
    +    }
    +
    +    // Get or load from the cache optionally pinning the entry
    +    // so that it wont get evicted from the cache
    +    private V getOrLoad(K key, boolean shouldPin) {
    +        V val;
    +        if (shouldPin) {
    +            try {
    +                lock.lock();
    +                val = load(key);
    +                pin(key);
    +            } finally {
    +                lock.unlock();
    +            }
    +        } else {
    +            val = map.get(key);
    +            if (val == null) {
    +                try {
    +                    lock.lock();
    +                    val = load(key);
    +                } finally {
    +                    lock.unlock();
    +                }
    +            }
    +        }
    +
    +        return val;
    +    }
    +
    +    private V load(K key) {
    +        V val = map.get(key);
    +        if (val == null) {
    +            val = cacheLoader.load(key);
    +            if (val == null) {
    +                throw new NullPointerException("Null value");
    --- End diff --
    
    good point.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r131119473
  
    --- Diff: storm-client/test/jvm/org/apache/storm/topology/SimpleWindowPartitionCacheTest.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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.utils.Utils;
    +import org.apache.storm.windowing.persistence.SimpleWindowPartitionCache;
    +import org.apache.storm.windowing.persistence.WindowPartitionCache;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.concurrent.FutureTask;
    +
    +/**
    + * Unit tests for {@link SimpleWindowPartitionCache}
    + */
    +public class SimpleWindowPartitionCacheTest {
    +
    +    @Before
    +    public void setUp() throws Exception {
    +    }
    +
    +    @Test(expected = IllegalArgumentException.class)
    +    public void testBuildInvalid1() throws Exception {
    +        SimpleWindowPartitionCache.<Integer, Integer>newBuilder()
    +            .maximumSize(0)
    +            .build(null);
    +    }
    +
    +    @Test(expected = IllegalArgumentException.class)
    +    public void testBuildInvalid2() throws Exception {
    +        SimpleWindowPartitionCache.<Integer, Integer>newBuilder()
    +            .maximumSize(-1)
    +            .build(null);
    +    }
    +
    +    @Test(expected = NullPointerException.class)
    +    public void testBuildInvalid3() throws Exception {
    +        SimpleWindowPartitionCache.<Integer, Integer>newBuilder()
    +            .maximumSize(1)
    +            .build(null);
    +    }
    +
    +    @Test
    +    public void testBuildOk() throws Exception {
    +        SimpleWindowPartitionCache.<Integer, Integer>newBuilder()
    +            .maximumSize(1)
    +            .removalListener((key, val, removalCause) -> {
    +            })
    +            .build(key -> key);
    +    }
    +
    +    @Test
    +    public void testGet() throws Exception {
    +        List<Integer> removed = new ArrayList<>();
    +        List<Integer> loaded = new ArrayList<>();
    +        SimpleWindowPartitionCache<Integer, Integer> cache =
    +            SimpleWindowPartitionCache.<Integer, Integer>newBuilder()
    +                .maximumSize(2)
    +                .removalListener((key, val, removalCause) -> removed.add(key))
    +                .build(key -> {
    +                    loaded.add(key);
    +                    return key;
    +                });
    +
    +        cache.get(1);
    +        cache.get(2);
    +        cache.get(3);
    +        Assert.assertEquals(Arrays.asList(1, 2, 3), loaded);
    +        Assert.assertEquals(Collections.singletonList(2), removed);
    --- End diff --
    
    Why is 2 removed if the size is 2 and the access order is 1, 2, 3? Shouldn't it be 1?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130225984
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutor.java ---
    @@ -0,0 +1,596 @@
    +/**
    + * 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 java.util.AbstractCollection;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Objects;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentLinkedQueue;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.locks.ReentrantLock;
    +import java.util.function.Supplier;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.state.KeyValueState;
    +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.topology.base.BaseWindowedBolt;
    +import org.apache.storm.tuple.Tuple;
    +import org.apache.storm.windowing.DefaultEvictionContext;
    +import org.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.EventImpl;
    +import org.apache.storm.windowing.WindowLifecycleListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static java.util.Collections.emptyIterator;
    +import static org.apache.storm.topology.WindowPartitionCache.CacheLoader;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalCause;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalListener;
    +
    +/**
    + * Wraps a {@link IStatefulWindowedBolt} and handles the execution. Uses state and the underlying
    + * checkpointing mechanisms to save the tuples in window to state. The tuples are also kept in-memory
    + * by transparently caching the window partitions and checkpointing them as needed.
    + */
    +public class PersistentWindowedBoltExecutor<T extends State> extends WindowedBoltExecutor implements IStatefulBolt<T> {
    +    private static final Logger LOG = LoggerFactory.getLogger(PersistentWindowedBoltExecutor.class);
    +    private final IStatefulWindowedBolt<T> statefulWindowedBolt;
    +    private transient TopologyContext topologyContext;
    +    private transient OutputCollector outputCollector;
    +    private transient WindowState<Tuple> state;
    +    private transient boolean stateInitialized;
    +    private transient boolean prePrepared;
    +
    +    public PersistentWindowedBoltExecutor(IStatefulWindowedBolt<T> bolt) {
    +        super(bolt);
    +        statefulWindowedBolt = bolt;
    +    }
    +
    +    @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));
    +    }
    +
    +    @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);
    +        }
    +    }
    +
    +    private int getCheckpointIntervalMillis(Map<String, Object> topoConf) {
    +        int checkpointInterval = Integer.MAX_VALUE;
    +        if (topoConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL) != null) {
    +            checkpointInterval = ((Number) topoConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL)).intValue();
    +        }
    +        return checkpointInterval;
    +    }
    +
    +    // package access for unit tests
    +    void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                 KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                 KeyValueState<String, Deque<Long>> partitionState,
    +                 KeyValueState<String, Optional<?>> windowSystemState) {
    +        init(topoConf, context, collector, windowState, partitionState, windowSystemState);
    +        doPrepare(topoConf, context, new NoAckOutputCollector(collector), state, true);
    +        Map<String, Optional<?>> wstate = new HashMap<>();
    +        windowSystemState.forEach(s -> wstate.put(s.getKey(), s.getValue()));
    +        restoreState(wstate);
    +    }
    +
    +    @Override
    +    protected void start() {
    +        if (stateInitialized) {
    +            super.start();
    +        } else {
    +            LOG.debug("Will invoke start after state is initialized");
    +        }
    +    }
    +
    +    @Override
    +    public void execute(Tuple input) {
    +        if (!stateInitialized) {
    +            throw new IllegalStateException("execute invoked before initState with input tuple " + input);
    +        }
    +        super.execute(input);
    +        // StatefulBoltExecutor does the actual ack when the state is saved.
    +        outputCollector.ack(input);
    +    }
    +
    +    @Override
    +    public void initState(T state) {
    +        if (stateInitialized) {
    +            String msg = "initState invoked when the state is already initialized";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        } else {
    +            statefulWindowedBolt.initState(state);
    +            stateInitialized = true;
    +            start();
    +        }
    +    }
    +
    +    @Override
    +    public void prePrepare(long txid) {
    +        if (stateInitialized) {
    +            LOG.debug("Prepare streamState, txid {}", txid);
    +            state.prepareCommit(txid);
    +            prePrepared = true;
    +        } else {
    +            String msg = "Cannot prepare before initState";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        }
    +    }
    +
    +    @Override
    +    public void preCommit(long txid) {
    +        // preCommit can be invoked during recovery before the state is initialized
    +        if (prePrepared || !stateInitialized) {
    +            LOG.debug("Commit streamState, txid {}", txid);
    +            state.commit(txid);
    +        } else {
    +            String msg = "preCommit before prePrepare in initialized state";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        }
    +    }
    +
    +    @Override
    +    public void preRollback() {
    +        LOG.debug("Rollback streamState, stateInitialized {}", stateInitialized);
    +        state.rollback();
    +    }
    +
    +    @Override
    +    protected WindowLifecycleListener<Tuple> newWindowLifecycleListener() {
    +        return new WindowLifecycleListener<Tuple>() {
    +            @Override
    +            public void onExpiry(List<Tuple> events) {
    +                /*
    +                 * NO-OP: the events are ack-ed in execute
    +                 */
    +            }
    +
    +            @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();
    +            }
    +        };
    +    }
    +
    +    private void init(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                      KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                      KeyValueState<String, Deque<Long>> partitionState,
    +                      KeyValueState<String, Optional<?>> windowSystemState) {
    +        topologyContext = context;
    +        outputCollector = collector;
    +        state = new WindowState<>(windowState, partitionState, windowSystemState, this::getState,
    +            statefulWindowedBolt.maxEventsInMemory());
    +    }
    +
    +    private KeyValueState<Long, WindowPartition<Tuple>> getWindowState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window";
    +        return (KeyValueState<Long, WindowPartition<Tuple>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Deque<Long>> getPartitionState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-partitions";
    +        return (KeyValueState<String, Deque<Long>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Optional<?>> getWindowSystemState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-systemstate";
    +        return (KeyValueState<String, Optional<?>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    // a wrapper around the window related states that are checkpointed
    +    private static class WindowState<T> extends AbstractCollection<Event<T>> {
    +        // number of events per window-partition
    +        private static final int MAX_PARTITION_EVENTS = 1000;
    +        private static final int MIN_PARTITIONS = 10;
    +        private static final String PARTITION_IDS_KEY = "pk";
    +        private final KeyValueState<String, Deque<Long>> partitionIdsState;
    +        private final KeyValueState<Long, WindowPartition<T>> windowPartitionsState;
    +        private final KeyValueState<String, Optional<?>> windowSystemState;
    +        // ordered partition keys
    +        private Deque<Long> partitionIds;
    +        private volatile long latestPartitionId;
    +        private WindowPartitionCache<Long, WindowPartition<T>> cache;
    +        private Supplier<Map<String, Optional<?>>> windowSystemStateSupplier;
    +        private final ReentrantLock partitionIdsLock = new ReentrantLock(true);
    +        private final ReentrantLock windowPartitionsLock = new ReentrantLock(true);
    +        private final long maxEventsInMemory;
    +        private WindowPartition<T> latestPartition;
    +        private Set<Long> iteratorPins = new HashSet<>();
    +
    +        WindowState(KeyValueState<Long, WindowPartition<T>> windowPartitionsState,
    +                    KeyValueState<String, Deque<Long>> partitionIdsState,
    +                    KeyValueState<String, Optional<?>> windowSystemState,
    +                    Supplier<Map<String, Optional<?>>> windowSystemStateSupplier,
    +                    long maxEventsInMemory) {
    +            this.windowPartitionsState = windowPartitionsState;
    +            this.partitionIdsState = partitionIdsState;
    +            this.windowSystemState = windowSystemState;
    +            this.windowSystemStateSupplier = windowSystemStateSupplier;
    +            this.maxEventsInMemory = Math.max(MAX_PARTITION_EVENTS * MIN_PARTITIONS, maxEventsInMemory);
    +            initCache();
    +            initPartitions();
    +        }
    +
    +        @Override
    +        public boolean add(Event<T> event) {
    +            if (latestPartition.size() >= MAX_PARTITION_EVENTS) {
    +                cache.unpin(latestPartition.getId());
    +                latestPartition = getPinnedPartition(getNextPartitionId());
    +            }
    +            latestPartition.add(event);
    +            return true;
    +        }
    +
    +        @Override
    +        public Iterator<Event<T>> iterator() {
    +
    +            return new Iterator<Event<T>>() {
    +                private Iterator<Long> ids = getIds();
    +                private Iterator<Event<T>> current = emptyIterator();
    +                private Iterator<Event<T>> removeFrom;
    +                private WindowPartition<T> curPartition;
    +
    +                private Iterator<Long> getIds() {
    +                    try {
    +                        partitionIdsLock.lock();
    +                        LOG.debug("Iterator partitionIds: {}", partitionIds);
    +                        return new ArrayList<>(partitionIds).iterator();
    +                    } finally {
    +                        partitionIdsLock.unlock();
    +                    }
    +                }
    +
    +                @Override
    +                public void remove() {
    +                    if (removeFrom == null) {
    +                        throw new IllegalStateException("No calls to next() since last call to remove()");
    +                    }
    +                    removeFrom.remove();
    +                    removeFrom = null;
    +                }
    +
    +                @Override
    +                public boolean hasNext() {
    +                    boolean curHasNext = current.hasNext();
    +                    while (!curHasNext && ids.hasNext()) {
    +                        if (curPartition != null) {
    +                            unpin(curPartition.getId());
    +                        }
    +                        curPartition = getPinnedPartition(ids.next());
    +                        if (curPartition != null) {
    +                            iteratorPins.add(curPartition.getId());
    +                            current = curPartition.iterator();
    +                            curHasNext = current.hasNext();
    +                        }
    +                    }
    +                    // un-pin the last partition
    +                    if (!curHasNext && curPartition != null) {
    +                        unpin(curPartition.getId());
    +                        curPartition = null;
    +                    }
    +                    return curHasNext;
    +                }
    +
    +                @Override
    +                public Event<T> next() {
    +                    if (!hasNext()) {
    +                        throw new NoSuchElementException();
    +                    }
    +                    removeFrom = current;
    +                    return current.next();
    +                }
    +
    +                private void unpin(long id) {
    +                    cache.unpin(id);
    +                    iteratorPins.remove(id);
    +                }
    +            };
    +        }
    +
    +        void clearIteratorPins() {
    +            LOG.debug("clearIteratorPins '{}'", iteratorPins);
    +            Iterator<Long> it = iteratorPins.iterator();
    +            while (it.hasNext()) {
    +                cache.unpin(it.next());
    +                it.remove();
    +            }
    +        }
    +
    +        @Override
    +        public int size() {
    +            throw new UnsupportedOperationException();
    +        }
    +
    +        void prepareCommit(long txid) {
    +            flush();
    +            partitionIdsState.prepareCommit(txid);
    +            windowPartitionsState.prepareCommit(txid);
    +            windowSystemState.prepareCommit(txid);
    +        }
    +
    +        void commit(long txid) {
    +            partitionIdsState.commit(txid);
    +            windowPartitionsState.commit(txid);
    +            windowSystemState.commit(txid);
    +        }
    +
    +        void rollback() {
    +            partitionIdsState.rollback();
    +            windowPartitionsState.rollback();
    +            windowSystemState.rollback();
    +        }
    +
    +        private void initPartitions() {
    +            partitionIds = partitionIdsState.get(PARTITION_IDS_KEY, new LinkedList<>());
    +            if (partitionIds.isEmpty()) {
    +                partitionIds.add(0L);
    +                partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    +            }
    +            latestPartitionId = partitionIds.peekLast();
    +            latestPartition = cache.getPinned(latestPartitionId);
    +        }
    +
    +        private void initCache() {
    +            long size = maxEventsInMemory / MAX_PARTITION_EVENTS;
    +            LOG.info("maxEventsInMemory: {}, partition size: {}, number of partitions: {}",
    +                maxEventsInMemory, MAX_PARTITION_EVENTS, size);
    +            cache = SimpleWindowPartitionCache.<Long, WindowPartition<T>>newBuilder()
    +                .maximumSize(size)
    +                .removalListener(new RemovalListener<Long, WindowPartition<T>>() {
    +                    @Override
    +                    public void onRemoval(Long pid, WindowPartition<T> p, RemovalCause removalCause) {
    +                        Objects.requireNonNull(pid, "Null partition id");
    +                        Objects.requireNonNull(p, "Null window partition");
    +                        LOG.debug("onRemoval for id '{}', WindowPartition '{}'", pid, p);
    +                        try {
    +                            windowPartitionsLock.lock();
    +                            if (p.isEmpty() && pid != latestPartitionId) {
    +                                // if the empty partition was not invalidated by flush, but evicted from cache
    +                                if (removalCause != RemovalCause.EXPLICIT) {
    +                                    deletePartition(pid);
    +                                    windowPartitionsState.delete(pid);
    +                                }
    +                            } else if (p.isModified()) {
    +                                windowPartitionsState.put(pid, p);
    +                            } else {
    +                                LOG.debug("WindowPartition '{}' is not modified", pid);
    +                            }
    +                        } finally {
    +                            windowPartitionsLock.unlock();
    +                        }
    +                    }
    +                }).build(new CacheLoader<Long, WindowPartition<T>>() {
    +                    @Override
    +                   public WindowPartition<T> load(Long id) {
    +                        LOG.debug("Load partition: {}", id);
    +                        // load from state
    +                        try {
    +                            windowPartitionsLock.lock();
    +                            return windowPartitionsState.get(id, new WindowPartition<>(id));
    +                        } finally {
    +                            windowPartitionsLock.unlock();
    +                        }
    +                    }
    +                    }
    +                );
    +        }
    +
    +        private void deletePartition(long pid) {
    +            LOG.debug("Delete partition: {}", pid);
    +            try {
    +                partitionIdsLock.lock();
    +                partitionIds.remove(pid);
    +                partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    +            } finally {
    +                partitionIdsLock.unlock();
    +            }
    +        }
    +
    +        private long getNextPartitionId() {
    +            try {
    +                partitionIdsLock.lock();
    +                partitionIds.add(++latestPartitionId);
    +                partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    +            } finally {
    +                partitionIdsLock.unlock();
    +            }
    +            return latestPartitionId;
    +        }
    +
    +        private WindowPartition<T> getPinnedPartition(long id) {
    +            return cache.getPinned(id);
    +        }
    +
    +        private void flush() {
    +            LOG.debug("Flushing modified partitions");
    +            cache.asMap().forEach((pid, p) -> {
    +                Long pidToInvalidate = null;
    +                try {
    +                    windowPartitionsLock.lock();
    +                    if (p.isEmpty() && pid != latestPartitionId) {
    +                        LOG.debug("Invalidating empty partition {}", pid);
    +                        deletePartition(pid);
    +                        windowPartitionsState.delete(pid);
    +                        pidToInvalidate = pid;
    +                    } else if (p.isModified()) {
    +                        LOG.debug("Updating modified partition {}", pid);
    +                        p.clearModified();
    +                        windowPartitionsState.put(pid, p);
    +                    }
    +                } finally {
    +                    windowPartitionsLock.unlock();
    +                }
    +                // invalidate after releasing the lock
    +                if (pidToInvalidate != null) {
    +                    cache.invalidate(pidToInvalidate);
    +                }
    +            });
    +            windowSystemStateSupplier.get().forEach(windowSystemState::put);
    +        }
    +    }
    +
    +    // the window partition that holds the events
    +    public static class WindowPartition<T> implements Iterable<Event<T>> {
    --- End diff --
    
    It might be better to put some of these classes in their own files, maybe create a new package for them?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130227210
  
    --- Diff: storm-client/test/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutorTest.java ---
    @@ -0,0 +1,261 @@
    +/**
    + * 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.Config;
    +import org.apache.storm.generated.GlobalStreamId;
    +import org.apache.storm.state.KeyValueState;
    +import org.apache.storm.streams.Pair;
    +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.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.TimestampExtractor;
    +import org.apache.storm.windowing.TupleWindow;
    +import org.apache.storm.windowing.WaterMarkEvent;
    +import org.apache.storm.windowing.WaterMarkEventGenerator;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.mockito.ArgumentCaptor;
    +import org.mockito.Captor;
    +import org.mockito.Mockito;
    +import org.mockito.MockitoAnnotations;
    +import org.mockito.invocation.InvocationOnMock;
    +import org.mockito.runners.MockitoJUnitRunner;
    +import org.mockito.stubbing.Answer;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.stream.Collectors;
    +import java.util.stream.LongStream;
    +
    +import static org.mockito.AdditionalAnswers.returnsArgAt;
    +
    +/**
    + * Unit tests for {@link PersistentWindowedBoltExecutor}
    + */
    +@RunWith(MockitoJUnitRunner.class)
    +public class PersistentWindowedBoltExecutorTest {
    +    public static final String LATE_STREAM = "late_stream";
    +    private PersistentWindowedBoltExecutor<KeyValueState<String, String>> executor;
    +    private IStatefulWindowedBolt<KeyValueState<String, String>> mockBolt;
    +    private OutputCollector mockOutputCollector;
    +    private TopologyContext mockTopologyContext;
    +    private Map<String, Object> mockStormConf = new HashMap<>();
    --- End diff --
    
    Nit: This isn't really a mock


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130225941
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutor.java ---
    @@ -0,0 +1,596 @@
    +/**
    + * 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 java.util.AbstractCollection;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Objects;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentLinkedQueue;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.locks.ReentrantLock;
    +import java.util.function.Supplier;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.state.KeyValueState;
    +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.topology.base.BaseWindowedBolt;
    +import org.apache.storm.tuple.Tuple;
    +import org.apache.storm.windowing.DefaultEvictionContext;
    +import org.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.EventImpl;
    +import org.apache.storm.windowing.WindowLifecycleListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static java.util.Collections.emptyIterator;
    +import static org.apache.storm.topology.WindowPartitionCache.CacheLoader;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalCause;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalListener;
    +
    +/**
    + * Wraps a {@link IStatefulWindowedBolt} and handles the execution. Uses state and the underlying
    + * checkpointing mechanisms to save the tuples in window to state. The tuples are also kept in-memory
    + * by transparently caching the window partitions and checkpointing them as needed.
    + */
    +public class PersistentWindowedBoltExecutor<T extends State> extends WindowedBoltExecutor implements IStatefulBolt<T> {
    +    private static final Logger LOG = LoggerFactory.getLogger(PersistentWindowedBoltExecutor.class);
    +    private final IStatefulWindowedBolt<T> statefulWindowedBolt;
    +    private transient TopologyContext topologyContext;
    +    private transient OutputCollector outputCollector;
    +    private transient WindowState<Tuple> state;
    +    private transient boolean stateInitialized;
    +    private transient boolean prePrepared;
    +
    +    public PersistentWindowedBoltExecutor(IStatefulWindowedBolt<T> bolt) {
    +        super(bolt);
    +        statefulWindowedBolt = bolt;
    +    }
    +
    +    @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));
    +    }
    +
    +    @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);
    +        }
    +    }
    +
    +    private int getCheckpointIntervalMillis(Map<String, Object> topoConf) {
    +        int checkpointInterval = Integer.MAX_VALUE;
    +        if (topoConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL) != null) {
    +            checkpointInterval = ((Number) topoConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL)).intValue();
    +        }
    +        return checkpointInterval;
    +    }
    +
    +    // package access for unit tests
    +    void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                 KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                 KeyValueState<String, Deque<Long>> partitionState,
    +                 KeyValueState<String, Optional<?>> windowSystemState) {
    +        init(topoConf, context, collector, windowState, partitionState, windowSystemState);
    +        doPrepare(topoConf, context, new NoAckOutputCollector(collector), state, true);
    +        Map<String, Optional<?>> wstate = new HashMap<>();
    +        windowSystemState.forEach(s -> wstate.put(s.getKey(), s.getValue()));
    +        restoreState(wstate);
    +    }
    +
    +    @Override
    +    protected void start() {
    +        if (stateInitialized) {
    +            super.start();
    +        } else {
    +            LOG.debug("Will invoke start after state is initialized");
    +        }
    +    }
    +
    +    @Override
    +    public void execute(Tuple input) {
    +        if (!stateInitialized) {
    +            throw new IllegalStateException("execute invoked before initState with input tuple " + input);
    +        }
    +        super.execute(input);
    +        // StatefulBoltExecutor does the actual ack when the state is saved.
    +        outputCollector.ack(input);
    +    }
    +
    +    @Override
    +    public void initState(T state) {
    +        if (stateInitialized) {
    +            String msg = "initState invoked when the state is already initialized";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        } else {
    +            statefulWindowedBolt.initState(state);
    +            stateInitialized = true;
    +            start();
    +        }
    +    }
    +
    +    @Override
    +    public void prePrepare(long txid) {
    +        if (stateInitialized) {
    +            LOG.debug("Prepare streamState, txid {}", txid);
    +            state.prepareCommit(txid);
    +            prePrepared = true;
    +        } else {
    +            String msg = "Cannot prepare before initState";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        }
    +    }
    +
    +    @Override
    +    public void preCommit(long txid) {
    +        // preCommit can be invoked during recovery before the state is initialized
    +        if (prePrepared || !stateInitialized) {
    +            LOG.debug("Commit streamState, txid {}", txid);
    +            state.commit(txid);
    +        } else {
    +            String msg = "preCommit before prePrepare in initialized state";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        }
    +    }
    +
    +    @Override
    +    public void preRollback() {
    +        LOG.debug("Rollback streamState, stateInitialized {}", stateInitialized);
    +        state.rollback();
    +    }
    +
    +    @Override
    +    protected WindowLifecycleListener<Tuple> newWindowLifecycleListener() {
    +        return new WindowLifecycleListener<Tuple>() {
    +            @Override
    +            public void onExpiry(List<Tuple> events) {
    +                /*
    +                 * NO-OP: the events are ack-ed in execute
    +                 */
    +            }
    +
    +            @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();
    +            }
    +        };
    +    }
    +
    +    private void init(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                      KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                      KeyValueState<String, Deque<Long>> partitionState,
    +                      KeyValueState<String, Optional<?>> windowSystemState) {
    +        topologyContext = context;
    +        outputCollector = collector;
    +        state = new WindowState<>(windowState, partitionState, windowSystemState, this::getState,
    +            statefulWindowedBolt.maxEventsInMemory());
    +    }
    +
    +    private KeyValueState<Long, WindowPartition<Tuple>> getWindowState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window";
    +        return (KeyValueState<Long, WindowPartition<Tuple>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Deque<Long>> getPartitionState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-partitions";
    +        return (KeyValueState<String, Deque<Long>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Optional<?>> getWindowSystemState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-systemstate";
    +        return (KeyValueState<String, Optional<?>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    // a wrapper around the window related states that are checkpointed
    +    private static class WindowState<T> extends AbstractCollection<Event<T>> {
    +        // number of events per window-partition
    +        private static final int MAX_PARTITION_EVENTS = 1000;
    +        private static final int MIN_PARTITIONS = 10;
    +        private static final String PARTITION_IDS_KEY = "pk";
    +        private final KeyValueState<String, Deque<Long>> partitionIdsState;
    +        private final KeyValueState<Long, WindowPartition<T>> windowPartitionsState;
    +        private final KeyValueState<String, Optional<?>> windowSystemState;
    +        // ordered partition keys
    +        private Deque<Long> partitionIds;
    +        private volatile long latestPartitionId;
    +        private WindowPartitionCache<Long, WindowPartition<T>> cache;
    +        private Supplier<Map<String, Optional<?>>> windowSystemStateSupplier;
    +        private final ReentrantLock partitionIdsLock = new ReentrantLock(true);
    +        private final ReentrantLock windowPartitionsLock = new ReentrantLock(true);
    +        private final long maxEventsInMemory;
    +        private WindowPartition<T> latestPartition;
    +        private Set<Long> iteratorPins = new HashSet<>();
    +
    +        WindowState(KeyValueState<Long, WindowPartition<T>> windowPartitionsState,
    +                    KeyValueState<String, Deque<Long>> partitionIdsState,
    +                    KeyValueState<String, Optional<?>> windowSystemState,
    +                    Supplier<Map<String, Optional<?>>> windowSystemStateSupplier,
    +                    long maxEventsInMemory) {
    +            this.windowPartitionsState = windowPartitionsState;
    +            this.partitionIdsState = partitionIdsState;
    +            this.windowSystemState = windowSystemState;
    +            this.windowSystemStateSupplier = windowSystemStateSupplier;
    +            this.maxEventsInMemory = Math.max(MAX_PARTITION_EVENTS * MIN_PARTITIONS, maxEventsInMemory);
    +            initCache();
    +            initPartitions();
    +        }
    +
    +        @Override
    +        public boolean add(Event<T> event) {
    +            if (latestPartition.size() >= MAX_PARTITION_EVENTS) {
    +                cache.unpin(latestPartition.getId());
    +                latestPartition = getPinnedPartition(getNextPartitionId());
    +            }
    +            latestPartition.add(event);
    +            return true;
    +        }
    +
    +        @Override
    +        public Iterator<Event<T>> iterator() {
    +
    +            return new Iterator<Event<T>>() {
    +                private Iterator<Long> ids = getIds();
    +                private Iterator<Event<T>> current = emptyIterator();
    +                private Iterator<Event<T>> removeFrom;
    +                private WindowPartition<T> curPartition;
    +
    +                private Iterator<Long> getIds() {
    +                    try {
    +                        partitionIdsLock.lock();
    +                        LOG.debug("Iterator partitionIds: {}", partitionIds);
    +                        return new ArrayList<>(partitionIds).iterator();
    +                    } finally {
    +                        partitionIdsLock.unlock();
    +                    }
    +                }
    +
    +                @Override
    +                public void remove() {
    +                    if (removeFrom == null) {
    +                        throw new IllegalStateException("No calls to next() since last call to remove()");
    +                    }
    +                    removeFrom.remove();
    +                    removeFrom = null;
    +                }
    +
    +                @Override
    +                public boolean hasNext() {
    +                    boolean curHasNext = current.hasNext();
    +                    while (!curHasNext && ids.hasNext()) {
    +                        if (curPartition != null) {
    +                            unpin(curPartition.getId());
    +                        }
    +                        curPartition = getPinnedPartition(ids.next());
    +                        if (curPartition != null) {
    +                            iteratorPins.add(curPartition.getId());
    +                            current = curPartition.iterator();
    +                            curHasNext = current.hasNext();
    +                        }
    +                    }
    +                    // un-pin the last partition
    +                    if (!curHasNext && curPartition != null) {
    +                        unpin(curPartition.getId());
    +                        curPartition = null;
    +                    }
    +                    return curHasNext;
    +                }
    +
    +                @Override
    +                public Event<T> next() {
    +                    if (!hasNext()) {
    +                        throw new NoSuchElementException();
    +                    }
    +                    removeFrom = current;
    +                    return current.next();
    +                }
    +
    +                private void unpin(long id) {
    +                    cache.unpin(id);
    +                    iteratorPins.remove(id);
    +                }
    +            };
    +        }
    +
    +        void clearIteratorPins() {
    +            LOG.debug("clearIteratorPins '{}'", iteratorPins);
    +            Iterator<Long> it = iteratorPins.iterator();
    +            while (it.hasNext()) {
    +                cache.unpin(it.next());
    +                it.remove();
    +            }
    +        }
    +
    +        @Override
    +        public int size() {
    +            throw new UnsupportedOperationException();
    +        }
    +
    +        void prepareCommit(long txid) {
    +            flush();
    +            partitionIdsState.prepareCommit(txid);
    +            windowPartitionsState.prepareCommit(txid);
    +            windowSystemState.prepareCommit(txid);
    +        }
    +
    +        void commit(long txid) {
    +            partitionIdsState.commit(txid);
    +            windowPartitionsState.commit(txid);
    +            windowSystemState.commit(txid);
    +        }
    +
    +        void rollback() {
    +            partitionIdsState.rollback();
    +            windowPartitionsState.rollback();
    +            windowSystemState.rollback();
    +        }
    +
    +        private void initPartitions() {
    +            partitionIds = partitionIdsState.get(PARTITION_IDS_KEY, new LinkedList<>());
    +            if (partitionIds.isEmpty()) {
    +                partitionIds.add(0L);
    +                partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    +            }
    +            latestPartitionId = partitionIds.peekLast();
    +            latestPartition = cache.getPinned(latestPartitionId);
    +        }
    +
    +        private void initCache() {
    +            long size = maxEventsInMemory / MAX_PARTITION_EVENTS;
    +            LOG.info("maxEventsInMemory: {}, partition size: {}, number of partitions: {}",
    +                maxEventsInMemory, MAX_PARTITION_EVENTS, size);
    +            cache = SimpleWindowPartitionCache.<Long, WindowPartition<T>>newBuilder()
    +                .maximumSize(size)
    +                .removalListener(new RemovalListener<Long, WindowPartition<T>>() {
    +                    @Override
    +                    public void onRemoval(Long pid, WindowPartition<T> p, RemovalCause removalCause) {
    +                        Objects.requireNonNull(pid, "Null partition id");
    +                        Objects.requireNonNull(p, "Null window partition");
    +                        LOG.debug("onRemoval for id '{}', WindowPartition '{}'", pid, p);
    +                        try {
    +                            windowPartitionsLock.lock();
    +                            if (p.isEmpty() && pid != latestPartitionId) {
    +                                // if the empty partition was not invalidated by flush, but evicted from cache
    +                                if (removalCause != RemovalCause.EXPLICIT) {
    +                                    deletePartition(pid);
    +                                    windowPartitionsState.delete(pid);
    +                                }
    +                            } else if (p.isModified()) {
    +                                windowPartitionsState.put(pid, p);
    +                            } else {
    +                                LOG.debug("WindowPartition '{}' is not modified", pid);
    +                            }
    +                        } finally {
    +                            windowPartitionsLock.unlock();
    +                        }
    +                    }
    +                }).build(new CacheLoader<Long, WindowPartition<T>>() {
    +                    @Override
    +                   public WindowPartition<T> load(Long id) {
    +                        LOG.debug("Load partition: {}", id);
    +                        // load from state
    +                        try {
    +                            windowPartitionsLock.lock();
    +                            return windowPartitionsState.get(id, new WindowPartition<>(id));
    +                        } finally {
    +                            windowPartitionsLock.unlock();
    +                        }
    +                    }
    +                    }
    +                );
    +        }
    +
    +        private void deletePartition(long pid) {
    +            LOG.debug("Delete partition: {}", pid);
    +            try {
    +                partitionIdsLock.lock();
    +                partitionIds.remove(pid);
    +                partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    --- End diff --
    
    Why is it necessary to do this?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by satishd <gi...@git.apache.org>.
Github user satishd commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129809356
  
    --- Diff: docs/Windowing.md ---
    @@ -266,3 +266,108 @@ tuples can be received within the timeout period.
     An example toplogy `SlidingWindowTopology` shows how to use the apis to compute a sliding window sum and a tumbling window 
     average.
     
    +## Stateful windowing
    +The default windowing implementation in storm stores the tuples in memory until they are processed and expired from the 
    +window. This limits the use cases to windows that
    +fit entirely in memory. Also the source tuples cannot be ack-ed until the window expiry requiring large message timeouts
    +(topology.message.timeout.secs should be larger than the window length + sliding interval). This also puts extra loads 
    +due to the complex acking and anchoring requirements.
    + 
    +To address the above limitations and to support larger window sizes, storm provides stateful windowing support via `IStatefulWindowedBolt`. 
    +User bolts should typically extend `BaseStatefulWindowedBolt` for the windowing operations with the framework automatically 
    +managing the state of the window in the background.
    +
    +If the sources provide a monotonically increasing identifier as a part of the message, the framework can use this
    +to periodically checkpoint the last expired and evaluated message ids, to avoid duplicate window evaluations in case of 
    +failures or restarts. During recovery, the tuples with message ids lower than last expired id are discarded and tuples with 
    +message id between the last expired and last evaluated message ids are fed into the system without activating any previously
    +activated windows.
    +The tuples beyond the last evaluated message ids are processed as usual. This can be enabled by setting
    +the `messageIdField` as shown below,
    +
    +```java
    +topologyBuilder.setBolt("mybolt",
    +                   new MyStatefulWindowedBolt()
    +                   .withWindow(...) // windowing configuarations
    +                   .withMessageIdField("msgid"), // a monotonically increasing 'long' field in the tuple
    +                   parallelism)
    +               .shuffleGrouping("spout");
    +```
    +
    +However, this option is feasible only if the sources can provide a monotonically increasing identifier in the tuple and the same is maintained
    +while re-emitting the messages in case of failures. With this option the tuples are still buffered in memory until processed
    +and expired from the window.
    +
    +For more details take a look at the sample topology in storm-starter [StatefulWindowingTopology](../examples/storm-starter/src/jvm/org/apache/storm/starter/StatefulWindowingTopology.java) which will help you get started.
    +
    +### Window checkpointing
    +
    +With window checkpointing, the monotonically increasing id is no longer required since the framework transparently saves the state of the window periodically into the configured state backend.
    +The state that is saved includes the tuples in the window, any system state that is required to recover the state of processing
    +and also the user state.
    +
    +```java
    +topologyBuilder.setBolt("mybolt",
    +                   new MyStatefulPersistentWindowedBolt()
    +                   .withWindow(...) // windowing configuarations
    +                   .withPersistence() // persist the window state
    +                   .withMaxEventsInMemory(25000), // max number of events to be cached in memory
    +                    parallelism)
    +               .shuffleGrouping("spout");
    +
    +```
    +
    +The `withPersistence` instructs the framework to transparently save the tuples in window along with
    +any associated system and user state to the state backend. The `withMaxEventsInMemory` is an optional 
    +configuration that specifies the maximum number of tuples that may be kept in memory. The tuples are transparently loaded from 
    +the state backend as required and the ones that are most likely to be used again are retained in memory.
    +
    +The state backend can be configured by setting the topology state provider config,
    +
    +```java
    +// use redis for state persistence
    +conf.put(Config.TOPOLOGY_STATE_PROVIDER, "org.apache.storm.redis.state.RedisKeyValueStateProvider");
    +
    +```
    +Currently storm supports Redis and HBase as state backends and uses the underlying state-checkpointing
    +framework for saving the window state. For more details on state checkpointing see [State-checkpointing.md](State-checkpointing.md)
    +
    +Here is an example of a persistent windowed bolt that uses the window checkpointing to save its state. The `initState`
    +is invoked with the last saved state (user state) at initialization time. The execute method is invoked based on the configured
    +windowing parameters and the tuples in the active window can be accessed via an `iterator` as shown below.
    +
    +```java
    +public class MyStatefulPersistentWindowedBolt extends BaseStatefulWindowedBolt<K, V> {
    +  private KeyValueState<K, V> state;
    +  
    +  @Override
    +  public void initState(KeyValueState<K, V> state) {
    +    this.state = state;
    +   // ...
    +   // restore the state from the last saved state.
    +   // ...
    +  }
    +  
    +  @Override
    +  public void execute(TupleWindow window) {      
    +    // iterate over tuples in the current window
    +    Iterator<Tuple> it = window.getIter();
    +    while (it.hasNext()) {
    +        // compute some result based on the tuples in window
    +    }
    +    
    +    // possibly update any state to be maintained across windows
    +    state.put(STATE_KEY, updatedValue);
    +    
    +    // emit the results downstream
    +    collector.emit(new Values(result));
    +  }
    +}
    +```
    +
    +Note: In case of persistent windowed bolts, use `TupleWindow.getIter` to retrieve an iterator over the 
    +events in the window. If the number of tuples in windows is huge, invoking `TupleWindow.get` would
    +try to load all the tuples into memory and may throw an OOM exception.
    --- End diff --
    
    Do we also want to say `getNew` and `getExpired` are not supported with persistent windowed bolts? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130228071
  
    --- Diff: storm-client/test/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutorTest.java ---
    @@ -0,0 +1,261 @@
    +/**
    + * 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.Config;
    +import org.apache.storm.generated.GlobalStreamId;
    +import org.apache.storm.state.KeyValueState;
    +import org.apache.storm.streams.Pair;
    +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.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.TimestampExtractor;
    +import org.apache.storm.windowing.TupleWindow;
    +import org.apache.storm.windowing.WaterMarkEvent;
    +import org.apache.storm.windowing.WaterMarkEventGenerator;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.mockito.ArgumentCaptor;
    +import org.mockito.Captor;
    +import org.mockito.Mockito;
    +import org.mockito.MockitoAnnotations;
    +import org.mockito.invocation.InvocationOnMock;
    +import org.mockito.runners.MockitoJUnitRunner;
    +import org.mockito.stubbing.Answer;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.stream.Collectors;
    +import java.util.stream.LongStream;
    +
    +import static org.mockito.AdditionalAnswers.returnsArgAt;
    +
    +/**
    + * Unit tests for {@link PersistentWindowedBoltExecutor}
    + */
    +@RunWith(MockitoJUnitRunner.class)
    +public class PersistentWindowedBoltExecutorTest {
    +    public static final String LATE_STREAM = "late_stream";
    +    private PersistentWindowedBoltExecutor<KeyValueState<String, String>> executor;
    +    private IStatefulWindowedBolt<KeyValueState<String, String>> mockBolt;
    +    private OutputCollector mockOutputCollector;
    +    private TopologyContext mockTopologyContext;
    +    private Map<String, Object> mockStormConf = new HashMap<>();
    +    private TimestampExtractor mockTimestampExtractor;
    +    private WaterMarkEventGenerator mockWaterMarkEventGenerator;
    +    private KeyValueState mockPartitionState;
    +    private KeyValueState mockWindowState;
    +    private KeyValueState mockSystemState;
    +
    +    private static final String PARTITION_KEY = "pk";
    +    private static final String EVICTION_STATE_KEY = "es";
    +    private static final String TRIGGER_STATE_KEY = "ts";
    +    private long tupleTs;
    +
    +    @Captor
    +    private ArgumentCaptor<Tuple> tupleCaptor;
    +    @Captor
    +    private ArgumentCaptor<Collection<Tuple>> anchorCaptor;
    +    @Captor
    +    private ArgumentCaptor<String> stringCaptor;
    +    @Captor
    +    private ArgumentCaptor<Long> longCaptor;
    +    @Captor
    +    private ArgumentCaptor<Values> valuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<TupleWindow> tupleWindowCaptor;
    +    @Captor
    +    private ArgumentCaptor<Deque<Long>> partitionValuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<PersistentWindowedBoltExecutor.WindowPartition<Event<Tuple>>> windowValuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<Object> systemValuesCaptor;
    +
    +    @Before
    +    public void setUp() throws Exception {
    +        MockitoAnnotations.initMocks(this);
    +        mockBolt = Mockito.mock(IStatefulWindowedBolt.class);
    +        mockWaterMarkEventGenerator = Mockito.mock(WaterMarkEventGenerator.class);
    +        mockTimestampExtractor = Mockito.mock(TimestampExtractor.class);
    +        tupleTs = System.currentTimeMillis();
    +        Mockito.when(mockTimestampExtractor.extractTimestamp(Mockito.any())).thenReturn(tupleTs);
    +        Mockito.when(mockBolt.getTimestampExtractor()).thenReturn(mockTimestampExtractor);
    +        Mockito.when(mockBolt.isPersistent()).thenReturn(true);
    +        mockPartitionState = Mockito.mock(KeyValueState.class);
    +        mockWindowState = Mockito.mock(KeyValueState.class);
    +        mockSystemState = Mockito.mock(KeyValueState.class);
    +        mockTopologyContext = Mockito.mock(TopologyContext.class);
    +        Mockito.when(mockTopologyContext.getThisStreams()).thenReturn(Collections.singleton(LATE_STREAM));
    +        mockOutputCollector = Mockito.mock(OutputCollector.class);
    +        executor = new PersistentWindowedBoltExecutor<>(mockBolt);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_WINDOW_LENGTH_COUNT, 5);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_SLIDING_INTERVAL_COUNT, 5);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_LATE_TUPLE_STREAM, LATE_STREAM);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_WATERMARK_EVENT_INTERVAL_MS, 100_000);
    +        mockStormConf.put(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS, 30);
    +        mockStormConf.put(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL, 1000);
    +        Mockito.when(mockPartitionState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(mockWindowState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(mockSystemState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        executor.prepare(mockStormConf, mockTopologyContext, mockOutputCollector,
    +            mockWindowState, mockPartitionState, mockSystemState);
    +    }
    +
    +    @Test
    +    public void testExecuteTuple() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), 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);
    +    }
    +
    +    @Test
    +    public void testExecuteLatetuple() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), 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
    +        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());
    +    }
    +
    +    @Test
    +    public void testActivation() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), Mockito.anyLong())).thenReturn(true);
    +        executor.initState(null);
    +        executor.waterMarkEventGenerator = mockWaterMarkEventGenerator;
    +
    +        List<Tuple> mockTuples = getMockTuples(5);
    +        mockTuples.forEach(t -> executor.execute(t));
    +        // all tuples acked
    +        Mockito.verify(mockOutputCollector, Mockito.times(5)).ack(tupleCaptor.capture());
    +        Assert.assertArrayEquals(mockTuples.toArray(), tupleCaptor.getAllValues().toArray());
    +
    +        Mockito.doAnswer(new Answer<Void>() {
    +            @Override
    +            public Void answer(InvocationOnMock invocation) throws Throwable {
    +                TupleWindow window = (TupleWindow) invocation.getArguments()[0];
    +                // iterate the tuples
    +                Assert.assertEquals(5, window.get().size());
    +                // iterating multiple times should produce same events
    +                Assert.assertEquals(5, window.get().size());
    +                Assert.assertEquals(5, window.get().size());
    +                return null;
    +            }
    +        }).when(mockBolt).execute(Mockito.any());
    +        // trigger the window
    +        long activationTs = tupleTs + 1000;
    +        executor.getWindowManager().add(new WaterMarkEvent<>(activationTs));
    +        executor.prePrepare(0);
    +
    +        // partition ids
    +        Mockito.verify(mockPartitionState, Mockito.times(1)).put(stringCaptor.capture(), partitionValuesCaptor.capture());
    +        Assert.assertEquals(PARTITION_KEY, stringCaptor.getAllValues().get(0));
    +        Assert.assertEquals(Collections.singletonList(0L), partitionValuesCaptor.getAllValues().get(0));
    +
    +        // window partitions
    +        Mockito.verify(mockWindowState, Mockito.times(1)).put(longCaptor.capture(), windowValuesCaptor.capture());
    +        Assert.assertEquals(0L, (long) longCaptor.getAllValues().get(0));
    +        Assert.assertEquals(5, windowValuesCaptor.getAllValues().get(0).size());
    +        List<Event<?>> tuples = windowValuesCaptor.getAllValues().get(0)
    +            .getEvents().stream().map(Event::get).collect(Collectors.toList());
    +        Assert.assertArrayEquals(mockTuples.toArray(), tuples.toArray());
    +
    +        // window system state
    +        Mockito.verify(mockSystemState, Mockito.times(2)).put(stringCaptor.capture(), systemValuesCaptor.capture());
    +        Assert.assertEquals(EVICTION_STATE_KEY, stringCaptor.getAllValues().get(1));
    +        Assert.assertEquals(Optional.of(Pair.of(5L, 5L)), systemValuesCaptor.getAllValues().get(0));
    +        Assert.assertEquals(TRIGGER_STATE_KEY, stringCaptor.getAllValues().get(2));
    +        Assert.assertEquals(Optional.of(tupleTs), systemValuesCaptor.getAllValues().get(1));
    +    }
    +
    +    @Test
    +    public void testCacheEviction() {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), Mockito.anyLong())).thenReturn(true);
    +        executor.initState(null);
    +        executor.waterMarkEventGenerator = mockWaterMarkEventGenerator;
    +        List<Tuple> mockTuples = getMockTuples(20000);
    +        mockTuples.forEach(t -> executor.execute(t));
    +
    +        Mockito.verify(mockWindowState, Mockito.times(10)).put(longCaptor.capture(), windowValuesCaptor.capture());
    +        Assert.assertEquals(10, longCaptor.getAllValues().size());
    +        Assert.assertEquals(10, windowValuesCaptor.getAllValues().size());
    +        // number of evicted events
    +        Assert.assertEquals(10_000, windowValuesCaptor.getAllValues().stream()
    +            .mapToInt(x -> x.size()).sum());
    +
    +        Map<Long, PersistentWindowedBoltExecutor.WindowPartition<Event<Tuple>>> partitionMap = new HashMap<>();
    +        windowValuesCaptor.getAllValues().forEach(v -> partitionMap.put(v.getId(), v));
    +
    +        Mockito.verify(mockPartitionState, Mockito.times(20)).put(stringCaptor.capture(), partitionValuesCaptor.capture());
    +        Assert.assertEquals(LongStream.range(0, 20).boxed().collect(Collectors.toList()), partitionValuesCaptor.getAllValues().get(19));
    +
    +        Mockito.when(mockWindowState.get(Mockito.any(), Mockito.any())).then(new Answer<Object>() {
    +            @Override
    +            public Object answer(InvocationOnMock invocation) throws Throwable {
    +                Object[] args = invocation.getArguments();
    +                PersistentWindowedBoltExecutor.WindowPartition<Event<Tuple>> evicted = partitionMap.get(args[0]);
    +                return evicted != null ? evicted : args[1];
    +            }
    +        });
    +
    +        Mockito.doAnswer(new Answer<Void>() {
    +            @Override
    +            public Void answer(InvocationOnMock invocation) throws Throwable {
    +                Object[] args = invocation.getArguments();
    +                partitionMap.put((long)args[0], (PersistentWindowedBoltExecutor.WindowPartition<Event<Tuple>>)args[1]);
    +                return null;
    +            }
    +        }).when(mockWindowState).put(Mockito.any(), Mockito.any());
    +
    +        // trigger the window
    +        long activationTs = tupleTs + 1000;
    +        executor.getWindowManager().add(new WaterMarkEvent<>(activationTs));
    +
    +        Mockito.verify(mockBolt, Mockito.times(4000)).execute(Mockito.any());
    --- End diff --
    
    Why 4000?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130273744
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutor.java ---
    @@ -0,0 +1,596 @@
    +/**
    + * 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 java.util.AbstractCollection;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Objects;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentLinkedQueue;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.locks.ReentrantLock;
    +import java.util.function.Supplier;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.state.KeyValueState;
    +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.topology.base.BaseWindowedBolt;
    +import org.apache.storm.tuple.Tuple;
    +import org.apache.storm.windowing.DefaultEvictionContext;
    +import org.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.EventImpl;
    +import org.apache.storm.windowing.WindowLifecycleListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static java.util.Collections.emptyIterator;
    +import static org.apache.storm.topology.WindowPartitionCache.CacheLoader;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalCause;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalListener;
    +
    +/**
    + * Wraps a {@link IStatefulWindowedBolt} and handles the execution. Uses state and the underlying
    + * checkpointing mechanisms to save the tuples in window to state. The tuples are also kept in-memory
    + * by transparently caching the window partitions and checkpointing them as needed.
    + */
    +public class PersistentWindowedBoltExecutor<T extends State> extends WindowedBoltExecutor implements IStatefulBolt<T> {
    +    private static final Logger LOG = LoggerFactory.getLogger(PersistentWindowedBoltExecutor.class);
    +    private final IStatefulWindowedBolt<T> statefulWindowedBolt;
    +    private transient TopologyContext topologyContext;
    +    private transient OutputCollector outputCollector;
    +    private transient WindowState<Tuple> state;
    +    private transient boolean stateInitialized;
    +    private transient boolean prePrepared;
    +
    +    public PersistentWindowedBoltExecutor(IStatefulWindowedBolt<T> bolt) {
    +        super(bolt);
    +        statefulWindowedBolt = bolt;
    +    }
    +
    +    @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));
    +    }
    +
    +    @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);
    +        }
    +    }
    +
    +    private int getCheckpointIntervalMillis(Map<String, Object> topoConf) {
    +        int checkpointInterval = Integer.MAX_VALUE;
    +        if (topoConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL) != null) {
    +            checkpointInterval = ((Number) topoConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL)).intValue();
    +        }
    +        return checkpointInterval;
    +    }
    +
    +    // package access for unit tests
    +    void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                 KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                 KeyValueState<String, Deque<Long>> partitionState,
    +                 KeyValueState<String, Optional<?>> windowSystemState) {
    +        init(topoConf, context, collector, windowState, partitionState, windowSystemState);
    +        doPrepare(topoConf, context, new NoAckOutputCollector(collector), state, true);
    +        Map<String, Optional<?>> wstate = new HashMap<>();
    +        windowSystemState.forEach(s -> wstate.put(s.getKey(), s.getValue()));
    +        restoreState(wstate);
    +    }
    +
    +    @Override
    +    protected void start() {
    +        if (stateInitialized) {
    +            super.start();
    +        } else {
    +            LOG.debug("Will invoke start after state is initialized");
    +        }
    +    }
    +
    +    @Override
    +    public void execute(Tuple input) {
    +        if (!stateInitialized) {
    +            throw new IllegalStateException("execute invoked before initState with input tuple " + input);
    +        }
    +        super.execute(input);
    +        // StatefulBoltExecutor does the actual ack when the state is saved.
    +        outputCollector.ack(input);
    +    }
    +
    +    @Override
    +    public void initState(T state) {
    +        if (stateInitialized) {
    +            String msg = "initState invoked when the state is already initialized";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        } else {
    +            statefulWindowedBolt.initState(state);
    +            stateInitialized = true;
    +            start();
    +        }
    +    }
    +
    +    @Override
    +    public void prePrepare(long txid) {
    +        if (stateInitialized) {
    +            LOG.debug("Prepare streamState, txid {}", txid);
    +            state.prepareCommit(txid);
    +            prePrepared = true;
    +        } else {
    +            String msg = "Cannot prepare before initState";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        }
    +    }
    +
    +    @Override
    +    public void preCommit(long txid) {
    +        // preCommit can be invoked during recovery before the state is initialized
    +        if (prePrepared || !stateInitialized) {
    +            LOG.debug("Commit streamState, txid {}", txid);
    +            state.commit(txid);
    +        } else {
    +            String msg = "preCommit before prePrepare in initialized state";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        }
    +    }
    +
    +    @Override
    +    public void preRollback() {
    +        LOG.debug("Rollback streamState, stateInitialized {}", stateInitialized);
    +        state.rollback();
    +    }
    +
    +    @Override
    +    protected WindowLifecycleListener<Tuple> newWindowLifecycleListener() {
    +        return new WindowLifecycleListener<Tuple>() {
    +            @Override
    +            public void onExpiry(List<Tuple> events) {
    +                /*
    +                 * NO-OP: the events are ack-ed in execute
    +                 */
    +            }
    +
    +            @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();
    +            }
    +        };
    +    }
    +
    +    private void init(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                      KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                      KeyValueState<String, Deque<Long>> partitionState,
    +                      KeyValueState<String, Optional<?>> windowSystemState) {
    +        topologyContext = context;
    +        outputCollector = collector;
    +        state = new WindowState<>(windowState, partitionState, windowSystemState, this::getState,
    +            statefulWindowedBolt.maxEventsInMemory());
    +    }
    +
    +    private KeyValueState<Long, WindowPartition<Tuple>> getWindowState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window";
    +        return (KeyValueState<Long, WindowPartition<Tuple>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Deque<Long>> getPartitionState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-partitions";
    +        return (KeyValueState<String, Deque<Long>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Optional<?>> getWindowSystemState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-systemstate";
    +        return (KeyValueState<String, Optional<?>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    // a wrapper around the window related states that are checkpointed
    +    private static class WindowState<T> extends AbstractCollection<Event<T>> {
    +        // number of events per window-partition
    +        private static final int MAX_PARTITION_EVENTS = 1000;
    +        private static final int MIN_PARTITIONS = 10;
    +        private static final String PARTITION_IDS_KEY = "pk";
    +        private final KeyValueState<String, Deque<Long>> partitionIdsState;
    +        private final KeyValueState<Long, WindowPartition<T>> windowPartitionsState;
    +        private final KeyValueState<String, Optional<?>> windowSystemState;
    +        // ordered partition keys
    +        private Deque<Long> partitionIds;
    +        private volatile long latestPartitionId;
    +        private WindowPartitionCache<Long, WindowPartition<T>> cache;
    +        private Supplier<Map<String, Optional<?>>> windowSystemStateSupplier;
    +        private final ReentrantLock partitionIdsLock = new ReentrantLock(true);
    +        private final ReentrantLock windowPartitionsLock = new ReentrantLock(true);
    +        private final long maxEventsInMemory;
    +        private WindowPartition<T> latestPartition;
    +        private Set<Long> iteratorPins = new HashSet<>();
    +
    +        WindowState(KeyValueState<Long, WindowPartition<T>> windowPartitionsState,
    +                    KeyValueState<String, Deque<Long>> partitionIdsState,
    +                    KeyValueState<String, Optional<?>> windowSystemState,
    +                    Supplier<Map<String, Optional<?>>> windowSystemStateSupplier,
    +                    long maxEventsInMemory) {
    +            this.windowPartitionsState = windowPartitionsState;
    +            this.partitionIdsState = partitionIdsState;
    +            this.windowSystemState = windowSystemState;
    +            this.windowSystemStateSupplier = windowSystemStateSupplier;
    +            this.maxEventsInMemory = Math.max(MAX_PARTITION_EVENTS * MIN_PARTITIONS, maxEventsInMemory);
    +            initCache();
    +            initPartitions();
    +        }
    +
    +        @Override
    +        public boolean add(Event<T> event) {
    +            if (latestPartition.size() >= MAX_PARTITION_EVENTS) {
    +                cache.unpin(latestPartition.getId());
    +                latestPartition = getPinnedPartition(getNextPartitionId());
    +            }
    +            latestPartition.add(event);
    +            return true;
    +        }
    +
    +        @Override
    +        public Iterator<Event<T>> iterator() {
    +
    +            return new Iterator<Event<T>>() {
    +                private Iterator<Long> ids = getIds();
    +                private Iterator<Event<T>> current = emptyIterator();
    +                private Iterator<Event<T>> removeFrom;
    +                private WindowPartition<T> curPartition;
    +
    +                private Iterator<Long> getIds() {
    +                    try {
    +                        partitionIdsLock.lock();
    +                        LOG.debug("Iterator partitionIds: {}", partitionIds);
    +                        return new ArrayList<>(partitionIds).iterator();
    +                    } finally {
    +                        partitionIdsLock.unlock();
    +                    }
    +                }
    +
    +                @Override
    +                public void remove() {
    +                    if (removeFrom == null) {
    +                        throw new IllegalStateException("No calls to next() since last call to remove()");
    +                    }
    +                    removeFrom.remove();
    +                    removeFrom = null;
    +                }
    +
    +                @Override
    +                public boolean hasNext() {
    +                    boolean curHasNext = current.hasNext();
    +                    while (!curHasNext && ids.hasNext()) {
    +                        if (curPartition != null) {
    +                            unpin(curPartition.getId());
    +                        }
    +                        curPartition = getPinnedPartition(ids.next());
    +                        if (curPartition != null) {
    +                            iteratorPins.add(curPartition.getId());
    +                            current = curPartition.iterator();
    +                            curHasNext = current.hasNext();
    +                        }
    +                    }
    +                    // un-pin the last partition
    +                    if (!curHasNext && curPartition != null) {
    +                        unpin(curPartition.getId());
    +                        curPartition = null;
    +                    }
    +                    return curHasNext;
    +                }
    +
    +                @Override
    +                public Event<T> next() {
    +                    if (!hasNext()) {
    +                        throw new NoSuchElementException();
    +                    }
    +                    removeFrom = current;
    +                    return current.next();
    +                }
    +
    +                private void unpin(long id) {
    +                    cache.unpin(id);
    +                    iteratorPins.remove(id);
    +                }
    +            };
    +        }
    +
    +        void clearIteratorPins() {
    +            LOG.debug("clearIteratorPins '{}'", iteratorPins);
    +            Iterator<Long> it = iteratorPins.iterator();
    +            while (it.hasNext()) {
    +                cache.unpin(it.next());
    +                it.remove();
    +            }
    +        }
    +
    +        @Override
    +        public int size() {
    +            throw new UnsupportedOperationException();
    +        }
    +
    +        void prepareCommit(long txid) {
    +            flush();
    +            partitionIdsState.prepareCommit(txid);
    +            windowPartitionsState.prepareCommit(txid);
    +            windowSystemState.prepareCommit(txid);
    +        }
    +
    +        void commit(long txid) {
    +            partitionIdsState.commit(txid);
    +            windowPartitionsState.commit(txid);
    +            windowSystemState.commit(txid);
    +        }
    +
    +        void rollback() {
    +            partitionIdsState.rollback();
    +            windowPartitionsState.rollback();
    +            windowSystemState.rollback();
    +        }
    +
    +        private void initPartitions() {
    +            partitionIds = partitionIdsState.get(PARTITION_IDS_KEY, new LinkedList<>());
    +            if (partitionIds.isEmpty()) {
    +                partitionIds.add(0L);
    +                partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    +            }
    +            latestPartitionId = partitionIds.peekLast();
    +            latestPartition = cache.getPinned(latestPartitionId);
    +        }
    +
    +        private void initCache() {
    +            long size = maxEventsInMemory / MAX_PARTITION_EVENTS;
    +            LOG.info("maxEventsInMemory: {}, partition size: {}, number of partitions: {}",
    +                maxEventsInMemory, MAX_PARTITION_EVENTS, size);
    +            cache = SimpleWindowPartitionCache.<Long, WindowPartition<T>>newBuilder()
    +                .maximumSize(size)
    +                .removalListener(new RemovalListener<Long, WindowPartition<T>>() {
    +                    @Override
    +                    public void onRemoval(Long pid, WindowPartition<T> p, RemovalCause removalCause) {
    +                        Objects.requireNonNull(pid, "Null partition id");
    +                        Objects.requireNonNull(p, "Null window partition");
    +                        LOG.debug("onRemoval for id '{}', WindowPartition '{}'", pid, p);
    +                        try {
    +                            windowPartitionsLock.lock();
    +                            if (p.isEmpty() && pid != latestPartitionId) {
    +                                // if the empty partition was not invalidated by flush, but evicted from cache
    +                                if (removalCause != RemovalCause.EXPLICIT) {
    +                                    deletePartition(pid);
    +                                    windowPartitionsState.delete(pid);
    +                                }
    +                            } else if (p.isModified()) {
    +                                windowPartitionsState.put(pid, p);
    +                            } else {
    +                                LOG.debug("WindowPartition '{}' is not modified", pid);
    +                            }
    +                        } finally {
    +                            windowPartitionsLock.unlock();
    +                        }
    +                    }
    +                }).build(new CacheLoader<Long, WindowPartition<T>>() {
    +                    @Override
    +                   public WindowPartition<T> load(Long id) {
    +                        LOG.debug("Load partition: {}", id);
    +                        // load from state
    +                        try {
    +                            windowPartitionsLock.lock();
    +                            return windowPartitionsState.get(id, new WindowPartition<>(id));
    +                        } finally {
    +                            windowPartitionsLock.unlock();
    +                        }
    +                    }
    +                    }
    +                );
    +        }
    +
    +        private void deletePartition(long pid) {
    +            LOG.debug("Delete partition: {}", pid);
    +            try {
    +                partitionIdsLock.lock();
    +                partitionIds.remove(pid);
    +                partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    +            } finally {
    +                partitionIdsLock.unlock();
    +            }
    +        }
    +
    +        private long getNextPartitionId() {
    +            try {
    +                partitionIdsLock.lock();
    +                partitionIds.add(++latestPartitionId);
    +                partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    +            } finally {
    +                partitionIdsLock.unlock();
    +            }
    +            return latestPartitionId;
    +        }
    +
    +        private WindowPartition<T> getPinnedPartition(long id) {
    +            return cache.getPinned(id);
    +        }
    +
    +        private void flush() {
    +            LOG.debug("Flushing modified partitions");
    +            cache.asMap().forEach((pid, p) -> {
    +                Long pidToInvalidate = null;
    +                try {
    +                    windowPartitionsLock.lock();
    +                    if (p.isEmpty() && pid != latestPartitionId) {
    +                        LOG.debug("Invalidating empty partition {}", pid);
    +                        deletePartition(pid);
    +                        windowPartitionsState.delete(pid);
    +                        pidToInvalidate = pid;
    +                    } else if (p.isModified()) {
    +                        LOG.debug("Updating modified partition {}", pid);
    +                        p.clearModified();
    +                        windowPartitionsState.put(pid, p);
    +                    }
    +                } finally {
    +                    windowPartitionsLock.unlock();
    +                }
    +                // invalidate after releasing the lock
    +                if (pidToInvalidate != null) {
    +                    cache.invalidate(pidToInvalidate);
    +                }
    +            });
    +            windowSystemStateSupplier.get().forEach(windowSystemState::put);
    +        }
    +    }
    +
    +    // the window partition that holds the events
    +    public static class WindowPartition<T> implements Iterable<Event<T>> {
    +        private final ConcurrentLinkedQueue<Event<T>> events = new ConcurrentLinkedQueue<>();
    +        private final AtomicInteger size = new AtomicInteger();
    +        private final long id;
    +        private transient volatile boolean modified;
    +
    +        public WindowPartition(long id) {
    +            this.id = id;
    +        }
    +
    +        void add(Event<T> event) {
    +            events.add(event);
    +            size.incrementAndGet();
    +            setModified();
    +        }
    +
    +        boolean isModified() {
    +            return modified;
    +        }
    +
    +        void setModified() {
    +            if (!modified) {
    +                modified = true;
    +            }
    +        }
    +
    +        void clearModified() {
    +            if (modified) {
    --- End diff --
    
    yes, since its not too frequent its not required.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129110842
  
    --- Diff: docs/Windowing.md ---
    @@ -266,3 +266,105 @@ tuples can be received within the timeout period.
     An example toplogy `SlidingWindowTopology` shows how to use the apis to compute a sliding window sum and a tumbling window 
     average.
     
    +## Stateful windowing
    +The default windowing implementation in storm stores the tuples in memory until they are processed and expired from the 
    +window. This limits the use cases to windows that
    +fit entirely in memory. Also the source tuples cannot be ack-ed until the window expiry requiring large message timeouts
    +(topology.message.timeout.secs should be larger than the window length + sliding interval). This also puts extra loads 
    +due to the complex acking and anchoring requirements.
    + 
    +To address the above limitations and to support larger window sizes, storm provides stateful windowing support via `IStatefulWindowedBolt`. 
    +User bolts should typically extend `BaseStatefulWindowedBolt` for the windowing operations with the framework automatically 
    +managing the state of the window in the background.
    +
    +If the sources provide a monotonically increasing identifier as a part of the message, the framework can use this
    +to periodically checkpoint the last expired and evaluated message ids, to avoid duplicate window evaluations in case of 
    +failures or restarts. During recovery, the tuples with message ids lower than last expired id are discarded and tuples with 
    +message id between the last expired and last evaluated message ids are fed into the system without activating any triggers. 
    +The tuples beyond the last evaluated message ids are processed as usual. This can be enabled by setting
    +the `messageIdField` as shown below,
    +
    +```java
    +topologyBuilder.setBolt("mybolt",
    +                   new MyStatefulWindowedBolt()
    +                   .withWindow(...) // windowing configuarations
    +                   .withMessageIdField("msgid"), // a monotonically increasing 'long' field in the tuple
    +                   parallelism)
    +               .shuffleGrouping("spout");
    +```
    +
    +However, this option is feasible only if the sources can provide a monotonically increasing identifier in the tuple and the same is maintained
    +while re-emitting the messages in case of failures. Here the tuples in window are still held in memory.
    +
    +For more details take a look at the sample topology in storm starter `StatefulWindowingTopology` which will help you get started.
    +
    +### Window checkpointing
    +
    +With window checkpointing, the monotonically increasing id is no-longer required since the framework transparently saves the state of the window periodically into the configured state backend.
    +The state that is saved includes the tuples in the window, any system state that is required to recover the state of processing
    +and also the user state.
    +
    +```java
    +topologyBuilder.setBolt("mybolt",
    +                   new MyStatefulPersistentWindowedBolt()
    +                   .withWindow(...) // windowing configuarations
    +                   .withPersistence() // persist the window state
    +                   .withMaxEventsInMemory(25000), // max number of events to be cached in memory
    +                    parallelism)
    +               .shuffleGrouping("spout");
    +
    +```
    +
    +The `withPersistence` instructs the framework to transparently save the tuples in window along with
    +any associated system and user state to the state backend. The `withMaxEventsInMemory` is an optional 
    +configuration that specifies the maximum number of tuples that may be kept in memory. The tuples are transparently loaded from 
    +the state backend as required and the ones that are most likely to be used again are retained in memory.
    +
    +The state backend can be configured by setting the topology state provider config,
    +
    +```java
    +// use redis for state persistence
    +conf.put(Config.TOPOLOGY_STATE_PROVIDER, "org.apache.storm.redis.state.RedisKeyValueStateProvider");
    +
    +```
    +Currently storm supports Redis and HBase as state backends and uses the underlying state-checkpointing
    +framework for saving the window state. For more details on state checkpointing see [State-checkpointing.md](State-checkpointing.md)
    +
    +Here is an example of a persistent windowed bolt that uses the window checkpointing to save its state. The `initState`
    +is invoked with the last saved state (user state) at initialization time. The execute method is invoked based on the configured
    +windowing parameters and the tuples in the active window can be accessed via an `iterator` as shown below.
    +
    +```java
    +public class MyStatefulPersistentWindowedBolt extends BaseStatefulWindowedBolt<K, V> {
    +  private KeyValueState<K, V> state;
    +  
    +  @Override
    +  public void initState(KeyValueState<K, V> state) {
    +    this.state = state;
    +   // ...
    +   // restore the state from the last saved state.
    +   // ...
    +  }
    +  
    +  @Override
    +  public void execute(TupleWindow window) {      
    +    // iterate over tuples in the current window
    +    Iterator<Tuple> it = window.getIter();
    +    while (it.hasNext()) {
    +        // compute some result based on the tuples in window
    +    }
    +    
    +    // possibly update any state to be maintained across windows
    +    state.put(STATE_KEY, updatedValue);
    +    
    +    // emit the results downstream
    +    collector.emit(new Values(result));
    +  }
    +}
    +```
    +
    +Note: In case of persistent windowed bolts, use `TupleWindow.getIter` to retrieve an iterator over the 
    +events in the window. If the number of tuples in windows are huge, invoking `TupleWindow.get` would
    +try to load all the tuples into memory and may throw an OOM exception.
    +
    +For more details take a look at the sample topology in storm starter `PersistentWindowingTopology` which will help you get started.
    --- End diff --
    
    Nit: storm-starter, and maybe link


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r128200537
  
    --- Diff: storm-client/src/jvm/org/apache/storm/state/DefaultStateEncoder.java ---
    @@ -18,7 +18,8 @@
     
     package org.apache.storm.state;
     
    -import com.google.common.base.Optional;
    +
    +import java.util.Optional;
    --- End diff --
    
    Great to replace a Guava thing with Java 8 support.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129121999
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/IStatefulWindowedBolt.java ---
    @@ -15,12 +15,33 @@
      * 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 windowed bolt abstraction for supporting windowing operation with state
    + * A windowed bolt abstraction for supporting windowing operation with state.
      */
     public interface IStatefulWindowedBolt<T extends State> extends IStatefulComponent<T>, IWindowedBolt {
    +    /**
    +     * If the stateful windowed bolt should have its windows persisted in state and maintain a subset
    +     * (recent events) in memory.
    +     * <p>
    --- End diff --
    
    Nit: On purpose that there are two paragraph tags here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129763359
  
    --- Diff: docs/Windowing.md ---
    @@ -266,3 +266,105 @@ tuples can be received within the timeout period.
     An example toplogy `SlidingWindowTopology` shows how to use the apis to compute a sliding window sum and a tumbling window 
     average.
     
    +## Stateful windowing
    +The default windowing implementation in storm stores the tuples in memory until they are processed and expired from the 
    +window. This limits the use cases to windows that
    +fit entirely in memory. Also the source tuples cannot be ack-ed until the window expiry requiring large message timeouts
    +(topology.message.timeout.secs should be larger than the window length + sliding interval). This also puts extra loads 
    +due to the complex acking and anchoring requirements.
    + 
    +To address the above limitations and to support larger window sizes, storm provides stateful windowing support via `IStatefulWindowedBolt`. 
    +User bolts should typically extend `BaseStatefulWindowedBolt` for the windowing operations with the framework automatically 
    +managing the state of the window in the background.
    +
    +If the sources provide a monotonically increasing identifier as a part of the message, the framework can use this
    +to periodically checkpoint the last expired and evaluated message ids, to avoid duplicate window evaluations in case of 
    +failures or restarts. During recovery, the tuples with message ids lower than last expired id are discarded and tuples with 
    +message id between the last expired and last evaluated message ids are fed into the system without activating any triggers. 
    +The tuples beyond the last evaluated message ids are processed as usual. This can be enabled by setting
    +the `messageIdField` as shown below,
    +
    +```java
    +topologyBuilder.setBolt("mybolt",
    +                   new MyStatefulWindowedBolt()
    +                   .withWindow(...) // windowing configuarations
    +                   .withMessageIdField("msgid"), // a monotonically increasing 'long' field in the tuple
    +                   parallelism)
    +               .shuffleGrouping("spout");
    +```
    +
    +However, this option is feasible only if the sources can provide a monotonically increasing identifier in the tuple and the same is maintained
    +while re-emitting the messages in case of failures. Here the tuples in window are still held in memory.
    --- End diff --
    
    will reword: With this option the tuples are still buffered in memory until processed and expired from the window.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by satishd <gi...@git.apache.org>.
Github user satishd commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r128991661
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/base/BaseStatefulWindowedBolt.java ---
    @@ -151,6 +158,38 @@
             return this;
         }
     
    +    /**
    +     * If set, the stateful windowed bolt would use the backend state for window persistence and
    +     * only keep a sub-set of events in memory as specified by {@link #withMaxEventsInMemory(long)}.
    +     */
    +    public BaseStatefulWindowedBolt<T> withPersistence() {
    +        persistent = true;
    +        return this;
    +    }
    +
    +    /**
    +     * The maximum number of window events to keep in memory. This is meaningful only if
    +     * {@link #withPersistence()} is also set. As the number of events in memory grows close
    +     * to the maximum, the events that are less likely to be used again are evicted and persisted.
    +     * The default value for this is {@code 1,000,000}.
    +     *
    +     * @param maxEventsInMemory the maximum number of window events to keep in memory
    +     */
    +    public BaseStatefulWindowedBolt<T> withMaxEventsInMemory(long maxEventsInMemory) {
    --- End diff --
    
    We can explore later limiting with memory size as that would give better handle to the user on memory resources, which can be with onheap/offheap cache. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130227886
  
    --- Diff: storm-client/test/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutorTest.java ---
    @@ -0,0 +1,261 @@
    +/**
    + * 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.Config;
    +import org.apache.storm.generated.GlobalStreamId;
    +import org.apache.storm.state.KeyValueState;
    +import org.apache.storm.streams.Pair;
    +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.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.TimestampExtractor;
    +import org.apache.storm.windowing.TupleWindow;
    +import org.apache.storm.windowing.WaterMarkEvent;
    +import org.apache.storm.windowing.WaterMarkEventGenerator;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.mockito.ArgumentCaptor;
    +import org.mockito.Captor;
    +import org.mockito.Mockito;
    +import org.mockito.MockitoAnnotations;
    +import org.mockito.invocation.InvocationOnMock;
    +import org.mockito.runners.MockitoJUnitRunner;
    +import org.mockito.stubbing.Answer;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.stream.Collectors;
    +import java.util.stream.LongStream;
    +
    +import static org.mockito.AdditionalAnswers.returnsArgAt;
    +
    +/**
    + * Unit tests for {@link PersistentWindowedBoltExecutor}
    + */
    +@RunWith(MockitoJUnitRunner.class)
    +public class PersistentWindowedBoltExecutorTest {
    +    public static final String LATE_STREAM = "late_stream";
    +    private PersistentWindowedBoltExecutor<KeyValueState<String, String>> executor;
    +    private IStatefulWindowedBolt<KeyValueState<String, String>> mockBolt;
    +    private OutputCollector mockOutputCollector;
    +    private TopologyContext mockTopologyContext;
    +    private Map<String, Object> mockStormConf = new HashMap<>();
    +    private TimestampExtractor mockTimestampExtractor;
    +    private WaterMarkEventGenerator mockWaterMarkEventGenerator;
    +    private KeyValueState mockPartitionState;
    --- End diff --
    
    Nit: Could probably avoid rawtypes complaints by using [at]Mock similar to what you've done with Captor


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #2218: STORM-2614: Enhance stateful windowing to persist the win...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on the issue:

    https://github.com/apache/storm/pull/2218
  
    @srdo , @satishd,  thanks for the review. Pushed some changes to address the latest review comments.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130274015
  
    --- Diff: storm-client/src/jvm/org/apache/storm/windowing/TupleWindowIterImpl.java ---
    @@ -0,0 +1,80 @@
    +/**
    + * 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.windowing;
    +
    +import com.google.common.collect.Iterators;
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.function.Supplier;
    +import org.apache.storm.tuple.Tuple;
    +
    +/**
    + * An iterator based implementation over the events in a window.
    + */
    +public class TupleWindowIterImpl implements TupleWindow {
    +    private final Supplier<Iterator<Tuple>> tuplesIt;
    +    private final Supplier<Iterator<Tuple>> newTuplesIt;
    +    private final Supplier<Iterator<Tuple>> expiredTuplesIt;
    +    private final Long startTimestamp;
    +    private final Long endTimestamp;
    +
    +    public TupleWindowIterImpl(Supplier<Iterator<Tuple>> tuplesIt,
    +                               Supplier<Iterator<Tuple>> newTuplesIt,
    +                               Supplier<Iterator<Tuple>> expiredTuplesIt,
    +                               Long startTimestamp, Long endTimestamp) {
    +        this.tuplesIt = tuplesIt;
    +        this.newTuplesIt = newTuplesIt;
    +        this.expiredTuplesIt = expiredTuplesIt;
    +        this.startTimestamp = startTimestamp;
    +        this.endTimestamp = endTimestamp;
    +    }
    +
    +    @Override
    +    public List<Tuple> get() {
    --- End diff --
    
    yes, its mentioned in the doc. Will add a Javadoc comment.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #2218: STORM-2614: Enhance stateful windowing to persist the win...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on the issue:

    https://github.com/apache/storm/pull/2218
  
    This builds on top of the existing state checkpointing mechanism (documented here - https://github.com/apache/storm/blob/master/docs/State-checkpointing.md). Theres nothing extra added to the underlying checkpointing mechanism itself and its pretty straightforward. 
    
    The tuples in window (think a FIFO queue) are split into multiple partitions so that they are more manageable and can be distributed/sharded via the underlying key-value state (redis/hbase etc). The modified partitions are saved during a checkpoint. During iteration the partition are loaded on demand from the underlying state backend as they are accessed. A subset of the most recently used partitions are cached in memory (backed by a guava Loading Cache).
    
    During checkpoint, the following are saved :
    1. Any modified or newly created window partitions.
    2. Any state needed to recover the Trigger/Eviction policies.
    3. State thats exposed to the user where the user may have saved some values.
    
    Since the KV state does not guarantee any specific ordering of the keys during iteration, a separate structure is maintained to store the ordered partition Ids which is used during iteration to retrieve the partitions in order. This is also saved during the checkpoint.
    
    The above mechanism kicks in only if user choses to use the windowed state persistence, otherwise the current behavior (keeping the tuples in an in-memory queue) is retained.
    
    I would like to capture the design details in the PR/Documentation itself as far as possible than adding it in separate docs which tend to get lost.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130273878
  
    --- Diff: storm-client/test/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutorTest.java ---
    @@ -0,0 +1,261 @@
    +/**
    + * 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.Config;
    +import org.apache.storm.generated.GlobalStreamId;
    +import org.apache.storm.state.KeyValueState;
    +import org.apache.storm.streams.Pair;
    +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.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.TimestampExtractor;
    +import org.apache.storm.windowing.TupleWindow;
    +import org.apache.storm.windowing.WaterMarkEvent;
    +import org.apache.storm.windowing.WaterMarkEventGenerator;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.mockito.ArgumentCaptor;
    +import org.mockito.Captor;
    +import org.mockito.Mockito;
    +import org.mockito.MockitoAnnotations;
    +import org.mockito.invocation.InvocationOnMock;
    +import org.mockito.runners.MockitoJUnitRunner;
    +import org.mockito.stubbing.Answer;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.stream.Collectors;
    +import java.util.stream.LongStream;
    +
    +import static org.mockito.AdditionalAnswers.returnsArgAt;
    +
    +/**
    + * Unit tests for {@link PersistentWindowedBoltExecutor}
    + */
    +@RunWith(MockitoJUnitRunner.class)
    +public class PersistentWindowedBoltExecutorTest {
    +    public static final String LATE_STREAM = "late_stream";
    +    private PersistentWindowedBoltExecutor<KeyValueState<String, String>> executor;
    +    private IStatefulWindowedBolt<KeyValueState<String, String>> mockBolt;
    +    private OutputCollector mockOutputCollector;
    +    private TopologyContext mockTopologyContext;
    +    private Map<String, Object> mockStormConf = new HashMap<>();
    +    private TimestampExtractor mockTimestampExtractor;
    +    private WaterMarkEventGenerator mockWaterMarkEventGenerator;
    +    private KeyValueState mockPartitionState;
    +    private KeyValueState mockWindowState;
    +    private KeyValueState mockSystemState;
    +
    +    private static final String PARTITION_KEY = "pk";
    +    private static final String EVICTION_STATE_KEY = "es";
    +    private static final String TRIGGER_STATE_KEY = "ts";
    +    private long tupleTs;
    +
    +    @Captor
    +    private ArgumentCaptor<Tuple> tupleCaptor;
    +    @Captor
    +    private ArgumentCaptor<Collection<Tuple>> anchorCaptor;
    +    @Captor
    +    private ArgumentCaptor<String> stringCaptor;
    +    @Captor
    +    private ArgumentCaptor<Long> longCaptor;
    +    @Captor
    +    private ArgumentCaptor<Values> valuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<TupleWindow> tupleWindowCaptor;
    +    @Captor
    +    private ArgumentCaptor<Deque<Long>> partitionValuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<PersistentWindowedBoltExecutor.WindowPartition<Event<Tuple>>> windowValuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<Object> systemValuesCaptor;
    +
    +    @Before
    +    public void setUp() throws Exception {
    +        MockitoAnnotations.initMocks(this);
    +        mockBolt = Mockito.mock(IStatefulWindowedBolt.class);
    +        mockWaterMarkEventGenerator = Mockito.mock(WaterMarkEventGenerator.class);
    +        mockTimestampExtractor = Mockito.mock(TimestampExtractor.class);
    +        tupleTs = System.currentTimeMillis();
    +        Mockito.when(mockTimestampExtractor.extractTimestamp(Mockito.any())).thenReturn(tupleTs);
    +        Mockito.when(mockBolt.getTimestampExtractor()).thenReturn(mockTimestampExtractor);
    +        Mockito.when(mockBolt.isPersistent()).thenReturn(true);
    +        mockPartitionState = Mockito.mock(KeyValueState.class);
    +        mockWindowState = Mockito.mock(KeyValueState.class);
    +        mockSystemState = Mockito.mock(KeyValueState.class);
    +        mockTopologyContext = Mockito.mock(TopologyContext.class);
    +        Mockito.when(mockTopologyContext.getThisStreams()).thenReturn(Collections.singleton(LATE_STREAM));
    +        mockOutputCollector = Mockito.mock(OutputCollector.class);
    +        executor = new PersistentWindowedBoltExecutor<>(mockBolt);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_WINDOW_LENGTH_COUNT, 5);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_SLIDING_INTERVAL_COUNT, 5);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_LATE_TUPLE_STREAM, LATE_STREAM);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_WATERMARK_EVENT_INTERVAL_MS, 100_000);
    +        mockStormConf.put(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS, 30);
    +        mockStormConf.put(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL, 1000);
    +        Mockito.when(mockPartitionState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(mockWindowState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(mockSystemState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        executor.prepare(mockStormConf, mockTopologyContext, mockOutputCollector,
    +            mockWindowState, mockPartitionState, mockSystemState);
    +    }
    +
    +    @Test
    +    public void testExecuteTuple() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), 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);
    +    }
    +
    +    @Test
    +    public void testExecuteLatetuple() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), 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
    +        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());
    +    }
    +
    +    @Test
    +    public void testActivation() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), Mockito.anyLong())).thenReturn(true);
    +        executor.initState(null);
    +        executor.waterMarkEventGenerator = mockWaterMarkEventGenerator;
    +
    +        List<Tuple> mockTuples = getMockTuples(5);
    +        mockTuples.forEach(t -> executor.execute(t));
    +        // all tuples acked
    +        Mockito.verify(mockOutputCollector, Mockito.times(5)).ack(tupleCaptor.capture());
    +        Assert.assertArrayEquals(mockTuples.toArray(), tupleCaptor.getAllValues().toArray());
    +
    +        Mockito.doAnswer(new Answer<Void>() {
    +            @Override
    +            public Void answer(InvocationOnMock invocation) throws Throwable {
    +                TupleWindow window = (TupleWindow) invocation.getArguments()[0];
    +                // iterate the tuples
    +                Assert.assertEquals(5, window.get().size());
    +                // iterating multiple times should produce same events
    +                Assert.assertEquals(5, window.get().size());
    +                Assert.assertEquals(5, window.get().size());
    +                return null;
    +            }
    +        }).when(mockBolt).execute(Mockito.any());
    +        // trigger the window
    +        long activationTs = tupleTs + 1000;
    +        executor.getWindowManager().add(new WaterMarkEvent<>(activationTs));
    +        executor.prePrepare(0);
    +
    +        // partition ids
    +        Mockito.verify(mockPartitionState, Mockito.times(1)).put(stringCaptor.capture(), partitionValuesCaptor.capture());
    +        Assert.assertEquals(PARTITION_KEY, stringCaptor.getAllValues().get(0));
    +        Assert.assertEquals(Collections.singletonList(0L), partitionValuesCaptor.getAllValues().get(0));
    +
    +        // window partitions
    +        Mockito.verify(mockWindowState, Mockito.times(1)).put(longCaptor.capture(), windowValuesCaptor.capture());
    +        Assert.assertEquals(0L, (long) longCaptor.getAllValues().get(0));
    +        Assert.assertEquals(5, windowValuesCaptor.getAllValues().get(0).size());
    +        List<Event<?>> tuples = windowValuesCaptor.getAllValues().get(0)
    +            .getEvents().stream().map(Event::get).collect(Collectors.toList());
    +        Assert.assertArrayEquals(mockTuples.toArray(), tuples.toArray());
    +
    +        // window system state
    +        Mockito.verify(mockSystemState, Mockito.times(2)).put(stringCaptor.capture(), systemValuesCaptor.capture());
    +        Assert.assertEquals(EVICTION_STATE_KEY, stringCaptor.getAllValues().get(1));
    --- End diff --
    
    thanks, can use local captor variables here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by satishd <gi...@git.apache.org>.
Github user satishd commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130233654
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/SimpleWindowPartitionCache.java ---
    @@ -0,0 +1,191 @@
    +/**
    + * 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 java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.concurrent.ConcurrentMap;
    +import java.util.concurrent.ConcurrentSkipListMap;
    +import java.util.concurrent.locks.ReentrantLock;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * A simple implementation that evicts the largest un-pinned entry from the cache. This works well
    + * for caching window partitions since the access pattern is mostly sequential scans.
    + */
    +public class SimpleWindowPartitionCache<K, V> implements WindowPartitionCache<K, V> {
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleWindowPartitionCache.class);
    +
    +    private final ConcurrentSkipListMap<K, V> map = new ConcurrentSkipListMap<>();
    +    private final Map<K, Long> pinned = new HashMap<>();
    +    private final long maximumSize;
    +    private final RemovalListener<K, V> removalListener;
    +    private final CacheLoader<K, V> cacheLoader;
    +    private final ReentrantLock lock = new ReentrantLock(true);
    +    private int size;
    --- End diff --
    
    `ConcurrentSkipList#size` is not a constant time operation and it may need to traverse all of them and its value also may change when size() is invoked. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130227637
  
    --- Diff: storm-client/test/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutorTest.java ---
    @@ -0,0 +1,261 @@
    +/**
    + * 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.Config;
    +import org.apache.storm.generated.GlobalStreamId;
    +import org.apache.storm.state.KeyValueState;
    +import org.apache.storm.streams.Pair;
    +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.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.TimestampExtractor;
    +import org.apache.storm.windowing.TupleWindow;
    +import org.apache.storm.windowing.WaterMarkEvent;
    +import org.apache.storm.windowing.WaterMarkEventGenerator;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.mockito.ArgumentCaptor;
    +import org.mockito.Captor;
    +import org.mockito.Mockito;
    +import org.mockito.MockitoAnnotations;
    +import org.mockito.invocation.InvocationOnMock;
    +import org.mockito.runners.MockitoJUnitRunner;
    +import org.mockito.stubbing.Answer;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.stream.Collectors;
    +import java.util.stream.LongStream;
    +
    +import static org.mockito.AdditionalAnswers.returnsArgAt;
    +
    +/**
    + * Unit tests for {@link PersistentWindowedBoltExecutor}
    + */
    +@RunWith(MockitoJUnitRunner.class)
    +public class PersistentWindowedBoltExecutorTest {
    +    public static final String LATE_STREAM = "late_stream";
    +    private PersistentWindowedBoltExecutor<KeyValueState<String, String>> executor;
    +    private IStatefulWindowedBolt<KeyValueState<String, String>> mockBolt;
    +    private OutputCollector mockOutputCollector;
    +    private TopologyContext mockTopologyContext;
    +    private Map<String, Object> mockStormConf = new HashMap<>();
    +    private TimestampExtractor mockTimestampExtractor;
    +    private WaterMarkEventGenerator mockWaterMarkEventGenerator;
    +    private KeyValueState mockPartitionState;
    +    private KeyValueState mockWindowState;
    +    private KeyValueState mockSystemState;
    +
    +    private static final String PARTITION_KEY = "pk";
    +    private static final String EVICTION_STATE_KEY = "es";
    +    private static final String TRIGGER_STATE_KEY = "ts";
    +    private long tupleTs;
    +
    +    @Captor
    +    private ArgumentCaptor<Tuple> tupleCaptor;
    +    @Captor
    +    private ArgumentCaptor<Collection<Tuple>> anchorCaptor;
    +    @Captor
    +    private ArgumentCaptor<String> stringCaptor;
    +    @Captor
    +    private ArgumentCaptor<Long> longCaptor;
    +    @Captor
    +    private ArgumentCaptor<Values> valuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<TupleWindow> tupleWindowCaptor;
    +    @Captor
    +    private ArgumentCaptor<Deque<Long>> partitionValuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<PersistentWindowedBoltExecutor.WindowPartition<Event<Tuple>>> windowValuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<Object> systemValuesCaptor;
    +
    +    @Before
    +    public void setUp() throws Exception {
    +        MockitoAnnotations.initMocks(this);
    +        mockBolt = Mockito.mock(IStatefulWindowedBolt.class);
    +        mockWaterMarkEventGenerator = Mockito.mock(WaterMarkEventGenerator.class);
    +        mockTimestampExtractor = Mockito.mock(TimestampExtractor.class);
    +        tupleTs = System.currentTimeMillis();
    +        Mockito.when(mockTimestampExtractor.extractTimestamp(Mockito.any())).thenReturn(tupleTs);
    +        Mockito.when(mockBolt.getTimestampExtractor()).thenReturn(mockTimestampExtractor);
    +        Mockito.when(mockBolt.isPersistent()).thenReturn(true);
    +        mockPartitionState = Mockito.mock(KeyValueState.class);
    +        mockWindowState = Mockito.mock(KeyValueState.class);
    +        mockSystemState = Mockito.mock(KeyValueState.class);
    +        mockTopologyContext = Mockito.mock(TopologyContext.class);
    +        Mockito.when(mockTopologyContext.getThisStreams()).thenReturn(Collections.singleton(LATE_STREAM));
    +        mockOutputCollector = Mockito.mock(OutputCollector.class);
    +        executor = new PersistentWindowedBoltExecutor<>(mockBolt);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_WINDOW_LENGTH_COUNT, 5);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_SLIDING_INTERVAL_COUNT, 5);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_LATE_TUPLE_STREAM, LATE_STREAM);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_WATERMARK_EVENT_INTERVAL_MS, 100_000);
    +        mockStormConf.put(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS, 30);
    +        mockStormConf.put(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL, 1000);
    +        Mockito.when(mockPartitionState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(mockWindowState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(mockSystemState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        executor.prepare(mockStormConf, mockTopologyContext, mockOutputCollector,
    +            mockWindowState, mockPartitionState, mockSystemState);
    +    }
    +
    +    @Test
    +    public void testExecuteTuple() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), 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);
    +    }
    +
    +    @Test
    +    public void testExecuteLatetuple() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), 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
    +        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());
    +    }
    +
    +    @Test
    +    public void testActivation() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), Mockito.anyLong())).thenReturn(true);
    +        executor.initState(null);
    +        executor.waterMarkEventGenerator = mockWaterMarkEventGenerator;
    +
    +        List<Tuple> mockTuples = getMockTuples(5);
    +        mockTuples.forEach(t -> executor.execute(t));
    +        // all tuples acked
    +        Mockito.verify(mockOutputCollector, Mockito.times(5)).ack(tupleCaptor.capture());
    +        Assert.assertArrayEquals(mockTuples.toArray(), tupleCaptor.getAllValues().toArray());
    +
    +        Mockito.doAnswer(new Answer<Void>() {
    +            @Override
    +            public Void answer(InvocationOnMock invocation) throws Throwable {
    +                TupleWindow window = (TupleWindow) invocation.getArguments()[0];
    +                // iterate the tuples
    +                Assert.assertEquals(5, window.get().size());
    +                // iterating multiple times should produce same events
    +                Assert.assertEquals(5, window.get().size());
    +                Assert.assertEquals(5, window.get().size());
    +                return null;
    +            }
    +        }).when(mockBolt).execute(Mockito.any());
    +        // trigger the window
    +        long activationTs = tupleTs + 1000;
    +        executor.getWindowManager().add(new WaterMarkEvent<>(activationTs));
    +        executor.prePrepare(0);
    +
    +        // partition ids
    +        Mockito.verify(mockPartitionState, Mockito.times(1)).put(stringCaptor.capture(), partitionValuesCaptor.capture());
    +        Assert.assertEquals(PARTITION_KEY, stringCaptor.getAllValues().get(0));
    +        Assert.assertEquals(Collections.singletonList(0L), partitionValuesCaptor.getAllValues().get(0));
    +
    +        // window partitions
    +        Mockito.verify(mockWindowState, Mockito.times(1)).put(longCaptor.capture(), windowValuesCaptor.capture());
    +        Assert.assertEquals(0L, (long) longCaptor.getAllValues().get(0));
    +        Assert.assertEquals(5, windowValuesCaptor.getAllValues().get(0).size());
    +        List<Event<?>> tuples = windowValuesCaptor.getAllValues().get(0)
    +            .getEvents().stream().map(Event::get).collect(Collectors.toList());
    +        Assert.assertArrayEquals(mockTuples.toArray(), tuples.toArray());
    +
    +        // window system state
    +        Mockito.verify(mockSystemState, Mockito.times(2)).put(stringCaptor.capture(), systemValuesCaptor.capture());
    +        Assert.assertEquals(EVICTION_STATE_KEY, stringCaptor.getAllValues().get(1));
    --- End diff --
    
    Nit: Reusing the captor like this is pretty brittle when someone needs to add extra verification to this test. Consider just declaring the captors as local variables and use a captor per verify. I don't think you gain anything by using [at]Captor for non-generic classes anyway.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129763340
  
    --- Diff: docs/Windowing.md ---
    @@ -266,3 +266,105 @@ tuples can be received within the timeout period.
     An example toplogy `SlidingWindowTopology` shows how to use the apis to compute a sliding window sum and a tumbling window 
     average.
     
    +## Stateful windowing
    +The default windowing implementation in storm stores the tuples in memory until they are processed and expired from the 
    +window. This limits the use cases to windows that
    +fit entirely in memory. Also the source tuples cannot be ack-ed until the window expiry requiring large message timeouts
    +(topology.message.timeout.secs should be larger than the window length + sliding interval). This also puts extra loads 
    +due to the complex acking and anchoring requirements.
    + 
    +To address the above limitations and to support larger window sizes, storm provides stateful windowing support via `IStatefulWindowedBolt`. 
    +User bolts should typically extend `BaseStatefulWindowedBolt` for the windowing operations with the framework automatically 
    +managing the state of the window in the background.
    +
    +If the sources provide a monotonically increasing identifier as a part of the message, the framework can use this
    +to periodically checkpoint the last expired and evaluated message ids, to avoid duplicate window evaluations in case of 
    +failures or restarts. During recovery, the tuples with message ids lower than last expired id are discarded and tuples with 
    +message id between the last expired and last evaluated message ids are fed into the system without activating any triggers. 
    --- End diff --
    
    yes its the trigger policy instances. will reword.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129109828
  
    --- Diff: docs/Windowing.md ---
    @@ -266,3 +266,105 @@ tuples can be received within the timeout period.
     An example toplogy `SlidingWindowTopology` shows how to use the apis to compute a sliding window sum and a tumbling window 
     average.
     
    +## Stateful windowing
    +The default windowing implementation in storm stores the tuples in memory until they are processed and expired from the 
    +window. This limits the use cases to windows that
    +fit entirely in memory. Also the source tuples cannot be ack-ed until the window expiry requiring large message timeouts
    +(topology.message.timeout.secs should be larger than the window length + sliding interval). This also puts extra loads 
    +due to the complex acking and anchoring requirements.
    + 
    +To address the above limitations and to support larger window sizes, storm provides stateful windowing support via `IStatefulWindowedBolt`. 
    +User bolts should typically extend `BaseStatefulWindowedBolt` for the windowing operations with the framework automatically 
    +managing the state of the window in the background.
    +
    +If the sources provide a monotonically increasing identifier as a part of the message, the framework can use this
    +to periodically checkpoint the last expired and evaluated message ids, to avoid duplicate window evaluations in case of 
    +failures or restarts. During recovery, the tuples with message ids lower than last expired id are discarded and tuples with 
    +message id between the last expired and last evaluated message ids are fed into the system without activating any triggers. 
    +The tuples beyond the last evaluated message ids are processed as usual. This can be enabled by setting
    +the `messageIdField` as shown below,
    +
    +```java
    +topologyBuilder.setBolt("mybolt",
    +                   new MyStatefulWindowedBolt()
    +                   .withWindow(...) // windowing configuarations
    +                   .withMessageIdField("msgid"), // a monotonically increasing 'long' field in the tuple
    +                   parallelism)
    +               .shuffleGrouping("spout");
    +```
    +
    +However, this option is feasible only if the sources can provide a monotonically increasing identifier in the tuple and the same is maintained
    +while re-emitting the messages in case of failures. Here the tuples in window are still held in memory.
    +
    +For more details take a look at the sample topology in storm starter `StatefulWindowingTopology` which will help you get started.
    --- End diff --
    
    Nit: storm-starter. Maybe we should link to storm-starter here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129763573
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutor.java ---
    @@ -0,0 +1,563 @@
    +/**
    + * 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 java.util.AbstractCollection;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentLinkedQueue;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.locks.ReentrantLock;
    +import java.util.function.Supplier;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.state.KeyValueState;
    +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.windowing.DefaultEvictionContext;
    +import org.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.EventImpl;
    +import org.apache.storm.windowing.WindowLifecycleListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static java.util.Collections.emptyIterator;
    +import static org.apache.storm.topology.WindowPartitionCache.CacheLoader;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalCause;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalListener;
    +
    +/**
    + * Wraps a {@link IStatefulWindowedBolt} and handles the execution. Uses state and the underlying
    + * checkpointing mechanisms to save the tuples in window to state. The tuples are also kept in-memory
    + * by transparently caching the window partitions and checkpointing them as needed.
    + */
    +public class PersistentWindowedBoltExecutor<T extends State> extends WindowedBoltExecutor implements IStatefulBolt<T> {
    +    private static final Logger LOG = LoggerFactory.getLogger(PersistentWindowedBoltExecutor.class);
    +    private final IStatefulWindowedBolt<T> statefulWindowedBolt;
    +    private transient TopologyContext topologyContext;
    +    private transient OutputCollector outputCollector;
    +    private transient WindowState<Tuple> state;
    +    private transient boolean stateInitialized;
    +    private transient boolean prePrepared;
    +
    +    public PersistentWindowedBoltExecutor(IStatefulWindowedBolt<T> bolt) {
    +        super(bolt);
    +        statefulWindowedBolt = bolt;
    +    }
    +
    +    @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));
    +    }
    +
    +    // package access for unit tests
    +    void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                 KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                 KeyValueState<String, Deque<Long>> partitionState,
    +                 KeyValueState<String, Optional<?>> windowSystemState) {
    +        init(topoConf, context, collector, windowState, partitionState, windowSystemState);
    +        doPrepare(topoConf, context, new NoAckOutputCollector(collector), state, true);
    +        Map<String, Optional<?>> wstate = new HashMap<>();
    +        windowSystemState.forEach(s -> wstate.put(s.getKey(), s.getValue()));
    +        restoreState(wstate);
    +    }
    +
    +    @Override
    +    protected void start() {
    +        if (stateInitialized) {
    +            super.start();
    +        } else {
    +            LOG.debug("Will invoke start after state is initialized");
    +        }
    +    }
    +
    +    @Override
    +    public void execute(Tuple input) {
    +        if (!stateInitialized) {
    +            throw new IllegalStateException("execute invoked before initState with input tuple " + input);
    +        }
    +        super.execute(input);
    +        // StatefulBoltExecutor does the actual ack when the state is saved.
    +        outputCollector.ack(input);
    +    }
    +
    +    @Override
    +    public void initState(T state) {
    +        if (stateInitialized) {
    +            LOG.warn("State is already initialized. Ignoring initState");
    +        } else {
    +            statefulWindowedBolt.initState(state);
    +            stateInitialized = true;
    +            start();
    +        }
    +    }
    +
    +    @Override
    +    public void prePrepare(long txid) {
    +        if (stateInitialized) {
    +            LOG.debug("Prepare streamState, txid {}", txid);
    +            state.prepareCommit(txid);
    +            prePrepared = true;
    +        } else {
    +            LOG.warn("Cannot prepare before initState");
    +        }
    +    }
    +
    +    @Override
    +    public void preCommit(long txid) {
    +        if (prePrepared) {
    +            LOG.debug("Commit streamState, txid {}", txid);
    +            state.commit(txid);
    +        } else {
    +            LOG.debug("Ignoring preCommit and not committing streamState.");
    +        }
    +    }
    +
    +    @Override
    +    public void preRollback() {
    +        LOG.debug("Rollback streamState, stateInitialized {}", stateInitialized);
    +        state.rollback();
    +    }
    +
    +    @Override
    +    protected WindowLifecycleListener<Tuple> newWindowLifecycleListener() {
    +        return new WindowLifecycleListener<Tuple>() {
    +            @Override
    +            public void onExpiry(List<Tuple> events) {
    +                /*
    +                 * NO-OP: the events are ack-ed in execute
    +                 */
    +            }
    +
    +            @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();
    +            }
    +        };
    +    }
    +
    +    private void init(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                      KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                      KeyValueState<String, Deque<Long>> partitionState,
    +                      KeyValueState<String, Optional<?>> windowSystemState) {
    +        topologyContext = context;
    +        outputCollector = collector;
    +        state = new WindowState<>(windowState, partitionState, windowSystemState, this::getState,
    +            statefulWindowedBolt.maxEventsInMemory());
    +    }
    +
    +    private KeyValueState<Long, WindowPartition<Tuple>> getWindowState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window";
    +        return (KeyValueState<Long, WindowPartition<Tuple>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Deque<Long>> getPartitionState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-partitions";
    +        return (KeyValueState<String, Deque<Long>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Optional<?>> getWindowSystemState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-systemstate";
    +        return (KeyValueState<String, Optional<?>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    // a wrapper around the window related states that are checkpointed
    +    private static class WindowState<T> extends AbstractCollection<Event<T>> {
    +        // number of events per window-partition
    +        private static final int PARTITION_SZ = 1000;
    +        private static final int MIN_PARTITIONS = 10;
    +        private static final String PARTITION_IDS_KEY = "pk";
    +        private final KeyValueState<String, Deque<Long>> partitionIds;
    +        private final KeyValueState<Long, WindowPartition<T>> windowPartitions;
    +        private final KeyValueState<String, Optional<?>> windowSystemState;
    +        // ordered partition keys
    +        private Deque<Long> pids;
    +        private volatile long latestPartitionId;
    +        private WindowPartitionCache<Long, WindowPartition<T>> cache;
    +        private Supplier<Map<String, Optional<?>>> windowSystemStateSupplier;
    +        private final ReentrantLock partitionIdsLock = new ReentrantLock(true);
    +        private final ReentrantLock windowPartitionsLock = new ReentrantLock(true);
    +        private final long maxEventsInMemory;
    +        private WindowPartition<T> latestPartition;
    +        private Set<Long> iteratorPins = new HashSet<>();
    +
    +        WindowState(KeyValueState<Long, WindowPartition<T>> windowPartitions,
    +                    KeyValueState<String, Deque<Long>> partitionIds,
    +                    KeyValueState<String, Optional<?>> windowSystemState,
    +                    Supplier<Map<String, Optional<?>>> windowSystemStateSupplier,
    +                    long maxEventsInMemory) {
    +            this.windowPartitions = windowPartitions;
    +            this.partitionIds = partitionIds;
    +            this.windowSystemState = windowSystemState;
    +            this.windowSystemStateSupplier = windowSystemStateSupplier;
    +            this.maxEventsInMemory = Math.max(PARTITION_SZ * MIN_PARTITIONS, maxEventsInMemory);
    +            initCache();
    +            initPartitions();
    +        }
    +
    +        @Override
    +        public boolean add(Event<T> event) {
    +            if (latestPartition.size() >= PARTITION_SZ) {
    +                cache.unpin(latestPartition.getId());
    +                latestPartition = getPinnedPartition(getNextPartitionId());
    +            }
    +            latestPartition.add(event);
    +            return true;
    +        }
    +
    +        @Override
    +        public Iterator<Event<T>> iterator() {
    +
    +            return new Iterator<Event<T>>() {
    +                private Iterator<Long> ids = getIds();
    +                private Iterator<Event<T>> current = emptyIterator();
    +                private Iterator<Event<T>> removeFrom;
    +                private WindowPartition<T> curPartition;
    +
    +                private Iterator<Long> getIds() {
    +                    try {
    +                        partitionIdsLock.lock();
    +                        LOG.debug("Iterator pids: {}", pids);
    +                        return new ArrayList<>(pids).iterator();
    +                    } finally {
    +                        partitionIdsLock.unlock();
    +                    }
    +                }
    +
    +                @Override
    +                public void remove() {
    +                    if (removeFrom == null) {
    +                        throw new IllegalStateException("No calls to next() since last call to remove()");
    +                    }
    +                    removeFrom.remove();
    +                    removeFrom = null;
    +                }
    +
    +                @Override
    +                public boolean hasNext() {
    +                    boolean curHasNext = current.hasNext();
    +                    while (!curHasNext && ids.hasNext()) {
    +                        if (curPartition != null) {
    +                            unpin(curPartition.getId());
    +                        }
    +                        curPartition = getPinnedPartition(ids.next());
    +                        iteratorPins.add(curPartition.getId());
    +                        if (curPartition != null) {
    +                            current = curPartition.iterator();
    +                            curHasNext = current.hasNext();
    +                        }
    +                    }
    +                    // un-pin the last partition
    +                    if (!curHasNext && curPartition != null) {
    +                        unpin(curPartition.getId());
    +                        curPartition = null;
    +                    }
    +                    return curHasNext;
    +                }
    +
    +                @Override
    +                public Event<T> next() {
    +                    if (!hasNext()) {
    +                        throw new NoSuchElementException();
    +                    }
    +                    removeFrom = current;
    +                    return current.next();
    +                }
    +
    +                private void unpin(long id) {
    +                    cache.unpin(id);
    +                    iteratorPins.remove(id);
    +                }
    +            };
    +        }
    +
    +        void clearIteratorPins() {
    +            LOG.debug("clearIteratorPins '{}'", iteratorPins);
    +            Iterator<Long> it = iteratorPins.iterator();
    +            while (it.hasNext()) {
    +                cache.unpin(it.next());
    +                it.remove();
    +            }
    +        }
    +
    +        @Override
    +        public int size() {
    +            throw new UnsupportedOperationException();
    +        }
    +
    +        void prepareCommit(long txid) {
    +            flush();
    +            partitionIds.prepareCommit(txid);
    +            windowPartitions.prepareCommit(txid);
    +            windowSystemState.prepareCommit(txid);
    +        }
    +
    +        void commit(long txid) {
    +            partitionIds.commit(txid);
    +            windowPartitions.commit(txid);
    +            windowSystemState.commit(txid);
    +        }
    +
    +        void rollback() {
    +            partitionIds.rollback();
    +            windowPartitions.rollback();
    +            windowSystemState.rollback();
    +        }
    +
    +        private void initPartitions() {
    +            pids = partitionIds.get(PARTITION_IDS_KEY, new LinkedList<>());
    +            if (pids.isEmpty()) {
    +                pids.add(0L);
    +                partitionIds.put(PARTITION_IDS_KEY, pids);
    +            } else {
    +                latestPartitionId = pids.peekLast();
    --- End diff --
    
    yes, it was initialized to 0 being a member variable, anyways I will remove the else to make it clearer.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130225676
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutor.java ---
    @@ -0,0 +1,596 @@
    +/**
    + * 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 java.util.AbstractCollection;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Objects;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentLinkedQueue;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.locks.ReentrantLock;
    +import java.util.function.Supplier;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.state.KeyValueState;
    +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.topology.base.BaseWindowedBolt;
    +import org.apache.storm.tuple.Tuple;
    +import org.apache.storm.windowing.DefaultEvictionContext;
    +import org.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.EventImpl;
    +import org.apache.storm.windowing.WindowLifecycleListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static java.util.Collections.emptyIterator;
    +import static org.apache.storm.topology.WindowPartitionCache.CacheLoader;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalCause;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalListener;
    +
    +/**
    + * Wraps a {@link IStatefulWindowedBolt} and handles the execution. Uses state and the underlying
    + * checkpointing mechanisms to save the tuples in window to state. The tuples are also kept in-memory
    + * by transparently caching the window partitions and checkpointing them as needed.
    + */
    +public class PersistentWindowedBoltExecutor<T extends State> extends WindowedBoltExecutor implements IStatefulBolt<T> {
    +    private static final Logger LOG = LoggerFactory.getLogger(PersistentWindowedBoltExecutor.class);
    +    private final IStatefulWindowedBolt<T> statefulWindowedBolt;
    +    private transient TopologyContext topologyContext;
    +    private transient OutputCollector outputCollector;
    +    private transient WindowState<Tuple> state;
    +    private transient boolean stateInitialized;
    +    private transient boolean prePrepared;
    +
    +    public PersistentWindowedBoltExecutor(IStatefulWindowedBolt<T> bolt) {
    +        super(bolt);
    +        statefulWindowedBolt = bolt;
    +    }
    +
    +    @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));
    +    }
    +
    +    @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);
    +        }
    +    }
    +
    +    private int getCheckpointIntervalMillis(Map<String, Object> topoConf) {
    +        int checkpointInterval = Integer.MAX_VALUE;
    +        if (topoConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL) != null) {
    +            checkpointInterval = ((Number) topoConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL)).intValue();
    +        }
    +        return checkpointInterval;
    +    }
    +
    +    // package access for unit tests
    +    void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                 KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                 KeyValueState<String, Deque<Long>> partitionState,
    +                 KeyValueState<String, Optional<?>> windowSystemState) {
    +        init(topoConf, context, collector, windowState, partitionState, windowSystemState);
    +        doPrepare(topoConf, context, new NoAckOutputCollector(collector), state, true);
    +        Map<String, Optional<?>> wstate = new HashMap<>();
    +        windowSystemState.forEach(s -> wstate.put(s.getKey(), s.getValue()));
    +        restoreState(wstate);
    +    }
    +
    +    @Override
    +    protected void start() {
    +        if (stateInitialized) {
    +            super.start();
    +        } else {
    +            LOG.debug("Will invoke start after state is initialized");
    +        }
    +    }
    +
    +    @Override
    +    public void execute(Tuple input) {
    +        if (!stateInitialized) {
    +            throw new IllegalStateException("execute invoked before initState with input tuple " + input);
    +        }
    +        super.execute(input);
    +        // StatefulBoltExecutor does the actual ack when the state is saved.
    +        outputCollector.ack(input);
    +    }
    +
    +    @Override
    +    public void initState(T state) {
    +        if (stateInitialized) {
    +            String msg = "initState invoked when the state is already initialized";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        } else {
    +            statefulWindowedBolt.initState(state);
    +            stateInitialized = true;
    +            start();
    +        }
    +    }
    +
    +    @Override
    +    public void prePrepare(long txid) {
    +        if (stateInitialized) {
    +            LOG.debug("Prepare streamState, txid {}", txid);
    +            state.prepareCommit(txid);
    +            prePrepared = true;
    +        } else {
    +            String msg = "Cannot prepare before initState";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        }
    +    }
    +
    +    @Override
    +    public void preCommit(long txid) {
    +        // preCommit can be invoked during recovery before the state is initialized
    +        if (prePrepared || !stateInitialized) {
    +            LOG.debug("Commit streamState, txid {}", txid);
    +            state.commit(txid);
    +        } else {
    +            String msg = "preCommit before prePrepare in initialized state";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        }
    +    }
    +
    +    @Override
    +    public void preRollback() {
    +        LOG.debug("Rollback streamState, stateInitialized {}", stateInitialized);
    +        state.rollback();
    +    }
    +
    +    @Override
    +    protected WindowLifecycleListener<Tuple> newWindowLifecycleListener() {
    +        return new WindowLifecycleListener<Tuple>() {
    +            @Override
    +            public void onExpiry(List<Tuple> events) {
    +                /*
    +                 * NO-OP: the events are ack-ed in execute
    +                 */
    +            }
    +
    +            @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();
    +            }
    +        };
    +    }
    +
    +    private void init(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                      KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                      KeyValueState<String, Deque<Long>> partitionState,
    +                      KeyValueState<String, Optional<?>> windowSystemState) {
    +        topologyContext = context;
    +        outputCollector = collector;
    +        state = new WindowState<>(windowState, partitionState, windowSystemState, this::getState,
    +            statefulWindowedBolt.maxEventsInMemory());
    +    }
    +
    +    private KeyValueState<Long, WindowPartition<Tuple>> getWindowState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window";
    +        return (KeyValueState<Long, WindowPartition<Tuple>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Deque<Long>> getPartitionState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-partitions";
    +        return (KeyValueState<String, Deque<Long>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Optional<?>> getWindowSystemState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-systemstate";
    +        return (KeyValueState<String, Optional<?>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    // a wrapper around the window related states that are checkpointed
    +    private static class WindowState<T> extends AbstractCollection<Event<T>> {
    +        // number of events per window-partition
    +        private static final int MAX_PARTITION_EVENTS = 1000;
    +        private static final int MIN_PARTITIONS = 10;
    +        private static final String PARTITION_IDS_KEY = "pk";
    +        private final KeyValueState<String, Deque<Long>> partitionIdsState;
    +        private final KeyValueState<Long, WindowPartition<T>> windowPartitionsState;
    +        private final KeyValueState<String, Optional<?>> windowSystemState;
    +        // ordered partition keys
    +        private Deque<Long> partitionIds;
    +        private volatile long latestPartitionId;
    +        private WindowPartitionCache<Long, WindowPartition<T>> cache;
    +        private Supplier<Map<String, Optional<?>>> windowSystemStateSupplier;
    +        private final ReentrantLock partitionIdsLock = new ReentrantLock(true);
    +        private final ReentrantLock windowPartitionsLock = new ReentrantLock(true);
    +        private final long maxEventsInMemory;
    +        private WindowPartition<T> latestPartition;
    +        private Set<Long> iteratorPins = new HashSet<>();
    +
    +        WindowState(KeyValueState<Long, WindowPartition<T>> windowPartitionsState,
    +                    KeyValueState<String, Deque<Long>> partitionIdsState,
    +                    KeyValueState<String, Optional<?>> windowSystemState,
    +                    Supplier<Map<String, Optional<?>>> windowSystemStateSupplier,
    +                    long maxEventsInMemory) {
    +            this.windowPartitionsState = windowPartitionsState;
    +            this.partitionIdsState = partitionIdsState;
    +            this.windowSystemState = windowSystemState;
    +            this.windowSystemStateSupplier = windowSystemStateSupplier;
    +            this.maxEventsInMemory = Math.max(MAX_PARTITION_EVENTS * MIN_PARTITIONS, maxEventsInMemory);
    +            initCache();
    +            initPartitions();
    +        }
    +
    +        @Override
    +        public boolean add(Event<T> event) {
    +            if (latestPartition.size() >= MAX_PARTITION_EVENTS) {
    +                cache.unpin(latestPartition.getId());
    +                latestPartition = getPinnedPartition(getNextPartitionId());
    +            }
    +            latestPartition.add(event);
    +            return true;
    +        }
    +
    +        @Override
    +        public Iterator<Event<T>> iterator() {
    +
    +            return new Iterator<Event<T>>() {
    +                private Iterator<Long> ids = getIds();
    +                private Iterator<Event<T>> current = emptyIterator();
    +                private Iterator<Event<T>> removeFrom;
    +                private WindowPartition<T> curPartition;
    +
    +                private Iterator<Long> getIds() {
    +                    try {
    +                        partitionIdsLock.lock();
    +                        LOG.debug("Iterator partitionIds: {}", partitionIds);
    +                        return new ArrayList<>(partitionIds).iterator();
    +                    } finally {
    +                        partitionIdsLock.unlock();
    +                    }
    +                }
    +
    +                @Override
    +                public void remove() {
    +                    if (removeFrom == null) {
    +                        throw new IllegalStateException("No calls to next() since last call to remove()");
    +                    }
    +                    removeFrom.remove();
    +                    removeFrom = null;
    +                }
    +
    +                @Override
    +                public boolean hasNext() {
    +                    boolean curHasNext = current.hasNext();
    +                    while (!curHasNext && ids.hasNext()) {
    +                        if (curPartition != null) {
    +                            unpin(curPartition.getId());
    +                        }
    +                        curPartition = getPinnedPartition(ids.next());
    +                        if (curPartition != null) {
    +                            iteratorPins.add(curPartition.getId());
    +                            current = curPartition.iterator();
    +                            curHasNext = current.hasNext();
    +                        }
    +                    }
    +                    // un-pin the last partition
    +                    if (!curHasNext && curPartition != null) {
    +                        unpin(curPartition.getId());
    +                        curPartition = null;
    +                    }
    +                    return curHasNext;
    +                }
    +
    +                @Override
    +                public Event<T> next() {
    +                    if (!hasNext()) {
    +                        throw new NoSuchElementException();
    +                    }
    +                    removeFrom = current;
    +                    return current.next();
    +                }
    +
    +                private void unpin(long id) {
    +                    cache.unpin(id);
    +                    iteratorPins.remove(id);
    +                }
    +            };
    +        }
    +
    +        void clearIteratorPins() {
    +            LOG.debug("clearIteratorPins '{}'", iteratorPins);
    +            Iterator<Long> it = iteratorPins.iterator();
    +            while (it.hasNext()) {
    +                cache.unpin(it.next());
    +                it.remove();
    +            }
    +        }
    +
    +        @Override
    +        public int size() {
    +            throw new UnsupportedOperationException();
    +        }
    +
    +        void prepareCommit(long txid) {
    +            flush();
    +            partitionIdsState.prepareCommit(txid);
    +            windowPartitionsState.prepareCommit(txid);
    +            windowSystemState.prepareCommit(txid);
    +        }
    +
    +        void commit(long txid) {
    +            partitionIdsState.commit(txid);
    +            windowPartitionsState.commit(txid);
    +            windowSystemState.commit(txid);
    +        }
    +
    +        void rollback() {
    +            partitionIdsState.rollback();
    +            windowPartitionsState.rollback();
    +            windowSystemState.rollback();
    +        }
    +
    +        private void initPartitions() {
    +            partitionIds = partitionIdsState.get(PARTITION_IDS_KEY, new LinkedList<>());
    +            if (partitionIds.isEmpty()) {
    +                partitionIds.add(0L);
    +                partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    +            }
    +            latestPartitionId = partitionIds.peekLast();
    +            latestPartition = cache.getPinned(latestPartitionId);
    +        }
    +
    +        private void initCache() {
    +            long size = maxEventsInMemory / MAX_PARTITION_EVENTS;
    +            LOG.info("maxEventsInMemory: {}, partition size: {}, number of partitions: {}",
    +                maxEventsInMemory, MAX_PARTITION_EVENTS, size);
    +            cache = SimpleWindowPartitionCache.<Long, WindowPartition<T>>newBuilder()
    +                .maximumSize(size)
    +                .removalListener(new RemovalListener<Long, WindowPartition<T>>() {
    +                    @Override
    +                    public void onRemoval(Long pid, WindowPartition<T> p, RemovalCause removalCause) {
    +                        Objects.requireNonNull(pid, "Null partition id");
    +                        Objects.requireNonNull(p, "Null window partition");
    +                        LOG.debug("onRemoval for id '{}', WindowPartition '{}'", pid, p);
    +                        try {
    +                            windowPartitionsLock.lock();
    +                            if (p.isEmpty() && pid != latestPartitionId) {
    +                                // if the empty partition was not invalidated by flush, but evicted from cache
    +                                if (removalCause != RemovalCause.EXPLICIT) {
    +                                    deletePartition(pid);
    +                                    windowPartitionsState.delete(pid);
    +                                }
    +                            } else if (p.isModified()) {
    +                                windowPartitionsState.put(pid, p);
    +                            } else {
    +                                LOG.debug("WindowPartition '{}' is not modified", pid);
    +                            }
    +                        } finally {
    +                            windowPartitionsLock.unlock();
    +                        }
    +                    }
    +                }).build(new CacheLoader<Long, WindowPartition<T>>() {
    +                    @Override
    +                   public WindowPartition<T> load(Long id) {
    +                        LOG.debug("Load partition: {}", id);
    +                        // load from state
    +                        try {
    +                            windowPartitionsLock.lock();
    +                            return windowPartitionsState.get(id, new WindowPartition<>(id));
    +                        } finally {
    +                            windowPartitionsLock.unlock();
    +                        }
    +                    }
    +                    }
    +                );
    +        }
    +
    +        private void deletePartition(long pid) {
    +            LOG.debug("Delete partition: {}", pid);
    +            try {
    +                partitionIdsLock.lock();
    +                partitionIds.remove(pid);
    +                partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    +            } finally {
    +                partitionIdsLock.unlock();
    +            }
    +        }
    +
    +        private long getNextPartitionId() {
    +            try {
    +                partitionIdsLock.lock();
    +                partitionIds.add(++latestPartitionId);
    +                partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    +            } finally {
    +                partitionIdsLock.unlock();
    +            }
    +            return latestPartitionId;
    +        }
    +
    +        private WindowPartition<T> getPinnedPartition(long id) {
    +            return cache.getPinned(id);
    +        }
    +
    +        private void flush() {
    +            LOG.debug("Flushing modified partitions");
    +            cache.asMap().forEach((pid, p) -> {
    +                Long pidToInvalidate = null;
    +                try {
    +                    windowPartitionsLock.lock();
    +                    if (p.isEmpty() && pid != latestPartitionId) {
    +                        LOG.debug("Invalidating empty partition {}", pid);
    +                        deletePartition(pid);
    +                        windowPartitionsState.delete(pid);
    +                        pidToInvalidate = pid;
    +                    } else if (p.isModified()) {
    +                        LOG.debug("Updating modified partition {}", pid);
    +                        p.clearModified();
    +                        windowPartitionsState.put(pid, p);
    +                    }
    +                } finally {
    +                    windowPartitionsLock.unlock();
    +                }
    +                // invalidate after releasing the lock
    +                if (pidToInvalidate != null) {
    +                    cache.invalidate(pidToInvalidate);
    +                }
    +            });
    +            windowSystemStateSupplier.get().forEach(windowSystemState::put);
    +        }
    +    }
    +
    +    // the window partition that holds the events
    +    public static class WindowPartition<T> implements Iterable<Event<T>> {
    +        private final ConcurrentLinkedQueue<Event<T>> events = new ConcurrentLinkedQueue<>();
    +        private final AtomicInteger size = new AtomicInteger();
    --- End diff --
    
    Why is this maintained separately from the events queue?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130273860
  
    --- Diff: storm-client/test/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutorTest.java ---
    @@ -0,0 +1,261 @@
    +/**
    + * 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.Config;
    +import org.apache.storm.generated.GlobalStreamId;
    +import org.apache.storm.state.KeyValueState;
    +import org.apache.storm.streams.Pair;
    +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.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.TimestampExtractor;
    +import org.apache.storm.windowing.TupleWindow;
    +import org.apache.storm.windowing.WaterMarkEvent;
    +import org.apache.storm.windowing.WaterMarkEventGenerator;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.mockito.ArgumentCaptor;
    +import org.mockito.Captor;
    +import org.mockito.Mockito;
    +import org.mockito.MockitoAnnotations;
    +import org.mockito.invocation.InvocationOnMock;
    +import org.mockito.runners.MockitoJUnitRunner;
    +import org.mockito.stubbing.Answer;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.stream.Collectors;
    +import java.util.stream.LongStream;
    +
    +import static org.mockito.AdditionalAnswers.returnsArgAt;
    +
    +/**
    + * Unit tests for {@link PersistentWindowedBoltExecutor}
    + */
    +@RunWith(MockitoJUnitRunner.class)
    +public class PersistentWindowedBoltExecutorTest {
    +    public static final String LATE_STREAM = "late_stream";
    +    private PersistentWindowedBoltExecutor<KeyValueState<String, String>> executor;
    +    private IStatefulWindowedBolt<KeyValueState<String, String>> mockBolt;
    +    private OutputCollector mockOutputCollector;
    +    private TopologyContext mockTopologyContext;
    +    private Map<String, Object> mockStormConf = new HashMap<>();
    +    private TimestampExtractor mockTimestampExtractor;
    +    private WaterMarkEventGenerator mockWaterMarkEventGenerator;
    +    private KeyValueState mockPartitionState;
    +    private KeyValueState mockWindowState;
    +    private KeyValueState mockSystemState;
    +
    +    private static final String PARTITION_KEY = "pk";
    +    private static final String EVICTION_STATE_KEY = "es";
    +    private static final String TRIGGER_STATE_KEY = "ts";
    +    private long tupleTs;
    +
    +    @Captor
    +    private ArgumentCaptor<Tuple> tupleCaptor;
    +    @Captor
    +    private ArgumentCaptor<Collection<Tuple>> anchorCaptor;
    +    @Captor
    +    private ArgumentCaptor<String> stringCaptor;
    +    @Captor
    +    private ArgumentCaptor<Long> longCaptor;
    +    @Captor
    +    private ArgumentCaptor<Values> valuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<TupleWindow> tupleWindowCaptor;
    +    @Captor
    +    private ArgumentCaptor<Deque<Long>> partitionValuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<PersistentWindowedBoltExecutor.WindowPartition<Event<Tuple>>> windowValuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<Object> systemValuesCaptor;
    +
    +    @Before
    +    public void setUp() throws Exception {
    +        MockitoAnnotations.initMocks(this);
    +        mockBolt = Mockito.mock(IStatefulWindowedBolt.class);
    +        mockWaterMarkEventGenerator = Mockito.mock(WaterMarkEventGenerator.class);
    +        mockTimestampExtractor = Mockito.mock(TimestampExtractor.class);
    +        tupleTs = System.currentTimeMillis();
    +        Mockito.when(mockTimestampExtractor.extractTimestamp(Mockito.any())).thenReturn(tupleTs);
    +        Mockito.when(mockBolt.getTimestampExtractor()).thenReturn(mockTimestampExtractor);
    +        Mockito.when(mockBolt.isPersistent()).thenReturn(true);
    +        mockPartitionState = Mockito.mock(KeyValueState.class);
    +        mockWindowState = Mockito.mock(KeyValueState.class);
    +        mockSystemState = Mockito.mock(KeyValueState.class);
    +        mockTopologyContext = Mockito.mock(TopologyContext.class);
    +        Mockito.when(mockTopologyContext.getThisStreams()).thenReturn(Collections.singleton(LATE_STREAM));
    +        mockOutputCollector = Mockito.mock(OutputCollector.class);
    +        executor = new PersistentWindowedBoltExecutor<>(mockBolt);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_WINDOW_LENGTH_COUNT, 5);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_SLIDING_INTERVAL_COUNT, 5);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_LATE_TUPLE_STREAM, LATE_STREAM);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_WATERMARK_EVENT_INTERVAL_MS, 100_000);
    +        mockStormConf.put(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS, 30);
    +        mockStormConf.put(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL, 1000);
    +        Mockito.when(mockPartitionState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(mockWindowState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(mockSystemState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        executor.prepare(mockStormConf, mockTopologyContext, mockOutputCollector,
    +            mockWindowState, mockPartitionState, mockSystemState);
    +    }
    +
    +    @Test
    +    public void testExecuteTuple() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), 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);
    +    }
    +
    +    @Test
    +    public void testExecuteLatetuple() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), 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
    +        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());
    +    }
    +
    +    @Test
    +    public void testActivation() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), Mockito.anyLong())).thenReturn(true);
    +        executor.initState(null);
    +        executor.waterMarkEventGenerator = mockWaterMarkEventGenerator;
    +
    +        List<Tuple> mockTuples = getMockTuples(5);
    +        mockTuples.forEach(t -> executor.execute(t));
    +        // all tuples acked
    +        Mockito.verify(mockOutputCollector, Mockito.times(5)).ack(tupleCaptor.capture());
    +        Assert.assertArrayEquals(mockTuples.toArray(), tupleCaptor.getAllValues().toArray());
    +
    +        Mockito.doAnswer(new Answer<Void>() {
    +            @Override
    +            public Void answer(InvocationOnMock invocation) throws Throwable {
    +                TupleWindow window = (TupleWindow) invocation.getArguments()[0];
    +                // iterate the tuples
    +                Assert.assertEquals(5, window.get().size());
    +                // iterating multiple times should produce same events
    +                Assert.assertEquals(5, window.get().size());
    +                Assert.assertEquals(5, window.get().size());
    +                return null;
    +            }
    +        }).when(mockBolt).execute(Mockito.any());
    +        // trigger the window
    +        long activationTs = tupleTs + 1000;
    +        executor.getWindowManager().add(new WaterMarkEvent<>(activationTs));
    +        executor.prePrepare(0);
    +
    +        // partition ids
    +        Mockito.verify(mockPartitionState, Mockito.times(1)).put(stringCaptor.capture(), partitionValuesCaptor.capture());
    +        Assert.assertEquals(PARTITION_KEY, stringCaptor.getAllValues().get(0));
    --- End diff --
    
    yes, can use getValue


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130226675
  
    --- Diff: storm-client/src/jvm/org/apache/storm/windowing/WindowManager.java ---
    @@ -289,4 +414,22 @@ public String toString() {
                     ", triggerPolicy=" + triggerPolicy +
                     '}';
         }
    +
    +    public void restoreState(Map<String, Optional<?>> state) {
    +        if (state != null) {
    --- End diff --
    
    Can this be null?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130273891
  
    --- Diff: storm-client/test/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutorTest.java ---
    @@ -0,0 +1,261 @@
    +/**
    + * 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.Config;
    +import org.apache.storm.generated.GlobalStreamId;
    +import org.apache.storm.state.KeyValueState;
    +import org.apache.storm.streams.Pair;
    +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.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.TimestampExtractor;
    +import org.apache.storm.windowing.TupleWindow;
    +import org.apache.storm.windowing.WaterMarkEvent;
    +import org.apache.storm.windowing.WaterMarkEventGenerator;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.mockito.ArgumentCaptor;
    +import org.mockito.Captor;
    +import org.mockito.Mockito;
    +import org.mockito.MockitoAnnotations;
    +import org.mockito.invocation.InvocationOnMock;
    +import org.mockito.runners.MockitoJUnitRunner;
    +import org.mockito.stubbing.Answer;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.stream.Collectors;
    +import java.util.stream.LongStream;
    +
    +import static org.mockito.AdditionalAnswers.returnsArgAt;
    +
    +/**
    + * Unit tests for {@link PersistentWindowedBoltExecutor}
    + */
    +@RunWith(MockitoJUnitRunner.class)
    +public class PersistentWindowedBoltExecutorTest {
    +    public static final String LATE_STREAM = "late_stream";
    +    private PersistentWindowedBoltExecutor<KeyValueState<String, String>> executor;
    +    private IStatefulWindowedBolt<KeyValueState<String, String>> mockBolt;
    +    private OutputCollector mockOutputCollector;
    +    private TopologyContext mockTopologyContext;
    +    private Map<String, Object> mockStormConf = new HashMap<>();
    +    private TimestampExtractor mockTimestampExtractor;
    +    private WaterMarkEventGenerator mockWaterMarkEventGenerator;
    +    private KeyValueState mockPartitionState;
    +    private KeyValueState mockWindowState;
    +    private KeyValueState mockSystemState;
    +
    +    private static final String PARTITION_KEY = "pk";
    +    private static final String EVICTION_STATE_KEY = "es";
    +    private static final String TRIGGER_STATE_KEY = "ts";
    +    private long tupleTs;
    +
    +    @Captor
    +    private ArgumentCaptor<Tuple> tupleCaptor;
    +    @Captor
    +    private ArgumentCaptor<Collection<Tuple>> anchorCaptor;
    +    @Captor
    +    private ArgumentCaptor<String> stringCaptor;
    +    @Captor
    +    private ArgumentCaptor<Long> longCaptor;
    +    @Captor
    +    private ArgumentCaptor<Values> valuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<TupleWindow> tupleWindowCaptor;
    +    @Captor
    +    private ArgumentCaptor<Deque<Long>> partitionValuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<PersistentWindowedBoltExecutor.WindowPartition<Event<Tuple>>> windowValuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<Object> systemValuesCaptor;
    +
    +    @Before
    +    public void setUp() throws Exception {
    +        MockitoAnnotations.initMocks(this);
    +        mockBolt = Mockito.mock(IStatefulWindowedBolt.class);
    +        mockWaterMarkEventGenerator = Mockito.mock(WaterMarkEventGenerator.class);
    +        mockTimestampExtractor = Mockito.mock(TimestampExtractor.class);
    +        tupleTs = System.currentTimeMillis();
    +        Mockito.when(mockTimestampExtractor.extractTimestamp(Mockito.any())).thenReturn(tupleTs);
    +        Mockito.when(mockBolt.getTimestampExtractor()).thenReturn(mockTimestampExtractor);
    +        Mockito.when(mockBolt.isPersistent()).thenReturn(true);
    +        mockPartitionState = Mockito.mock(KeyValueState.class);
    +        mockWindowState = Mockito.mock(KeyValueState.class);
    +        mockSystemState = Mockito.mock(KeyValueState.class);
    +        mockTopologyContext = Mockito.mock(TopologyContext.class);
    +        Mockito.when(mockTopologyContext.getThisStreams()).thenReturn(Collections.singleton(LATE_STREAM));
    +        mockOutputCollector = Mockito.mock(OutputCollector.class);
    +        executor = new PersistentWindowedBoltExecutor<>(mockBolt);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_WINDOW_LENGTH_COUNT, 5);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_SLIDING_INTERVAL_COUNT, 5);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_LATE_TUPLE_STREAM, LATE_STREAM);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_WATERMARK_EVENT_INTERVAL_MS, 100_000);
    +        mockStormConf.put(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS, 30);
    +        mockStormConf.put(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL, 1000);
    +        Mockito.when(mockPartitionState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(mockWindowState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(mockSystemState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        executor.prepare(mockStormConf, mockTopologyContext, mockOutputCollector,
    +            mockWindowState, mockPartitionState, mockSystemState);
    +    }
    +
    +    @Test
    +    public void testExecuteTuple() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), 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);
    +    }
    +
    +    @Test
    +    public void testExecuteLatetuple() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), 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
    +        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());
    +    }
    +
    +    @Test
    +    public void testActivation() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), Mockito.anyLong())).thenReturn(true);
    +        executor.initState(null);
    +        executor.waterMarkEventGenerator = mockWaterMarkEventGenerator;
    +
    +        List<Tuple> mockTuples = getMockTuples(5);
    +        mockTuples.forEach(t -> executor.execute(t));
    +        // all tuples acked
    +        Mockito.verify(mockOutputCollector, Mockito.times(5)).ack(tupleCaptor.capture());
    +        Assert.assertArrayEquals(mockTuples.toArray(), tupleCaptor.getAllValues().toArray());
    +
    +        Mockito.doAnswer(new Answer<Void>() {
    +            @Override
    +            public Void answer(InvocationOnMock invocation) throws Throwable {
    +                TupleWindow window = (TupleWindow) invocation.getArguments()[0];
    +                // iterate the tuples
    +                Assert.assertEquals(5, window.get().size());
    +                // iterating multiple times should produce same events
    +                Assert.assertEquals(5, window.get().size());
    +                Assert.assertEquals(5, window.get().size());
    +                return null;
    +            }
    +        }).when(mockBolt).execute(Mockito.any());
    +        // trigger the window
    +        long activationTs = tupleTs + 1000;
    +        executor.getWindowManager().add(new WaterMarkEvent<>(activationTs));
    +        executor.prePrepare(0);
    +
    +        // partition ids
    +        Mockito.verify(mockPartitionState, Mockito.times(1)).put(stringCaptor.capture(), partitionValuesCaptor.capture());
    +        Assert.assertEquals(PARTITION_KEY, stringCaptor.getAllValues().get(0));
    +        Assert.assertEquals(Collections.singletonList(0L), partitionValuesCaptor.getAllValues().get(0));
    +
    +        // window partitions
    +        Mockito.verify(mockWindowState, Mockito.times(1)).put(longCaptor.capture(), windowValuesCaptor.capture());
    +        Assert.assertEquals(0L, (long) longCaptor.getAllValues().get(0));
    +        Assert.assertEquals(5, windowValuesCaptor.getAllValues().get(0).size());
    +        List<Event<?>> tuples = windowValuesCaptor.getAllValues().get(0)
    +            .getEvents().stream().map(Event::get).collect(Collectors.toList());
    +        Assert.assertArrayEquals(mockTuples.toArray(), tuples.toArray());
    +
    +        // window system state
    +        Mockito.verify(mockSystemState, Mockito.times(2)).put(stringCaptor.capture(), systemValuesCaptor.capture());
    +        Assert.assertEquals(EVICTION_STATE_KEY, stringCaptor.getAllValues().get(1));
    +        Assert.assertEquals(Optional.of(Pair.of(5L, 5L)), systemValuesCaptor.getAllValues().get(0));
    +        Assert.assertEquals(TRIGGER_STATE_KEY, stringCaptor.getAllValues().get(2));
    +        Assert.assertEquals(Optional.of(tupleTs), systemValuesCaptor.getAllValues().get(1));
    +    }
    +
    +    @Test
    +    public void testCacheEviction() {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), Mockito.anyLong())).thenReturn(true);
    +        executor.initState(null);
    +        executor.waterMarkEventGenerator = mockWaterMarkEventGenerator;
    +        List<Tuple> mockTuples = getMockTuples(20000);
    +        mockTuples.forEach(t -> executor.execute(t));
    +
    +        Mockito.verify(mockWindowState, Mockito.times(10)).put(longCaptor.capture(), windowValuesCaptor.capture());
    +        Assert.assertEquals(10, longCaptor.getAllValues().size());
    +        Assert.assertEquals(10, windowValuesCaptor.getAllValues().size());
    +        // number of evicted events
    +        Assert.assertEquals(10_000, windowValuesCaptor.getAllValues().stream()
    --- End diff --
    
    Here its checking if the `PersistentWindowedBoltExecutor` is writing the evicted values out to the state. Will add separate unit tests for WindowState.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130227123
  
    --- Diff: storm-client/src/jvm/org/apache/storm/windowing/WindowManager.java ---
    @@ -111,14 +125,86 @@ public void add(Event<T> windowEvent) {
                 LOG.debug("Got watermark event with ts {}", windowEvent.getTimestamp());
             }
             track(windowEvent);
    -        compactWindow();
    +        if (!stateful) {
    +            compactWindow();
    +        }
         }
     
         /**
          * The callback invoked by the trigger policy.
          */
         @Override
         public boolean onTrigger() {
    +        return stateful ? doOnTriggerStateful() : doOnTrigger();
    --- End diff --
    
    I'm wondering if this would be nicer as a strategy or handled with an abstract superclass? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by satishd <gi...@git.apache.org>.
Github user satishd commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129003027
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutor.java ---
    @@ -0,0 +1,563 @@
    +/**
    + * 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 java.util.AbstractCollection;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentLinkedQueue;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.locks.ReentrantLock;
    +import java.util.function.Supplier;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.state.KeyValueState;
    +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.windowing.DefaultEvictionContext;
    +import org.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.EventImpl;
    +import org.apache.storm.windowing.WindowLifecycleListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static java.util.Collections.emptyIterator;
    +import static org.apache.storm.topology.WindowPartitionCache.CacheLoader;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalCause;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalListener;
    +
    +/**
    + * Wraps a {@link IStatefulWindowedBolt} and handles the execution. Uses state and the underlying
    + * checkpointing mechanisms to save the tuples in window to state. The tuples are also kept in-memory
    + * by transparently caching the window partitions and checkpointing them as needed.
    + */
    +public class PersistentWindowedBoltExecutor<T extends State> extends WindowedBoltExecutor implements IStatefulBolt<T> {
    +    private static final Logger LOG = LoggerFactory.getLogger(PersistentWindowedBoltExecutor.class);
    +    private final IStatefulWindowedBolt<T> statefulWindowedBolt;
    +    private transient TopologyContext topologyContext;
    +    private transient OutputCollector outputCollector;
    +    private transient WindowState<Tuple> state;
    +    private transient boolean stateInitialized;
    +    private transient boolean prePrepared;
    +
    +    public PersistentWindowedBoltExecutor(IStatefulWindowedBolt<T> bolt) {
    +        super(bolt);
    +        statefulWindowedBolt = bolt;
    +    }
    +
    +    @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));
    +    }
    +
    +    // package access for unit tests
    +    void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                 KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                 KeyValueState<String, Deque<Long>> partitionState,
    +                 KeyValueState<String, Optional<?>> windowSystemState) {
    +        init(topoConf, context, collector, windowState, partitionState, windowSystemState);
    +        doPrepare(topoConf, context, new NoAckOutputCollector(collector), state, true);
    +        Map<String, Optional<?>> wstate = new HashMap<>();
    +        windowSystemState.forEach(s -> wstate.put(s.getKey(), s.getValue()));
    +        restoreState(wstate);
    +    }
    +
    +    @Override
    +    protected void start() {
    +        if (stateInitialized) {
    +            super.start();
    +        } else {
    +            LOG.debug("Will invoke start after state is initialized");
    +        }
    +    }
    +
    +    @Override
    +    public void execute(Tuple input) {
    +        if (!stateInitialized) {
    +            throw new IllegalStateException("execute invoked before initState with input tuple " + input);
    +        }
    +        super.execute(input);
    +        // StatefulBoltExecutor does the actual ack when the state is saved.
    +        outputCollector.ack(input);
    +    }
    +
    +    @Override
    +    public void initState(T state) {
    +        if (stateInitialized) {
    +            LOG.warn("State is already initialized. Ignoring initState");
    +        } else {
    +            statefulWindowedBolt.initState(state);
    +            stateInitialized = true;
    +            start();
    +        }
    +    }
    +
    +    @Override
    +    public void prePrepare(long txid) {
    +        if (stateInitialized) {
    +            LOG.debug("Prepare streamState, txid {}", txid);
    +            state.prepareCommit(txid);
    +            prePrepared = true;
    +        } else {
    +            LOG.warn("Cannot prepare before initState");
    +        }
    +    }
    +
    +    @Override
    +    public void preCommit(long txid) {
    +        if (prePrepared) {
    +            LOG.debug("Commit streamState, txid {}", txid);
    +            state.commit(txid);
    +        } else {
    +            LOG.debug("Ignoring preCommit and not committing streamState.");
    +        }
    +    }
    +
    +    @Override
    +    public void preRollback() {
    +        LOG.debug("Rollback streamState, stateInitialized {}", stateInitialized);
    +        state.rollback();
    +    }
    +
    +    @Override
    +    protected WindowLifecycleListener<Tuple> newWindowLifecycleListener() {
    +        return new WindowLifecycleListener<Tuple>() {
    +            @Override
    +            public void onExpiry(List<Tuple> events) {
    +                /*
    +                 * NO-OP: the events are ack-ed in execute
    +                 */
    +            }
    +
    +            @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();
    +            }
    +        };
    +    }
    +
    +    private void init(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    --- End diff --
    
    `topoConf` is never used, may want to remove that.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129763418
  
    --- Diff: examples/storm-starter/src/jvm/org/apache/storm/starter/PersistentWindowingTopology.java ---
    @@ -0,0 +1,176 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.storm.starter;
    +
    +import static org.apache.storm.topology.base.BaseWindowedBolt.Duration;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.concurrent.TimeUnit;
    +import org.apache.storm.Config;
    +import org.apache.storm.StormSubmitter;
    +import org.apache.storm.starter.spout.RandomIntegerSpout;
    +import org.apache.storm.state.KeyValueState;
    +import org.apache.storm.streams.Pair;
    +import org.apache.storm.task.OutputCollector;
    +import org.apache.storm.task.TopologyContext;
    +import org.apache.storm.topology.OutputFieldsDeclarer;
    +import org.apache.storm.topology.TopologyBuilder;
    +import org.apache.storm.topology.base.BaseStatefulWindowedBolt;
    +import org.apache.storm.tuple.Fields;
    +import org.apache.storm.tuple.Tuple;
    +import org.apache.storm.tuple.Values;
    +import org.apache.storm.windowing.TupleWindow;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * An example that demonstrates the usage of {@link org.apache.storm.topology.IStatefulWindowedBolt} with window
    + * persistence.
    + * <p>
    + * The framework automatically checkpoints the tuples in window along with the bolt's state and restores the same
    + * during restarts.
    + * </p>
    + *
    + * <p>
    + * This topology uses 'redis' for state persistence, so you should also start a redis instance before deploying.
    + * If you are running in local mode you can just start a redis server locally which will be used for storing the state. The default
    + * RedisKeyValueStateProvider parameters can be overridden by setting "topology.state.provider.config", for e.g.
    + * <pre>
    + * {
    + *   "jedisPoolConfig": {
    + *     "host": "redis-server-host",
    + *     "port": 6379,
    + *     "timeout": 2000,
    + *     "database": 0,
    + *     "password": "xyz"
    + *   }
    + * }
    + * </pre>
    + * </p>
    + */
    +public class PersistentWindowingTopology {
    +    private static final Logger LOG = LoggerFactory.getLogger(PersistentWindowingTopology.class);
    +
    +    // wrapper to hold global and window averages
    +    private static class Averages {
    +        private final double global;
    +        private final double window;
    +
    +        Averages(double global, double window) {
    +            this.global = global;
    +            this.window = window;
    +        }
    +
    +        @Override
    +        public String toString() {
    +            return "Averages{" + "global=" + String.format("%.2f", global) + ", window=" + String.format("%.2f", window) + '}';
    +        }
    +    }
    +
    +    /**
    +     * A bolt that uses stateful persistence to store the windows along with the state (global avg).
    +     */
    +    private static class AvgBolt extends BaseStatefulWindowedBolt<KeyValueState<String, Pair<Long, Long>>> {
    +        private static final String STATE_KEY = "avg";
    +
    +        private OutputCollector collector;
    +        private KeyValueState<String, Pair<Long, Long>> state;
    +        private Pair<Long, Long> globalAvg;
    +
    +        @Override
    +        public void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector) {
    +            this.collector = collector;
    +        }
    +
    +        @Override
    +        public void initState(KeyValueState<String, Pair<Long, Long>> state) {
    +            this.state = state;
    +            globalAvg = state.get(STATE_KEY, Pair.of(0L, 0L));
    +            LOG.info("initState with global avg [" + (double) globalAvg._1 / globalAvg._2 + "]");
    --- End diff --
    
    _1 and _2 are exposed to be consistent with other `TupleN` classes. Will use the getters here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by satishd <gi...@git.apache.org>.
Github user satishd commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129791075
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutor.java ---
    @@ -0,0 +1,596 @@
    +/**
    + * 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 java.util.AbstractCollection;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Objects;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentLinkedQueue;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.locks.ReentrantLock;
    +import java.util.function.Supplier;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.state.KeyValueState;
    +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.topology.base.BaseWindowedBolt;
    +import org.apache.storm.tuple.Tuple;
    +import org.apache.storm.windowing.DefaultEvictionContext;
    +import org.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.EventImpl;
    +import org.apache.storm.windowing.WindowLifecycleListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static java.util.Collections.emptyIterator;
    +import static org.apache.storm.topology.WindowPartitionCache.CacheLoader;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalCause;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalListener;
    +
    +/**
    + * Wraps a {@link IStatefulWindowedBolt} and handles the execution. Uses state and the underlying
    + * checkpointing mechanisms to save the tuples in window to state. The tuples are also kept in-memory
    + * by transparently caching the window partitions and checkpointing them as needed.
    + */
    +public class PersistentWindowedBoltExecutor<T extends State> extends WindowedBoltExecutor implements IStatefulBolt<T> {
    +    private static final Logger LOG = LoggerFactory.getLogger(PersistentWindowedBoltExecutor.class);
    +    private final IStatefulWindowedBolt<T> statefulWindowedBolt;
    +    private transient TopologyContext topologyContext;
    +    private transient OutputCollector outputCollector;
    +    private transient WindowState<Tuple> state;
    +    private transient boolean stateInitialized;
    +    private transient boolean prePrepared;
    +
    +    public PersistentWindowedBoltExecutor(IStatefulWindowedBolt<T> bolt) {
    +        super(bolt);
    +        statefulWindowedBolt = bolt;
    +    }
    +
    +    @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));
    +    }
    +
    +    @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);
    +        }
    +    }
    +
    +    private int getCheckpointIntervalMillis(Map<String, Object> topoConf) {
    +        int checkpointInterval = Integer.MAX_VALUE;
    +        if (topoConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL) != null) {
    +            checkpointInterval = ((Number) topoConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL)).intValue();
    +        }
    +        return checkpointInterval;
    +    }
    +
    +    // package access for unit tests
    +    void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                 KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                 KeyValueState<String, Deque<Long>> partitionState,
    +                 KeyValueState<String, Optional<?>> windowSystemState) {
    +        init(topoConf, context, collector, windowState, partitionState, windowSystemState);
    +        doPrepare(topoConf, context, new NoAckOutputCollector(collector), state, true);
    +        Map<String, Optional<?>> wstate = new HashMap<>();
    +        windowSystemState.forEach(s -> wstate.put(s.getKey(), s.getValue()));
    +        restoreState(wstate);
    +    }
    +
    +    @Override
    +    protected void start() {
    +        if (stateInitialized) {
    +            super.start();
    +        } else {
    +            LOG.debug("Will invoke start after state is initialized");
    +        }
    +    }
    +
    +    @Override
    +    public void execute(Tuple input) {
    +        if (!stateInitialized) {
    +            throw new IllegalStateException("execute invoked before initState with input tuple " + input);
    +        }
    +        super.execute(input);
    +        // StatefulBoltExecutor does the actual ack when the state is saved.
    +        outputCollector.ack(input);
    +    }
    +
    +    @Override
    +    public void initState(T state) {
    +        if (stateInitialized) {
    +            String msg = "initState invoked when the state is already initialized";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        } else {
    +            statefulWindowedBolt.initState(state);
    +            stateInitialized = true;
    +            start();
    +        }
    +    }
    +
    +    @Override
    +    public void prePrepare(long txid) {
    +        if (stateInitialized) {
    +            LOG.debug("Prepare streamState, txid {}", txid);
    +            state.prepareCommit(txid);
    +            prePrepared = true;
    +        } else {
    +            String msg = "Cannot prepare before initState";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        }
    +    }
    +
    +    @Override
    +    public void preCommit(long txid) {
    +        // preCommit can be invoked during recovery before the state is initialized
    +        if (prePrepared || !stateInitialized) {
    +            LOG.debug("Commit streamState, txid {}", txid);
    +            state.commit(txid);
    +        } else {
    +            String msg = "preCommit before prePrepare in initialized state";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        }
    +    }
    +
    +    @Override
    +    public void preRollback() {
    +        LOG.debug("Rollback streamState, stateInitialized {}", stateInitialized);
    +        state.rollback();
    +    }
    +
    +    @Override
    +    protected WindowLifecycleListener<Tuple> newWindowLifecycleListener() {
    +        return new WindowLifecycleListener<Tuple>() {
    +            @Override
    +            public void onExpiry(List<Tuple> events) {
    +                /*
    +                 * NO-OP: the events are ack-ed in execute
    +                 */
    +            }
    +
    +            @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();
    +            }
    +        };
    +    }
    +
    +    private void init(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                      KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                      KeyValueState<String, Deque<Long>> partitionState,
    +                      KeyValueState<String, Optional<?>> windowSystemState) {
    +        topologyContext = context;
    +        outputCollector = collector;
    +        state = new WindowState<>(windowState, partitionState, windowSystemState, this::getState,
    +            statefulWindowedBolt.maxEventsInMemory());
    +    }
    +
    +    private KeyValueState<Long, WindowPartition<Tuple>> getWindowState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window";
    +        return (KeyValueState<Long, WindowPartition<Tuple>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Deque<Long>> getPartitionState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-partitions";
    +        return (KeyValueState<String, Deque<Long>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Optional<?>> getWindowSystemState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-systemstate";
    +        return (KeyValueState<String, Optional<?>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    // a wrapper around the window related states that are checkpointed
    +    private static class WindowState<T> extends AbstractCollection<Event<T>> {
    --- End diff --
    
    Can we move this class out to package level(may be a new package) and add unit tests for this? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129111019
  
    --- Diff: examples/storm-starter/src/jvm/org/apache/storm/starter/PersistentWindowingTopology.java ---
    @@ -0,0 +1,176 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.storm.starter;
    +
    +import static org.apache.storm.topology.base.BaseWindowedBolt.Duration;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.concurrent.TimeUnit;
    +import org.apache.storm.Config;
    +import org.apache.storm.StormSubmitter;
    +import org.apache.storm.starter.spout.RandomIntegerSpout;
    +import org.apache.storm.state.KeyValueState;
    +import org.apache.storm.streams.Pair;
    +import org.apache.storm.task.OutputCollector;
    +import org.apache.storm.task.TopologyContext;
    +import org.apache.storm.topology.OutputFieldsDeclarer;
    +import org.apache.storm.topology.TopologyBuilder;
    +import org.apache.storm.topology.base.BaseStatefulWindowedBolt;
    +import org.apache.storm.tuple.Fields;
    +import org.apache.storm.tuple.Tuple;
    +import org.apache.storm.tuple.Values;
    +import org.apache.storm.windowing.TupleWindow;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * An example that demonstrates the usage of {@link org.apache.storm.topology.IStatefulWindowedBolt} with window
    + * persistence.
    + * <p>
    + * The framework automatically checkpoints the tuples in window along with the bolt's state and restores the same
    --- End diff --
    
    Nit: in the window


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r128228066
  
    --- Diff: storm-client/src/jvm/org/apache/storm/windowing/WindowManager.java ---
    @@ -289,4 +414,24 @@ public String toString() {
                     ", triggerPolicy=" + triggerPolicy +
                     '}';
         }
    +
    +    public void restoreState(Map<String, ?> state) {
    +        if (state != null) {
    +            Optional<?> es = (Optional<?>) state.get(EVICTION_STATE_KEY);
    --- End diff --
    
    Maybe better to have Optional.empty() as default value and remove checking null. Same applies as ts.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129649500
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/SimpleWindowPartitionCache.java ---
    @@ -0,0 +1,191 @@
    +/**
    + * 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 java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.concurrent.ConcurrentMap;
    +import java.util.concurrent.ConcurrentSkipListMap;
    +import java.util.concurrent.locks.ReentrantLock;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * A simple implementation that evicts the largest un-pinned entry from the cache. This works well
    + * for caching window partitions since the access pattern is mostly sequential scans.
    + */
    +public class SimpleWindowPartitionCache<K, V> implements WindowPartitionCache<K, V> {
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleWindowPartitionCache.class);
    +
    +    private final ConcurrentSkipListMap<K, V> map = new ConcurrentSkipListMap<>();
    +    private final Map<K, Long> pinned = new HashMap<>();
    +    private final long maximumSize;
    +    private final RemovalListener<K, V> removalListener;
    +    private final CacheLoader<K, V> cacheLoader;
    +    private final ReentrantLock lock = new ReentrantLock(true);
    +    private int size;
    +
    +    @Override
    +    public V get(K key) {
    +        return getOrLoad(key, false);
    +    }
    +
    +    @Override
    +    public V getPinned(K key) {
    +        return getOrLoad(key, true);
    +    }
    +
    +    @Override
    +    public boolean unpin(K key) {
    +        LOG.debug("unpin '{}'", key);
    +        boolean res = false;
    +        try {
    +            lock.lock();
    +            if (pinned.compute(key, (k, v) -> v == null ? 0 : v - 1) <= 0) {
    +                pinned.remove(key);
    +                res = true;
    +            }
    +        } finally {
    +            lock.unlock();
    +        }
    +        LOG.debug("pinned '{}'", pinned);
    +        return res;
    +    }
    +
    +    @Override
    +    public ConcurrentMap<K, V> asMap() {
    +        return map;
    +    }
    +
    +    @Override
    +    public void invalidate(K key) {
    +        try {
    +            lock.lock();
    +            if (isPinned(key)) {
    +                LOG.debug("Entry '{}' is pinned, skipping invalidation", key);
    +            } else {
    +                LOG.debug("Invalidating entry '{}'", key);
    +                V val = map.remove(key);
    +                if (val != null) {
    +                    --size;
    +                    pinned.remove(key);
    +                    removalListener.onRemoval(key, val, RemovalCause.EXPLICIT);
    +                }
    +            }
    +        } finally {
    +            lock.unlock();
    +        }
    +    }
    +
    +    // Get or load from the cache optionally pinning the entry
    +    // so that it wont get evicted from the cache
    +    private V getOrLoad(K key, boolean shouldPin) {
    +        V val;
    +        if (shouldPin) {
    +            try {
    +                lock.lock();
    +                val = load(key);
    +                pin(key);
    +            } finally {
    +                lock.unlock();
    +            }
    +        } else {
    +            val = map.get(key);
    +            if (val == null) {
    +                try {
    +                    lock.lock();
    +                    val = load(key);
    +                } finally {
    +                    lock.unlock();
    +                }
    +            }
    +        }
    +
    +        return val;
    +    }
    +
    +    private V load(K key) {
    +        V val = map.get(key);
    +        if (val == null) {
    +            val = cacheLoader.load(key);
    +            if (val == null) {
    +                throw new NullPointerException("Null value");
    +            }
    +            ensureCapacity();
    +            map.put(key, val);
    +            ++size;
    +        }
    +        return val;
    +    }
    +
    +    private void ensureCapacity() {
    +        if (size >= maximumSize) {
    +            Iterator<Map.Entry<K, V>> it = map.descendingMap().entrySet().iterator();
    +            while (it.hasNext()) {
    +                Map.Entry<K, V> next = it.next();
    +                if (!isPinned(next.getKey())) {
    +                    it.remove();
    +                    removalListener.onRemoval(next.getKey(), next.getValue(), RemovalCause.REPLACED);
    +                    --size;
    +                    break;
    +                }
    +            }
    +        }
    +    }
    +
    +    private void pin(K key) {
    +        LOG.debug("pin '{}'", key);
    +        pinned.compute(key, (k, v) -> v == null ? 1L : v + 1);
    +        LOG.debug("pinned '{}'", pinned);
    +    }
    +
    +    private boolean isPinned(K key) {
    +        return pinned.getOrDefault(key, 0L) > 0;
    --- End diff --
    
    Is it valid to have pinned contain a key with 0 value? Otherwise wouldn't pinned.contains work just as well?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130228748
  
    --- Diff: storm-client/src/jvm/org/apache/storm/windowing/WindowManager.java ---
    @@ -111,14 +125,86 @@ public void add(Event<T> windowEvent) {
                 LOG.debug("Got watermark event with ts {}", windowEvent.getTimestamp());
             }
             track(windowEvent);
    -        compactWindow();
    +        if (!stateful) {
    +            compactWindow();
    +        }
         }
     
         /**
          * The callback invoked by the trigger policy.
          */
         @Override
         public boolean onTrigger() {
    +        return stateful ? doOnTriggerStateful() : doOnTrigger();
    +    }
    +
    +    private static class IteratorStatus {
    +        private boolean valid = true;
    +
    +        void invalidate() {
    +            valid = false;
    +        }
    +
    +        boolean isValid() {
    +            return valid;
    +        }
    +    }
    +
    +    private static<T> Iterator<T> expiringIterator(Iterator<T> inner, IteratorStatus status) {
    +        return new Iterator<T>() {
    +            @Override
    +            public boolean hasNext() {
    +                if (status.isValid()) {
    +                    return inner.hasNext();
    +                }
    +                throw new IllegalStateException("Stale iterator");
    +            }
    +
    +            @Override
    +            public T next() {
    +                if (status.isValid()) {
    +                    return inner.next();
    +                }
    +                throw new IllegalStateException("Stale iterator");
    +            }
    +        };
    +    }
    +
    +    private boolean doOnTriggerStateful() {
    +        Supplier<Iterator<T>> scanEventsStateful = this::scanEventsStateful;
    +        Iterator<T> it = scanEventsStateful.get();
    +        boolean hasEvents = it.hasNext();
    +        if (hasEvents) {
    +            final IteratorStatus status = new IteratorStatus();
    +            LOG.debug("invoking windowLifecycleListener onActivation with iterator");
    +            // reuse the retrieved iterator
    +            Supplier<Iterator<T>> wrapper = new Supplier<Iterator<T>>() {
    +                Iterator<T> initial = it;
    +                @Override
    +                public Iterator<T> get() {
    +                    if (status.isValid()) {
    --- End diff --
    
    Could you explain how this mechanism works?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130273758
  
    --- Diff: storm-client/src/jvm/org/apache/storm/windowing/WindowManager.java ---
    @@ -53,28 +61,34 @@
         public static final int EXPIRE_EVENTS_THRESHOLD = 100;
     
         private final WindowLifecycleListener<T> windowLifecycleListener;
    -    private final ConcurrentLinkedQueue<Event<T>> queue;
    +    private final Collection<Event<T>> queue;
         private final List<T> expiredEvents;
         private final Set<Event<T>> prevWindowEvents;
         private final AtomicInteger eventsSinceLastExpiry;
         private final ReentrantLock lock;
    -    private EvictionPolicy<T> evictionPolicy;
    -    private TriggerPolicy<T> triggerPolicy;
    -
    +    private final boolean stateful;
    +    private EvictionPolicy<T, ?> evictionPolicy;
    +    private TriggerPolicy<T, ?> triggerPolicy;
         public WindowManager(WindowLifecycleListener<T> lifecycleListener) {
    +        this(lifecycleListener, new ConcurrentLinkedQueue<>(), false);
    +    }
    +
    +    public WindowManager(WindowLifecycleListener<T> lifecycleListener, Collection<Event<T>> queue, boolean stateful) {
    --- End diff --
    
    Ok.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130273666
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/SimpleWindowPartitionCache.java ---
    @@ -0,0 +1,191 @@
    +/**
    + * 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 java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.concurrent.ConcurrentMap;
    +import java.util.concurrent.ConcurrentSkipListMap;
    +import java.util.concurrent.locks.ReentrantLock;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * A simple implementation that evicts the largest un-pinned entry from the cache. This works well
    + * for caching window partitions since the access pattern is mostly sequential scans.
    + */
    +public class SimpleWindowPartitionCache<K, V> implements WindowPartitionCache<K, V> {
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleWindowPartitionCache.class);
    +
    +    private final ConcurrentSkipListMap<K, V> map = new ConcurrentSkipListMap<>();
    +    private final Map<K, Long> pinned = new HashMap<>();
    +    private final long maximumSize;
    +    private final RemovalListener<K, V> removalListener;
    +    private final CacheLoader<K, V> cacheLoader;
    +    private final ReentrantLock lock = new ReentrantLock(true);
    +    private int size;
    +
    +    @Override
    +    public V get(K key) {
    +        return getOrLoad(key, false);
    +    }
    +
    +    @Override
    +    public V getPinned(K key) {
    +        return getOrLoad(key, true);
    +    }
    +
    +    @Override
    +    public boolean unpin(K key) {
    +        LOG.debug("unpin '{}'", key);
    +        boolean res = false;
    +        try {
    +            lock.lock();
    +            if (pinned.compute(key, (k, v) -> v == null ? 0 : v - 1) <= 0) {
    +                pinned.remove(key);
    +                res = true;
    +            }
    +        } finally {
    +            lock.unlock();
    +        }
    +        LOG.debug("pinned '{}'", pinned);
    +        return res;
    +    }
    +
    +    @Override
    +    public ConcurrentMap<K, V> asMap() {
    +        return map;
    +    }
    +
    +    @Override
    +    public void invalidate(K key) {
    +        try {
    +            lock.lock();
    +            if (isPinned(key)) {
    +                LOG.debug("Entry '{}' is pinned, skipping invalidation", key);
    +            } else {
    +                LOG.debug("Invalidating entry '{}'", key);
    +                V val = map.remove(key);
    +                if (val != null) {
    +                    --size;
    +                    pinned.remove(key);
    +                    removalListener.onRemoval(key, val, RemovalCause.EXPLICIT);
    +                }
    +            }
    +        } finally {
    +            lock.unlock();
    +        }
    +    }
    +
    +    // Get or load from the cache optionally pinning the entry
    +    // so that it wont get evicted from the cache
    +    private V getOrLoad(K key, boolean shouldPin) {
    +        V val;
    +        if (shouldPin) {
    +            try {
    +                lock.lock();
    +                val = load(key);
    +                pin(key);
    +            } finally {
    +                lock.unlock();
    +            }
    +        } else {
    +            val = map.get(key);
    --- End diff --
    
    An optimization to avoid the lock if the entry is already present.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130273783
  
    --- Diff: storm-client/src/jvm/org/apache/storm/windowing/WindowManager.java ---
    @@ -111,14 +125,86 @@ public void add(Event<T> windowEvent) {
                 LOG.debug("Got watermark event with ts {}", windowEvent.getTimestamp());
             }
             track(windowEvent);
    -        compactWindow();
    +        if (!stateful) {
    +            compactWindow();
    +        }
         }
     
         /**
          * The callback invoked by the trigger policy.
          */
         @Override
         public boolean onTrigger() {
    +        return stateful ? doOnTriggerStateful() : doOnTrigger();
    --- End diff --
    
    Strategy is not straightforward without a lot of refactoring. Will move it into sub class to keep the stateful case separate.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129763781
  
    --- Diff: storm-client/src/jvm/org/apache/storm/windowing/TupleWindowIterImpl.java ---
    @@ -0,0 +1,80 @@
    +/**
    + * 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.windowing;
    +
    +import com.google.common.collect.Iterators;
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.function.Supplier;
    +import org.apache.storm.tuple.Tuple;
    +
    +/**
    + * An iterator based implementation over the events in a window.
    + */
    +public class TupleWindowIterImpl implements TupleWindow {
    +    private final Supplier<Iterator<Tuple>> tuplesIt;
    +    private final Supplier<Iterator<Tuple>> newTuplesIt;
    +    private final Supplier<Iterator<Tuple>> expiredTuplesIt;
    +    private final Long startTimestamp;
    +    private final Long endTimestamp;
    +
    +    public TupleWindowIterImpl(Supplier<Iterator<Tuple>> tuplesIt,
    +                               Supplier<Iterator<Tuple>> newTuplesIt,
    +                               Supplier<Iterator<Tuple>> expiredTuplesIt,
    +                               Long startTimestamp, Long endTimestamp) {
    +        this.tuplesIt = tuplesIt;
    +        this.newTuplesIt = newTuplesIt;
    +        this.expiredTuplesIt = expiredTuplesIt;
    +        this.startTimestamp = startTimestamp;
    +        this.endTimestamp = endTimestamp;
    +    }
    +
    +    @Override
    +    public List<Tuple> get() {
    +        List<Tuple> tuples = new ArrayList<>();
    +        tuplesIt.get().forEachRemaining(t -> tuples.add(t));
    +        return tuples;
    +    }
    +
    +    @Override
    +    public List<Tuple> getNew() {
    +        throw new UnsupportedOperationException("Not implemented");
    --- End diff --
    
    it not straightforward to return the new and expired tuples iterators without persisting the tuples separately and maintaining the state, so for now its unsupported. The newTuplesIt and expiredTuplesIt receives null values for now


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130528860
  
    --- Diff: storm-client/src/jvm/org/apache/storm/windowing/WindowManager.java ---
    @@ -111,14 +125,86 @@ public void add(Event<T> windowEvent) {
                 LOG.debug("Got watermark event with ts {}", windowEvent.getTimestamp());
             }
             track(windowEvent);
    -        compactWindow();
    +        if (!stateful) {
    +            compactWindow();
    +        }
         }
     
         /**
          * The callback invoked by the trigger policy.
          */
         @Override
         public boolean onTrigger() {
    +        return stateful ? doOnTriggerStateful() : doOnTrigger();
    +    }
    +
    +    private static class IteratorStatus {
    +        private boolean valid = true;
    +
    +        void invalidate() {
    +            valid = false;
    +        }
    +
    +        boolean isValid() {
    +            return valid;
    +        }
    +    }
    +
    +    private static<T> Iterator<T> expiringIterator(Iterator<T> inner, IteratorStatus status) {
    +        return new Iterator<T>() {
    +            @Override
    +            public boolean hasNext() {
    +                if (status.isValid()) {
    +                    return inner.hasNext();
    +                }
    +                throw new IllegalStateException("Stale iterator");
    +            }
    +
    +            @Override
    +            public T next() {
    +                if (status.isValid()) {
    +                    return inner.next();
    +                }
    +                throw new IllegalStateException("Stale iterator");
    +            }
    +        };
    +    }
    +
    +    private boolean doOnTriggerStateful() {
    +        Supplier<Iterator<T>> scanEventsStateful = this::scanEventsStateful;
    +        Iterator<T> it = scanEventsStateful.get();
    +        boolean hasEvents = it.hasNext();
    +        if (hasEvents) {
    +            final IteratorStatus status = new IteratorStatus();
    +            LOG.debug("invoking windowLifecycleListener onActivation with iterator");
    +            // reuse the retrieved iterator
    +            Supplier<Iterator<T>> wrapper = new Supplier<Iterator<T>>() {
    +                Iterator<T> initial = it;
    +                @Override
    +                public Iterator<T> get() {
    +                    if (status.isValid()) {
    --- End diff --
    
    Will update the message. 
    Within execute, a bolt may invoke `get`/`getIter` multiple times. So here we have a supplier and return a new iterator each time.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #2218: STORM-2614: Enhance stateful windowing to persist the win...

Posted by ben-manes <gi...@git.apache.org>.
Github user ben-manes commented on the issue:

    https://github.com/apache/storm/pull/2218
  
    fyi, you can emulate pinning by setting the entry's weight to zero. Then the entry will not be evicted, but is eligible for expiration, etc. Of course zero weight entries means the cache allows more entries overall, whereas pinning might still restrict overall capacity. But its not clear how to a cache should handle evictions when not enough space can be freed from unpinned entries.
    
    The other approach is less elegant. A `CacheWriter` can intercept an eviction and a `CacheLoader` intercepts gets to fetch a missing entry. Since recursive computations are disallowed, the writer could re-put the entry asynchronously. This races with a `get`, so the loader could fetch from the map prior to some other data store. In some ways not too dissimilar to your own pinned cache.
    
    In general pinning breaks the caching model of recomputable transient state and the eviction policy will try to avoid pollution by removing low-value entries early. Generally it can lead to O(n) evictions unless maintained in a dedicated LRU queue so that potential victims are not evaluated. Since pinning vs capacity is confusing, right now Caffeine favors zero weights since the use-cases are unclear.
    
    Hope that helps. I agree your own logic may be more straightforward and preferable.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/storm/pull/2218


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130227746
  
    --- Diff: storm-client/test/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutorTest.java ---
    @@ -0,0 +1,261 @@
    +/**
    + * 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.Config;
    +import org.apache.storm.generated.GlobalStreamId;
    +import org.apache.storm.state.KeyValueState;
    +import org.apache.storm.streams.Pair;
    +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.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.TimestampExtractor;
    +import org.apache.storm.windowing.TupleWindow;
    +import org.apache.storm.windowing.WaterMarkEvent;
    +import org.apache.storm.windowing.WaterMarkEventGenerator;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.mockito.ArgumentCaptor;
    +import org.mockito.Captor;
    +import org.mockito.Mockito;
    +import org.mockito.MockitoAnnotations;
    +import org.mockito.invocation.InvocationOnMock;
    +import org.mockito.runners.MockitoJUnitRunner;
    +import org.mockito.stubbing.Answer;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.stream.Collectors;
    +import java.util.stream.LongStream;
    +
    +import static org.mockito.AdditionalAnswers.returnsArgAt;
    +
    +/**
    + * Unit tests for {@link PersistentWindowedBoltExecutor}
    + */
    +@RunWith(MockitoJUnitRunner.class)
    +public class PersistentWindowedBoltExecutorTest {
    +    public static final String LATE_STREAM = "late_stream";
    +    private PersistentWindowedBoltExecutor<KeyValueState<String, String>> executor;
    +    private IStatefulWindowedBolt<KeyValueState<String, String>> mockBolt;
    +    private OutputCollector mockOutputCollector;
    +    private TopologyContext mockTopologyContext;
    +    private Map<String, Object> mockStormConf = new HashMap<>();
    +    private TimestampExtractor mockTimestampExtractor;
    +    private WaterMarkEventGenerator mockWaterMarkEventGenerator;
    +    private KeyValueState mockPartitionState;
    +    private KeyValueState mockWindowState;
    +    private KeyValueState mockSystemState;
    +
    +    private static final String PARTITION_KEY = "pk";
    +    private static final String EVICTION_STATE_KEY = "es";
    +    private static final String TRIGGER_STATE_KEY = "ts";
    +    private long tupleTs;
    +
    +    @Captor
    +    private ArgumentCaptor<Tuple> tupleCaptor;
    +    @Captor
    +    private ArgumentCaptor<Collection<Tuple>> anchorCaptor;
    +    @Captor
    +    private ArgumentCaptor<String> stringCaptor;
    +    @Captor
    +    private ArgumentCaptor<Long> longCaptor;
    +    @Captor
    +    private ArgumentCaptor<Values> valuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<TupleWindow> tupleWindowCaptor;
    +    @Captor
    +    private ArgumentCaptor<Deque<Long>> partitionValuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<PersistentWindowedBoltExecutor.WindowPartition<Event<Tuple>>> windowValuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<Object> systemValuesCaptor;
    +
    +    @Before
    +    public void setUp() throws Exception {
    +        MockitoAnnotations.initMocks(this);
    +        mockBolt = Mockito.mock(IStatefulWindowedBolt.class);
    +        mockWaterMarkEventGenerator = Mockito.mock(WaterMarkEventGenerator.class);
    +        mockTimestampExtractor = Mockito.mock(TimestampExtractor.class);
    +        tupleTs = System.currentTimeMillis();
    +        Mockito.when(mockTimestampExtractor.extractTimestamp(Mockito.any())).thenReturn(tupleTs);
    +        Mockito.when(mockBolt.getTimestampExtractor()).thenReturn(mockTimestampExtractor);
    +        Mockito.when(mockBolt.isPersistent()).thenReturn(true);
    +        mockPartitionState = Mockito.mock(KeyValueState.class);
    +        mockWindowState = Mockito.mock(KeyValueState.class);
    +        mockSystemState = Mockito.mock(KeyValueState.class);
    +        mockTopologyContext = Mockito.mock(TopologyContext.class);
    +        Mockito.when(mockTopologyContext.getThisStreams()).thenReturn(Collections.singleton(LATE_STREAM));
    +        mockOutputCollector = Mockito.mock(OutputCollector.class);
    +        executor = new PersistentWindowedBoltExecutor<>(mockBolt);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_WINDOW_LENGTH_COUNT, 5);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_SLIDING_INTERVAL_COUNT, 5);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_LATE_TUPLE_STREAM, LATE_STREAM);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_WATERMARK_EVENT_INTERVAL_MS, 100_000);
    +        mockStormConf.put(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS, 30);
    +        mockStormConf.put(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL, 1000);
    +        Mockito.when(mockPartitionState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(mockWindowState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(mockSystemState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        executor.prepare(mockStormConf, mockTopologyContext, mockOutputCollector,
    +            mockWindowState, mockPartitionState, mockSystemState);
    +    }
    +
    +    @Test
    +    public void testExecuteTuple() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), 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);
    +    }
    +
    +    @Test
    +    public void testExecuteLatetuple() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), 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
    +        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());
    +    }
    +
    +    @Test
    +    public void testActivation() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), Mockito.anyLong())).thenReturn(true);
    +        executor.initState(null);
    +        executor.waterMarkEventGenerator = mockWaterMarkEventGenerator;
    +
    +        List<Tuple> mockTuples = getMockTuples(5);
    +        mockTuples.forEach(t -> executor.execute(t));
    +        // all tuples acked
    +        Mockito.verify(mockOutputCollector, Mockito.times(5)).ack(tupleCaptor.capture());
    +        Assert.assertArrayEquals(mockTuples.toArray(), tupleCaptor.getAllValues().toArray());
    +
    +        Mockito.doAnswer(new Answer<Void>() {
    +            @Override
    +            public Void answer(InvocationOnMock invocation) throws Throwable {
    +                TupleWindow window = (TupleWindow) invocation.getArguments()[0];
    +                // iterate the tuples
    +                Assert.assertEquals(5, window.get().size());
    +                // iterating multiple times should produce same events
    +                Assert.assertEquals(5, window.get().size());
    +                Assert.assertEquals(5, window.get().size());
    +                return null;
    +            }
    +        }).when(mockBolt).execute(Mockito.any());
    +        // trigger the window
    +        long activationTs = tupleTs + 1000;
    +        executor.getWindowManager().add(new WaterMarkEvent<>(activationTs));
    +        executor.prePrepare(0);
    +
    +        // partition ids
    +        Mockito.verify(mockPartitionState, Mockito.times(1)).put(stringCaptor.capture(), partitionValuesCaptor.capture());
    +        Assert.assertEquals(PARTITION_KEY, stringCaptor.getAllValues().get(0));
    +        Assert.assertEquals(Collections.singletonList(0L), partitionValuesCaptor.getAllValues().get(0));
    +
    +        // window partitions
    +        Mockito.verify(mockWindowState, Mockito.times(1)).put(longCaptor.capture(), windowValuesCaptor.capture());
    +        Assert.assertEquals(0L, (long) longCaptor.getAllValues().get(0));
    +        Assert.assertEquals(5, windowValuesCaptor.getAllValues().get(0).size());
    +        List<Event<?>> tuples = windowValuesCaptor.getAllValues().get(0)
    +            .getEvents().stream().map(Event::get).collect(Collectors.toList());
    +        Assert.assertArrayEquals(mockTuples.toArray(), tuples.toArray());
    +
    +        // window system state
    +        Mockito.verify(mockSystemState, Mockito.times(2)).put(stringCaptor.capture(), systemValuesCaptor.capture());
    +        Assert.assertEquals(EVICTION_STATE_KEY, stringCaptor.getAllValues().get(1));
    +        Assert.assertEquals(Optional.of(Pair.of(5L, 5L)), systemValuesCaptor.getAllValues().get(0));
    +        Assert.assertEquals(TRIGGER_STATE_KEY, stringCaptor.getAllValues().get(2));
    +        Assert.assertEquals(Optional.of(tupleTs), systemValuesCaptor.getAllValues().get(1));
    +    }
    +
    +    @Test
    +    public void testCacheEviction() {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), Mockito.anyLong())).thenReturn(true);
    +        executor.initState(null);
    +        executor.waterMarkEventGenerator = mockWaterMarkEventGenerator;
    +        List<Tuple> mockTuples = getMockTuples(20000);
    +        mockTuples.forEach(t -> executor.execute(t));
    +
    +        Mockito.verify(mockWindowState, Mockito.times(10)).put(longCaptor.capture(), windowValuesCaptor.capture());
    +        Assert.assertEquals(10, longCaptor.getAllValues().size());
    +        Assert.assertEquals(10, windowValuesCaptor.getAllValues().size());
    +        // number of evicted events
    +        Assert.assertEquals(10_000, windowValuesCaptor.getAllValues().stream()
    --- End diff --
    
    I think you should make MAX_PARTITION_EVENTS and MIN_PARTITIONS public so you can refer to them here. Much of this test might also fit better into a WindowCache test


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by satishd <gi...@git.apache.org>.
Github user satishd commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129760482
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutor.java ---
    @@ -0,0 +1,563 @@
    +/**
    + * 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 java.util.AbstractCollection;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentLinkedQueue;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.locks.ReentrantLock;
    +import java.util.function.Supplier;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.state.KeyValueState;
    +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.windowing.DefaultEvictionContext;
    +import org.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.EventImpl;
    +import org.apache.storm.windowing.WindowLifecycleListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static java.util.Collections.emptyIterator;
    +import static org.apache.storm.topology.WindowPartitionCache.CacheLoader;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalCause;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalListener;
    +
    +/**
    + * Wraps a {@link IStatefulWindowedBolt} and handles the execution. Uses state and the underlying
    + * checkpointing mechanisms to save the tuples in window to state. The tuples are also kept in-memory
    + * by transparently caching the window partitions and checkpointing them as needed.
    + */
    +public class PersistentWindowedBoltExecutor<T extends State> extends WindowedBoltExecutor implements IStatefulBolt<T> {
    +    private static final Logger LOG = LoggerFactory.getLogger(PersistentWindowedBoltExecutor.class);
    +    private final IStatefulWindowedBolt<T> statefulWindowedBolt;
    +    private transient TopologyContext topologyContext;
    +    private transient OutputCollector outputCollector;
    +    private transient WindowState<Tuple> state;
    +    private transient boolean stateInitialized;
    +    private transient boolean prePrepared;
    +
    +    public PersistentWindowedBoltExecutor(IStatefulWindowedBolt<T> bolt) {
    +        super(bolt);
    +        statefulWindowedBolt = bolt;
    +    }
    +
    +    @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));
    +    }
    +
    +    // package access for unit tests
    +    void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                 KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                 KeyValueState<String, Deque<Long>> partitionState,
    +                 KeyValueState<String, Optional<?>> windowSystemState) {
    +        init(topoConf, context, collector, windowState, partitionState, windowSystemState);
    --- End diff --
    
    better to inline this as init assigns two fields and instantiates `WindowState`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130462981
  
    --- Diff: storm-client/src/jvm/org/apache/storm/windowing/WindowManager.java ---
    @@ -111,14 +125,86 @@ public void add(Event<T> windowEvent) {
                 LOG.debug("Got watermark event with ts {}", windowEvent.getTimestamp());
             }
             track(windowEvent);
    -        compactWindow();
    +        if (!stateful) {
    +            compactWindow();
    +        }
         }
     
         /**
          * The callback invoked by the trigger policy.
          */
         @Override
         public boolean onTrigger() {
    +        return stateful ? doOnTriggerStateful() : doOnTrigger();
    +    }
    +
    +    private static class IteratorStatus {
    +        private boolean valid = true;
    +
    +        void invalidate() {
    +            valid = false;
    +        }
    +
    +        boolean isValid() {
    +            return valid;
    +        }
    +    }
    +
    +    private static<T> Iterator<T> expiringIterator(Iterator<T> inner, IteratorStatus status) {
    +        return new Iterator<T>() {
    +            @Override
    +            public boolean hasNext() {
    +                if (status.isValid()) {
    +                    return inner.hasNext();
    +                }
    +                throw new IllegalStateException("Stale iterator");
    +            }
    +
    +            @Override
    +            public T next() {
    +                if (status.isValid()) {
    +                    return inner.next();
    +                }
    +                throw new IllegalStateException("Stale iterator");
    +            }
    +        };
    +    }
    +
    +    private boolean doOnTriggerStateful() {
    +        Supplier<Iterator<T>> scanEventsStateful = this::scanEventsStateful;
    +        Iterator<T> it = scanEventsStateful.get();
    +        boolean hasEvents = it.hasNext();
    +        if (hasEvents) {
    +            final IteratorStatus status = new IteratorStatus();
    +            LOG.debug("invoking windowLifecycleListener onActivation with iterator");
    +            // reuse the retrieved iterator
    +            Supplier<Iterator<T>> wrapper = new Supplier<Iterator<T>>() {
    +                Iterator<T> initial = it;
    +                @Override
    +                public Iterator<T> get() {
    +                    if (status.isValid()) {
    --- End diff --
    
    Makes sense, thanks. Could you expand the IllegalStateException to mention this? Why is the iterator being updated in line 191 btw?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130227459
  
    --- Diff: storm-client/test/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutorTest.java ---
    @@ -0,0 +1,261 @@
    +/**
    + * 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.Config;
    +import org.apache.storm.generated.GlobalStreamId;
    +import org.apache.storm.state.KeyValueState;
    +import org.apache.storm.streams.Pair;
    +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.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.TimestampExtractor;
    +import org.apache.storm.windowing.TupleWindow;
    +import org.apache.storm.windowing.WaterMarkEvent;
    +import org.apache.storm.windowing.WaterMarkEventGenerator;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.mockito.ArgumentCaptor;
    +import org.mockito.Captor;
    +import org.mockito.Mockito;
    +import org.mockito.MockitoAnnotations;
    +import org.mockito.invocation.InvocationOnMock;
    +import org.mockito.runners.MockitoJUnitRunner;
    +import org.mockito.stubbing.Answer;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.stream.Collectors;
    +import java.util.stream.LongStream;
    +
    +import static org.mockito.AdditionalAnswers.returnsArgAt;
    +
    +/**
    + * Unit tests for {@link PersistentWindowedBoltExecutor}
    + */
    +@RunWith(MockitoJUnitRunner.class)
    +public class PersistentWindowedBoltExecutorTest {
    +    public static final String LATE_STREAM = "late_stream";
    +    private PersistentWindowedBoltExecutor<KeyValueState<String, String>> executor;
    +    private IStatefulWindowedBolt<KeyValueState<String, String>> mockBolt;
    +    private OutputCollector mockOutputCollector;
    +    private TopologyContext mockTopologyContext;
    +    private Map<String, Object> mockStormConf = new HashMap<>();
    +    private TimestampExtractor mockTimestampExtractor;
    +    private WaterMarkEventGenerator mockWaterMarkEventGenerator;
    +    private KeyValueState mockPartitionState;
    +    private KeyValueState mockWindowState;
    +    private KeyValueState mockSystemState;
    +
    +    private static final String PARTITION_KEY = "pk";
    +    private static final String EVICTION_STATE_KEY = "es";
    +    private static final String TRIGGER_STATE_KEY = "ts";
    +    private long tupleTs;
    +
    +    @Captor
    +    private ArgumentCaptor<Tuple> tupleCaptor;
    +    @Captor
    +    private ArgumentCaptor<Collection<Tuple>> anchorCaptor;
    +    @Captor
    +    private ArgumentCaptor<String> stringCaptor;
    +    @Captor
    +    private ArgumentCaptor<Long> longCaptor;
    +    @Captor
    +    private ArgumentCaptor<Values> valuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<TupleWindow> tupleWindowCaptor;
    +    @Captor
    +    private ArgumentCaptor<Deque<Long>> partitionValuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<PersistentWindowedBoltExecutor.WindowPartition<Event<Tuple>>> windowValuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<Object> systemValuesCaptor;
    +
    +    @Before
    +    public void setUp() throws Exception {
    +        MockitoAnnotations.initMocks(this);
    +        mockBolt = Mockito.mock(IStatefulWindowedBolt.class);
    +        mockWaterMarkEventGenerator = Mockito.mock(WaterMarkEventGenerator.class);
    +        mockTimestampExtractor = Mockito.mock(TimestampExtractor.class);
    +        tupleTs = System.currentTimeMillis();
    +        Mockito.when(mockTimestampExtractor.extractTimestamp(Mockito.any())).thenReturn(tupleTs);
    +        Mockito.when(mockBolt.getTimestampExtractor()).thenReturn(mockTimestampExtractor);
    +        Mockito.when(mockBolt.isPersistent()).thenReturn(true);
    +        mockPartitionState = Mockito.mock(KeyValueState.class);
    +        mockWindowState = Mockito.mock(KeyValueState.class);
    +        mockSystemState = Mockito.mock(KeyValueState.class);
    +        mockTopologyContext = Mockito.mock(TopologyContext.class);
    +        Mockito.when(mockTopologyContext.getThisStreams()).thenReturn(Collections.singleton(LATE_STREAM));
    +        mockOutputCollector = Mockito.mock(OutputCollector.class);
    +        executor = new PersistentWindowedBoltExecutor<>(mockBolt);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_WINDOW_LENGTH_COUNT, 5);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_SLIDING_INTERVAL_COUNT, 5);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_LATE_TUPLE_STREAM, LATE_STREAM);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_WATERMARK_EVENT_INTERVAL_MS, 100_000);
    +        mockStormConf.put(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS, 30);
    +        mockStormConf.put(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL, 1000);
    +        Mockito.when(mockPartitionState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(mockWindowState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(mockSystemState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        executor.prepare(mockStormConf, mockTopologyContext, mockOutputCollector,
    +            mockWindowState, mockPartitionState, mockSystemState);
    +    }
    +
    +    @Test
    +    public void testExecuteTuple() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), 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);
    +    }
    +
    +    @Test
    +    public void testExecuteLatetuple() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), 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
    +        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());
    +    }
    +
    +    @Test
    +    public void testActivation() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), Mockito.anyLong())).thenReturn(true);
    +        executor.initState(null);
    +        executor.waterMarkEventGenerator = mockWaterMarkEventGenerator;
    +
    +        List<Tuple> mockTuples = getMockTuples(5);
    +        mockTuples.forEach(t -> executor.execute(t));
    +        // all tuples acked
    +        Mockito.verify(mockOutputCollector, Mockito.times(5)).ack(tupleCaptor.capture());
    +        Assert.assertArrayEquals(mockTuples.toArray(), tupleCaptor.getAllValues().toArray());
    +
    +        Mockito.doAnswer(new Answer<Void>() {
    +            @Override
    +            public Void answer(InvocationOnMock invocation) throws Throwable {
    +                TupleWindow window = (TupleWindow) invocation.getArguments()[0];
    +                // iterate the tuples
    +                Assert.assertEquals(5, window.get().size());
    +                // iterating multiple times should produce same events
    +                Assert.assertEquals(5, window.get().size());
    +                Assert.assertEquals(5, window.get().size());
    +                return null;
    +            }
    +        }).when(mockBolt).execute(Mockito.any());
    +        // trigger the window
    +        long activationTs = tupleTs + 1000;
    +        executor.getWindowManager().add(new WaterMarkEvent<>(activationTs));
    +        executor.prePrepare(0);
    +
    +        // partition ids
    +        Mockito.verify(mockPartitionState, Mockito.times(1)).put(stringCaptor.capture(), partitionValuesCaptor.capture());
    +        Assert.assertEquals(PARTITION_KEY, stringCaptor.getAllValues().get(0));
    --- End diff --
    
    There's a getValue you can use instead I think. You already know there is exactly one value due to the verify above


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r131123215
  
    --- Diff: storm-client/test/jvm/org/apache/storm/windowing/persistence/WindowStateTest.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.windowing.persistence;
    +
    +import org.apache.storm.state.KeyValueState;
    +import org.apache.storm.tuple.Tuple;
    +import org.apache.storm.windowing.Event;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.mockito.ArgumentCaptor;
    +import org.mockito.Captor;
    +import org.mockito.Mock;
    +import org.mockito.Mockito;
    +import org.mockito.MockitoAnnotations;
    +import org.mockito.invocation.InvocationOnMock;
    +import org.mockito.runners.MockitoJUnitRunner;
    +import org.mockito.stubbing.Answer;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.function.Consumer;
    +import java.util.function.Supplier;
    +
    +import static org.mockito.AdditionalAnswers.returnsArgAt;
    +
    +/**
    + * Unit tests for {@link WindowState}
    + */
    +@RunWith(MockitoJUnitRunner.class)
    +public class WindowStateTest {
    +
    +    @Mock
    +    private KeyValueState<Long, WindowState.WindowPartition<Integer>> windowState;
    +    @Mock
    +    private KeyValueState<String, Deque<Long>> partitionIdsState;
    +    @Mock
    +    private KeyValueState<String, Optional<?>> systemState;
    +    @Mock
    +    private Supplier<Map<String, Optional<?>>> supplier;
    +    @Captor
    +    private ArgumentCaptor<Long> longCaptor;
    +    @Captor
    +    private ArgumentCaptor<WindowState.WindowPartition<Integer>> windowValuesCaptor;
    +
    +    private static final int MAX_EVENTS_PER_PARTITION = 1000;
    +
    +    @Before
    +    public void setUp() throws Exception {
    +        MockitoAnnotations.initMocks(this);
    +    }
    +
    +    @Test
    +    public void testAdd() throws Exception {
    +        Mockito.when(partitionIdsState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(windowState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +
    +        WindowState<Integer> ws = getWindowState(10 * MAX_EVENTS_PER_PARTITION);
    +
    +        long partitions = 15;
    +        long numEvents = partitions * MAX_EVENTS_PER_PARTITION;
    +        for (int i = 0; i < numEvents; i++) {
    +            ws.add(getEvent(i));
    +        }
    +        // 5 events evicted to window state
    +        Mockito.verify(windowState, Mockito.times(5)).put(longCaptor.capture(), windowValuesCaptor.capture());
    +        Assert.assertEquals(5, longCaptor.getAllValues().size());
    +        // each evicted partition has MAX_EVENTS_PER_PARTITION
    +        windowValuesCaptor.getAllValues().forEach(wp -> {
    +            Assert.assertEquals(MAX_EVENTS_PER_PARTITION, wp.size());
    +        });
    +        // last partition is not evicted
    +        Assert.assertFalse(longCaptor.getAllValues().contains(partitions - 1));
    +    }
    +
    +    @Test
    +    public void testIterator() throws Exception {
    +        Map<Long, WindowState.WindowPartition<Event<Tuple>>> partitionMap = new HashMap<>();
    +        Mockito.when(partitionIdsState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(windowState.get(Mockito.any(), Mockito.any())).then(new Answer<Object>() {
    +            @Override
    +            public Object answer(InvocationOnMock invocation) throws Throwable {
    +                Object[] args = invocation.getArguments();
    +                WindowState.WindowPartition<Event<Tuple>> evicted = partitionMap.get(args[0]);
    +                return evicted != null ? evicted : args[1];
    +            }
    +        });
    +
    +        Mockito.doAnswer(new Answer<Void>() {
    +            @Override
    +            public Void answer(InvocationOnMock invocation) throws Throwable {
    +                Object[] args = invocation.getArguments();
    +                partitionMap.put((long)args[0], (WindowState.WindowPartition<Event<Tuple>>)args[1]);
    +                return null;
    +            }
    +        }).when(windowState).put(Mockito.any(), Mockito.any());
    +
    +        Mockito.doAnswer(new Answer<Void>() {
    +            @Override
    +            public Void answer(InvocationOnMock invocation) throws Throwable {
    +                Object[] args = invocation.getArguments();
    +                partitionMap.remove(args[0]);
    +                return null;
    +            }
    +        }).when(windowState).delete(Mockito.anyLong());
    +
    +        Mockito.when(supplier.get()).thenReturn(Collections.emptyMap());
    +
    +        WindowState<Integer> ws = getWindowState(10 * MAX_EVENTS_PER_PARTITION);
    +
    +        long partitions = 15;
    +
    +        long numEvents = partitions * MAX_EVENTS_PER_PARTITION;
    +        List<Event<Integer>> expected = new ArrayList<>();
    +        for (int i = 0; i < numEvents; i++) {
    +            Event<Integer> event = getEvent(i);
    +            expected.add(event);
    +            ws.add(event);
    +        }
    +
    +        Assert.assertEquals(5, partitionMap.size());
    +        Iterator<Event<Integer>> it = ws.iterator();
    +        List<Event<Integer>> actual = new ArrayList<>();
    +        it.forEachRemaining(actual::add);
    +        Assert.assertEquals(expected, actual);
    +
    +        // iterate again
    +        it = ws.iterator();
    +        actual.clear();
    +        it.forEachRemaining(actual::add);
    +        Assert.assertEquals(expected, actual);
    +
    +        // remove
    +        it = ws.iterator();
    +        while (it.hasNext()) {
    +            it.next();
    +            it.remove();
    +        }
    +
    +        it = ws.iterator();
    +        actual.clear();
    +        it.forEachRemaining(actual::add);
    +        Assert.assertEquals(Collections.emptyList(), actual);
    +    }
    +
    +    @Test
    +    public void testIteratorPartitionNotEvicted() throws Exception {
    +        Map<Long, WindowState.WindowPartition<Event<Tuple>>> partitionMap = new HashMap<>();
    +        Mockito.when(partitionIdsState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(windowState.get(Mockito.any(), Mockito.any())).then(new Answer<Object>() {
    +            @Override
    +            public Object answer(InvocationOnMock invocation) throws Throwable {
    +                Object[] args = invocation.getArguments();
    +                WindowState.WindowPartition<Event<Tuple>> evicted = partitionMap.get(args[0]);
    +                return evicted != null ? evicted : args[1];
    +            }
    +        });
    +
    +        Mockito.doAnswer(new Answer<Void>() {
    +            @Override
    +            public Void answer(InvocationOnMock invocation) throws Throwable {
    +                Object[] args = invocation.getArguments();
    +                partitionMap.put((long)args[0], (WindowState.WindowPartition<Event<Tuple>>)args[1]);
    +                return null;
    +            }
    +        }).when(windowState).put(Mockito.any(), Mockito.any());
    +
    +        Mockito.when(supplier.get()).thenReturn(Collections.emptyMap());
    +
    +        WindowState<Integer> ws = getWindowState(10 * MAX_EVENTS_PER_PARTITION);
    +
    +        long partitions = 10;
    +
    +        long numEvents = partitions * MAX_EVENTS_PER_PARTITION;
    +        List<Event<Integer>> expected = new ArrayList<>();
    +        for (int i = 0; i < numEvents; i++) {
    +            Event<Integer> event = getEvent(i);
    +            expected.add(event);
    +            ws.add(event);
    +        }
    +
    +        Iterator<Event<Integer>> it = ws.iterator();
    +        for(int i=0; i<9500; i++) {
    +            it.next();
    +        }
    +
    +        for (int i = 0; i < numEvents; i++) {
    +            Event<Integer> event = getEvent(i);
    +            expected.add(event);
    +            ws.add(event);
    +        }
    +
    +        // 9 th partition should not have been evicted
    --- End diff --
    
    Isn't this the 10th?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r131219599
  
    --- Diff: storm-client/test/jvm/org/apache/storm/windowing/persistence/WindowStateTest.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.windowing.persistence;
    +
    +import org.apache.storm.state.KeyValueState;
    +import org.apache.storm.tuple.Tuple;
    +import org.apache.storm.windowing.Event;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.mockito.ArgumentCaptor;
    +import org.mockito.Captor;
    +import org.mockito.Mock;
    +import org.mockito.Mockito;
    +import org.mockito.MockitoAnnotations;
    +import org.mockito.invocation.InvocationOnMock;
    +import org.mockito.runners.MockitoJUnitRunner;
    +import org.mockito.stubbing.Answer;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.function.Consumer;
    +import java.util.function.Supplier;
    +
    +import static org.mockito.AdditionalAnswers.returnsArgAt;
    +
    +/**
    + * Unit tests for {@link WindowState}
    + */
    +@RunWith(MockitoJUnitRunner.class)
    +public class WindowStateTest {
    +
    +    @Mock
    +    private KeyValueState<Long, WindowState.WindowPartition<Integer>> windowState;
    +    @Mock
    +    private KeyValueState<String, Deque<Long>> partitionIdsState;
    +    @Mock
    +    private KeyValueState<String, Optional<?>> systemState;
    +    @Mock
    +    private Supplier<Map<String, Optional<?>>> supplier;
    +    @Captor
    +    private ArgumentCaptor<Long> longCaptor;
    +    @Captor
    +    private ArgumentCaptor<WindowState.WindowPartition<Integer>> windowValuesCaptor;
    +
    +    private static final int MAX_EVENTS_PER_PARTITION = 1000;
    +
    +    @Before
    +    public void setUp() throws Exception {
    +        MockitoAnnotations.initMocks(this);
    +    }
    +
    +    @Test
    +    public void testAdd() throws Exception {
    +        Mockito.when(partitionIdsState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(windowState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +
    +        WindowState<Integer> ws = getWindowState(10 * MAX_EVENTS_PER_PARTITION);
    +
    +        long partitions = 15;
    +        long numEvents = partitions * MAX_EVENTS_PER_PARTITION;
    +        for (int i = 0; i < numEvents; i++) {
    +            ws.add(getEvent(i));
    +        }
    +        // 5 events evicted to window state
    +        Mockito.verify(windowState, Mockito.times(5)).put(longCaptor.capture(), windowValuesCaptor.capture());
    +        Assert.assertEquals(5, longCaptor.getAllValues().size());
    +        // each evicted partition has MAX_EVENTS_PER_PARTITION
    +        windowValuesCaptor.getAllValues().forEach(wp -> {
    +            Assert.assertEquals(MAX_EVENTS_PER_PARTITION, wp.size());
    +        });
    +        // last partition is not evicted
    +        Assert.assertFalse(longCaptor.getAllValues().contains(partitions - 1));
    +    }
    +
    +    @Test
    +    public void testIterator() throws Exception {
    +        Map<Long, WindowState.WindowPartition<Event<Tuple>>> partitionMap = new HashMap<>();
    +        Mockito.when(partitionIdsState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(windowState.get(Mockito.any(), Mockito.any())).then(new Answer<Object>() {
    +            @Override
    +            public Object answer(InvocationOnMock invocation) throws Throwable {
    +                Object[] args = invocation.getArguments();
    +                WindowState.WindowPartition<Event<Tuple>> evicted = partitionMap.get(args[0]);
    +                return evicted != null ? evicted : args[1];
    +            }
    +        });
    +
    +        Mockito.doAnswer(new Answer<Void>() {
    +            @Override
    +            public Void answer(InvocationOnMock invocation) throws Throwable {
    +                Object[] args = invocation.getArguments();
    +                partitionMap.put((long)args[0], (WindowState.WindowPartition<Event<Tuple>>)args[1]);
    +                return null;
    +            }
    +        }).when(windowState).put(Mockito.any(), Mockito.any());
    +
    +        Mockito.doAnswer(new Answer<Void>() {
    +            @Override
    +            public Void answer(InvocationOnMock invocation) throws Throwable {
    +                Object[] args = invocation.getArguments();
    +                partitionMap.remove(args[0]);
    +                return null;
    +            }
    +        }).when(windowState).delete(Mockito.anyLong());
    +
    +        Mockito.when(supplier.get()).thenReturn(Collections.emptyMap());
    +
    +        WindowState<Integer> ws = getWindowState(10 * MAX_EVENTS_PER_PARTITION);
    +
    +        long partitions = 15;
    +
    +        long numEvents = partitions * MAX_EVENTS_PER_PARTITION;
    +        List<Event<Integer>> expected = new ArrayList<>();
    +        for (int i = 0; i < numEvents; i++) {
    +            Event<Integer> event = getEvent(i);
    +            expected.add(event);
    +            ws.add(event);
    +        }
    +
    +        Assert.assertEquals(5, partitionMap.size());
    +        Iterator<Event<Integer>> it = ws.iterator();
    +        List<Event<Integer>> actual = new ArrayList<>();
    +        it.forEachRemaining(actual::add);
    +        Assert.assertEquals(expected, actual);
    +
    +        // iterate again
    +        it = ws.iterator();
    +        actual.clear();
    +        it.forEachRemaining(actual::add);
    +        Assert.assertEquals(expected, actual);
    +
    +        // remove
    +        it = ws.iterator();
    +        while (it.hasNext()) {
    +            it.next();
    +            it.remove();
    +        }
    +
    +        it = ws.iterator();
    +        actual.clear();
    +        it.forEachRemaining(actual::add);
    +        Assert.assertEquals(Collections.emptyList(), actual);
    +    }
    +
    +    @Test
    +    public void testIteratorPartitionNotEvicted() throws Exception {
    +        Map<Long, WindowState.WindowPartition<Event<Tuple>>> partitionMap = new HashMap<>();
    +        Mockito.when(partitionIdsState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(windowState.get(Mockito.any(), Mockito.any())).then(new Answer<Object>() {
    +            @Override
    +            public Object answer(InvocationOnMock invocation) throws Throwable {
    +                Object[] args = invocation.getArguments();
    +                WindowState.WindowPartition<Event<Tuple>> evicted = partitionMap.get(args[0]);
    +                return evicted != null ? evicted : args[1];
    +            }
    +        });
    +
    +        Mockito.doAnswer(new Answer<Void>() {
    +            @Override
    +            public Void answer(InvocationOnMock invocation) throws Throwable {
    +                Object[] args = invocation.getArguments();
    +                partitionMap.put((long)args[0], (WindowState.WindowPartition<Event<Tuple>>)args[1]);
    +                return null;
    +            }
    +        }).when(windowState).put(Mockito.any(), Mockito.any());
    +
    +        Mockito.when(supplier.get()).thenReturn(Collections.emptyMap());
    +
    +        WindowState<Integer> ws = getWindowState(10 * MAX_EVENTS_PER_PARTITION);
    +
    +        long partitions = 10;
    +
    +        long numEvents = partitions * MAX_EVENTS_PER_PARTITION;
    +        List<Event<Integer>> expected = new ArrayList<>();
    +        for (int i = 0; i < numEvents; i++) {
    +            Event<Integer> event = getEvent(i);
    +            expected.add(event);
    +            ws.add(event);
    +        }
    +
    +        Iterator<Event<Integer>> it = ws.iterator();
    +        for(int i=0; i<9500; i++) {
    +            it.next();
    +        }
    +
    +        for (int i = 0; i < numEvents; i++) {
    +            Event<Integer> event = getEvent(i);
    +            expected.add(event);
    +            ws.add(event);
    +        }
    +
    +        // 9 th partition should not have been evicted
    --- End diff --
    
    right.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r131220681
  
    --- Diff: storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java ---
    @@ -2363,7 +2363,7 @@ private CommonTopoInfo getCommonTopoInfo(String topoId, String operation) throws
                 ret.launchTimeSecs = 0;
             }
             ret.assignment = state.assignmentInfo(topoId, null);
    -        ret.beats = Utils.OR(heartbeatsCache.get().get(topoId), Collections.emptyMap());
    +        ret.beats = Utils.<Map<List<Integer>, Map<String, Object>>>OR(heartbeatsCache.get().get(topoId), Collections.emptyMap());
    --- End diff --
    
    Yes thanks, after upgrading the java 8 to the latest release it appears to work. Removed the type hint.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130528810
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutor.java ---
    @@ -0,0 +1,596 @@
    +/**
    + * 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 java.util.AbstractCollection;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Objects;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentLinkedQueue;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.locks.ReentrantLock;
    +import java.util.function.Supplier;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.state.KeyValueState;
    +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.topology.base.BaseWindowedBolt;
    +import org.apache.storm.tuple.Tuple;
    +import org.apache.storm.windowing.DefaultEvictionContext;
    +import org.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.EventImpl;
    +import org.apache.storm.windowing.WindowLifecycleListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static java.util.Collections.emptyIterator;
    +import static org.apache.storm.topology.WindowPartitionCache.CacheLoader;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalCause;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalListener;
    +
    +/**
    + * Wraps a {@link IStatefulWindowedBolt} and handles the execution. Uses state and the underlying
    + * checkpointing mechanisms to save the tuples in window to state. The tuples are also kept in-memory
    + * by transparently caching the window partitions and checkpointing them as needed.
    + */
    +public class PersistentWindowedBoltExecutor<T extends State> extends WindowedBoltExecutor implements IStatefulBolt<T> {
    +    private static final Logger LOG = LoggerFactory.getLogger(PersistentWindowedBoltExecutor.class);
    +    private final IStatefulWindowedBolt<T> statefulWindowedBolt;
    +    private transient TopologyContext topologyContext;
    +    private transient OutputCollector outputCollector;
    +    private transient WindowState<Tuple> state;
    +    private transient boolean stateInitialized;
    +    private transient boolean prePrepared;
    +
    +    public PersistentWindowedBoltExecutor(IStatefulWindowedBolt<T> bolt) {
    +        super(bolt);
    +        statefulWindowedBolt = bolt;
    +    }
    +
    +    @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));
    +    }
    +
    +    @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);
    +        }
    +    }
    +
    +    private int getCheckpointIntervalMillis(Map<String, Object> topoConf) {
    +        int checkpointInterval = Integer.MAX_VALUE;
    +        if (topoConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL) != null) {
    +            checkpointInterval = ((Number) topoConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL)).intValue();
    +        }
    +        return checkpointInterval;
    +    }
    +
    +    // package access for unit tests
    +    void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                 KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                 KeyValueState<String, Deque<Long>> partitionState,
    +                 KeyValueState<String, Optional<?>> windowSystemState) {
    +        init(topoConf, context, collector, windowState, partitionState, windowSystemState);
    +        doPrepare(topoConf, context, new NoAckOutputCollector(collector), state, true);
    +        Map<String, Optional<?>> wstate = new HashMap<>();
    +        windowSystemState.forEach(s -> wstate.put(s.getKey(), s.getValue()));
    +        restoreState(wstate);
    +    }
    +
    +    @Override
    +    protected void start() {
    +        if (stateInitialized) {
    +            super.start();
    +        } else {
    +            LOG.debug("Will invoke start after state is initialized");
    +        }
    +    }
    +
    +    @Override
    +    public void execute(Tuple input) {
    +        if (!stateInitialized) {
    +            throw new IllegalStateException("execute invoked before initState with input tuple " + input);
    +        }
    +        super.execute(input);
    +        // StatefulBoltExecutor does the actual ack when the state is saved.
    +        outputCollector.ack(input);
    +    }
    +
    +    @Override
    +    public void initState(T state) {
    +        if (stateInitialized) {
    +            String msg = "initState invoked when the state is already initialized";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        } else {
    +            statefulWindowedBolt.initState(state);
    +            stateInitialized = true;
    +            start();
    +        }
    +    }
    +
    +    @Override
    +    public void prePrepare(long txid) {
    +        if (stateInitialized) {
    +            LOG.debug("Prepare streamState, txid {}", txid);
    +            state.prepareCommit(txid);
    +            prePrepared = true;
    +        } else {
    +            String msg = "Cannot prepare before initState";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        }
    +    }
    +
    +    @Override
    +    public void preCommit(long txid) {
    +        // preCommit can be invoked during recovery before the state is initialized
    +        if (prePrepared || !stateInitialized) {
    +            LOG.debug("Commit streamState, txid {}", txid);
    +            state.commit(txid);
    +        } else {
    +            String msg = "preCommit before prePrepare in initialized state";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        }
    +    }
    +
    +    @Override
    +    public void preRollback() {
    +        LOG.debug("Rollback streamState, stateInitialized {}", stateInitialized);
    +        state.rollback();
    +    }
    +
    +    @Override
    +    protected WindowLifecycleListener<Tuple> newWindowLifecycleListener() {
    +        return new WindowLifecycleListener<Tuple>() {
    +            @Override
    +            public void onExpiry(List<Tuple> events) {
    +                /*
    +                 * NO-OP: the events are ack-ed in execute
    +                 */
    +            }
    +
    +            @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();
    +            }
    +        };
    +    }
    +
    +    private void init(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                      KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                      KeyValueState<String, Deque<Long>> partitionState,
    +                      KeyValueState<String, Optional<?>> windowSystemState) {
    +        topologyContext = context;
    +        outputCollector = collector;
    +        state = new WindowState<>(windowState, partitionState, windowSystemState, this::getState,
    +            statefulWindowedBolt.maxEventsInMemory());
    +    }
    +
    +    private KeyValueState<Long, WindowPartition<Tuple>> getWindowState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window";
    +        return (KeyValueState<Long, WindowPartition<Tuple>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Deque<Long>> getPartitionState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-partitions";
    +        return (KeyValueState<String, Deque<Long>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Optional<?>> getWindowSystemState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-systemstate";
    +        return (KeyValueState<String, Optional<?>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    // a wrapper around the window related states that are checkpointed
    +    private static class WindowState<T> extends AbstractCollection<Event<T>> {
    +        // number of events per window-partition
    +        private static final int MAX_PARTITION_EVENTS = 1000;
    +        private static final int MIN_PARTITIONS = 10;
    +        private static final String PARTITION_IDS_KEY = "pk";
    +        private final KeyValueState<String, Deque<Long>> partitionIdsState;
    +        private final KeyValueState<Long, WindowPartition<T>> windowPartitionsState;
    +        private final KeyValueState<String, Optional<?>> windowSystemState;
    +        // ordered partition keys
    +        private Deque<Long> partitionIds;
    +        private volatile long latestPartitionId;
    +        private WindowPartitionCache<Long, WindowPartition<T>> cache;
    +        private Supplier<Map<String, Optional<?>>> windowSystemStateSupplier;
    +        private final ReentrantLock partitionIdsLock = new ReentrantLock(true);
    +        private final ReentrantLock windowPartitionsLock = new ReentrantLock(true);
    +        private final long maxEventsInMemory;
    +        private WindowPartition<T> latestPartition;
    +        private Set<Long> iteratorPins = new HashSet<>();
    +
    +        WindowState(KeyValueState<Long, WindowPartition<T>> windowPartitionsState,
    +                    KeyValueState<String, Deque<Long>> partitionIdsState,
    +                    KeyValueState<String, Optional<?>> windowSystemState,
    +                    Supplier<Map<String, Optional<?>>> windowSystemStateSupplier,
    +                    long maxEventsInMemory) {
    +            this.windowPartitionsState = windowPartitionsState;
    +            this.partitionIdsState = partitionIdsState;
    +            this.windowSystemState = windowSystemState;
    +            this.windowSystemStateSupplier = windowSystemStateSupplier;
    +            this.maxEventsInMemory = Math.max(MAX_PARTITION_EVENTS * MIN_PARTITIONS, maxEventsInMemory);
    +            initCache();
    +            initPartitions();
    +        }
    +
    +        @Override
    +        public boolean add(Event<T> event) {
    +            if (latestPartition.size() >= MAX_PARTITION_EVENTS) {
    +                cache.unpin(latestPartition.getId());
    +                latestPartition = getPinnedPartition(getNextPartitionId());
    +            }
    +            latestPartition.add(event);
    +            return true;
    +        }
    +
    +        @Override
    +        public Iterator<Event<T>> iterator() {
    +
    +            return new Iterator<Event<T>>() {
    +                private Iterator<Long> ids = getIds();
    +                private Iterator<Event<T>> current = emptyIterator();
    +                private Iterator<Event<T>> removeFrom;
    +                private WindowPartition<T> curPartition;
    +
    +                private Iterator<Long> getIds() {
    +                    try {
    +                        partitionIdsLock.lock();
    +                        LOG.debug("Iterator partitionIds: {}", partitionIds);
    +                        return new ArrayList<>(partitionIds).iterator();
    +                    } finally {
    +                        partitionIdsLock.unlock();
    +                    }
    +                }
    +
    +                @Override
    +                public void remove() {
    +                    if (removeFrom == null) {
    +                        throw new IllegalStateException("No calls to next() since last call to remove()");
    +                    }
    +                    removeFrom.remove();
    +                    removeFrom = null;
    +                }
    +
    +                @Override
    +                public boolean hasNext() {
    +                    boolean curHasNext = current.hasNext();
    +                    while (!curHasNext && ids.hasNext()) {
    +                        if (curPartition != null) {
    +                            unpin(curPartition.getId());
    +                        }
    +                        curPartition = getPinnedPartition(ids.next());
    +                        if (curPartition != null) {
    +                            iteratorPins.add(curPartition.getId());
    +                            current = curPartition.iterator();
    +                            curHasNext = current.hasNext();
    +                        }
    +                    }
    +                    // un-pin the last partition
    +                    if (!curHasNext && curPartition != null) {
    +                        unpin(curPartition.getId());
    +                        curPartition = null;
    +                    }
    +                    return curHasNext;
    +                }
    +
    +                @Override
    +                public Event<T> next() {
    +                    if (!hasNext()) {
    +                        throw new NoSuchElementException();
    +                    }
    +                    removeFrom = current;
    +                    return current.next();
    +                }
    +
    +                private void unpin(long id) {
    +                    cache.unpin(id);
    +                    iteratorPins.remove(id);
    +                }
    +            };
    +        }
    +
    +        void clearIteratorPins() {
    +            LOG.debug("clearIteratorPins '{}'", iteratorPins);
    +            Iterator<Long> it = iteratorPins.iterator();
    +            while (it.hasNext()) {
    +                cache.unpin(it.next());
    +                it.remove();
    +            }
    +        }
    +
    +        @Override
    +        public int size() {
    +            throw new UnsupportedOperationException();
    +        }
    +
    +        void prepareCommit(long txid) {
    +            flush();
    +            partitionIdsState.prepareCommit(txid);
    +            windowPartitionsState.prepareCommit(txid);
    +            windowSystemState.prepareCommit(txid);
    +        }
    +
    +        void commit(long txid) {
    +            partitionIdsState.commit(txid);
    +            windowPartitionsState.commit(txid);
    +            windowSystemState.commit(txid);
    +        }
    +
    +        void rollback() {
    +            partitionIdsState.rollback();
    +            windowPartitionsState.rollback();
    +            windowSystemState.rollback();
    +        }
    +
    +        private void initPartitions() {
    +            partitionIds = partitionIdsState.get(PARTITION_IDS_KEY, new LinkedList<>());
    +            if (partitionIds.isEmpty()) {
    +                partitionIds.add(0L);
    +                partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    +            }
    +            latestPartitionId = partitionIds.peekLast();
    +            latestPartition = cache.getPinned(latestPartitionId);
    +        }
    +
    +        private void initCache() {
    +            long size = maxEventsInMemory / MAX_PARTITION_EVENTS;
    +            LOG.info("maxEventsInMemory: {}, partition size: {}, number of partitions: {}",
    +                maxEventsInMemory, MAX_PARTITION_EVENTS, size);
    +            cache = SimpleWindowPartitionCache.<Long, WindowPartition<T>>newBuilder()
    +                .maximumSize(size)
    +                .removalListener(new RemovalListener<Long, WindowPartition<T>>() {
    +                    @Override
    +                    public void onRemoval(Long pid, WindowPartition<T> p, RemovalCause removalCause) {
    +                        Objects.requireNonNull(pid, "Null partition id");
    +                        Objects.requireNonNull(p, "Null window partition");
    +                        LOG.debug("onRemoval for id '{}', WindowPartition '{}'", pid, p);
    +                        try {
    +                            windowPartitionsLock.lock();
    +                            if (p.isEmpty() && pid != latestPartitionId) {
    +                                // if the empty partition was not invalidated by flush, but evicted from cache
    +                                if (removalCause != RemovalCause.EXPLICIT) {
    +                                    deletePartition(pid);
    +                                    windowPartitionsState.delete(pid);
    +                                }
    +                            } else if (p.isModified()) {
    +                                windowPartitionsState.put(pid, p);
    +                            } else {
    +                                LOG.debug("WindowPartition '{}' is not modified", pid);
    +                            }
    +                        } finally {
    +                            windowPartitionsLock.unlock();
    +                        }
    +                    }
    +                }).build(new CacheLoader<Long, WindowPartition<T>>() {
    +                    @Override
    +                   public WindowPartition<T> load(Long id) {
    +                        LOG.debug("Load partition: {}", id);
    +                        // load from state
    +                        try {
    +                            windowPartitionsLock.lock();
    +                            return windowPartitionsState.get(id, new WindowPartition<>(id));
    +                        } finally {
    +                            windowPartitionsLock.unlock();
    +                        }
    +                    }
    +                    }
    +                );
    +        }
    +
    +        private void deletePartition(long pid) {
    +            LOG.debug("Delete partition: {}", pid);
    +            try {
    +                partitionIdsLock.lock();
    +                partitionIds.remove(pid);
    +                partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    +            } finally {
    +                partitionIdsLock.unlock();
    +            }
    +        }
    +
    +        private long getNextPartitionId() {
    +            try {
    +                partitionIdsLock.lock();
    +                partitionIds.add(++latestPartitionId);
    +                partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    +            } finally {
    +                partitionIdsLock.unlock();
    +            }
    +            return latestPartitionId;
    +        }
    +
    +        private WindowPartition<T> getPinnedPartition(long id) {
    +            return cache.getPinned(id);
    +        }
    +
    +        private void flush() {
    +            LOG.debug("Flushing modified partitions");
    +            cache.asMap().forEach((pid, p) -> {
    +                Long pidToInvalidate = null;
    +                try {
    +                    windowPartitionsLock.lock();
    +                    if (p.isEmpty() && pid != latestPartitionId) {
    +                        LOG.debug("Invalidating empty partition {}", pid);
    +                        deletePartition(pid);
    +                        windowPartitionsState.delete(pid);
    +                        pidToInvalidate = pid;
    +                    } else if (p.isModified()) {
    +                        LOG.debug("Updating modified partition {}", pid);
    +                        p.clearModified();
    +                        windowPartitionsState.put(pid, p);
    +                    }
    +                } finally {
    +                    windowPartitionsLock.unlock();
    +                }
    +                // invalidate after releasing the lock
    +                if (pidToInvalidate != null) {
    +                    cache.invalidate(pidToInvalidate);
    +                }
    +            });
    +            windowSystemStateSupplier.get().forEach(windowSystemState::put);
    +        }
    +    }
    +
    +    // the window partition that holds the events
    +    public static class WindowPartition<T> implements Iterable<Event<T>> {
    +        private final ConcurrentLinkedQueue<Event<T>> events = new ConcurrentLinkedQueue<>();
    +        private final AtomicInteger size = new AtomicInteger();
    +        private final long id;
    +        private transient volatile boolean modified;
    +
    +        public WindowPartition(long id) {
    +            this.id = id;
    +        }
    +
    +        void add(Event<T> event) {
    +            events.add(event);
    +            size.incrementAndGet();
    +            setModified();
    +        }
    +
    +        boolean isModified() {
    +            return modified;
    +        }
    +
    +        void setModified() {
    +            if (!modified) {
    +                modified = true;
    +            }
    +        }
    +
    +        void clearModified() {
    +            if (modified) {
    --- End diff --
    
    In `clearModified` its probably ok to skip the check since its invoked only during flush. 
    
    `setModified` is invoked in `add` so we can use it to avoid frequent writes to the volatile.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r131126237
  
    --- Diff: storm-client/src/jvm/org/apache/storm/windowing/persistence/WindowState.java ---
    @@ -0,0 +1,422 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.windowing.persistence;
    +
    +import com.google.common.collect.ImmutableMap;
    +import java.util.AbstractCollection;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Objects;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentLinkedQueue;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.locks.ReentrantLock;
    +import java.util.function.Supplier;
    +import org.apache.storm.state.KeyValueState;
    +import org.apache.storm.windowing.Event;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * A wrapper around the window related states that are checkpointed.
    + */
    +public class WindowState<T> extends AbstractCollection<Event<T>> {
    +    private static final Logger LOG = LoggerFactory.getLogger(WindowState.class);
    +
    +    // number of events per window-partition
    +    public static final int MAX_PARTITION_EVENTS = 1000;
    +    public static final int MIN_PARTITIONS = 10;
    +    private static final String PARTITION_IDS_KEY = "pk";
    +    private final KeyValueState<String, Deque<Long>> partitionIdsState;
    +    private final KeyValueState<Long, WindowPartition<T>> windowPartitionsState;
    +    private final KeyValueState<String, Optional<?>> windowSystemState;
    +    // ordered partition keys
    +    private volatile Deque<Long> partitionIds;
    +    private volatile long latestPartitionId;
    +    private volatile WindowPartition<T> latestPartition;
    +    private volatile WindowPartitionCache<Long, WindowPartition<T>> cache;
    +    private Supplier<Map<String, Optional<?>>> windowSystemStateSupplier;
    +    private final ReentrantLock partitionIdsLock = new ReentrantLock(true);
    +    private final WindowPartitionLock windowPartitionsLock = new WindowPartitionLock();
    +    private final long maxEventsInMemory;
    +    private Set<Long> iteratorPins = new HashSet<>();
    +
    +    public WindowState(KeyValueState<Long, WindowPartition<T>> windowPartitionsState,
    +                KeyValueState<String, Deque<Long>> partitionIdsState,
    +                KeyValueState<String, Optional<?>> windowSystemState,
    +                Supplier<Map<String, Optional<?>>> windowSystemStateSupplier,
    +                long maxEventsInMemory) {
    +        this.windowPartitionsState = windowPartitionsState;
    +        this.partitionIdsState = partitionIdsState;
    +        this.windowSystemState = windowSystemState;
    +        this.windowSystemStateSupplier = windowSystemStateSupplier;
    +        this.maxEventsInMemory = Math.max(MAX_PARTITION_EVENTS * MIN_PARTITIONS, maxEventsInMemory);
    +        init();
    +    }
    +
    +    @Override
    +    public boolean add(Event<T> event) {
    +        if (latestPartition.size() >= MAX_PARTITION_EVENTS) {
    +            cache.unpin(latestPartition.getId());
    +            latestPartition = getPinnedPartition(getNextPartitionId());
    +        }
    +        latestPartition.add(event);
    +        return true;
    +    }
    +
    +    @Override
    +    public Iterator<Event<T>> iterator() {
    +
    +        return new Iterator<Event<T>>() {
    +            private Iterator<Long> ids = getIds();
    +            private Iterator<Event<T>> current = Collections.emptyIterator();
    +            private Iterator<Event<T>> removeFrom;
    +            private WindowPartition<T> curPartition;
    +
    +            private Iterator<Long> getIds() {
    +                try {
    +                    partitionIdsLock.lock();
    +                    LOG.debug("Iterator partitionIds: {}", partitionIds);
    +                    return new ArrayList<>(partitionIds).iterator();
    +                } finally {
    +                    partitionIdsLock.unlock();
    +                }
    +            }
    +
    +            @Override
    +            public void remove() {
    +                if (removeFrom == null) {
    +                    throw new IllegalStateException("No calls to next() since last call to remove()");
    +                }
    +                removeFrom.remove();
    +                removeFrom = null;
    +            }
    +
    +            @Override
    +            public boolean hasNext() {
    +                boolean curHasNext = current.hasNext();
    +                while (!curHasNext && ids.hasNext()) {
    +                    if (curPartition != null) {
    +                        unpin(curPartition.getId());
    +                    }
    +                    curPartition = getPinnedPartition(ids.next());
    +                    if (curPartition != null) {
    +                        iteratorPins.add(curPartition.getId());
    +                        current = curPartition.iterator();
    +                        curHasNext = current.hasNext();
    +                    }
    +                }
    +                // un-pin the last partition
    +                if (!curHasNext && curPartition != null) {
    +                    unpin(curPartition.getId());
    +                    curPartition = null;
    +                }
    +                return curHasNext;
    +            }
    +
    +            @Override
    +            public Event<T> next() {
    +                if (!hasNext()) {
    +                    throw new NoSuchElementException();
    +                }
    +                removeFrom = current;
    +                return current.next();
    +            }
    +
    +            private void unpin(long id) {
    +                cache.unpin(id);
    +                iteratorPins.remove(id);
    +            }
    +        };
    +    }
    +
    +    public void clearIteratorPins() {
    +        LOG.debug("clearIteratorPins '{}'", iteratorPins);
    +        Iterator<Long> it = iteratorPins.iterator();
    +        while (it.hasNext()) {
    +            cache.unpin(it.next());
    +            it.remove();
    +        }
    +    }
    +
    +    @Override
    +    public int size() {
    +        throw new UnsupportedOperationException();
    +    }
    +
    +    /**
    +     * Prepares the {@link WindowState} for commit.
    +     *
    +     * @param txid the transaction id
    +     */
    +    public void prepareCommit(long txid) {
    +        flush();
    +        partitionIdsState.prepareCommit(txid);
    +        windowPartitionsState.prepareCommit(txid);
    +        windowSystemState.prepareCommit(txid);
    +    }
    +
    +    /**
    +     * Commits the {@link WindowState}.
    +     *
    +     * @param txid the transaction id
    +     */
    +    public void commit(long txid) {
    +        partitionIdsState.commit(txid);
    +        windowPartitionsState.commit(txid);
    +        windowSystemState.commit(txid);
    +    }
    +
    +    /**
    +     * Rolls back the {@link WindowState}.
    +     *
    +     * @param reInit if the members should be synced with the values from the state.
    +     */
    +    public void rollback(boolean reInit) {
    +        partitionIdsState.rollback();
    +        windowPartitionsState.rollback();
    +        windowSystemState.rollback();
    +        // re-init cache and partitions
    +        if (reInit) {
    +            init();
    +        }
    +    }
    +
    +    private void init() {
    +        initCache();
    +        initPartitions();
    +    }
    +
    +    private void initPartitions() {
    +        partitionIds = partitionIdsState.get(PARTITION_IDS_KEY, new LinkedList<>());
    +        if (partitionIds.isEmpty()) {
    +            partitionIds.add(0L);
    +            partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    +        }
    +        latestPartitionId = partitionIds.peekLast();
    +        latestPartition = cache.pinAndGet(latestPartitionId);
    +    }
    +
    +    private void initCache() {
    +        long size = maxEventsInMemory / MAX_PARTITION_EVENTS;
    +        LOG.info("maxEventsInMemory: {}, partition size: {}, number of partitions: {}",
    +            maxEventsInMemory, MAX_PARTITION_EVENTS, size);
    +        cache = SimpleWindowPartitionCache.<Long, WindowPartition<T>>newBuilder()
    +            .maximumSize(size)
    +            .removalListener(new WindowPartitionCache.RemovalListener<Long, WindowPartition<T>>() {
    +                @Override
    +                public void onRemoval(Long pid, WindowPartition<T> p, WindowPartitionCache.RemovalCause removalCause) {
    +                    Objects.requireNonNull(pid, "Null partition id");
    +                    Objects.requireNonNull(p, "Null window partition");
    +                    LOG.debug("onRemoval for id '{}', WindowPartition '{}'", pid, p);
    +                    try {
    +                        windowPartitionsLock.lock(pid);
    +                        if (p.isEmpty() && pid != latestPartitionId) {
    +                            // if the empty partition was not invalidated by flush, but evicted from cache
    +                            if (removalCause != WindowPartitionCache.RemovalCause.EXPLICIT) {
    +                                deletePartition(pid);
    +                                windowPartitionsState.delete(pid);
    +                            }
    +                        } else if (p.isModified()) {
    +                            windowPartitionsState.put(pid, p);
    +                        } else {
    +                            LOG.debug("WindowPartition '{}' is not modified", pid);
    +                        }
    +                    } finally {
    +                        windowPartitionsLock.unlock(pid);
    +                    }
    +                }
    +            }).build(new WindowPartitionCache.CacheLoader<Long, WindowPartition<T>>() {
    +                @Override
    +                public WindowPartition<T> load(Long id) {
    +                    LOG.debug("Load partition: {}", id);
    +                    // load from state
    +                    try {
    +                        windowPartitionsLock.lock(id);
    +                        return windowPartitionsState.get(id, new WindowPartition<>(id));
    +                    } finally {
    +                        windowPartitionsLock.unlock(id);
    +                    }
    +                }
    +            });
    +    }
    +
    +    private void deletePartition(long pid) {
    +        LOG.debug("Delete partition: {}", pid);
    +        try {
    +            partitionIdsLock.lock();
    +            partitionIds.remove(pid);
    +            partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    +        } finally {
    +            partitionIdsLock.unlock();
    +        }
    +    }
    +
    +    private long getNextPartitionId() {
    +        try {
    +            partitionIdsLock.lock();
    +            partitionIds.add(++latestPartitionId);
    +            partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    +        } finally {
    +            partitionIdsLock.unlock();
    +        }
    +        return latestPartitionId;
    +    }
    +
    +    private WindowPartition<T> getPinnedPartition(long id) {
    +        return cache.pinAndGet(id);
    +    }
    +
    +    private void flush() {
    +        LOG.debug("Flushing modified partitions");
    +        cache.asMap().forEach((pid, p) -> {
    +            Long pidToInvalidate = null;
    +            try {
    +                windowPartitionsLock.lock(pid);
    +                if (p.isEmpty() && pid != latestPartitionId) {
    +                    LOG.debug("Invalidating empty partition {}", pid);
    +                    deletePartition(pid);
    +                    windowPartitionsState.delete(pid);
    +                    pidToInvalidate = pid;
    +                } else if (p.isModified()) {
    +                    LOG.debug("Updating modified partition {}", pid);
    +                    p.clearModified();
    +                    windowPartitionsState.put(pid, p);
    +                }
    +            } finally {
    +                windowPartitionsLock.unlock(pid);
    +            }
    +            // invalidate after releasing the lock
    +            if (pidToInvalidate != null) {
    +                cache.invalidate(pidToInvalidate);
    --- End diff --
    
    If this id is pinned when this happens, when is it removed?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r131127445
  
    --- Diff: storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java ---
    @@ -2363,7 +2363,7 @@ private CommonTopoInfo getCommonTopoInfo(String topoId, String operation) throws
                 ret.launchTimeSecs = 0;
             }
             ret.assignment = state.assignmentInfo(topoId, null);
    -        ret.beats = Utils.OR(heartbeatsCache.get().get(topoId), Collections.emptyMap());
    +        ret.beats = Utils.<Map<List<Integer>, Map<String, Object>>>OR(heartbeatsCache.get().get(topoId), Collections.emptyMap());
    --- End diff --
    
    I'd guess it's a type inference error in your JDK. Could you try updating and see if that fixes it? I know some of the earlier Java 8 releases had some bugs in this regard.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by ben-manes <gi...@git.apache.org>.
Github user ben-manes commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129211231
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/SimpleWindowPartitionCache.java ---
    @@ -0,0 +1,182 @@
    +/**
    + * 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 java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.concurrent.ConcurrentMap;
    +import java.util.concurrent.ConcurrentSkipListMap;
    +import java.util.concurrent.locks.ReentrantLock;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * A simple implementation that evicts the largest un-pinned entry from the cache. This works well
    + * for caching window partitions since the access pattern is mostly sequential scans.
    + */
    +public class SimpleWindowPartitionCache<K, V> implements WindowPartitionCache<K, V> {
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleWindowPartitionCache.class);
    +
    +    private final ConcurrentSkipListMap<K, V> map = new ConcurrentSkipListMap<>();
    +    private final Map<K, Long> pinned = new HashMap<>();
    +    private final long maximumSize;
    +    private final RemovalListener<K, V> removalListener;
    +    private final CacheLoader<K, V> cacheLoader;
    +    private final ReentrantLock lock = new ReentrantLock(true);
    +    private int size;
    +
    +    @Override
    +    public V get(K key) {
    +        return getOrLoad(key, false);
    +    }
    +
    +    @Override
    +    public V getPinned(K key) {
    +        return getOrLoad(key, true);
    +    }
    +
    +    @Override
    +    public boolean unpin(K key) {
    +        LOG.debug("unpin '{}'", key);
    +        boolean res = false;
    +        try {
    +            lock.lock();
    +            if (pinned.compute(key, (k, v) -> v == null ? 0 : v - 1) <= 0) {
    +                pinned.remove(key);
    +                res = true;
    +            }
    +        } finally {
    +            lock.unlock();
    +        }
    +        LOG.debug("pinned '{}'", pinned);
    +        return res;
    +    }
    +
    +    @Override
    +    public ConcurrentMap<K, V> asMap() {
    +        return map;
    +    }
    +
    +    @Override
    +    public void invalidate(K key) {
    +        if (isPinned(key)) {
    +            LOG.debug("Entry '{}' is pinned, skipping invalidation", key);
    +        } else {
    +            LOG.debug("Invalidating entry '{}'", key);
    +            V val = map.remove(key);
    +            try {
    +                lock.lock();
    +                --size;
    --- End diff --
    
    if the key isn't in the cache (e.g. concurrent invalidations) then this will decrease incorrectly. I think you need to check `val` and skip the locking code if not present.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129662087
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/WindowedBoltExecutor.java ---
    @@ -127,26 +154,30 @@ private void validate(Map<String, Object> topoConf, Count windowLengthCount, Dur
                 throw new IllegalArgumentException("Window length is not specified");
             }
     
    -        if (windowLengthDuration != null && slidingIntervalDuration != null) {
    -            ensureDurationLessThanTimeout(windowLengthDuration.value + slidingIntervalDuration.value, topologyTimeout);
    -        } else if (windowLengthDuration != null) {
    -            ensureDurationLessThanTimeout(windowLengthDuration.value, topologyTimeout);
    -        } else if (slidingIntervalDuration != null) {
    -            ensureDurationLessThanTimeout(slidingIntervalDuration.value, topologyTimeout);
    -        }
    +        if (isPersistent()) {
    +            ensureCheckpointIntervalLessThanTimeout(getCheckpointIntervalSecs(topoConf), topologyTimeout);
    --- End diff --
    
    Don't you want to do the normal validation as well?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130273903
  
    --- Diff: storm-client/test/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutorTest.java ---
    @@ -0,0 +1,261 @@
    +/**
    + * 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.Config;
    +import org.apache.storm.generated.GlobalStreamId;
    +import org.apache.storm.state.KeyValueState;
    +import org.apache.storm.streams.Pair;
    +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.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.TimestampExtractor;
    +import org.apache.storm.windowing.TupleWindow;
    +import org.apache.storm.windowing.WaterMarkEvent;
    +import org.apache.storm.windowing.WaterMarkEventGenerator;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.mockito.ArgumentCaptor;
    +import org.mockito.Captor;
    +import org.mockito.Mockito;
    +import org.mockito.MockitoAnnotations;
    +import org.mockito.invocation.InvocationOnMock;
    +import org.mockito.runners.MockitoJUnitRunner;
    +import org.mockito.stubbing.Answer;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.stream.Collectors;
    +import java.util.stream.LongStream;
    +
    +import static org.mockito.AdditionalAnswers.returnsArgAt;
    +
    +/**
    + * Unit tests for {@link PersistentWindowedBoltExecutor}
    + */
    +@RunWith(MockitoJUnitRunner.class)
    +public class PersistentWindowedBoltExecutorTest {
    +    public static final String LATE_STREAM = "late_stream";
    +    private PersistentWindowedBoltExecutor<KeyValueState<String, String>> executor;
    +    private IStatefulWindowedBolt<KeyValueState<String, String>> mockBolt;
    +    private OutputCollector mockOutputCollector;
    +    private TopologyContext mockTopologyContext;
    +    private Map<String, Object> mockStormConf = new HashMap<>();
    +    private TimestampExtractor mockTimestampExtractor;
    +    private WaterMarkEventGenerator mockWaterMarkEventGenerator;
    +    private KeyValueState mockPartitionState;
    +    private KeyValueState mockWindowState;
    +    private KeyValueState mockSystemState;
    +
    +    private static final String PARTITION_KEY = "pk";
    +    private static final String EVICTION_STATE_KEY = "es";
    +    private static final String TRIGGER_STATE_KEY = "ts";
    +    private long tupleTs;
    +
    +    @Captor
    +    private ArgumentCaptor<Tuple> tupleCaptor;
    +    @Captor
    +    private ArgumentCaptor<Collection<Tuple>> anchorCaptor;
    +    @Captor
    +    private ArgumentCaptor<String> stringCaptor;
    +    @Captor
    +    private ArgumentCaptor<Long> longCaptor;
    +    @Captor
    +    private ArgumentCaptor<Values> valuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<TupleWindow> tupleWindowCaptor;
    +    @Captor
    +    private ArgumentCaptor<Deque<Long>> partitionValuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<PersistentWindowedBoltExecutor.WindowPartition<Event<Tuple>>> windowValuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<Object> systemValuesCaptor;
    +
    +    @Before
    +    public void setUp() throws Exception {
    +        MockitoAnnotations.initMocks(this);
    +        mockBolt = Mockito.mock(IStatefulWindowedBolt.class);
    +        mockWaterMarkEventGenerator = Mockito.mock(WaterMarkEventGenerator.class);
    +        mockTimestampExtractor = Mockito.mock(TimestampExtractor.class);
    +        tupleTs = System.currentTimeMillis();
    +        Mockito.when(mockTimestampExtractor.extractTimestamp(Mockito.any())).thenReturn(tupleTs);
    +        Mockito.when(mockBolt.getTimestampExtractor()).thenReturn(mockTimestampExtractor);
    +        Mockito.when(mockBolt.isPersistent()).thenReturn(true);
    +        mockPartitionState = Mockito.mock(KeyValueState.class);
    +        mockWindowState = Mockito.mock(KeyValueState.class);
    +        mockSystemState = Mockito.mock(KeyValueState.class);
    +        mockTopologyContext = Mockito.mock(TopologyContext.class);
    +        Mockito.when(mockTopologyContext.getThisStreams()).thenReturn(Collections.singleton(LATE_STREAM));
    +        mockOutputCollector = Mockito.mock(OutputCollector.class);
    +        executor = new PersistentWindowedBoltExecutor<>(mockBolt);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_WINDOW_LENGTH_COUNT, 5);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_SLIDING_INTERVAL_COUNT, 5);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_LATE_TUPLE_STREAM, LATE_STREAM);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_WATERMARK_EVENT_INTERVAL_MS, 100_000);
    +        mockStormConf.put(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS, 30);
    +        mockStormConf.put(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL, 1000);
    +        Mockito.when(mockPartitionState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(mockWindowState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(mockSystemState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        executor.prepare(mockStormConf, mockTopologyContext, mockOutputCollector,
    +            mockWindowState, mockPartitionState, mockSystemState);
    +    }
    +
    +    @Test
    +    public void testExecuteTuple() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), 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);
    +    }
    +
    +    @Test
    +    public void testExecuteLatetuple() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), 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
    +        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());
    +    }
    +
    +    @Test
    +    public void testActivation() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), Mockito.anyLong())).thenReturn(true);
    +        executor.initState(null);
    +        executor.waterMarkEventGenerator = mockWaterMarkEventGenerator;
    +
    +        List<Tuple> mockTuples = getMockTuples(5);
    +        mockTuples.forEach(t -> executor.execute(t));
    +        // all tuples acked
    +        Mockito.verify(mockOutputCollector, Mockito.times(5)).ack(tupleCaptor.capture());
    +        Assert.assertArrayEquals(mockTuples.toArray(), tupleCaptor.getAllValues().toArray());
    +
    +        Mockito.doAnswer(new Answer<Void>() {
    +            @Override
    +            public Void answer(InvocationOnMock invocation) throws Throwable {
    +                TupleWindow window = (TupleWindow) invocation.getArguments()[0];
    +                // iterate the tuples
    +                Assert.assertEquals(5, window.get().size());
    +                // iterating multiple times should produce same events
    +                Assert.assertEquals(5, window.get().size());
    +                Assert.assertEquals(5, window.get().size());
    +                return null;
    +            }
    +        }).when(mockBolt).execute(Mockito.any());
    +        // trigger the window
    +        long activationTs = tupleTs + 1000;
    +        executor.getWindowManager().add(new WaterMarkEvent<>(activationTs));
    +        executor.prePrepare(0);
    +
    +        // partition ids
    +        Mockito.verify(mockPartitionState, Mockito.times(1)).put(stringCaptor.capture(), partitionValuesCaptor.capture());
    +        Assert.assertEquals(PARTITION_KEY, stringCaptor.getAllValues().get(0));
    +        Assert.assertEquals(Collections.singletonList(0L), partitionValuesCaptor.getAllValues().get(0));
    +
    +        // window partitions
    +        Mockito.verify(mockWindowState, Mockito.times(1)).put(longCaptor.capture(), windowValuesCaptor.capture());
    +        Assert.assertEquals(0L, (long) longCaptor.getAllValues().get(0));
    +        Assert.assertEquals(5, windowValuesCaptor.getAllValues().get(0).size());
    +        List<Event<?>> tuples = windowValuesCaptor.getAllValues().get(0)
    +            .getEvents().stream().map(Event::get).collect(Collectors.toList());
    +        Assert.assertArrayEquals(mockTuples.toArray(), tuples.toArray());
    +
    +        // window system state
    +        Mockito.verify(mockSystemState, Mockito.times(2)).put(stringCaptor.capture(), systemValuesCaptor.capture());
    +        Assert.assertEquals(EVICTION_STATE_KEY, stringCaptor.getAllValues().get(1));
    +        Assert.assertEquals(Optional.of(Pair.of(5L, 5L)), systemValuesCaptor.getAllValues().get(0));
    +        Assert.assertEquals(TRIGGER_STATE_KEY, stringCaptor.getAllValues().get(2));
    +        Assert.assertEquals(Optional.of(tupleTs), systemValuesCaptor.getAllValues().get(1));
    +    }
    +
    +    @Test
    +    public void testCacheEviction() {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), Mockito.anyLong())).thenReturn(true);
    +        executor.initState(null);
    +        executor.waterMarkEventGenerator = mockWaterMarkEventGenerator;
    +        List<Tuple> mockTuples = getMockTuples(20000);
    +        mockTuples.forEach(t -> executor.execute(t));
    +
    +        Mockito.verify(mockWindowState, Mockito.times(10)).put(longCaptor.capture(), windowValuesCaptor.capture());
    +        Assert.assertEquals(10, longCaptor.getAllValues().size());
    +        Assert.assertEquals(10, windowValuesCaptor.getAllValues().size());
    +        // number of evicted events
    +        Assert.assertEquals(10_000, windowValuesCaptor.getAllValues().stream()
    +            .mapToInt(x -> x.size()).sum());
    +
    +        Map<Long, PersistentWindowedBoltExecutor.WindowPartition<Event<Tuple>>> partitionMap = new HashMap<>();
    +        windowValuesCaptor.getAllValues().forEach(v -> partitionMap.put(v.getId(), v));
    +
    +        Mockito.verify(mockPartitionState, Mockito.times(20)).put(stringCaptor.capture(), partitionValuesCaptor.capture());
    +        Assert.assertEquals(LongStream.range(0, 20).boxed().collect(Collectors.toList()), partitionValuesCaptor.getAllValues().get(19));
    --- End diff --
    
    will add a comment. its the partition ids `0 .. 19`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129763707
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/WindowedBoltExecutor.java ---
    @@ -102,6 +109,18 @@ private int getMaxSpoutPending(Map<String, Object> topoConf) {
             return maxPending;
         }
     
    +    private int getCheckpointIntervalSecs(Map<String, Object> topoConf) {
    --- End diff --
    
    it can be refactored


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130433067
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutor.java ---
    @@ -0,0 +1,596 @@
    +/**
    + * 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 java.util.AbstractCollection;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Objects;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentLinkedQueue;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.locks.ReentrantLock;
    +import java.util.function.Supplier;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.state.KeyValueState;
    +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.topology.base.BaseWindowedBolt;
    +import org.apache.storm.tuple.Tuple;
    +import org.apache.storm.windowing.DefaultEvictionContext;
    +import org.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.EventImpl;
    +import org.apache.storm.windowing.WindowLifecycleListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static java.util.Collections.emptyIterator;
    +import static org.apache.storm.topology.WindowPartitionCache.CacheLoader;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalCause;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalListener;
    +
    +/**
    + * Wraps a {@link IStatefulWindowedBolt} and handles the execution. Uses state and the underlying
    + * checkpointing mechanisms to save the tuples in window to state. The tuples are also kept in-memory
    + * by transparently caching the window partitions and checkpointing them as needed.
    + */
    +public class PersistentWindowedBoltExecutor<T extends State> extends WindowedBoltExecutor implements IStatefulBolt<T> {
    +    private static final Logger LOG = LoggerFactory.getLogger(PersistentWindowedBoltExecutor.class);
    +    private final IStatefulWindowedBolt<T> statefulWindowedBolt;
    +    private transient TopologyContext topologyContext;
    +    private transient OutputCollector outputCollector;
    +    private transient WindowState<Tuple> state;
    +    private transient boolean stateInitialized;
    +    private transient boolean prePrepared;
    +
    +    public PersistentWindowedBoltExecutor(IStatefulWindowedBolt<T> bolt) {
    +        super(bolt);
    +        statefulWindowedBolt = bolt;
    +    }
    +
    +    @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));
    +    }
    +
    +    @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);
    +        }
    +    }
    +
    +    private int getCheckpointIntervalMillis(Map<String, Object> topoConf) {
    +        int checkpointInterval = Integer.MAX_VALUE;
    +        if (topoConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL) != null) {
    +            checkpointInterval = ((Number) topoConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL)).intValue();
    +        }
    +        return checkpointInterval;
    +    }
    +
    +    // package access for unit tests
    +    void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                 KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                 KeyValueState<String, Deque<Long>> partitionState,
    +                 KeyValueState<String, Optional<?>> windowSystemState) {
    +        init(topoConf, context, collector, windowState, partitionState, windowSystemState);
    +        doPrepare(topoConf, context, new NoAckOutputCollector(collector), state, true);
    +        Map<String, Optional<?>> wstate = new HashMap<>();
    +        windowSystemState.forEach(s -> wstate.put(s.getKey(), s.getValue()));
    +        restoreState(wstate);
    +    }
    +
    +    @Override
    +    protected void start() {
    +        if (stateInitialized) {
    +            super.start();
    +        } else {
    +            LOG.debug("Will invoke start after state is initialized");
    +        }
    +    }
    +
    +    @Override
    +    public void execute(Tuple input) {
    +        if (!stateInitialized) {
    +            throw new IllegalStateException("execute invoked before initState with input tuple " + input);
    +        }
    +        super.execute(input);
    +        // StatefulBoltExecutor does the actual ack when the state is saved.
    +        outputCollector.ack(input);
    +    }
    +
    +    @Override
    +    public void initState(T state) {
    +        if (stateInitialized) {
    +            String msg = "initState invoked when the state is already initialized";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        } else {
    +            statefulWindowedBolt.initState(state);
    +            stateInitialized = true;
    +            start();
    +        }
    +    }
    +
    +    @Override
    +    public void prePrepare(long txid) {
    +        if (stateInitialized) {
    +            LOG.debug("Prepare streamState, txid {}", txid);
    +            state.prepareCommit(txid);
    +            prePrepared = true;
    +        } else {
    +            String msg = "Cannot prepare before initState";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        }
    +    }
    +
    +    @Override
    +    public void preCommit(long txid) {
    +        // preCommit can be invoked during recovery before the state is initialized
    +        if (prePrepared || !stateInitialized) {
    +            LOG.debug("Commit streamState, txid {}", txid);
    +            state.commit(txid);
    +        } else {
    +            String msg = "preCommit before prePrepare in initialized state";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        }
    +    }
    +
    +    @Override
    +    public void preRollback() {
    +        LOG.debug("Rollback streamState, stateInitialized {}", stateInitialized);
    +        state.rollback();
    +    }
    +
    +    @Override
    +    protected WindowLifecycleListener<Tuple> newWindowLifecycleListener() {
    +        return new WindowLifecycleListener<Tuple>() {
    +            @Override
    +            public void onExpiry(List<Tuple> events) {
    +                /*
    +                 * NO-OP: the events are ack-ed in execute
    +                 */
    +            }
    +
    +            @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();
    +            }
    +        };
    +    }
    +
    +    private void init(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                      KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                      KeyValueState<String, Deque<Long>> partitionState,
    +                      KeyValueState<String, Optional<?>> windowSystemState) {
    +        topologyContext = context;
    +        outputCollector = collector;
    +        state = new WindowState<>(windowState, partitionState, windowSystemState, this::getState,
    +            statefulWindowedBolt.maxEventsInMemory());
    +    }
    +
    +    private KeyValueState<Long, WindowPartition<Tuple>> getWindowState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window";
    +        return (KeyValueState<Long, WindowPartition<Tuple>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Deque<Long>> getPartitionState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-partitions";
    +        return (KeyValueState<String, Deque<Long>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Optional<?>> getWindowSystemState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-systemstate";
    +        return (KeyValueState<String, Optional<?>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    // a wrapper around the window related states that are checkpointed
    +    private static class WindowState<T> extends AbstractCollection<Event<T>> {
    +        // number of events per window-partition
    +        private static final int MAX_PARTITION_EVENTS = 1000;
    +        private static final int MIN_PARTITIONS = 10;
    +        private static final String PARTITION_IDS_KEY = "pk";
    +        private final KeyValueState<String, Deque<Long>> partitionIdsState;
    +        private final KeyValueState<Long, WindowPartition<T>> windowPartitionsState;
    +        private final KeyValueState<String, Optional<?>> windowSystemState;
    +        // ordered partition keys
    +        private Deque<Long> partitionIds;
    +        private volatile long latestPartitionId;
    +        private WindowPartitionCache<Long, WindowPartition<T>> cache;
    +        private Supplier<Map<String, Optional<?>>> windowSystemStateSupplier;
    +        private final ReentrantLock partitionIdsLock = new ReentrantLock(true);
    +        private final ReentrantLock windowPartitionsLock = new ReentrantLock(true);
    +        private final long maxEventsInMemory;
    +        private WindowPartition<T> latestPartition;
    +        private Set<Long> iteratorPins = new HashSet<>();
    +
    +        WindowState(KeyValueState<Long, WindowPartition<T>> windowPartitionsState,
    +                    KeyValueState<String, Deque<Long>> partitionIdsState,
    +                    KeyValueState<String, Optional<?>> windowSystemState,
    +                    Supplier<Map<String, Optional<?>>> windowSystemStateSupplier,
    +                    long maxEventsInMemory) {
    +            this.windowPartitionsState = windowPartitionsState;
    +            this.partitionIdsState = partitionIdsState;
    +            this.windowSystemState = windowSystemState;
    +            this.windowSystemStateSupplier = windowSystemStateSupplier;
    +            this.maxEventsInMemory = Math.max(MAX_PARTITION_EVENTS * MIN_PARTITIONS, maxEventsInMemory);
    +            initCache();
    +            initPartitions();
    +        }
    +
    +        @Override
    +        public boolean add(Event<T> event) {
    +            if (latestPartition.size() >= MAX_PARTITION_EVENTS) {
    +                cache.unpin(latestPartition.getId());
    +                latestPartition = getPinnedPartition(getNextPartitionId());
    +            }
    +            latestPartition.add(event);
    +            return true;
    +        }
    +
    +        @Override
    +        public Iterator<Event<T>> iterator() {
    +
    +            return new Iterator<Event<T>>() {
    +                private Iterator<Long> ids = getIds();
    +                private Iterator<Event<T>> current = emptyIterator();
    +                private Iterator<Event<T>> removeFrom;
    +                private WindowPartition<T> curPartition;
    +
    +                private Iterator<Long> getIds() {
    +                    try {
    +                        partitionIdsLock.lock();
    +                        LOG.debug("Iterator partitionIds: {}", partitionIds);
    +                        return new ArrayList<>(partitionIds).iterator();
    +                    } finally {
    +                        partitionIdsLock.unlock();
    +                    }
    +                }
    +
    +                @Override
    +                public void remove() {
    +                    if (removeFrom == null) {
    +                        throw new IllegalStateException("No calls to next() since last call to remove()");
    +                    }
    +                    removeFrom.remove();
    +                    removeFrom = null;
    +                }
    +
    +                @Override
    +                public boolean hasNext() {
    +                    boolean curHasNext = current.hasNext();
    +                    while (!curHasNext && ids.hasNext()) {
    +                        if (curPartition != null) {
    +                            unpin(curPartition.getId());
    +                        }
    +                        curPartition = getPinnedPartition(ids.next());
    +                        if (curPartition != null) {
    +                            iteratorPins.add(curPartition.getId());
    +                            current = curPartition.iterator();
    +                            curHasNext = current.hasNext();
    +                        }
    +                    }
    +                    // un-pin the last partition
    +                    if (!curHasNext && curPartition != null) {
    +                        unpin(curPartition.getId());
    +                        curPartition = null;
    +                    }
    +                    return curHasNext;
    +                }
    +
    +                @Override
    +                public Event<T> next() {
    +                    if (!hasNext()) {
    +                        throw new NoSuchElementException();
    +                    }
    +                    removeFrom = current;
    +                    return current.next();
    +                }
    +
    +                private void unpin(long id) {
    +                    cache.unpin(id);
    +                    iteratorPins.remove(id);
    +                }
    +            };
    +        }
    +
    +        void clearIteratorPins() {
    +            LOG.debug("clearIteratorPins '{}'", iteratorPins);
    +            Iterator<Long> it = iteratorPins.iterator();
    +            while (it.hasNext()) {
    +                cache.unpin(it.next());
    +                it.remove();
    +            }
    +        }
    +
    +        @Override
    +        public int size() {
    +            throw new UnsupportedOperationException();
    +        }
    +
    +        void prepareCommit(long txid) {
    +            flush();
    +            partitionIdsState.prepareCommit(txid);
    +            windowPartitionsState.prepareCommit(txid);
    +            windowSystemState.prepareCommit(txid);
    +        }
    +
    +        void commit(long txid) {
    +            partitionIdsState.commit(txid);
    +            windowPartitionsState.commit(txid);
    +            windowSystemState.commit(txid);
    +        }
    +
    +        void rollback() {
    +            partitionIdsState.rollback();
    +            windowPartitionsState.rollback();
    +            windowSystemState.rollback();
    +        }
    +
    +        private void initPartitions() {
    +            partitionIds = partitionIdsState.get(PARTITION_IDS_KEY, new LinkedList<>());
    +            if (partitionIds.isEmpty()) {
    +                partitionIds.add(0L);
    +                partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    +            }
    +            latestPartitionId = partitionIds.peekLast();
    +            latestPartition = cache.getPinned(latestPartitionId);
    +        }
    +
    +        private void initCache() {
    +            long size = maxEventsInMemory / MAX_PARTITION_EVENTS;
    +            LOG.info("maxEventsInMemory: {}, partition size: {}, number of partitions: {}",
    +                maxEventsInMemory, MAX_PARTITION_EVENTS, size);
    +            cache = SimpleWindowPartitionCache.<Long, WindowPartition<T>>newBuilder()
    +                .maximumSize(size)
    +                .removalListener(new RemovalListener<Long, WindowPartition<T>>() {
    +                    @Override
    +                    public void onRemoval(Long pid, WindowPartition<T> p, RemovalCause removalCause) {
    +                        Objects.requireNonNull(pid, "Null partition id");
    +                        Objects.requireNonNull(p, "Null window partition");
    +                        LOG.debug("onRemoval for id '{}', WindowPartition '{}'", pid, p);
    +                        try {
    +                            windowPartitionsLock.lock();
    +                            if (p.isEmpty() && pid != latestPartitionId) {
    +                                // if the empty partition was not invalidated by flush, but evicted from cache
    +                                if (removalCause != RemovalCause.EXPLICIT) {
    +                                    deletePartition(pid);
    +                                    windowPartitionsState.delete(pid);
    +                                }
    +                            } else if (p.isModified()) {
    +                                windowPartitionsState.put(pid, p);
    +                            } else {
    +                                LOG.debug("WindowPartition '{}' is not modified", pid);
    +                            }
    +                        } finally {
    +                            windowPartitionsLock.unlock();
    +                        }
    +                    }
    +                }).build(new CacheLoader<Long, WindowPartition<T>>() {
    +                    @Override
    +                   public WindowPartition<T> load(Long id) {
    +                        LOG.debug("Load partition: {}", id);
    +                        // load from state
    +                        try {
    +                            windowPartitionsLock.lock();
    +                            return windowPartitionsState.get(id, new WindowPartition<>(id));
    +                        } finally {
    +                            windowPartitionsLock.unlock();
    +                        }
    +                    }
    +                    }
    +                );
    +        }
    +
    +        private void deletePartition(long pid) {
    +            LOG.debug("Delete partition: {}", pid);
    +            try {
    +                partitionIdsLock.lock();
    +                partitionIds.remove(pid);
    +                partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    +            } finally {
    +                partitionIdsLock.unlock();
    +            }
    +        }
    +
    +        private long getNextPartitionId() {
    +            try {
    +                partitionIdsLock.lock();
    +                partitionIds.add(++latestPartitionId);
    +                partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    +            } finally {
    +                partitionIdsLock.unlock();
    +            }
    +            return latestPartitionId;
    +        }
    +
    +        private WindowPartition<T> getPinnedPartition(long id) {
    +            return cache.getPinned(id);
    +        }
    +
    +        private void flush() {
    +            LOG.debug("Flushing modified partitions");
    +            cache.asMap().forEach((pid, p) -> {
    +                Long pidToInvalidate = null;
    +                try {
    +                    windowPartitionsLock.lock();
    +                    if (p.isEmpty() && pid != latestPartitionId) {
    +                        LOG.debug("Invalidating empty partition {}", pid);
    +                        deletePartition(pid);
    +                        windowPartitionsState.delete(pid);
    +                        pidToInvalidate = pid;
    +                    } else if (p.isModified()) {
    +                        LOG.debug("Updating modified partition {}", pid);
    +                        p.clearModified();
    +                        windowPartitionsState.put(pid, p);
    +                    }
    +                } finally {
    +                    windowPartitionsLock.unlock();
    +                }
    +                // invalidate after releasing the lock
    +                if (pidToInvalidate != null) {
    +                    cache.invalidate(pidToInvalidate);
    +                }
    +            });
    +            windowSystemStateSupplier.get().forEach(windowSystemState::put);
    +        }
    +    }
    +
    +    // the window partition that holds the events
    +    public static class WindowPartition<T> implements Iterable<Event<T>> {
    +        private final ConcurrentLinkedQueue<Event<T>> events = new ConcurrentLinkedQueue<>();
    +        private final AtomicInteger size = new AtomicInteger();
    +        private final long id;
    +        private transient volatile boolean modified;
    +
    +        public WindowPartition(long id) {
    +            this.id = id;
    +        }
    +
    +        void add(Event<T> event) {
    +            events.add(event);
    +            size.incrementAndGet();
    +            setModified();
    +        }
    +
    +        boolean isModified() {
    +            return modified;
    +        }
    +
    +        void setModified() {
    +            if (!modified) {
    +                modified = true;
    +            }
    +        }
    +
    +        void clearModified() {
    +            if (modified) {
    --- End diff --
    
    Sorry, I didn't consider that it was volatile. I'm guessing it's a performance optimization?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129663556
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/WindowedBoltExecutor.java ---
    @@ -102,6 +109,18 @@ private int getMaxSpoutPending(Map<String, Object> topoConf) {
             return maxPending;
         }
     
    +    private int getCheckpointIntervalSecs(Map<String, Object> topoConf) {
    --- End diff --
    
    Why does this need to "leak" up to the WindowedBoltExecutor instead of being kept in the subclass?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129344288
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/SimpleWindowPartitionCache.java ---
    @@ -0,0 +1,182 @@
    +/**
    + * 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 java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.concurrent.ConcurrentMap;
    +import java.util.concurrent.ConcurrentSkipListMap;
    +import java.util.concurrent.locks.ReentrantLock;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * A simple implementation that evicts the largest un-pinned entry from the cache. This works well
    + * for caching window partitions since the access pattern is mostly sequential scans.
    + */
    +public class SimpleWindowPartitionCache<K, V> implements WindowPartitionCache<K, V> {
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleWindowPartitionCache.class);
    +
    +    private final ConcurrentSkipListMap<K, V> map = new ConcurrentSkipListMap<>();
    +    private final Map<K, Long> pinned = new HashMap<>();
    +    private final long maximumSize;
    +    private final RemovalListener<K, V> removalListener;
    +    private final CacheLoader<K, V> cacheLoader;
    +    private final ReentrantLock lock = new ReentrantLock(true);
    +    private int size;
    +
    +    @Override
    +    public V get(K key) {
    +        return getOrLoad(key, false);
    +    }
    +
    +    @Override
    +    public V getPinned(K key) {
    +        return getOrLoad(key, true);
    +    }
    +
    +    @Override
    +    public boolean unpin(K key) {
    +        LOG.debug("unpin '{}'", key);
    +        boolean res = false;
    +        try {
    +            lock.lock();
    +            if (pinned.compute(key, (k, v) -> v == null ? 0 : v - 1) <= 0) {
    +                pinned.remove(key);
    +                res = true;
    +            }
    +        } finally {
    +            lock.unlock();
    +        }
    +        LOG.debug("pinned '{}'", pinned);
    +        return res;
    +    }
    +
    +    @Override
    +    public ConcurrentMap<K, V> asMap() {
    +        return map;
    +    }
    +
    +    @Override
    +    public void invalidate(K key) {
    +        if (isPinned(key)) {
    +            LOG.debug("Entry '{}' is pinned, skipping invalidation", key);
    +        } else {
    +            LOG.debug("Invalidating entry '{}'", key);
    +            V val = map.remove(key);
    +            try {
    +                lock.lock();
    +                --size;
    --- End diff --
    
    Yes, thanks for the observation. Refactored the code a bit to address this and a few other race conditions.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130226940
  
    --- Diff: storm-client/src/jvm/org/apache/storm/windowing/WatermarkCountEvictionPolicy.java ---
    @@ -17,20 +17,28 @@
      */
     package org.apache.storm.windowing;
     
    +import org.apache.storm.streams.Pair;
    +
    +import java.util.concurrent.atomic.AtomicLong;
    +
     /**
      * An eviction policy that tracks count based on watermark ts and
      * evicts events up to the watermark based on a threshold count.
      *
      * @param <T> the type of event tracked by this policy.
      */
    -public class WatermarkCountEvictionPolicy<T> extends CountEvictionPolicy<T> {
    +public class WatermarkCountEvictionPolicy<T> implements EvictionPolicy<T, Pair<Long, Long>> {
    --- End diff --
    
    Would it be possible to have this delegate to a CountEvictionPolicy instead of having to duplicate parts of that class here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129110774
  
    --- Diff: docs/Windowing.md ---
    @@ -266,3 +266,105 @@ tuples can be received within the timeout period.
     An example toplogy `SlidingWindowTopology` shows how to use the apis to compute a sliding window sum and a tumbling window 
     average.
     
    +## Stateful windowing
    +The default windowing implementation in storm stores the tuples in memory until they are processed and expired from the 
    +window. This limits the use cases to windows that
    +fit entirely in memory. Also the source tuples cannot be ack-ed until the window expiry requiring large message timeouts
    +(topology.message.timeout.secs should be larger than the window length + sliding interval). This also puts extra loads 
    +due to the complex acking and anchoring requirements.
    + 
    +To address the above limitations and to support larger window sizes, storm provides stateful windowing support via `IStatefulWindowedBolt`. 
    +User bolts should typically extend `BaseStatefulWindowedBolt` for the windowing operations with the framework automatically 
    +managing the state of the window in the background.
    +
    +If the sources provide a monotonically increasing identifier as a part of the message, the framework can use this
    +to periodically checkpoint the last expired and evaluated message ids, to avoid duplicate window evaluations in case of 
    +failures or restarts. During recovery, the tuples with message ids lower than last expired id are discarded and tuples with 
    +message id between the last expired and last evaluated message ids are fed into the system without activating any triggers. 
    +The tuples beyond the last evaluated message ids are processed as usual. This can be enabled by setting
    +the `messageIdField` as shown below,
    +
    +```java
    +topologyBuilder.setBolt("mybolt",
    +                   new MyStatefulWindowedBolt()
    +                   .withWindow(...) // windowing configuarations
    +                   .withMessageIdField("msgid"), // a monotonically increasing 'long' field in the tuple
    +                   parallelism)
    +               .shuffleGrouping("spout");
    +```
    +
    +However, this option is feasible only if the sources can provide a monotonically increasing identifier in the tuple and the same is maintained
    +while re-emitting the messages in case of failures. Here the tuples in window are still held in memory.
    +
    +For more details take a look at the sample topology in storm starter `StatefulWindowingTopology` which will help you get started.
    +
    +### Window checkpointing
    +
    +With window checkpointing, the monotonically increasing id is no-longer required since the framework transparently saves the state of the window periodically into the configured state backend.
    +The state that is saved includes the tuples in the window, any system state that is required to recover the state of processing
    +and also the user state.
    +
    +```java
    +topologyBuilder.setBolt("mybolt",
    +                   new MyStatefulPersistentWindowedBolt()
    +                   .withWindow(...) // windowing configuarations
    +                   .withPersistence() // persist the window state
    +                   .withMaxEventsInMemory(25000), // max number of events to be cached in memory
    +                    parallelism)
    +               .shuffleGrouping("spout");
    +
    +```
    +
    +The `withPersistence` instructs the framework to transparently save the tuples in window along with
    +any associated system and user state to the state backend. The `withMaxEventsInMemory` is an optional 
    +configuration that specifies the maximum number of tuples that may be kept in memory. The tuples are transparently loaded from 
    +the state backend as required and the ones that are most likely to be used again are retained in memory.
    +
    +The state backend can be configured by setting the topology state provider config,
    +
    +```java
    +// use redis for state persistence
    +conf.put(Config.TOPOLOGY_STATE_PROVIDER, "org.apache.storm.redis.state.RedisKeyValueStateProvider");
    +
    +```
    +Currently storm supports Redis and HBase as state backends and uses the underlying state-checkpointing
    +framework for saving the window state. For more details on state checkpointing see [State-checkpointing.md](State-checkpointing.md)
    +
    +Here is an example of a persistent windowed bolt that uses the window checkpointing to save its state. The `initState`
    +is invoked with the last saved state (user state) at initialization time. The execute method is invoked based on the configured
    +windowing parameters and the tuples in the active window can be accessed via an `iterator` as shown below.
    +
    +```java
    +public class MyStatefulPersistentWindowedBolt extends BaseStatefulWindowedBolt<K, V> {
    +  private KeyValueState<K, V> state;
    +  
    +  @Override
    +  public void initState(KeyValueState<K, V> state) {
    +    this.state = state;
    +   // ...
    +   // restore the state from the last saved state.
    +   // ...
    +  }
    +  
    +  @Override
    +  public void execute(TupleWindow window) {      
    +    // iterate over tuples in the current window
    +    Iterator<Tuple> it = window.getIter();
    +    while (it.hasNext()) {
    +        // compute some result based on the tuples in window
    +    }
    +    
    +    // possibly update any state to be maintained across windows
    +    state.put(STATE_KEY, updatedValue);
    +    
    +    // emit the results downstream
    +    collector.emit(new Values(result));
    +  }
    +}
    +```
    +
    +Note: In case of persistent windowed bolts, use `TupleWindow.getIter` to retrieve an iterator over the 
    +events in the window. If the number of tuples in windows are huge, invoking `TupleWindow.get` would
    --- End diff --
    
    Nit: If the number of tuples in the window is huge


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129109980
  
    --- Diff: docs/Windowing.md ---
    @@ -266,3 +266,105 @@ tuples can be received within the timeout period.
     An example toplogy `SlidingWindowTopology` shows how to use the apis to compute a sliding window sum and a tumbling window 
     average.
     
    +## Stateful windowing
    +The default windowing implementation in storm stores the tuples in memory until they are processed and expired from the 
    +window. This limits the use cases to windows that
    +fit entirely in memory. Also the source tuples cannot be ack-ed until the window expiry requiring large message timeouts
    +(topology.message.timeout.secs should be larger than the window length + sliding interval). This also puts extra loads 
    +due to the complex acking and anchoring requirements.
    + 
    +To address the above limitations and to support larger window sizes, storm provides stateful windowing support via `IStatefulWindowedBolt`. 
    +User bolts should typically extend `BaseStatefulWindowedBolt` for the windowing operations with the framework automatically 
    +managing the state of the window in the background.
    +
    +If the sources provide a monotonically increasing identifier as a part of the message, the framework can use this
    +to periodically checkpoint the last expired and evaluated message ids, to avoid duplicate window evaluations in case of 
    +failures or restarts. During recovery, the tuples with message ids lower than last expired id are discarded and tuples with 
    +message id between the last expired and last evaluated message ids are fed into the system without activating any triggers. 
    +The tuples beyond the last evaluated message ids are processed as usual. This can be enabled by setting
    +the `messageIdField` as shown below,
    +
    +```java
    +topologyBuilder.setBolt("mybolt",
    +                   new MyStatefulWindowedBolt()
    +                   .withWindow(...) // windowing configuarations
    +                   .withMessageIdField("msgid"), // a monotonically increasing 'long' field in the tuple
    +                   parallelism)
    +               .shuffleGrouping("spout");
    +```
    +
    +However, this option is feasible only if the sources can provide a monotonically increasing identifier in the tuple and the same is maintained
    +while re-emitting the messages in case of failures. Here the tuples in window are still held in memory.
    +
    +For more details take a look at the sample topology in storm starter `StatefulWindowingTopology` which will help you get started.
    +
    +### Window checkpointing
    +
    +With window checkpointing, the monotonically increasing id is no-longer required since the framework transparently saves the state of the window periodically into the configured state backend.
    --- End diff --
    
    Nit: no longer


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130273846
  
    --- Diff: storm-client/test/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutorTest.java ---
    @@ -0,0 +1,261 @@
    +/**
    + * 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.Config;
    +import org.apache.storm.generated.GlobalStreamId;
    +import org.apache.storm.state.KeyValueState;
    +import org.apache.storm.streams.Pair;
    +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.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.TimestampExtractor;
    +import org.apache.storm.windowing.TupleWindow;
    +import org.apache.storm.windowing.WaterMarkEvent;
    +import org.apache.storm.windowing.WaterMarkEventGenerator;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.mockito.ArgumentCaptor;
    +import org.mockito.Captor;
    +import org.mockito.Mockito;
    +import org.mockito.MockitoAnnotations;
    +import org.mockito.invocation.InvocationOnMock;
    +import org.mockito.runners.MockitoJUnitRunner;
    +import org.mockito.stubbing.Answer;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.stream.Collectors;
    +import java.util.stream.LongStream;
    +
    +import static org.mockito.AdditionalAnswers.returnsArgAt;
    +
    +/**
    + * Unit tests for {@link PersistentWindowedBoltExecutor}
    + */
    +@RunWith(MockitoJUnitRunner.class)
    +public class PersistentWindowedBoltExecutorTest {
    +    public static final String LATE_STREAM = "late_stream";
    +    private PersistentWindowedBoltExecutor<KeyValueState<String, String>> executor;
    +    private IStatefulWindowedBolt<KeyValueState<String, String>> mockBolt;
    +    private OutputCollector mockOutputCollector;
    +    private TopologyContext mockTopologyContext;
    +    private Map<String, Object> mockStormConf = new HashMap<>();
    +    private TimestampExtractor mockTimestampExtractor;
    +    private WaterMarkEventGenerator mockWaterMarkEventGenerator;
    +    private KeyValueState mockPartitionState;
    +    private KeyValueState mockWindowState;
    +    private KeyValueState mockSystemState;
    +
    +    private static final String PARTITION_KEY = "pk";
    +    private static final String EVICTION_STATE_KEY = "es";
    +    private static final String TRIGGER_STATE_KEY = "ts";
    +    private long tupleTs;
    +
    +    @Captor
    +    private ArgumentCaptor<Tuple> tupleCaptor;
    +    @Captor
    +    private ArgumentCaptor<Collection<Tuple>> anchorCaptor;
    +    @Captor
    +    private ArgumentCaptor<String> stringCaptor;
    +    @Captor
    +    private ArgumentCaptor<Long> longCaptor;
    +    @Captor
    +    private ArgumentCaptor<Values> valuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<TupleWindow> tupleWindowCaptor;
    +    @Captor
    +    private ArgumentCaptor<Deque<Long>> partitionValuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<PersistentWindowedBoltExecutor.WindowPartition<Event<Tuple>>> windowValuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<Object> systemValuesCaptor;
    +
    +    @Before
    +    public void setUp() throws Exception {
    +        MockitoAnnotations.initMocks(this);
    +        mockBolt = Mockito.mock(IStatefulWindowedBolt.class);
    +        mockWaterMarkEventGenerator = Mockito.mock(WaterMarkEventGenerator.class);
    +        mockTimestampExtractor = Mockito.mock(TimestampExtractor.class);
    +        tupleTs = System.currentTimeMillis();
    +        Mockito.when(mockTimestampExtractor.extractTimestamp(Mockito.any())).thenReturn(tupleTs);
    +        Mockito.when(mockBolt.getTimestampExtractor()).thenReturn(mockTimestampExtractor);
    +        Mockito.when(mockBolt.isPersistent()).thenReturn(true);
    +        mockPartitionState = Mockito.mock(KeyValueState.class);
    +        mockWindowState = Mockito.mock(KeyValueState.class);
    +        mockSystemState = Mockito.mock(KeyValueState.class);
    +        mockTopologyContext = Mockito.mock(TopologyContext.class);
    +        Mockito.when(mockTopologyContext.getThisStreams()).thenReturn(Collections.singleton(LATE_STREAM));
    +        mockOutputCollector = Mockito.mock(OutputCollector.class);
    +        executor = new PersistentWindowedBoltExecutor<>(mockBolt);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_WINDOW_LENGTH_COUNT, 5);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_SLIDING_INTERVAL_COUNT, 5);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_LATE_TUPLE_STREAM, LATE_STREAM);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_WATERMARK_EVENT_INTERVAL_MS, 100_000);
    +        mockStormConf.put(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS, 30);
    +        mockStormConf.put(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL, 1000);
    +        Mockito.when(mockPartitionState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(mockWindowState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(mockSystemState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        executor.prepare(mockStormConf, mockTopologyContext, mockOutputCollector,
    +            mockWindowState, mockPartitionState, mockSystemState);
    +    }
    +
    +    @Test
    +    public void testExecuteTuple() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), 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);
    +    }
    +
    +    @Test
    +    public void testExecuteLatetuple() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), 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
    +        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());
    +    }
    +
    +    @Test
    +    public void testActivation() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), Mockito.anyLong())).thenReturn(true);
    +        executor.initState(null);
    +        executor.waterMarkEventGenerator = mockWaterMarkEventGenerator;
    +
    +        List<Tuple> mockTuples = getMockTuples(5);
    +        mockTuples.forEach(t -> executor.execute(t));
    +        // all tuples acked
    +        Mockito.verify(mockOutputCollector, Mockito.times(5)).ack(tupleCaptor.capture());
    +        Assert.assertArrayEquals(mockTuples.toArray(), tupleCaptor.getAllValues().toArray());
    +
    +        Mockito.doAnswer(new Answer<Void>() {
    +            @Override
    +            public Void answer(InvocationOnMock invocation) throws Throwable {
    +                TupleWindow window = (TupleWindow) invocation.getArguments()[0];
    +                // iterate the tuples
    +                Assert.assertEquals(5, window.get().size());
    +                // iterating multiple times should produce same events
    --- End diff --
    
    Am pulling out the WindowState into a top level class and adding additional tests. Maybe this can still be left as is.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129763468
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutor.java ---
    @@ -0,0 +1,563 @@
    +/**
    + * 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 java.util.AbstractCollection;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentLinkedQueue;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.locks.ReentrantLock;
    +import java.util.function.Supplier;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.state.KeyValueState;
    +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.windowing.DefaultEvictionContext;
    +import org.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.EventImpl;
    +import org.apache.storm.windowing.WindowLifecycleListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static java.util.Collections.emptyIterator;
    +import static org.apache.storm.topology.WindowPartitionCache.CacheLoader;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalCause;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalListener;
    +
    +/**
    + * Wraps a {@link IStatefulWindowedBolt} and handles the execution. Uses state and the underlying
    + * checkpointing mechanisms to save the tuples in window to state. The tuples are also kept in-memory
    + * by transparently caching the window partitions and checkpointing them as needed.
    + */
    +public class PersistentWindowedBoltExecutor<T extends State> extends WindowedBoltExecutor implements IStatefulBolt<T> {
    +    private static final Logger LOG = LoggerFactory.getLogger(PersistentWindowedBoltExecutor.class);
    +    private final IStatefulWindowedBolt<T> statefulWindowedBolt;
    +    private transient TopologyContext topologyContext;
    +    private transient OutputCollector outputCollector;
    +    private transient WindowState<Tuple> state;
    +    private transient boolean stateInitialized;
    +    private transient boolean prePrepared;
    +
    +    public PersistentWindowedBoltExecutor(IStatefulWindowedBolt<T> bolt) {
    +        super(bolt);
    +        statefulWindowedBolt = bolt;
    +    }
    +
    +    @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));
    +    }
    +
    +    // package access for unit tests
    +    void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                 KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                 KeyValueState<String, Deque<Long>> partitionState,
    +                 KeyValueState<String, Optional<?>> windowSystemState) {
    +        init(topoConf, context, collector, windowState, partitionState, windowSystemState);
    +        doPrepare(topoConf, context, new NoAckOutputCollector(collector), state, true);
    +        Map<String, Optional<?>> wstate = new HashMap<>();
    +        windowSystemState.forEach(s -> wstate.put(s.getKey(), s.getValue()));
    +        restoreState(wstate);
    +    }
    +
    +    @Override
    +    protected void start() {
    +        if (stateInitialized) {
    +            super.start();
    +        } else {
    +            LOG.debug("Will invoke start after state is initialized");
    +        }
    +    }
    +
    +    @Override
    +    public void execute(Tuple input) {
    +        if (!stateInitialized) {
    +            throw new IllegalStateException("execute invoked before initState with input tuple " + input);
    +        }
    +        super.execute(input);
    +        // StatefulBoltExecutor does the actual ack when the state is saved.
    +        outputCollector.ack(input);
    +    }
    +
    +    @Override
    +    public void initState(T state) {
    +        if (stateInitialized) {
    +            LOG.warn("State is already initialized. Ignoring initState");
    --- End diff --
    
    If a worker crashes, all worker's state are rolled back and an initState message is sent across the topology so that crashed workers can initialize their state. This is already handled in `StatefulBoltExecutor` that wraps the `PersistentWindowedBoltExecutor` so here its Ok to throw an exception. Will address this and revisit the others.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130235112
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/SimpleWindowPartitionCache.java ---
    @@ -0,0 +1,191 @@
    +/**
    + * 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 java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.concurrent.ConcurrentMap;
    +import java.util.concurrent.ConcurrentSkipListMap;
    +import java.util.concurrent.locks.ReentrantLock;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * A simple implementation that evicts the largest un-pinned entry from the cache. This works well
    + * for caching window partitions since the access pattern is mostly sequential scans.
    + */
    +public class SimpleWindowPartitionCache<K, V> implements WindowPartitionCache<K, V> {
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleWindowPartitionCache.class);
    +
    +    private final ConcurrentSkipListMap<K, V> map = new ConcurrentSkipListMap<>();
    +    private final Map<K, Long> pinned = new HashMap<>();
    +    private final long maximumSize;
    +    private final RemovalListener<K, V> removalListener;
    +    private final CacheLoader<K, V> cacheLoader;
    +    private final ReentrantLock lock = new ReentrantLock(true);
    +    private int size;
    --- End diff --
    
    Thanks for explaining, this makes sense then.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130273957
  
    --- Diff: storm-client/src/jvm/org/apache/storm/windowing/WindowManager.java ---
    @@ -111,14 +125,86 @@ public void add(Event<T> windowEvent) {
                 LOG.debug("Got watermark event with ts {}", windowEvent.getTimestamp());
             }
             track(windowEvent);
    -        compactWindow();
    +        if (!stateful) {
    +            compactWindow();
    +        }
         }
     
         /**
          * The callback invoked by the trigger policy.
          */
         @Override
         public boolean onTrigger() {
    +        return stateful ? doOnTriggerStateful() : doOnTrigger();
    +    }
    +
    +    private static class IteratorStatus {
    +        private boolean valid = true;
    +
    +        void invalidate() {
    +            valid = false;
    +        }
    +
    +        boolean isValid() {
    +            return valid;
    +        }
    +    }
    +
    +    private static<T> Iterator<T> expiringIterator(Iterator<T> inner, IteratorStatus status) {
    +        return new Iterator<T>() {
    +            @Override
    +            public boolean hasNext() {
    +                if (status.isValid()) {
    +                    return inner.hasNext();
    +                }
    +                throw new IllegalStateException("Stale iterator");
    +            }
    +
    +            @Override
    +            public T next() {
    +                if (status.isValid()) {
    +                    return inner.next();
    +                }
    +                throw new IllegalStateException("Stale iterator");
    +            }
    +        };
    +    }
    +
    +    private boolean doOnTriggerStateful() {
    +        Supplier<Iterator<T>> scanEventsStateful = this::scanEventsStateful;
    +        Iterator<T> it = scanEventsStateful.get();
    --- End diff --
    
    The locks are required in `onTrigger` due to `compactEvents`. Here its not required since compact is not invoked. I am refactoring this into sub class.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r131121321
  
    --- Diff: storm-client/test/jvm/org/apache/storm/windowing/persistence/WindowStateTest.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.windowing.persistence;
    +
    +import org.apache.storm.state.KeyValueState;
    +import org.apache.storm.tuple.Tuple;
    +import org.apache.storm.windowing.Event;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.mockito.ArgumentCaptor;
    +import org.mockito.Captor;
    +import org.mockito.Mock;
    +import org.mockito.Mockito;
    +import org.mockito.MockitoAnnotations;
    +import org.mockito.invocation.InvocationOnMock;
    +import org.mockito.runners.MockitoJUnitRunner;
    +import org.mockito.stubbing.Answer;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.function.Consumer;
    +import java.util.function.Supplier;
    +
    +import static org.mockito.AdditionalAnswers.returnsArgAt;
    +
    +/**
    + * Unit tests for {@link WindowState}
    + */
    +@RunWith(MockitoJUnitRunner.class)
    +public class WindowStateTest {
    +
    +    @Mock
    +    private KeyValueState<Long, WindowState.WindowPartition<Integer>> windowState;
    +    @Mock
    +    private KeyValueState<String, Deque<Long>> partitionIdsState;
    +    @Mock
    +    private KeyValueState<String, Optional<?>> systemState;
    +    @Mock
    +    private Supplier<Map<String, Optional<?>>> supplier;
    +    @Captor
    +    private ArgumentCaptor<Long> longCaptor;
    +    @Captor
    +    private ArgumentCaptor<WindowState.WindowPartition<Integer>> windowValuesCaptor;
    +
    +    private static final int MAX_EVENTS_PER_PARTITION = 1000;
    +
    +    @Before
    +    public void setUp() throws Exception {
    +        MockitoAnnotations.initMocks(this);
    +    }
    +
    +    @Test
    +    public void testAdd() throws Exception {
    +        Mockito.when(partitionIdsState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(windowState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +
    +        WindowState<Integer> ws = getWindowState(10 * MAX_EVENTS_PER_PARTITION);
    +
    +        long partitions = 15;
    +        long numEvents = partitions * MAX_EVENTS_PER_PARTITION;
    +        for (int i = 0; i < numEvents; i++) {
    +            ws.add(getEvent(i));
    +        }
    +        // 5 events evicted to window state
    --- End diff --
    
    Isn't it 5 partitions?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130432729
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutor.java ---
    @@ -0,0 +1,596 @@
    +/**
    + * 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 java.util.AbstractCollection;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Objects;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentLinkedQueue;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.locks.ReentrantLock;
    +import java.util.function.Supplier;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.state.KeyValueState;
    +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.topology.base.BaseWindowedBolt;
    +import org.apache.storm.tuple.Tuple;
    +import org.apache.storm.windowing.DefaultEvictionContext;
    +import org.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.EventImpl;
    +import org.apache.storm.windowing.WindowLifecycleListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static java.util.Collections.emptyIterator;
    +import static org.apache.storm.topology.WindowPartitionCache.CacheLoader;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalCause;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalListener;
    +
    +/**
    + * Wraps a {@link IStatefulWindowedBolt} and handles the execution. Uses state and the underlying
    + * checkpointing mechanisms to save the tuples in window to state. The tuples are also kept in-memory
    + * by transparently caching the window partitions and checkpointing them as needed.
    + */
    +public class PersistentWindowedBoltExecutor<T extends State> extends WindowedBoltExecutor implements IStatefulBolt<T> {
    +    private static final Logger LOG = LoggerFactory.getLogger(PersistentWindowedBoltExecutor.class);
    +    private final IStatefulWindowedBolt<T> statefulWindowedBolt;
    +    private transient TopologyContext topologyContext;
    +    private transient OutputCollector outputCollector;
    +    private transient WindowState<Tuple> state;
    +    private transient boolean stateInitialized;
    +    private transient boolean prePrepared;
    +
    +    public PersistentWindowedBoltExecutor(IStatefulWindowedBolt<T> bolt) {
    +        super(bolt);
    +        statefulWindowedBolt = bolt;
    +    }
    +
    +    @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));
    +    }
    +
    +    @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);
    +        }
    +    }
    +
    +    private int getCheckpointIntervalMillis(Map<String, Object> topoConf) {
    +        int checkpointInterval = Integer.MAX_VALUE;
    +        if (topoConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL) != null) {
    +            checkpointInterval = ((Number) topoConf.get(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL)).intValue();
    +        }
    +        return checkpointInterval;
    +    }
    +
    +    // package access for unit tests
    +    void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                 KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                 KeyValueState<String, Deque<Long>> partitionState,
    +                 KeyValueState<String, Optional<?>> windowSystemState) {
    +        init(topoConf, context, collector, windowState, partitionState, windowSystemState);
    +        doPrepare(topoConf, context, new NoAckOutputCollector(collector), state, true);
    +        Map<String, Optional<?>> wstate = new HashMap<>();
    +        windowSystemState.forEach(s -> wstate.put(s.getKey(), s.getValue()));
    +        restoreState(wstate);
    +    }
    +
    +    @Override
    +    protected void start() {
    +        if (stateInitialized) {
    +            super.start();
    +        } else {
    +            LOG.debug("Will invoke start after state is initialized");
    +        }
    +    }
    +
    +    @Override
    +    public void execute(Tuple input) {
    +        if (!stateInitialized) {
    +            throw new IllegalStateException("execute invoked before initState with input tuple " + input);
    +        }
    +        super.execute(input);
    +        // StatefulBoltExecutor does the actual ack when the state is saved.
    +        outputCollector.ack(input);
    +    }
    +
    +    @Override
    +    public void initState(T state) {
    +        if (stateInitialized) {
    +            String msg = "initState invoked when the state is already initialized";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        } else {
    +            statefulWindowedBolt.initState(state);
    +            stateInitialized = true;
    +            start();
    +        }
    +    }
    +
    +    @Override
    +    public void prePrepare(long txid) {
    +        if (stateInitialized) {
    +            LOG.debug("Prepare streamState, txid {}", txid);
    +            state.prepareCommit(txid);
    +            prePrepared = true;
    +        } else {
    +            String msg = "Cannot prepare before initState";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        }
    +    }
    +
    +    @Override
    +    public void preCommit(long txid) {
    +        // preCommit can be invoked during recovery before the state is initialized
    +        if (prePrepared || !stateInitialized) {
    +            LOG.debug("Commit streamState, txid {}", txid);
    +            state.commit(txid);
    +        } else {
    +            String msg = "preCommit before prePrepare in initialized state";
    +            LOG.warn(msg);
    +            throw new IllegalStateException(msg);
    +        }
    +    }
    +
    +    @Override
    +    public void preRollback() {
    +        LOG.debug("Rollback streamState, stateInitialized {}", stateInitialized);
    +        state.rollback();
    +    }
    +
    +    @Override
    +    protected WindowLifecycleListener<Tuple> newWindowLifecycleListener() {
    +        return new WindowLifecycleListener<Tuple>() {
    +            @Override
    +            public void onExpiry(List<Tuple> events) {
    +                /*
    +                 * NO-OP: the events are ack-ed in execute
    +                 */
    +            }
    +
    +            @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();
    +            }
    +        };
    +    }
    +
    +    private void init(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                      KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                      KeyValueState<String, Deque<Long>> partitionState,
    +                      KeyValueState<String, Optional<?>> windowSystemState) {
    +        topologyContext = context;
    +        outputCollector = collector;
    +        state = new WindowState<>(windowState, partitionState, windowSystemState, this::getState,
    +            statefulWindowedBolt.maxEventsInMemory());
    +    }
    +
    +    private KeyValueState<Long, WindowPartition<Tuple>> getWindowState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window";
    +        return (KeyValueState<Long, WindowPartition<Tuple>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Deque<Long>> getPartitionState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-partitions";
    +        return (KeyValueState<String, Deque<Long>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Optional<?>> getWindowSystemState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-systemstate";
    +        return (KeyValueState<String, Optional<?>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    // a wrapper around the window related states that are checkpointed
    +    private static class WindowState<T> extends AbstractCollection<Event<T>> {
    +        // number of events per window-partition
    +        private static final int MAX_PARTITION_EVENTS = 1000;
    +        private static final int MIN_PARTITIONS = 10;
    +        private static final String PARTITION_IDS_KEY = "pk";
    +        private final KeyValueState<String, Deque<Long>> partitionIdsState;
    +        private final KeyValueState<Long, WindowPartition<T>> windowPartitionsState;
    +        private final KeyValueState<String, Optional<?>> windowSystemState;
    +        // ordered partition keys
    +        private Deque<Long> partitionIds;
    +        private volatile long latestPartitionId;
    +        private WindowPartitionCache<Long, WindowPartition<T>> cache;
    +        private Supplier<Map<String, Optional<?>>> windowSystemStateSupplier;
    +        private final ReentrantLock partitionIdsLock = new ReentrantLock(true);
    +        private final ReentrantLock windowPartitionsLock = new ReentrantLock(true);
    +        private final long maxEventsInMemory;
    +        private WindowPartition<T> latestPartition;
    +        private Set<Long> iteratorPins = new HashSet<>();
    +
    +        WindowState(KeyValueState<Long, WindowPartition<T>> windowPartitionsState,
    +                    KeyValueState<String, Deque<Long>> partitionIdsState,
    +                    KeyValueState<String, Optional<?>> windowSystemState,
    +                    Supplier<Map<String, Optional<?>>> windowSystemStateSupplier,
    +                    long maxEventsInMemory) {
    +            this.windowPartitionsState = windowPartitionsState;
    +            this.partitionIdsState = partitionIdsState;
    +            this.windowSystemState = windowSystemState;
    +            this.windowSystemStateSupplier = windowSystemStateSupplier;
    +            this.maxEventsInMemory = Math.max(MAX_PARTITION_EVENTS * MIN_PARTITIONS, maxEventsInMemory);
    +            initCache();
    +            initPartitions();
    +        }
    +
    +        @Override
    +        public boolean add(Event<T> event) {
    +            if (latestPartition.size() >= MAX_PARTITION_EVENTS) {
    +                cache.unpin(latestPartition.getId());
    +                latestPartition = getPinnedPartition(getNextPartitionId());
    +            }
    +            latestPartition.add(event);
    +            return true;
    +        }
    +
    +        @Override
    +        public Iterator<Event<T>> iterator() {
    +
    +            return new Iterator<Event<T>>() {
    +                private Iterator<Long> ids = getIds();
    +                private Iterator<Event<T>> current = emptyIterator();
    +                private Iterator<Event<T>> removeFrom;
    +                private WindowPartition<T> curPartition;
    +
    +                private Iterator<Long> getIds() {
    +                    try {
    +                        partitionIdsLock.lock();
    +                        LOG.debug("Iterator partitionIds: {}", partitionIds);
    +                        return new ArrayList<>(partitionIds).iterator();
    +                    } finally {
    +                        partitionIdsLock.unlock();
    +                    }
    +                }
    +
    +                @Override
    +                public void remove() {
    +                    if (removeFrom == null) {
    +                        throw new IllegalStateException("No calls to next() since last call to remove()");
    +                    }
    +                    removeFrom.remove();
    +                    removeFrom = null;
    +                }
    +
    +                @Override
    +                public boolean hasNext() {
    +                    boolean curHasNext = current.hasNext();
    +                    while (!curHasNext && ids.hasNext()) {
    +                        if (curPartition != null) {
    +                            unpin(curPartition.getId());
    +                        }
    +                        curPartition = getPinnedPartition(ids.next());
    +                        if (curPartition != null) {
    +                            iteratorPins.add(curPartition.getId());
    +                            current = curPartition.iterator();
    +                            curHasNext = current.hasNext();
    +                        }
    +                    }
    +                    // un-pin the last partition
    +                    if (!curHasNext && curPartition != null) {
    +                        unpin(curPartition.getId());
    +                        curPartition = null;
    +                    }
    +                    return curHasNext;
    +                }
    +
    +                @Override
    +                public Event<T> next() {
    +                    if (!hasNext()) {
    +                        throw new NoSuchElementException();
    +                    }
    +                    removeFrom = current;
    +                    return current.next();
    +                }
    +
    +                private void unpin(long id) {
    +                    cache.unpin(id);
    +                    iteratorPins.remove(id);
    +                }
    +            };
    +        }
    +
    +        void clearIteratorPins() {
    +            LOG.debug("clearIteratorPins '{}'", iteratorPins);
    +            Iterator<Long> it = iteratorPins.iterator();
    +            while (it.hasNext()) {
    +                cache.unpin(it.next());
    +                it.remove();
    +            }
    +        }
    +
    +        @Override
    +        public int size() {
    +            throw new UnsupportedOperationException();
    +        }
    +
    +        void prepareCommit(long txid) {
    +            flush();
    +            partitionIdsState.prepareCommit(txid);
    +            windowPartitionsState.prepareCommit(txid);
    +            windowSystemState.prepareCommit(txid);
    +        }
    +
    +        void commit(long txid) {
    +            partitionIdsState.commit(txid);
    +            windowPartitionsState.commit(txid);
    +            windowSystemState.commit(txid);
    +        }
    +
    +        void rollback() {
    +            partitionIdsState.rollback();
    +            windowPartitionsState.rollback();
    +            windowSystemState.rollback();
    +        }
    +
    +        private void initPartitions() {
    +            partitionIds = partitionIdsState.get(PARTITION_IDS_KEY, new LinkedList<>());
    +            if (partitionIds.isEmpty()) {
    +                partitionIds.add(0L);
    +                partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    +            }
    +            latestPartitionId = partitionIds.peekLast();
    +            latestPartition = cache.getPinned(latestPartitionId);
    +        }
    +
    +        private void initCache() {
    +            long size = maxEventsInMemory / MAX_PARTITION_EVENTS;
    +            LOG.info("maxEventsInMemory: {}, partition size: {}, number of partitions: {}",
    +                maxEventsInMemory, MAX_PARTITION_EVENTS, size);
    +            cache = SimpleWindowPartitionCache.<Long, WindowPartition<T>>newBuilder()
    +                .maximumSize(size)
    +                .removalListener(new RemovalListener<Long, WindowPartition<T>>() {
    +                    @Override
    +                    public void onRemoval(Long pid, WindowPartition<T> p, RemovalCause removalCause) {
    +                        Objects.requireNonNull(pid, "Null partition id");
    +                        Objects.requireNonNull(p, "Null window partition");
    +                        LOG.debug("onRemoval for id '{}', WindowPartition '{}'", pid, p);
    +                        try {
    +                            windowPartitionsLock.lock();
    +                            if (p.isEmpty() && pid != latestPartitionId) {
    +                                // if the empty partition was not invalidated by flush, but evicted from cache
    +                                if (removalCause != RemovalCause.EXPLICIT) {
    +                                    deletePartition(pid);
    +                                    windowPartitionsState.delete(pid);
    +                                }
    +                            } else if (p.isModified()) {
    +                                windowPartitionsState.put(pid, p);
    +                            } else {
    +                                LOG.debug("WindowPartition '{}' is not modified", pid);
    +                            }
    +                        } finally {
    +                            windowPartitionsLock.unlock();
    +                        }
    +                    }
    +                }).build(new CacheLoader<Long, WindowPartition<T>>() {
    +                    @Override
    +                   public WindowPartition<T> load(Long id) {
    +                        LOG.debug("Load partition: {}", id);
    +                        // load from state
    +                        try {
    +                            windowPartitionsLock.lock();
    +                            return windowPartitionsState.get(id, new WindowPartition<>(id));
    +                        } finally {
    +                            windowPartitionsLock.unlock();
    +                        }
    +                    }
    +                    }
    +                );
    +        }
    +
    +        private void deletePartition(long pid) {
    +            LOG.debug("Delete partition: {}", pid);
    +            try {
    +                partitionIdsLock.lock();
    +                partitionIds.remove(pid);
    +                partitionIdsState.put(PARTITION_IDS_KEY, partitionIds);
    --- End diff --
    
    Isn't that guaranteed to happen in initPartitions further up?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130227409
  
    --- Diff: storm-client/test/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutorTest.java ---
    @@ -0,0 +1,261 @@
    +/**
    + * 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.Config;
    +import org.apache.storm.generated.GlobalStreamId;
    +import org.apache.storm.state.KeyValueState;
    +import org.apache.storm.streams.Pair;
    +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.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.TimestampExtractor;
    +import org.apache.storm.windowing.TupleWindow;
    +import org.apache.storm.windowing.WaterMarkEvent;
    +import org.apache.storm.windowing.WaterMarkEventGenerator;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.mockito.ArgumentCaptor;
    +import org.mockito.Captor;
    +import org.mockito.Mockito;
    +import org.mockito.MockitoAnnotations;
    +import org.mockito.invocation.InvocationOnMock;
    +import org.mockito.runners.MockitoJUnitRunner;
    +import org.mockito.stubbing.Answer;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.stream.Collectors;
    +import java.util.stream.LongStream;
    +
    +import static org.mockito.AdditionalAnswers.returnsArgAt;
    +
    +/**
    + * Unit tests for {@link PersistentWindowedBoltExecutor}
    + */
    +@RunWith(MockitoJUnitRunner.class)
    +public class PersistentWindowedBoltExecutorTest {
    +    public static final String LATE_STREAM = "late_stream";
    +    private PersistentWindowedBoltExecutor<KeyValueState<String, String>> executor;
    +    private IStatefulWindowedBolt<KeyValueState<String, String>> mockBolt;
    +    private OutputCollector mockOutputCollector;
    +    private TopologyContext mockTopologyContext;
    +    private Map<String, Object> mockStormConf = new HashMap<>();
    +    private TimestampExtractor mockTimestampExtractor;
    +    private WaterMarkEventGenerator mockWaterMarkEventGenerator;
    +    private KeyValueState mockPartitionState;
    +    private KeyValueState mockWindowState;
    +    private KeyValueState mockSystemState;
    +
    +    private static final String PARTITION_KEY = "pk";
    +    private static final String EVICTION_STATE_KEY = "es";
    +    private static final String TRIGGER_STATE_KEY = "ts";
    +    private long tupleTs;
    +
    +    @Captor
    +    private ArgumentCaptor<Tuple> tupleCaptor;
    +    @Captor
    +    private ArgumentCaptor<Collection<Tuple>> anchorCaptor;
    +    @Captor
    +    private ArgumentCaptor<String> stringCaptor;
    +    @Captor
    +    private ArgumentCaptor<Long> longCaptor;
    +    @Captor
    +    private ArgumentCaptor<Values> valuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<TupleWindow> tupleWindowCaptor;
    +    @Captor
    +    private ArgumentCaptor<Deque<Long>> partitionValuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<PersistentWindowedBoltExecutor.WindowPartition<Event<Tuple>>> windowValuesCaptor;
    +    @Captor
    +    private ArgumentCaptor<Object> systemValuesCaptor;
    +
    +    @Before
    +    public void setUp() throws Exception {
    +        MockitoAnnotations.initMocks(this);
    +        mockBolt = Mockito.mock(IStatefulWindowedBolt.class);
    +        mockWaterMarkEventGenerator = Mockito.mock(WaterMarkEventGenerator.class);
    +        mockTimestampExtractor = Mockito.mock(TimestampExtractor.class);
    +        tupleTs = System.currentTimeMillis();
    +        Mockito.when(mockTimestampExtractor.extractTimestamp(Mockito.any())).thenReturn(tupleTs);
    +        Mockito.when(mockBolt.getTimestampExtractor()).thenReturn(mockTimestampExtractor);
    +        Mockito.when(mockBolt.isPersistent()).thenReturn(true);
    +        mockPartitionState = Mockito.mock(KeyValueState.class);
    +        mockWindowState = Mockito.mock(KeyValueState.class);
    +        mockSystemState = Mockito.mock(KeyValueState.class);
    +        mockTopologyContext = Mockito.mock(TopologyContext.class);
    +        Mockito.when(mockTopologyContext.getThisStreams()).thenReturn(Collections.singleton(LATE_STREAM));
    +        mockOutputCollector = Mockito.mock(OutputCollector.class);
    +        executor = new PersistentWindowedBoltExecutor<>(mockBolt);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_WINDOW_LENGTH_COUNT, 5);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_SLIDING_INTERVAL_COUNT, 5);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_LATE_TUPLE_STREAM, LATE_STREAM);
    +        mockStormConf.put(Config.TOPOLOGY_BOLTS_WATERMARK_EVENT_INTERVAL_MS, 100_000);
    +        mockStormConf.put(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS, 30);
    +        mockStormConf.put(Config.TOPOLOGY_STATE_CHECKPOINT_INTERVAL, 1000);
    +        Mockito.when(mockPartitionState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(mockWindowState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        Mockito.when(mockSystemState.get(Mockito.any(), Mockito.any())).then(returnsArgAt(1));
    +        executor.prepare(mockStormConf, mockTopologyContext, mockOutputCollector,
    +            mockWindowState, mockPartitionState, mockSystemState);
    +    }
    +
    +    @Test
    +    public void testExecuteTuple() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), 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);
    +    }
    +
    +    @Test
    +    public void testExecuteLatetuple() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), 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
    +        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());
    +    }
    +
    +    @Test
    +    public void testActivation() throws Exception {
    +        Mockito.when(mockWaterMarkEventGenerator.track(Mockito.any(GlobalStreamId.class), Mockito.anyLong())).thenReturn(true);
    +        executor.initState(null);
    +        executor.waterMarkEventGenerator = mockWaterMarkEventGenerator;
    +
    +        List<Tuple> mockTuples = getMockTuples(5);
    +        mockTuples.forEach(t -> executor.execute(t));
    +        // all tuples acked
    +        Mockito.verify(mockOutputCollector, Mockito.times(5)).ack(tupleCaptor.capture());
    +        Assert.assertArrayEquals(mockTuples.toArray(), tupleCaptor.getAllValues().toArray());
    +
    +        Mockito.doAnswer(new Answer<Void>() {
    +            @Override
    +            public Void answer(InvocationOnMock invocation) throws Throwable {
    +                TupleWindow window = (TupleWindow) invocation.getArguments()[0];
    +                // iterate the tuples
    +                Assert.assertEquals(5, window.get().size());
    +                // iterating multiple times should produce same events
    --- End diff --
    
    This feels like it belongs in a test of the iterator implementation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129763759
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/WindowedBoltExecutor.java ---
    @@ -127,26 +154,30 @@ private void validate(Map<String, Object> topoConf, Count windowLengthCount, Dur
                 throw new IllegalArgumentException("Window length is not specified");
             }
     
    -        if (windowLengthDuration != null && slidingIntervalDuration != null) {
    -            ensureDurationLessThanTimeout(windowLengthDuration.value + slidingIntervalDuration.value, topologyTimeout);
    -        } else if (windowLengthDuration != null) {
    -            ensureDurationLessThanTimeout(windowLengthDuration.value, topologyTimeout);
    -        } else if (slidingIntervalDuration != null) {
    -            ensureDurationLessThanTimeout(slidingIntervalDuration.value, topologyTimeout);
    -        }
    +        if (isPersistent()) {
    +            ensureCheckpointIntervalLessThanTimeout(getCheckpointIntervalSecs(topoConf), topologyTimeout);
    --- End diff --
    
    we don't need to validate `message timeout > window length + sliding interval` here since the tuples are acked during checkpoint. Hence ensuring the `timeout > checkpoint interval` would suffice.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by arunmahadevan <gi...@git.apache.org>.
Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r130273789
  
    --- Diff: storm-client/src/jvm/org/apache/storm/windowing/WindowManager.java ---
    @@ -289,4 +414,22 @@ public String toString() {
                     ", triggerPolicy=" + triggerPolicy +
                     '}';
         }
    +
    +    public void restoreState(Map<String, Optional<?>> state) {
    +        if (state != null) {
    --- End diff --
    
    appears that its not required now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2218: STORM-2614: Enhance stateful windowing to persist ...

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2218#discussion_r129644551
  
    --- Diff: storm-client/src/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutor.java ---
    @@ -0,0 +1,563 @@
    +/**
    + * 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 java.util.AbstractCollection;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentLinkedQueue;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.locks.ReentrantLock;
    +import java.util.function.Supplier;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.state.KeyValueState;
    +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.windowing.DefaultEvictionContext;
    +import org.apache.storm.windowing.Event;
    +import org.apache.storm.windowing.EventImpl;
    +import org.apache.storm.windowing.WindowLifecycleListener;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import static java.util.Collections.emptyIterator;
    +import static org.apache.storm.topology.WindowPartitionCache.CacheLoader;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalCause;
    +import static org.apache.storm.topology.WindowPartitionCache.RemovalListener;
    +
    +/**
    + * Wraps a {@link IStatefulWindowedBolt} and handles the execution. Uses state and the underlying
    + * checkpointing mechanisms to save the tuples in window to state. The tuples are also kept in-memory
    + * by transparently caching the window partitions and checkpointing them as needed.
    + */
    +public class PersistentWindowedBoltExecutor<T extends State> extends WindowedBoltExecutor implements IStatefulBolt<T> {
    +    private static final Logger LOG = LoggerFactory.getLogger(PersistentWindowedBoltExecutor.class);
    +    private final IStatefulWindowedBolt<T> statefulWindowedBolt;
    +    private transient TopologyContext topologyContext;
    +    private transient OutputCollector outputCollector;
    +    private transient WindowState<Tuple> state;
    +    private transient boolean stateInitialized;
    +    private transient boolean prePrepared;
    +
    +    public PersistentWindowedBoltExecutor(IStatefulWindowedBolt<T> bolt) {
    +        super(bolt);
    +        statefulWindowedBolt = bolt;
    +    }
    +
    +    @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));
    +    }
    +
    +    // package access for unit tests
    +    void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                 KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                 KeyValueState<String, Deque<Long>> partitionState,
    +                 KeyValueState<String, Optional<?>> windowSystemState) {
    +        init(topoConf, context, collector, windowState, partitionState, windowSystemState);
    +        doPrepare(topoConf, context, new NoAckOutputCollector(collector), state, true);
    +        Map<String, Optional<?>> wstate = new HashMap<>();
    +        windowSystemState.forEach(s -> wstate.put(s.getKey(), s.getValue()));
    +        restoreState(wstate);
    +    }
    +
    +    @Override
    +    protected void start() {
    +        if (stateInitialized) {
    +            super.start();
    +        } else {
    +            LOG.debug("Will invoke start after state is initialized");
    +        }
    +    }
    +
    +    @Override
    +    public void execute(Tuple input) {
    +        if (!stateInitialized) {
    +            throw new IllegalStateException("execute invoked before initState with input tuple " + input);
    +        }
    +        super.execute(input);
    +        // StatefulBoltExecutor does the actual ack when the state is saved.
    +        outputCollector.ack(input);
    +    }
    +
    +    @Override
    +    public void initState(T state) {
    +        if (stateInitialized) {
    +            LOG.warn("State is already initialized. Ignoring initState");
    +        } else {
    +            statefulWindowedBolt.initState(state);
    +            stateInitialized = true;
    +            start();
    +        }
    +    }
    +
    +    @Override
    +    public void prePrepare(long txid) {
    +        if (stateInitialized) {
    +            LOG.debug("Prepare streamState, txid {}", txid);
    +            state.prepareCommit(txid);
    +            prePrepared = true;
    +        } else {
    +            LOG.warn("Cannot prepare before initState");
    +        }
    +    }
    +
    +    @Override
    +    public void preCommit(long txid) {
    +        if (prePrepared) {
    +            LOG.debug("Commit streamState, txid {}", txid);
    +            state.commit(txid);
    +        } else {
    +            LOG.debug("Ignoring preCommit and not committing streamState.");
    +        }
    +    }
    +
    +    @Override
    +    public void preRollback() {
    +        LOG.debug("Rollback streamState, stateInitialized {}", stateInitialized);
    +        state.rollback();
    +    }
    +
    +    @Override
    +    protected WindowLifecycleListener<Tuple> newWindowLifecycleListener() {
    +        return new WindowLifecycleListener<Tuple>() {
    +            @Override
    +            public void onExpiry(List<Tuple> events) {
    +                /*
    +                 * NO-OP: the events are ack-ed in execute
    +                 */
    +            }
    +
    +            @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();
    +            }
    +        };
    +    }
    +
    +    private void init(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector,
    +                      KeyValueState<Long, WindowPartition<Tuple>> windowState,
    +                      KeyValueState<String, Deque<Long>> partitionState,
    +                      KeyValueState<String, Optional<?>> windowSystemState) {
    +        topologyContext = context;
    +        outputCollector = collector;
    +        state = new WindowState<>(windowState, partitionState, windowSystemState, this::getState,
    +            statefulWindowedBolt.maxEventsInMemory());
    +    }
    +
    +    private KeyValueState<Long, WindowPartition<Tuple>> getWindowState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window";
    +        return (KeyValueState<Long, WindowPartition<Tuple>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Deque<Long>> getPartitionState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-partitions";
    +        return (KeyValueState<String, Deque<Long>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    private KeyValueState<String, Optional<?>> getWindowSystemState(Map<String, Object> topoConf, TopologyContext context) {
    +        String namespace = context.getThisComponentId() + "-" + context.getThisTaskId() + "-window-systemstate";
    +        return (KeyValueState<String, Optional<?>>) StateFactory.getState(namespace, topoConf, context);
    +    }
    +
    +    // a wrapper around the window related states that are checkpointed
    +    private static class WindowState<T> extends AbstractCollection<Event<T>> {
    +        // number of events per window-partition
    +        private static final int PARTITION_SZ = 1000;
    +        private static final int MIN_PARTITIONS = 10;
    +        private static final String PARTITION_IDS_KEY = "pk";
    +        private final KeyValueState<String, Deque<Long>> partitionIds;
    +        private final KeyValueState<Long, WindowPartition<T>> windowPartitions;
    +        private final KeyValueState<String, Optional<?>> windowSystemState;
    +        // ordered partition keys
    +        private Deque<Long> pids;
    +        private volatile long latestPartitionId;
    +        private WindowPartitionCache<Long, WindowPartition<T>> cache;
    +        private Supplier<Map<String, Optional<?>>> windowSystemStateSupplier;
    +        private final ReentrantLock partitionIdsLock = new ReentrantLock(true);
    +        private final ReentrantLock windowPartitionsLock = new ReentrantLock(true);
    +        private final long maxEventsInMemory;
    +        private WindowPartition<T> latestPartition;
    +        private Set<Long> iteratorPins = new HashSet<>();
    +
    +        WindowState(KeyValueState<Long, WindowPartition<T>> windowPartitions,
    +                    KeyValueState<String, Deque<Long>> partitionIds,
    +                    KeyValueState<String, Optional<?>> windowSystemState,
    +                    Supplier<Map<String, Optional<?>>> windowSystemStateSupplier,
    +                    long maxEventsInMemory) {
    +            this.windowPartitions = windowPartitions;
    +            this.partitionIds = partitionIds;
    +            this.windowSystemState = windowSystemState;
    +            this.windowSystemStateSupplier = windowSystemStateSupplier;
    +            this.maxEventsInMemory = Math.max(PARTITION_SZ * MIN_PARTITIONS, maxEventsInMemory);
    +            initCache();
    +            initPartitions();
    +        }
    +
    +        @Override
    +        public boolean add(Event<T> event) {
    +            if (latestPartition.size() >= PARTITION_SZ) {
    +                cache.unpin(latestPartition.getId());
    +                latestPartition = getPinnedPartition(getNextPartitionId());
    +            }
    +            latestPartition.add(event);
    +            return true;
    +        }
    +
    +        @Override
    +        public Iterator<Event<T>> iterator() {
    +
    +            return new Iterator<Event<T>>() {
    +                private Iterator<Long> ids = getIds();
    +                private Iterator<Event<T>> current = emptyIterator();
    +                private Iterator<Event<T>> removeFrom;
    +                private WindowPartition<T> curPartition;
    +
    +                private Iterator<Long> getIds() {
    +                    try {
    +                        partitionIdsLock.lock();
    +                        LOG.debug("Iterator pids: {}", pids);
    +                        return new ArrayList<>(pids).iterator();
    +                    } finally {
    +                        partitionIdsLock.unlock();
    +                    }
    +                }
    +
    +                @Override
    +                public void remove() {
    +                    if (removeFrom == null) {
    +                        throw new IllegalStateException("No calls to next() since last call to remove()");
    +                    }
    +                    removeFrom.remove();
    +                    removeFrom = null;
    +                }
    +
    +                @Override
    +                public boolean hasNext() {
    +                    boolean curHasNext = current.hasNext();
    +                    while (!curHasNext && ids.hasNext()) {
    +                        if (curPartition != null) {
    +                            unpin(curPartition.getId());
    +                        }
    +                        curPartition = getPinnedPartition(ids.next());
    +                        iteratorPins.add(curPartition.getId());
    +                        if (curPartition != null) {
    --- End diff --
    
    The line above will throw an NPE if this is null


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---