You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by StefanRRichter <gi...@git.apache.org> on 2018/05/23 13:43:47 UTC

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

GitHub user StefanRRichter opened a pull request:

    https://github.com/apache/flink/pull/6062

    [FLINK-9423][state] Implement efficient deletes for heap-based timer …

    …service.
    
    ## What is the purpose of the change
    
    This PR introduces `InternalTimerHeap`, as data structure of in-flight timers that are stored in memory. This structure is a combination of the previous `PriorityQueue` and `HashSet` that have previously been used by the `HeapInternalTimerService`, enhanced by a support for faster deletion of timers.
    
    
    ## Brief change log
    
    - Made `InternalTimer` and interface, `TimerHeapInternalTimer` is currently the only implementation. The interface is in place to hide management methods like `setTimerHeapIndex(...)` from code that deals with the net information of timers.
    
    - Introduced `InternalTimerHeap` structure and replaced PQ+set in `HeapInternalTimerService`.
    
    
    ## Verifying this change
    
    Added the test `InternalTimerHeapTest` for `InternalTimerHeap`. Everything else is covered by existing tests.
    
    ## Does this pull request potentially affect one of the following parts:
    
      - Dependencies (does it add or upgrade a dependency): (no)
      - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (no)
      - The serializers: (no)
      - The runtime per-record code paths (performance sensitive): (yes)
      - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: (yes)
      - The S3 file system connector: (no)
    
    ## Documentation
    
      - Does this pull request introduce a new feature? (no)
      - If yes, how is the feature documented? (not applicable)


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

    $ git pull https://github.com/StefanRRichter/flink timer-effcient-deletes

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

    https://github.com/apache/flink/pull/6062.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 #6062
    
----
commit fffc7ff6750509de2c97ea7ed44d2404669acd35
Author: Stefan Richter <s....@...>
Date:   2018-05-08T14:00:55Z

    [FLINK-9423][state] Implement efficient deletes for heap-based timer service.

----


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r191730978
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TimerHeapInternalTimer.java ---
    @@ -0,0 +1,246 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.annotation.Internal;
    +import org.apache.flink.api.common.typeutils.CompatibilityResult;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
    +import org.apache.flink.api.common.typeutils.base.LongSerializer;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.core.memory.DataOutputView;
    +
    +import java.io.IOException;
    +
    +/**
    + * Implementation of {@link InternalTimer} for the {@link InternalTimerHeap}.
    + *
    + * @param <K> Type of the keys to which timers are scoped.
    + * @param <N> Type of the namespace to which timers are scoped.
    + */
    +@Internal
    +public final class TimerHeapInternalTimer<K, N> implements InternalTimer<K, N> {
    +
    +	/** The index that indicates that a tracked internal timer is not tracked. */
    +	private static final int NOT_MANAGED_BY_TIMER_QUEUE_INDEX = Integer.MIN_VALUE;
    +
    +	private final long timestamp;
    +
    +	private final K key;
    +
    +	private final N namespace;
    +
    +	/**
    +	 * This field holds the current physical index of this timer when it is managed by a timer heap so that we can
    +	 * support fast deletes.
    +	 */
    +	private transient int timerHeapIndex;
    --- End diff --
    
    I like to do this to communicate that this should not be considered for serialization, even if that means for custom serializers and not Java serialization.


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r190311650
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerHeap.java ---
    @@ -0,0 +1,504 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.annotation.VisibleForTesting;
    +import org.apache.flink.runtime.state.KeyGroupRange;
    +import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
    +
    +import javax.annotation.Nonnegative;
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Queue;
    +import java.util.Set;
    +
    +import static org.apache.flink.util.Preconditions.checkArgument;
    +
    +/**
    + * A heap-based priority queue for internal timers. This heap is supported by hash sets for fast contains
    + * (de-duplication) and deletes. The heap implementation is a simple binary tree stored inside an array. Element indexes
    + * in the heap array start at 1 instead of 0 to make array index computations a bit simpler in the hot methods.
    + *
    + * <p>Possible future improvements:
    + * <ul>
    + *  <li>We could also implement shrinking for the heap and the deduplication maps.</li>
    + *  <li>We could replace the deduplication maps with more efficient custom implementations. In particular, a hash set
    + * would be enough if it could return existing elements on unsuccessful adding, etc..</li>
    + * </ul>
    + *
    + * @param <K> type of the key of the internal timers managed by this priority queue.
    + * @param <N> type of the namespace of the internal timers managed by this priority queue.
    + */
    +public class InternalTimerHeap<K, N> implements Queue<TimerHeapInternalTimer<K, N>>, Set<TimerHeapInternalTimer<K, N>> {
    +
    +	/**
    +	 * A safe maximum size for arrays in the JVM.
    +	 */
    +	private static final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8;
    +
    +	/**
    +	 * Comparator for {@link TimerHeapInternalTimer}, based on the timestamp in ascending order.
    +	 */
    +	private static final Comparator<TimerHeapInternalTimer<?, ?>> COMPARATOR =
    +		(o1, o2) -> Long.compare(o1.getTimestamp(), o2.getTimestamp());
    +
    +	/**
    +	 * This array contains one hash set per key-group. The sets are used for fast de-duplication and deletes of timers.
    +	 */
    +	private final HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>>[] deduplicationMapsByKeyGroup;
    +
    +	/**
    +	 * The array that represents the heap-organized priority queue.
    +	 */
    +	private TimerHeapInternalTimer<K, N>[] queue;
    +
    +	/**
    +	 * The current size of the priority queue.
    +	 */
    +	private int size;
    +
    +	/**
    +	 * The key-group range of timers that are managed by this queue.
    +	 */
    +	private final KeyGroupRange keyGroupRange;
    +
    +	/**
    +	 * The total number of key-groups of the job.
    +	 */
    +	private final int totalNumberOfKeyGroups;
    +
    +
    +	/**
    +	 * Creates an empty {@link InternalTimerHeap} with the requested initial capacity.
    +	 *
    +	 * @param minimumCapacity the minimum and initial capacity of this priority queue.
    +	 */
    +	@SuppressWarnings("unchecked")
    +	InternalTimerHeap(
    +		@Nonnegative int minimumCapacity,
    +		@Nonnull KeyGroupRange keyGroupRange,
    +		@Nonnegative int totalNumberOfKeyGroups) {
    +
    +		this.totalNumberOfKeyGroups = totalNumberOfKeyGroups;
    +		this.keyGroupRange = keyGroupRange;
    +
    +		final int keyGroupsInLocalRange = keyGroupRange.getNumberOfKeyGroups();
    +		final int deduplicationSetSize = 1 + minimumCapacity / keyGroupsInLocalRange;
    +		this.deduplicationMapsByKeyGroup = new HashMap[keyGroupsInLocalRange];
    +		for (int i = 0; i < keyGroupsInLocalRange; ++i) {
    +			deduplicationMapsByKeyGroup[i] = new HashMap<>(deduplicationSetSize);
    +		}
    +
    +		this.queue = new TimerHeapInternalTimer[1 + minimumCapacity];
    +	}
    +
    +	/**
    +	 * @see Set#add(Object)
    +	 */
    +	@Override
    +	public boolean add(@Nonnull TimerHeapInternalTimer<K, N> timer) {
    +
    +		if (getDedupMapForKeyGroup(timer).putIfAbsent(timer, timer) == null) {
    +			final int newSize = ++this.size;
    +			checkCapacity(newSize);
    +			moveElementToIdx(timer, newSize);
    +			siftUp(newSize);
    +			return true;
    +		} else {
    +			return false;
    +		}
    +	}
    +
    +	/**
    +	 * This behaves like {@link #add(TimerHeapInternalTimer)}.
    +	 */
    +	@Override
    +	public boolean offer(@Nonnull TimerHeapInternalTimer<K, N> k) {
    +		return add(k);
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> poll() {
    +		return size() > 0 ? removeElementAtIndex(1) : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> remove() {
    +		TimerHeapInternalTimer<K, N> pollResult = poll();
    +		if (pollResult != null) {
    +			return pollResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> peek() {
    +		return size() > 0 ? queue[1] : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> element() {
    +		TimerHeapInternalTimer<K, N> peekResult = peek();
    +		if (peekResult != null) {
    +			return peekResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Override
    +	public boolean isEmpty() {
    +		return size() == 0;
    +	}
    +
    +	@Override
    +	public boolean contains(@Nullable Object o) {
    +		return (o instanceof TimerHeapInternalTimer)
    +			&& getDedupMapForKeyGroup((TimerHeapInternalTimer<?, ?>) o).containsKey(o);
    +	}
    +
    +	@Override
    +	public boolean remove(@Nullable Object o) {
    +		if (o instanceof TimerHeapInternalTimer) {
    +			return removeInternal((TimerHeapInternalTimer<?, ?>) o);
    +		}
    +		return false;
    +	}
    +
    +	@Override
    +	public boolean addAll(@Nullable Collection<? extends TimerHeapInternalTimer<K, N>> timers) {
    +
    +		if (timers == null) {
    +			return true;
    +		}
    +
    +		if (timers.size() > queue.length) {
    +			checkCapacity(timers.size());
    +		}
    +
    +		for (TimerHeapInternalTimer<K, N> k : timers) {
    +			add(k);
    +		}
    +
    +		return true;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public Object[] toArray() {
    +		return Arrays.copyOfRange(queue, 1, size + 1);
    +	}
    +
    +	@SuppressWarnings({"unchecked", "SuspiciousSystemArraycopy"})
    +	@Nonnull
    +	@Override
    +	public <T> T[] toArray(@Nonnull T[] array) {
    +		if (array.length >= size) {
    +			System.arraycopy(queue, 1, array, 0, size);
    +			return array;
    +		}
    +		return (T[]) Arrays.copyOfRange(queue, 1, size + 1, array.getClass());
    +	}
    +
    +	@Override
    +	public boolean removeAll(@Nullable Collection<?> toRemove) {
    +
    +		if (toRemove == null) {
    +			return false;
    +		}
    +
    +		int oldSize = size();
    +		for (Object o : toRemove) {
    +			remove(o);
    +		}
    +		return size() == oldSize;
    +	}
    +
    +	/**
    +	 * Returns an iterator over the elements in this queue. The iterator
    +	 * does not return the elements in any particular order.
    +	 *
    +	 * @return an iterator over the elements in this queue.
    +	 */
    +	@Nonnull
    +	@Override
    +	public Iterator<TimerHeapInternalTimer<K, N>> iterator() {
    +		return new InternalTimerPriorityQueueIterator();
    +	}
    +
    +	@Override
    +	public boolean containsAll(@Nullable Collection<?> toCheck) {
    +
    +		if (toCheck == null) {
    +			return true;
    +		}
    +
    +		for (Object o : toCheck) {
    +			if (!contains(o)) {
    +				return false;
    +			}
    +		}
    +
    +		return true;
    +	}
    +
    +	@Nonnegative
    +	@Override
    +	public int size() {
    +		return size;
    +	}
    +
    +	@Override
    +	public void clear() {
    +
    +		Arrays.fill(queue, null);
    +		for (HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> timerHashMap :
    +			deduplicationMapsByKeyGroup) {
    +			timerHashMap.clear();
    +		}
    +		size = 0;
    +	}
    +
    +	/**
    +	 * This method is currently not implemented.
    +	 */
    +	@Override
    +	public boolean retainAll(@Nullable Collection<?> toRetain) {
    +		throw new UnsupportedOperationException();
    +	}
    +
    +	/**
    +	 * Adds a new timer with the given timestamp, key, and namespace to the heap, if an identical timer was not yet
    +	 * registered.
    +	 *
    +	 * @param timestamp the timer timestamp.
    +	 * @param key the timer key.
    +	 * @param namespace the timer namespace.
    +	 * @return true iff a new timer with given timestamp, key, and namespace was added to the heap.
    +	 */
    +	boolean scheduleTimer(long timestamp, K key, N namespace) {
    +		return add(new TimerHeapInternalTimer<>(timestamp, key, namespace));
    +	}
    +
    +	/**
    +	 * Stops timer with the given timestamp, key, and namespace by removing it from the heap, if it exists on the heap.
    +	 *
    +	 * @param timestamp the timer timestamp.
    +	 * @param key the timer key.
    +	 * @param namespace the timer namespace.
    +	 * @return true iff a timer with given timestamp, key, and namespace was found and removed from the heap.
    +	 */
    +	boolean stopTimer(long timestamp, K key, N namespace) {
    +		return removeInternal(new TimerHeapInternalTimer<>(timestamp, key, namespace));
    +	}
    +
    +	/**
    +	 * This method adds all the given timers to the heap.
    +	 */
    +	void restoreTimers(Collection<? extends InternalTimer<K, N>> toAdd) {
    +		if (toAdd == null) {
    +			return;
    +		}
    +
    +		if (toAdd.size() > queue.length) {
    +			checkCapacity(toAdd.size());
    +		}
    +
    +		for (InternalTimer<K, N> k : toAdd) {
    +			if (k instanceof TimerHeapInternalTimer) {
    +				add((TimerHeapInternalTimer<K, N>) k);
    +			} else {
    +				scheduleTimer(k.getTimestamp(), k.getKey(), k.getNamespace());
    +			}
    +		}
    +	}
    +
    +	private boolean removeInternal(TimerHeapInternalTimer<?, ?> timerToRemove) {
    +
    +		TimerHeapInternalTimer<K, N> storedTimer = getDedupMapForKeyGroup(timerToRemove).remove(timerToRemove);
    +
    +		if (storedTimer != null) {
    +			removeElementAtIndex(storedTimer.getTimerHeapIndex());
    +			return true;
    +		}
    +
    +		return false;
    +	}
    +
    +	private TimerHeapInternalTimer<K, N> removeElementAtIndex(int removeIdx) {
    +		TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		TimerHeapInternalTimer<K, N> removedValue = heap[removeIdx];
    +
    +		assert removedValue.getTimerHeapIndex() == removeIdx;
    +
    +		final int oldSize = size;
    +
    +		if (removeIdx != oldSize) {
    +			TimerHeapInternalTimer<K, N> timer = heap[oldSize];
    +			moveElementToIdx(timer, removeIdx);
    +			siftDown(removeIdx);
    +			if (heap[removeIdx] == timer) {
    +				siftUp(removeIdx);
    +			}
    +		}
    +
    +		heap[oldSize] = null;
    +		getDedupMapForKeyGroup(removedValue).remove(removedValue);
    +
    +		--size;
    +		return removedValue;
    +	}
    +
    +	private void siftUp(int idx) {
    +		final TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		TimerHeapInternalTimer<K, N> currentTimer = heap[idx];
    +		int parentIdx = idx >>> 1;
    +
    +		while (parentIdx > 0 && COMPARATOR.compare(currentTimer, heap[parentIdx]) < 0) {
    +			moveElementToIdx(heap[parentIdx], idx);
    +			idx = parentIdx;
    +			parentIdx = parentIdx >>> 1;
    +		}
    +
    +		moveElementToIdx(currentTimer, idx);
    +	}
    +
    +	private void siftDown(int idx) {
    +		final TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		final int heapSize = this.size;
    +
    +		TimerHeapInternalTimer<K, N> currentTimer = heap[idx];
    +		int firstChildIdx = idx << 1;
    +		int secondChildIdx = firstChildIdx + 1;
    +
    +		if (secondChildIdx <= heapSize && COMPARATOR.compare(heap[secondChildIdx], heap[firstChildIdx]) < 0) {
    +			firstChildIdx = secondChildIdx;
    +		}
    +
    +		while (firstChildIdx <= heapSize && COMPARATOR.compare(heap[firstChildIdx], currentTimer) < 0) {
    +			moveElementToIdx(heap[firstChildIdx], idx);
    +			idx = firstChildIdx;
    +			firstChildIdx = idx << 1;
    +			secondChildIdx = firstChildIdx + 1;
    +
    +			if (secondChildIdx <= heapSize && COMPARATOR.compare(heap[secondChildIdx], heap[firstChildIdx]) < 0) {
    +				firstChildIdx = secondChildIdx;
    +			}
    +		}
    +
    +		moveElementToIdx(currentTimer, idx);
    +	}
    +
    +	/**
    +	 * Returns an unmodifiable set of all timers in the given key-group.
    +	 */
    +	Set<InternalTimer<K, N>> getTimersForKeyGroup(@Nonnegative int keyGroupIdx) {
    +		return Collections.unmodifiableSet(getDedupMapForKeyGroup(keyGroupIdx).keySet());
    +	}
    +
    +	private HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> getDedupMapForKeyGroup(
    +		@Nonnegative int keyGroupIdx) {
    +		return deduplicationMapsByKeyGroup[globalKeyGroupToLocalIndex(keyGroupIdx)];
    +	}
    +
    +	@VisibleForTesting
    +	@SuppressWarnings("unchecked")
    +	List<Set<InternalTimer<K, N>>> getTimersByKeyGroup() {
    +		List<Set<InternalTimer<K, N>>> result = new ArrayList<>(deduplicationMapsByKeyGroup.length);
    +		for (int i = 0; i < deduplicationMapsByKeyGroup.length; ++i) {
    +			result.add(i, Collections.unmodifiableSet(deduplicationMapsByKeyGroup[i].keySet()));
    +		}
    +		return result;
    +	}
    +
    +	private void moveElementToIdx(TimerHeapInternalTimer<K, N> element, int idx) {
    +		queue[idx] = element;
    +		element.setTimerHeapIndex(idx);
    +	}
    +
    +	private HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> getDedupMapForKeyGroup(
    +		TimerHeapInternalTimer<?, ?> timer) {
    +		int keyGroup = KeyGroupRangeAssignment.assignToKeyGroup(timer.getKey(), totalNumberOfKeyGroups);
    +		return getDedupMapForKeyGroup(keyGroup);
    +	}
    +
    +	private int globalKeyGroupToLocalIndex(int keyGroup) {
    +		checkArgument(keyGroupRange.contains(keyGroup));
    +		return keyGroup - keyGroupRange.getStartKeyGroup();
    +	}
    +
    +	private void checkCapacity(int requested) {
    +		int oldArraySize = queue.length;
    +
    +		if (requested >= oldArraySize) {
    +			final int grow = (oldArraySize < 64) ? oldArraySize + 2 : oldArraySize >> 1;
    +			int newArraySize = oldArraySize + grow;
    --- End diff --
    
    It seems that we can't ensure that `newArraySize >= requested`, this means that we may need to rescale the array multi times to satisfy the given `requested`.


---

[GitHub] flink issue #6062: [FLINK-9423][state] Implement efficient deletes for heap-...

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

    https://github.com/apache/flink/pull/6062
  
    CC @aljoscha 


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r190290965
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerHeap.java ---
    @@ -0,0 +1,504 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.annotation.VisibleForTesting;
    +import org.apache.flink.runtime.state.KeyGroupRange;
    +import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
    +
    +import javax.annotation.Nonnegative;
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Queue;
    +import java.util.Set;
    +
    +import static org.apache.flink.util.Preconditions.checkArgument;
    +
    +/**
    + * A heap-based priority queue for internal timers. This heap is supported by hash sets for fast contains
    + * (de-duplication) and deletes. The heap implementation is a simple binary tree stored inside an array. Element indexes
    + * in the heap array start at 1 instead of 0 to make array index computations a bit simpler in the hot methods.
    + *
    + * <p>Possible future improvements:
    + * <ul>
    + *  <li>We could also implement shrinking for the heap and the deduplication maps.</li>
    + *  <li>We could replace the deduplication maps with more efficient custom implementations. In particular, a hash set
    + * would be enough if it could return existing elements on unsuccessful adding, etc..</li>
    + * </ul>
    + *
    + * @param <K> type of the key of the internal timers managed by this priority queue.
    + * @param <N> type of the namespace of the internal timers managed by this priority queue.
    + */
    +public class InternalTimerHeap<K, N> implements Queue<TimerHeapInternalTimer<K, N>>, Set<TimerHeapInternalTimer<K, N>> {
    +
    +	/**
    +	 * A safe maximum size for arrays in the JVM.
    +	 */
    +	private static final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8;
    +
    +	/**
    +	 * Comparator for {@link TimerHeapInternalTimer}, based on the timestamp in ascending order.
    +	 */
    +	private static final Comparator<TimerHeapInternalTimer<?, ?>> COMPARATOR =
    +		(o1, o2) -> Long.compare(o1.getTimestamp(), o2.getTimestamp());
    +
    +	/**
    +	 * This array contains one hash set per key-group. The sets are used for fast de-duplication and deletes of timers.
    +	 */
    +	private final HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>>[] deduplicationMapsByKeyGroup;
    +
    +	/**
    +	 * The array that represents the heap-organized priority queue.
    +	 */
    +	private TimerHeapInternalTimer<K, N>[] queue;
    +
    +	/**
    +	 * The current size of the priority queue.
    +	 */
    +	private int size;
    +
    +	/**
    +	 * The key-group range of timers that are managed by this queue.
    +	 */
    +	private final KeyGroupRange keyGroupRange;
    +
    +	/**
    +	 * The total number of key-groups of the job.
    +	 */
    +	private final int totalNumberOfKeyGroups;
    +
    +
    +	/**
    +	 * Creates an empty {@link InternalTimerHeap} with the requested initial capacity.
    +	 *
    +	 * @param minimumCapacity the minimum and initial capacity of this priority queue.
    +	 */
    +	@SuppressWarnings("unchecked")
    +	InternalTimerHeap(
    +		@Nonnegative int minimumCapacity,
    +		@Nonnull KeyGroupRange keyGroupRange,
    +		@Nonnegative int totalNumberOfKeyGroups) {
    +
    +		this.totalNumberOfKeyGroups = totalNumberOfKeyGroups;
    +		this.keyGroupRange = keyGroupRange;
    +
    +		final int keyGroupsInLocalRange = keyGroupRange.getNumberOfKeyGroups();
    +		final int deduplicationSetSize = 1 + minimumCapacity / keyGroupsInLocalRange;
    +		this.deduplicationMapsByKeyGroup = new HashMap[keyGroupsInLocalRange];
    +		for (int i = 0; i < keyGroupsInLocalRange; ++i) {
    +			deduplicationMapsByKeyGroup[i] = new HashMap<>(deduplicationSetSize);
    +		}
    +
    +		this.queue = new TimerHeapInternalTimer[1 + minimumCapacity];
    +	}
    +
    +	/**
    +	 * @see Set#add(Object)
    +	 */
    +	@Override
    +	public boolean add(@Nonnull TimerHeapInternalTimer<K, N> timer) {
    +
    +		if (getDedupMapForKeyGroup(timer).putIfAbsent(timer, timer) == null) {
    +			final int newSize = ++this.size;
    +			checkCapacity(newSize);
    +			moveElementToIdx(timer, newSize);
    +			siftUp(newSize);
    +			return true;
    +		} else {
    +			return false;
    +		}
    +	}
    +
    +	/**
    +	 * This behaves like {@link #add(TimerHeapInternalTimer)}.
    +	 */
    +	@Override
    +	public boolean offer(@Nonnull TimerHeapInternalTimer<K, N> k) {
    +		return add(k);
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> poll() {
    +		return size() > 0 ? removeElementAtIndex(1) : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> remove() {
    +		TimerHeapInternalTimer<K, N> pollResult = poll();
    +		if (pollResult != null) {
    +			return pollResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> peek() {
    +		return size() > 0 ? queue[1] : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> element() {
    +		TimerHeapInternalTimer<K, N> peekResult = peek();
    +		if (peekResult != null) {
    +			return peekResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Override
    +	public boolean isEmpty() {
    +		return size() == 0;
    +	}
    +
    +	@Override
    +	public boolean contains(@Nullable Object o) {
    +		return (o instanceof TimerHeapInternalTimer)
    +			&& getDedupMapForKeyGroup((TimerHeapInternalTimer<?, ?>) o).containsKey(o);
    +	}
    +
    +	@Override
    +	public boolean remove(@Nullable Object o) {
    +		if (o instanceof TimerHeapInternalTimer) {
    +			return removeInternal((TimerHeapInternalTimer<?, ?>) o);
    +		}
    +		return false;
    +	}
    +
    +	@Override
    +	public boolean addAll(@Nullable Collection<? extends TimerHeapInternalTimer<K, N>> timers) {
    +
    +		if (timers == null) {
    +			return true;
    +		}
    +
    +		if (timers.size() > queue.length) {
    +			checkCapacity(timers.size());
    +		}
    +
    +		for (TimerHeapInternalTimer<K, N> k : timers) {
    +			add(k);
    +		}
    +
    +		return true;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public Object[] toArray() {
    +		return Arrays.copyOfRange(queue, 1, size + 1);
    +	}
    +
    +	@SuppressWarnings({"unchecked", "SuspiciousSystemArraycopy"})
    +	@Nonnull
    +	@Override
    +	public <T> T[] toArray(@Nonnull T[] array) {
    +		if (array.length >= size) {
    +			System.arraycopy(queue, 1, array, 0, size);
    +			return array;
    +		}
    +		return (T[]) Arrays.copyOfRange(queue, 1, size + 1, array.getClass());
    +	}
    +
    +	@Override
    +	public boolean removeAll(@Nullable Collection<?> toRemove) {
    +
    +		if (toRemove == null) {
    +			return false;
    +		}
    +
    +		int oldSize = size();
    +		for (Object o : toRemove) {
    +			remove(o);
    +		}
    +		return size() == oldSize;
    +	}
    +
    +	/**
    +	 * Returns an iterator over the elements in this queue. The iterator
    +	 * does not return the elements in any particular order.
    +	 *
    +	 * @return an iterator over the elements in this queue.
    +	 */
    +	@Nonnull
    +	@Override
    +	public Iterator<TimerHeapInternalTimer<K, N>> iterator() {
    +		return new InternalTimerPriorityQueueIterator();
    +	}
    +
    +	@Override
    +	public boolean containsAll(@Nullable Collection<?> toCheck) {
    +
    +		if (toCheck == null) {
    +			return true;
    +		}
    +
    +		for (Object o : toCheck) {
    +			if (!contains(o)) {
    +				return false;
    +			}
    +		}
    +
    +		return true;
    +	}
    +
    +	@Nonnegative
    +	@Override
    +	public int size() {
    +		return size;
    +	}
    +
    +	@Override
    +	public void clear() {
    +
    +		Arrays.fill(queue, null);
    +		for (HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> timerHashMap :
    +			deduplicationMapsByKeyGroup) {
    +			timerHashMap.clear();
    +		}
    +		size = 0;
    +	}
    +
    +	/**
    +	 * This method is currently not implemented.
    +	 */
    +	@Override
    +	public boolean retainAll(@Nullable Collection<?> toRetain) {
    +		throw new UnsupportedOperationException();
    +	}
    +
    +	/**
    +	 * Adds a new timer with the given timestamp, key, and namespace to the heap, if an identical timer was not yet
    +	 * registered.
    +	 *
    +	 * @param timestamp the timer timestamp.
    +	 * @param key the timer key.
    +	 * @param namespace the timer namespace.
    +	 * @return true iff a new timer with given timestamp, key, and namespace was added to the heap.
    +	 */
    +	boolean scheduleTimer(long timestamp, K key, N namespace) {
    +		return add(new TimerHeapInternalTimer<>(timestamp, key, namespace));
    +	}
    +
    +	/**
    +	 * Stops timer with the given timestamp, key, and namespace by removing it from the heap, if it exists on the heap.
    +	 *
    +	 * @param timestamp the timer timestamp.
    +	 * @param key the timer key.
    +	 * @param namespace the timer namespace.
    +	 * @return true iff a timer with given timestamp, key, and namespace was found and removed from the heap.
    +	 */
    +	boolean stopTimer(long timestamp, K key, N namespace) {
    +		return removeInternal(new TimerHeapInternalTimer<>(timestamp, key, namespace));
    +	}
    +
    +	/**
    +	 * This method adds all the given timers to the heap.
    +	 */
    +	void restoreTimers(Collection<? extends InternalTimer<K, N>> toAdd) {
    +		if (toAdd == null) {
    +			return;
    +		}
    +
    +		if (toAdd.size() > queue.length) {
    +			checkCapacity(toAdd.size());
    +		}
    +
    +		for (InternalTimer<K, N> k : toAdd) {
    +			if (k instanceof TimerHeapInternalTimer) {
    +				add((TimerHeapInternalTimer<K, N>) k);
    +			} else {
    +				scheduleTimer(k.getTimestamp(), k.getKey(), k.getNamespace());
    +			}
    +		}
    +	}
    +
    +	private boolean removeInternal(TimerHeapInternalTimer<?, ?> timerToRemove) {
    +
    +		TimerHeapInternalTimer<K, N> storedTimer = getDedupMapForKeyGroup(timerToRemove).remove(timerToRemove);
    +
    +		if (storedTimer != null) {
    +			removeElementAtIndex(storedTimer.getTimerHeapIndex());
    +			return true;
    +		}
    +
    +		return false;
    +	}
    +
    +	private TimerHeapInternalTimer<K, N> removeElementAtIndex(int removeIdx) {
    +		TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		TimerHeapInternalTimer<K, N> removedValue = heap[removeIdx];
    +
    +		assert removedValue.getTimerHeapIndex() == removeIdx;
    +
    +		final int oldSize = size;
    +
    +		if (removeIdx != oldSize) {
    +			TimerHeapInternalTimer<K, N> timer = heap[oldSize];
    +			moveElementToIdx(timer, removeIdx);
    +			siftDown(removeIdx);
    +			if (heap[removeIdx] == timer) {
    +				siftUp(removeIdx);
    +			}
    +		}
    +
    +		heap[oldSize] = null;
    +		getDedupMapForKeyGroup(removedValue).remove(removedValue);
    +
    +		--size;
    +		return removedValue;
    +	}
    +
    +	private void siftUp(int idx) {
    +		final TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		TimerHeapInternalTimer<K, N> currentTimer = heap[idx];
    +		int parentIdx = idx >>> 1;
    +
    +		while (parentIdx > 0 && COMPARATOR.compare(currentTimer, heap[parentIdx]) < 0) {
    +			moveElementToIdx(heap[parentIdx], idx);
    +			idx = parentIdx;
    +			parentIdx = parentIdx >>> 1;
    +		}
    +
    +		moveElementToIdx(currentTimer, idx);
    +	}
    +
    +	private void siftDown(int idx) {
    +		final TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		final int heapSize = this.size;
    +
    +		TimerHeapInternalTimer<K, N> currentTimer = heap[idx];
    +		int firstChildIdx = idx << 1;
    +		int secondChildIdx = firstChildIdx + 1;
    +
    +		if (secondChildIdx <= heapSize && COMPARATOR.compare(heap[secondChildIdx], heap[firstChildIdx]) < 0) {
    +			firstChildIdx = secondChildIdx;
    +		}
    +
    +		while (firstChildIdx <= heapSize && COMPARATOR.compare(heap[firstChildIdx], currentTimer) < 0) {
    +			moveElementToIdx(heap[firstChildIdx], idx);
    +			idx = firstChildIdx;
    +			firstChildIdx = idx << 1;
    +			secondChildIdx = firstChildIdx + 1;
    +
    +			if (secondChildIdx <= heapSize && COMPARATOR.compare(heap[secondChildIdx], heap[firstChildIdx]) < 0) {
    +				firstChildIdx = secondChildIdx;
    +			}
    +		}
    +
    +		moveElementToIdx(currentTimer, idx);
    +	}
    +
    +	/**
    +	 * Returns an unmodifiable set of all timers in the given key-group.
    +	 */
    +	Set<InternalTimer<K, N>> getTimersForKeyGroup(@Nonnegative int keyGroupIdx) {
    +		return Collections.unmodifiableSet(getDedupMapForKeyGroup(keyGroupIdx).keySet());
    +	}
    +
    +	private HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> getDedupMapForKeyGroup(
    +		@Nonnegative int keyGroupIdx) {
    +		return deduplicationMapsByKeyGroup[globalKeyGroupToLocalIndex(keyGroupIdx)];
    +	}
    +
    +	@VisibleForTesting
    +	@SuppressWarnings("unchecked")
    +	List<Set<InternalTimer<K, N>>> getTimersByKeyGroup() {
    +		List<Set<InternalTimer<K, N>>> result = new ArrayList<>(deduplicationMapsByKeyGroup.length);
    +		for (int i = 0; i < deduplicationMapsByKeyGroup.length; ++i) {
    +			result.add(i, Collections.unmodifiableSet(deduplicationMapsByKeyGroup[i].keySet()));
    +		}
    +		return result;
    +	}
    +
    +	private void moveElementToIdx(TimerHeapInternalTimer<K, N> element, int idx) {
    +		queue[idx] = element;
    +		element.setTimerHeapIndex(idx);
    +	}
    +
    +	private HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> getDedupMapForKeyGroup(
    +		TimerHeapInternalTimer<?, ?> timer) {
    +		int keyGroup = KeyGroupRangeAssignment.assignToKeyGroup(timer.getKey(), totalNumberOfKeyGroups);
    +		return getDedupMapForKeyGroup(keyGroup);
    +	}
    +
    +	private int globalKeyGroupToLocalIndex(int keyGroup) {
    +		checkArgument(keyGroupRange.contains(keyGroup));
    +		return keyGroup - keyGroupRange.getStartKeyGroup();
    +	}
    +
    +	private void checkCapacity(int requested) {
    +		int oldArraySize = queue.length;
    +
    +		if (requested >= oldArraySize) {
    --- End diff --
    
    Should this be `if (requested + size >= oldArraySize)`, and I feel that the `if` check outside(the places where we call this method) seems a bit redundant with this `if` check...


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r191450664
  
    --- Diff: flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/InternalTimerHeapTest.java ---
    @@ -0,0 +1,470 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.runtime.state.KeyGroupRange;
    +import org.apache.flink.runtime.state.VoidNamespace;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.junit.Assert;
    +import org.junit.Test;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Set;
    +import java.util.concurrent.ThreadLocalRandom;
    +import java.util.function.BiFunction;
    +import java.util.function.Function;
    +
    +/**
    + * Tests for {@link InternalTimerHeap}.
    + */
    +public class InternalTimerHeapTest {
    +
    +	private static final KeyGroupRange KEY_GROUP_RANGE = new KeyGroupRange(0, 1);
    +
    +	private static void insertRandomTimers(
    +		InternalTimerHeap<Integer, VoidNamespace> timerPriorityQueue,
    +		int count) {
    +		insertRandomTimers(timerPriorityQueue, null, count);
    +	}
    +
    +	private static void insertRandomTimers(
    +		InternalTimerHeap<Integer, VoidNamespace> timerPriorityQueue,
    +		Set<TimerHeapInternalTimer<Integer, VoidNamespace>> checkSet,
    +		int count) {
    +
    +		ThreadLocalRandom localRandom = ThreadLocalRandom.current();
    +
    +		for (int i = 0; i < count; ++i) {
    +			TimerHeapInternalTimer<Integer, VoidNamespace> timer =
    +				new TimerHeapInternalTimer<>(localRandom.nextLong(), i, VoidNamespace.INSTANCE);
    +			if (checkSet != null) {
    +				Preconditions.checkState(checkSet.add(timer));
    +			}
    +			Assert.assertTrue(timerPriorityQueue.add(timer));
    +		}
    +	}
    +
    +	private static InternalTimerHeap<Integer, VoidNamespace> newPriorityQueue(int initialCapacity) {
    +		return new InternalTimerHeap<>(
    +			initialCapacity,
    +			KEY_GROUP_RANGE,
    +			KEY_GROUP_RANGE.getNumberOfKeyGroups());
    +	}
    +
    +	@Test
    +	public void testCombined() {
    +		final int initialCapacity = 4;
    +		final int testSize = 1000;
    +		InternalTimerHeap<Integer, VoidNamespace> timerPriorityQueue = newPriorityQueue(initialCapacity);
    +		HashSet<TimerHeapInternalTimer<Integer, VoidNamespace>> checkSet = new HashSet<>(testSize);
    +
    +		insertRandomTimers(timerPriorityQueue, checkSet, testSize);
    +
    +		long lastTimestamp = Long.MIN_VALUE;
    +		int lastSize = timerPriorityQueue.size();
    +		Assert.assertEquals(testSize, lastSize);
    +		TimerHeapInternalTimer<Integer, VoidNamespace> timer;
    +		while ((timer = timerPriorityQueue.peek()) != null) {
    +			Assert.assertFalse(timerPriorityQueue.isEmpty());
    +			Assert.assertEquals(lastSize, timerPriorityQueue.size());
    +			Assert.assertEquals(timer, timerPriorityQueue.poll());
    +			Assert.assertTrue(checkSet.remove(timer));
    +			Assert.assertTrue(timer.getTimestamp() >= lastTimestamp);
    +			lastTimestamp = timer.getTimestamp();
    +			--lastSize;
    +		}
    +
    +		Assert.assertTrue(timerPriorityQueue.isEmpty());
    +		Assert.assertEquals(0, timerPriorityQueue.size());
    +		Assert.assertEquals(0, checkSet.size());
    +	}
    +
    +	@Test
    +	public void testAdd() {
    +		testAddOfferCommon(InternalTimerHeap<Integer, VoidNamespace>::add);
    +	}
    +
    +	@Test
    +	public void testOffer() {
    +		testAddOfferCommon(InternalTimerHeap<Integer, VoidNamespace>::offer);
    +	}
    +
    +	@Test
    +	public void testRemove() {
    +		InternalTimerHeap<Integer, VoidNamespace> timerPriorityQueue = newPriorityQueue(3);
    +
    +		try {
    +			timerPriorityQueue.remove();
    +			Assert.fail();
    +		} catch (NoSuchElementException ignore) {
    +		}
    +
    +		testRemovePollCommon(timerPriorityQueue, InternalTimerHeap::remove);
    +
    +		try {
    +			timerPriorityQueue.remove();
    +			Assert.fail();
    +		} catch (NoSuchElementException ignore) {
    +		}
    +	}
    +
    +	@Test
    +	public void testPoll() {
    +		InternalTimerHeap<Integer, VoidNamespace> timerPriorityQueue = newPriorityQueue(3);
    +
    +		Assert.assertNull(timerPriorityQueue.poll());
    +
    +		testRemovePollCommon(timerPriorityQueue, InternalTimerHeap::poll);
    +
    +		Assert.assertNull(timerPriorityQueue.poll());
    +	}
    +
    +	@Test
    +	public void testRemoveTimer() {
    +
    +		InternalTimerHeap<Integer, VoidNamespace> timerPriorityQueue = newPriorityQueue(3);
    +
    +		final int testSize = 345;
    +		HashSet<TimerHeapInternalTimer<Integer, VoidNamespace>> checkSet = new HashSet<>(testSize);
    +
    +		insertRandomTimers(timerPriorityQueue, checkSet, testSize);
    +
    +		// check that the whole set is still in order
    +		while (!checkSet.isEmpty()) {
    +
    +			Iterator<TimerHeapInternalTimer<Integer, VoidNamespace>> iterator = checkSet.iterator();
    +			InternalTimer<Integer, VoidNamespace> timer = iterator.next();
    +			iterator.remove();
    +			Assert.assertTrue(timerPriorityQueue.remove(timer));
    +			Assert.assertEquals(checkSet.size(), timerPriorityQueue.size());
    +
    +			long lastTimestamp = Long.MIN_VALUE;
    +
    +			while ((timer = timerPriorityQueue.poll()) != null) {
    +				Assert.assertTrue(timer.getTimestamp() >= lastTimestamp);
    +				lastTimestamp = timer.getTimestamp();
    +			}
    +
    +			Assert.assertTrue(timerPriorityQueue.isEmpty());
    +
    +			timerPriorityQueue.addAll(checkSet);
    +		}
    +	}
    +
    +	@Test
    +	public void testPeek() {
    +		InternalTimerHeap<Integer, VoidNamespace> timerPriorityQueue =
    +			newPriorityQueue(1);
    +
    +		Assert.assertNull(timerPriorityQueue.peek());
    +
    +		testPeekElementCommon(timerPriorityQueue, InternalTimerHeap::peek);
    +	}
    +
    +	@Test
    +	public void testElement() {
    +		InternalTimerHeap<Integer, VoidNamespace> timerPriorityQueue =
    +			newPriorityQueue(1);
    +		try {
    +			timerPriorityQueue.element();
    +			Assert.fail();
    +		} catch (NoSuchElementException ignore) {
    +		}
    +		testPeekElementCommon(timerPriorityQueue, InternalTimerHeap::element);
    +	}
    +
    +	@Test
    +	public void testIsEmpty() {
    +		InternalTimerHeap<Integer, VoidNamespace> timerPriorityQueue =
    +			newPriorityQueue(1);
    +
    +		Assert.assertTrue(timerPriorityQueue.isEmpty());
    +
    +		TimerHeapInternalTimer<Integer, VoidNamespace> timer =
    +			new TimerHeapInternalTimer<>(42L, 4711, VoidNamespace.INSTANCE);
    +
    +		timerPriorityQueue.add(timer);
    +		Assert.assertFalse(timerPriorityQueue.isEmpty());
    +
    +		timerPriorityQueue.poll();
    +		Assert.assertTrue(timerPriorityQueue.isEmpty());
    +	}
    +
    +	@Test
    +	public void testContains() {
    +		InternalTimerHeap<Integer, VoidNamespace> timerPriorityQueue =
    +			newPriorityQueue(1);
    +		TimerHeapInternalTimer<Integer, VoidNamespace> timer =
    +			new TimerHeapInternalTimer<>(42L, 4711, VoidNamespace.INSTANCE);
    +
    +		Assert.assertFalse(timerPriorityQueue.contains(timer));
    +
    +		timerPriorityQueue.add(timer);
    +		Assert.assertTrue(timerPriorityQueue.contains(timer));
    +
    +		timerPriorityQueue.remove(timer);
    +		Assert.assertFalse(timerPriorityQueue.contains(timer));
    +	}
    +
    +	@Test
    +	public void testAddAll() {
    +		final int testSize = 10;
    +		HashSet<TimerHeapInternalTimer<Integer, VoidNamespace>> timerSet = new HashSet<>(testSize);
    +		for (int i = 0; i < testSize; ++i) {
    +			timerSet.add(new TimerHeapInternalTimer<>(i, i, VoidNamespace.INSTANCE));
    +		}
    +
    +		List<TimerHeapInternalTimer<Integer, VoidNamespace>> twoTimesTimerSet = new ArrayList<>(timerSet.size() * 2);
    +		twoTimesTimerSet.addAll(timerSet);
    +		twoTimesTimerSet.addAll(timerSet);
    +
    +		InternalTimerHeap<Integer, VoidNamespace> timerPriorityQueue =
    +			newPriorityQueue(1);
    +
    +		Assert.assertTrue(timerPriorityQueue.addAll(twoTimesTimerSet));
    +		Assert.assertFalse(timerPriorityQueue.addAll(twoTimesTimerSet));
    +
    +		Assert.assertEquals(timerSet.size(), timerPriorityQueue.size());
    +
    +		for (TimerHeapInternalTimer<Integer, VoidNamespace> timer : timerPriorityQueue) {
    +			Assert.assertTrue(timerSet.remove(timer));
    +		}
    +
    +		Assert.assertTrue(timerSet.isEmpty());
    +	}
    +
    +	@Test
    +	public void testToArray() {
    +		final int testSize = 10;
    +		HashSet<TimerHeapInternalTimer<Integer, VoidNamespace>> checkSet = new HashSet<>(testSize);
    +		InternalTimerHeap<Integer, VoidNamespace> timerPriorityQueue =
    +			newPriorityQueue(1);
    +
    +		Assert.assertEquals(0, timerPriorityQueue.toArray().length);
    +
    +		insertRandomTimers(timerPriorityQueue, checkSet, testSize);
    +
    +		Object[] toArray = timerPriorityQueue.toArray();
    +		Assert.assertEquals(timerPriorityQueue.size(), toArray.length);
    +
    +		for (Object o : toArray) {
    +			if (o instanceof TimerHeapInternalTimer) {
    +				Assert.assertTrue(checkSet.remove(o));
    +			}
    +		}
    +
    +		Assert.assertTrue(checkSet.isEmpty());
    +	}
    +
    +	@Test
    +	public void testToArrayGeneric() {
    +		final int testSize = 10;
    +		HashSet<TimerHeapInternalTimer<Integer, VoidNamespace>> checkSet = new HashSet<>(testSize);
    +		InternalTimerHeap<Integer, VoidNamespace> timerPriorityQueue =
    +			newPriorityQueue(1);
    +
    +		Assert.assertEquals(0, timerPriorityQueue.toArray().length);
    +
    +		insertRandomTimers(timerPriorityQueue, checkSet, testSize);
    +
    +		int[] arraySizes = new int[]{0, timerPriorityQueue.size(), timerPriorityQueue.size() + 2};
    +
    +		for (int arraySize : arraySizes) {
    +			HashSet<TimerHeapInternalTimer<Integer, VoidNamespace>> checkSetCopy = new HashSet<>(checkSet);
    +			TimerHeapInternalTimer[] toArray = timerPriorityQueue.toArray(new TimerHeapInternalTimer[arraySize]);
    +			Assert.assertEquals(Math.max(timerPriorityQueue.size(), arraySize), toArray.length);
    +
    +			for (TimerHeapInternalTimer o : toArray) {
    +				if (o != null) {
    +					Assert.assertTrue(checkSetCopy.remove(o));
    +				}
    +			}
    +			Assert.assertTrue(checkSetCopy.isEmpty());
    +		}
    +	}
    +
    +	@Test
    +	public void testRemoveAll() {
    +		final int testSize = 10;
    +		InternalTimerHeap<Integer, VoidNamespace> timerPriorityQueue =
    +			newPriorityQueue(1);
    +		timerPriorityQueue.removeAll(Collections.<TimerHeapInternalTimer<Integer, VoidNamespace>>emptyList());
    +		HashSet<TimerHeapInternalTimer<Integer, VoidNamespace>> checkSet = new HashSet<>(testSize);
    +		insertRandomTimers(timerPriorityQueue, checkSet, testSize);
    +
    +	}
    +
    +	@Test
    +	public void testRetainAll() {
    +		InternalTimerHeap<Integer, VoidNamespace> timerPriorityQueue =
    +			newPriorityQueue(1);
    +		try {
    +			timerPriorityQueue.retainAll(Collections.emptyList());
    +			Assert.fail();
    +		} catch (UnsupportedOperationException ignore) {
    +		}
    +	}
    +
    +	@Test
    +	public void testIterator() {
    +		InternalTimerHeap<Integer, VoidNamespace> timerPriorityQueue =
    +			newPriorityQueue(1);
    +
    +		// test empty iterator
    +		Iterator<TimerHeapInternalTimer<Integer, VoidNamespace>> iterator = timerPriorityQueue.iterator();
    +		Assert.assertFalse(iterator.hasNext());
    +		try {
    +			iterator.next();
    +			Assert.fail();
    +		} catch (NoSuchElementException ignore) {
    +		}
    +
    +		// iterate some data
    +		final int testSize = 10;
    +		HashSet<TimerHeapInternalTimer<Integer, VoidNamespace>> checkSet = new HashSet<>(testSize);
    +		insertRandomTimers(timerPriorityQueue, checkSet, testSize);
    +		iterator = timerPriorityQueue.iterator();
    +		while (iterator.hasNext()) {
    +			Assert.assertTrue(checkSet.remove(iterator.next()));
    +		}
    +		Assert.assertTrue(checkSet.isEmpty());
    +
    +		// test remove is not supported
    +		try {
    +			iterator.remove();
    +			Assert.fail();
    +		} catch (UnsupportedOperationException ignore) {
    +		}
    +	}
    +
    +	@Test
    +	public void testClear() {
    +		InternalTimerHeap<Integer, VoidNamespace> timerPriorityQueue =
    +			newPriorityQueue(1);
    +
    +		int count = 10;
    +		insertRandomTimers(timerPriorityQueue, count);
    +		Assert.assertEquals(count, timerPriorityQueue.size());
    +		timerPriorityQueue.clear();
    +		Assert.assertEquals(0, timerPriorityQueue.size());
    +	}
    +
    +	@Test
    +	public void testScheduleTimer() {
    +		InternalTimerHeap<Integer, VoidNamespace> timerPriorityQueue =
    +			newPriorityQueue(1);
    +
    +		final long timestamp = 42L;
    +		final Integer key = 4711;
    +		Assert.assertTrue(timerPriorityQueue.scheduleTimer(timestamp, key, VoidNamespace.INSTANCE));
    +		Assert.assertFalse(timerPriorityQueue.scheduleTimer(timestamp, key, VoidNamespace.INSTANCE));
    +		Assert.assertEquals(1, timerPriorityQueue.size());
    +		final InternalTimer<Integer, VoidNamespace> timer = timerPriorityQueue.remove();
    +		Assert.assertEquals(timestamp, timer.getTimestamp());
    +		Assert.assertEquals(key, timer.getKey());
    +		Assert.assertEquals(VoidNamespace.INSTANCE, timer.getNamespace());
    +	}
    +
    +	@Test
    +	public void testStopTimer() {
    +		InternalTimerHeap<Integer, VoidNamespace> timerPriorityQueue =
    +			newPriorityQueue(1);
    +
    +		final long timestamp = 42L;
    +		final Integer key = 4711;
    +		Assert.assertFalse(timerPriorityQueue.stopTimer(timestamp, key, VoidNamespace.INSTANCE));
    +		Assert.assertTrue(timerPriorityQueue.scheduleTimer(timestamp, key, VoidNamespace.INSTANCE));
    +		Assert.assertTrue(timerPriorityQueue.stopTimer(timestamp, key, VoidNamespace.INSTANCE));
    +		Assert.assertFalse(timerPriorityQueue.stopTimer(timestamp, key, VoidNamespace.INSTANCE));
    +		Assert.assertTrue(timerPriorityQueue.isEmpty());
    +	}
    +
    +	private void testAddOfferCommon(
    +		BiFunction<InternalTimerHeap, TimerHeapInternalTimer<Integer, VoidNamespace>, Boolean> testMethod) {
    +
    +		InternalTimerHeap<Integer, VoidNamespace> timerPriorityQueue = newPriorityQueue(1);
    +
    +		Assert.assertTrue(timerPriorityQueue.isEmpty());
    +
    +		TimerHeapInternalTimer<Integer, VoidNamespace> timer1 =
    +			new TimerHeapInternalTimer<>(42L, 4711, VoidNamespace.INSTANCE);
    +		TimerHeapInternalTimer<Integer, VoidNamespace> timer2 =
    +			new TimerHeapInternalTimer<>(43L, 4712, VoidNamespace.INSTANCE);
    +
    +		// add first timer
    +		Assert.assertTrue(testMethod.apply(timerPriorityQueue, timer1));
    +		Assert.assertEquals(1, timerPriorityQueue.size());
    +
    +		// add second timer
    +		Assert.assertTrue(testMethod.apply(timerPriorityQueue, timer2));
    +		Assert.assertEquals(2, timerPriorityQueue.size());
    +
    +		// check adding first timer again, duplicate should not be added
    +		Assert.assertFalse(testMethod.apply(timerPriorityQueue, timer1));
    +		Assert.assertEquals(2, timerPriorityQueue.size());
    +	}
    +
    +	private void testPeekElementCommon(
    +		InternalTimerHeap<Integer, VoidNamespace> timerPriorityQueue,
    +		Function<InternalTimerHeap, TimerHeapInternalTimer<Integer, VoidNamespace>> testFun) {
    +
    +		TimerHeapInternalTimer<Integer, VoidNamespace> timer1 =
    +			new TimerHeapInternalTimer<>(42L, 4711, VoidNamespace.INSTANCE);
    +		timerPriorityQueue.add(timer1);
    +
    +		Assert.assertEquals(timer1, testFun.apply(timerPriorityQueue));
    +
    +		TimerHeapInternalTimer<Integer, VoidNamespace> timer2 =
    +			new TimerHeapInternalTimer<>(43L, 4712, VoidNamespace.INSTANCE);
    +		timerPriorityQueue.add(timer2);
    +
    +		Assert.assertEquals(timer1, testFun.apply(timerPriorityQueue));
    +
    +		TimerHeapInternalTimer<Integer, VoidNamespace> timer3 =
    +			new TimerHeapInternalTimer<>(41L, 4712, VoidNamespace.INSTANCE);
    +		timerPriorityQueue.add(timer3);
    +
    +		Assert.assertEquals(timer3, testFun.apply(timerPriorityQueue));
    +		Assert.assertEquals(3, timerPriorityQueue.size());
    +	}
    +
    +	private void testRemovePollCommon(
    +		InternalTimerHeap<Integer, VoidNamespace> timerPriorityQueue,
    +		Function<InternalTimerHeap<Integer, VoidNamespace>, TimerHeapInternalTimer<Integer, VoidNamespace>> fun) {
    +
    +		final int testSize = 345;
    +		HashSet<TimerHeapInternalTimer<Integer, VoidNamespace>> checkSet = new HashSet<>(testSize);
    +		insertRandomTimers(timerPriorityQueue, checkSet, testSize);
    +
    +		long lastTimestamp = Long.MIN_VALUE;
    +		while (!timerPriorityQueue.isEmpty()) {
    +			TimerHeapInternalTimer<Integer, VoidNamespace> removed = fun.apply(timerPriorityQueue);
    +			Assert.assertTrue(checkSet.remove(removed));
    +			Assert.assertTrue(removed.getTimestamp() >= lastTimestamp);
    +			lastTimestamp = removed.getTimestamp();
    +		}
    +		Assert.assertTrue(checkSet.isEmpty());
    +	}
    --- End diff --
    
    Do we test somewhere the serialization and deserialization of the `TimerHeapInternalTimer`?


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r191440239
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TimerHeapInternalTimer.java ---
    @@ -0,0 +1,246 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.annotation.Internal;
    +import org.apache.flink.api.common.typeutils.CompatibilityResult;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
    +import org.apache.flink.api.common.typeutils.base.LongSerializer;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.core.memory.DataOutputView;
    +
    +import java.io.IOException;
    +
    +/**
    + * Implementation of {@link InternalTimer} for the {@link InternalTimerHeap}.
    + *
    + * @param <K> Type of the keys to which timers are scoped.
    + * @param <N> Type of the namespace to which timers are scoped.
    + */
    +@Internal
    +public final class TimerHeapInternalTimer<K, N> implements InternalTimer<K, N> {
    +
    +	/** The index that indicates that a tracked internal timer is not tracked. */
    +	private static final int NOT_MANAGED_BY_TIMER_QUEUE_INDEX = Integer.MIN_VALUE;
    +
    +	private final long timestamp;
    +
    +	private final K key;
    +
    +	private final N namespace;
    --- End diff --
    
    `null`?


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r191425505
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerHeap.java ---
    @@ -0,0 +1,511 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.annotation.VisibleForTesting;
    +import org.apache.flink.runtime.state.KeyGroupRange;
    +import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
    +
    +import javax.annotation.Nonnegative;
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Queue;
    +import java.util.Set;
    +
    +import static org.apache.flink.util.Preconditions.checkArgument;
    +
    +/**
    + * A heap-based priority queue for internal timers. This heap is supported by hash sets for fast contains
    + * (de-duplication) and deletes. The heap implementation is a simple binary tree stored inside an array. Element indexes
    + * in the heap array start at 1 instead of 0 to make array index computations a bit simpler in the hot methods.
    + *
    + * <p>Possible future improvements:
    + * <ul>
    + *  <li>We could also implement shrinking for the heap and the deduplication maps.</li>
    + *  <li>We could replace the deduplication maps with more efficient custom implementations. In particular, a hash set
    + * would be enough if it could return existing elements on unsuccessful adding, etc..</li>
    + * </ul>
    + *
    + * @param <K> type of the key of the internal timers managed by this priority queue.
    + * @param <N> type of the namespace of the internal timers managed by this priority queue.
    + */
    +public class InternalTimerHeap<K, N> implements Queue<TimerHeapInternalTimer<K, N>>, Set<TimerHeapInternalTimer<K, N>> {
    +
    +	/**
    +	 * A safe maximum size for arrays in the JVM.
    +	 */
    +	private static final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8;
    +
    +	/**
    +	 * Comparator for {@link TimerHeapInternalTimer}, based on the timestamp in ascending order.
    +	 */
    +	private static final Comparator<TimerHeapInternalTimer<?, ?>> COMPARATOR =
    +		(o1, o2) -> Long.compare(o1.getTimestamp(), o2.getTimestamp());
    +
    +	/**
    +	 * This array contains one hash set per key-group. The sets are used for fast de-duplication and deletes of timers.
    +	 */
    +	private final HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>>[] deduplicationMapsByKeyGroup;
    +
    +	/**
    +	 * The array that represents the heap-organized priority queue.
    +	 */
    +	private TimerHeapInternalTimer<K, N>[] queue;
    +
    +	/**
    +	 * The current size of the priority queue.
    +	 */
    +	private int size;
    +
    +	/**
    +	 * The key-group range of timers that are managed by this queue.
    +	 */
    +	private final KeyGroupRange keyGroupRange;
    +
    +	/**
    +	 * The total number of key-groups of the job.
    +	 */
    +	private final int totalNumberOfKeyGroups;
    +
    +
    +	/**
    +	 * Creates an empty {@link InternalTimerHeap} with the requested initial capacity.
    +	 *
    +	 * @param minimumCapacity the minimum and initial capacity of this priority queue.
    +	 */
    +	@SuppressWarnings("unchecked")
    +	InternalTimerHeap(
    +		@Nonnegative int minimumCapacity,
    +		@Nonnull KeyGroupRange keyGroupRange,
    +		@Nonnegative int totalNumberOfKeyGroups) {
    +
    +		this.totalNumberOfKeyGroups = totalNumberOfKeyGroups;
    +		this.keyGroupRange = keyGroupRange;
    +
    +		final int keyGroupsInLocalRange = keyGroupRange.getNumberOfKeyGroups();
    +		final int deduplicationSetSize = 1 + minimumCapacity / keyGroupsInLocalRange;
    +		this.deduplicationMapsByKeyGroup = new HashMap[keyGroupsInLocalRange];
    +		for (int i = 0; i < keyGroupsInLocalRange; ++i) {
    +			deduplicationMapsByKeyGroup[i] = new HashMap<>(deduplicationSetSize);
    +		}
    +
    +		this.queue = new TimerHeapInternalTimer[1 + minimumCapacity];
    +	}
    +
    +	/**
    +	 * @see Set#add(Object)
    +	 */
    +	@Override
    +	public boolean add(@Nonnull TimerHeapInternalTimer<K, N> timer) {
    +
    +		if (getDedupMapForKeyGroup(timer).putIfAbsent(timer, timer) == null) {
    +			final int newSize = ++this.size;
    +			growIfRequired(newSize);
    +			moveElementToIdx(timer, newSize);
    +			siftUp(newSize);
    +			return true;
    +		} else {
    +			return false;
    +		}
    +	}
    +
    +	/**
    +	 * This behaves like {@link #add(TimerHeapInternalTimer)}.
    +	 */
    +	@Override
    +	public boolean offer(@Nonnull TimerHeapInternalTimer<K, N> k) {
    +		return add(k);
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> poll() {
    +		return size() > 0 ? removeElementAtIndex(1) : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> remove() {
    +		TimerHeapInternalTimer<K, N> pollResult = poll();
    +		if (pollResult != null) {
    +			return pollResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> peek() {
    +		return size() > 0 ? queue[1] : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> element() {
    +		TimerHeapInternalTimer<K, N> peekResult = peek();
    +		if (peekResult != null) {
    +			return peekResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Override
    +	public boolean isEmpty() {
    +		return size() == 0;
    +	}
    +
    +	@Override
    +	public boolean contains(@Nullable Object toCheck) {
    +		return (toCheck instanceof TimerHeapInternalTimer)
    +			&& getDedupMapForKeyGroup((TimerHeapInternalTimer<?, ?>) toCheck).containsKey(toCheck);
    +	}
    +
    +	@Override
    +	public boolean remove(@Nullable Object toRemove) {
    +		if (toRemove instanceof TimerHeapInternalTimer) {
    +			return removeInternal((TimerHeapInternalTimer<?, ?>) toRemove);
    +		}
    +		return false;
    +	}
    +
    +	@Override
    +	public boolean addAll(@Nullable Collection<? extends TimerHeapInternalTimer<K, N>> timers) {
    +
    +		if (timers == null) {
    +			return true;
    +		}
    +
    +		final int oldSize = size();
    +		resizeForBulkLoad(oldSize + timers.size());
    +
    +		for (TimerHeapInternalTimer<K, N> k : timers) {
    +			add(k);
    +		}
    +
    +		return oldSize != size();
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public Object[] toArray() {
    +		return Arrays.copyOfRange(queue, 1, size + 1);
    +	}
    +
    +	@SuppressWarnings({"unchecked", "SuspiciousSystemArraycopy"})
    +	@Nonnull
    +	@Override
    +	public <T> T[] toArray(@Nonnull T[] array) {
    +		if (array.length >= size) {
    +			System.arraycopy(queue, 1, array, 0, size);
    +			return array;
    +		}
    +		return (T[]) Arrays.copyOfRange(queue, 1, size + 1, array.getClass());
    +	}
    +
    +	@Override
    +	public boolean removeAll(@Nullable Collection<?> toRemove) {
    +
    +		if (toRemove == null) {
    +			return false;
    +		}
    +
    +		int oldSize = size();
    +		for (Object o : toRemove) {
    +			remove(o);
    +		}
    +		return oldSize != size();
    +	}
    +
    +	/**
    +	 * Returns an iterator over the elements in this queue. The iterator
    +	 * does not return the elements in any particular order.
    +	 *
    +	 * @return an iterator over the elements in this queue.
    +	 */
    +	@Nonnull
    +	@Override
    +	public Iterator<TimerHeapInternalTimer<K, N>> iterator() {
    +		return new InternalTimerPriorityQueueIterator();
    +	}
    +
    +	@Override
    +	public boolean containsAll(@Nullable Collection<?> toCheck) {
    +
    +		if (toCheck == null) {
    +			return true;
    +		}
    +
    +		for (Object o : toCheck) {
    +			if (!contains(o)) {
    +				return false;
    +			}
    +		}
    +
    +		return true;
    +	}
    +
    +	@Nonnegative
    +	@Override
    +	public int size() {
    +		return size;
    +	}
    +
    +	@Override
    +	public void clear() {
    +
    +		Arrays.fill(queue, null);
    +		for (HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> timerHashMap :
    +			deduplicationMapsByKeyGroup) {
    +			timerHashMap.clear();
    +		}
    +		size = 0;
    +	}
    +
    +	/**
    +	 * This method is currently not implemented.
    +	 */
    +	@Override
    +	public boolean retainAll(@Nullable Collection<?> toRetain) {
    +		throw new UnsupportedOperationException();
    +	}
    +
    +	/**
    +	 * Adds a new timer with the given timestamp, key, and namespace to the heap, if an identical timer was not yet
    +	 * registered.
    +	 *
    +	 * @param timestamp the timer timestamp.
    +	 * @param key the timer key.
    +	 * @param namespace the timer namespace.
    +	 * @return true iff a new timer with given timestamp, key, and namespace was added to the heap.
    +	 */
    +	boolean scheduleTimer(long timestamp, K key, N namespace) {
    +		return add(new TimerHeapInternalTimer<>(timestamp, key, namespace));
    +	}
    +
    +	/**
    +	 * Stops timer with the given timestamp, key, and namespace by removing it from the heap, if it exists on the heap.
    +	 *
    +	 * @param timestamp the timer timestamp.
    +	 * @param key the timer key.
    +	 * @param namespace the timer namespace.
    +	 * @return true iff a timer with given timestamp, key, and namespace was found and removed from the heap.
    +	 */
    +	boolean stopTimer(long timestamp, K key, N namespace) {
    +		return removeInternal(new TimerHeapInternalTimer<>(timestamp, key, namespace));
    +	}
    +
    +	/**
    +	 * This method adds all the given timers to the heap.
    +	 */
    +	void addRestoredTimers(Collection<? extends InternalTimer<K, N>> restoredTimers) {
    +
    +		if (restoredTimers == null) {
    +			return;
    +		}
    +
    +		resizeForBulkLoad(restoredTimers.size());
    +
    +		for (InternalTimer<K, N> timer : restoredTimers) {
    +			if (timer instanceof TimerHeapInternalTimer) {
    +				add((TimerHeapInternalTimer<K, N>) timer);
    +			} else {
    +				scheduleTimer(timer.getTimestamp(), timer.getKey(), timer.getNamespace());
    +			}
    +		}
    +	}
    +
    +	private boolean removeInternal(TimerHeapInternalTimer<?, ?> timerToRemove) {
    +
    +		TimerHeapInternalTimer<K, N> storedTimer = getDedupMapForKeyGroup(timerToRemove).remove(timerToRemove);
    +
    +		if (storedTimer != null) {
    +			removeElementAtIndex(storedTimer.getTimerHeapIndex());
    +			return true;
    +		}
    +
    +		return false;
    +	}
    +
    +	private TimerHeapInternalTimer<K, N> removeElementAtIndex(int removeIdx) {
    +		TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		TimerHeapInternalTimer<K, N> removedValue = heap[removeIdx];
    +
    +		assert removedValue.getTimerHeapIndex() == removeIdx;
    +
    +		final int oldSize = size;
    +
    +		if (removeIdx != oldSize) {
    +			TimerHeapInternalTimer<K, N> timer = heap[oldSize];
    +			moveElementToIdx(timer, removeIdx);
    +			siftDown(removeIdx);
    +			if (heap[removeIdx] == timer) {
    +				siftUp(removeIdx);
    +			}
    +		}
    +
    +		heap[oldSize] = null;
    +		getDedupMapForKeyGroup(removedValue).remove(removedValue);
    +
    +		--size;
    +		return removedValue;
    +	}
    +
    +	private void siftUp(int idx) {
    +		final TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		TimerHeapInternalTimer<K, N> currentTimer = heap[idx];
    +		int parentIdx = idx >>> 1;
    +
    +		while (parentIdx > 0 && COMPARATOR.compare(currentTimer, heap[parentIdx]) < 0) {
    +			moveElementToIdx(heap[parentIdx], idx);
    +			idx = parentIdx;
    +			parentIdx = parentIdx >>> 1;
    --- End diff --
    
    could be replaced with `parentIdx >>>= 1`


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r191447618
  
    --- Diff: flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/InternalTimerHeapTest.java ---
    @@ -0,0 +1,470 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.runtime.state.KeyGroupRange;
    +import org.apache.flink.runtime.state.VoidNamespace;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.junit.Assert;
    +import org.junit.Test;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Set;
    +import java.util.concurrent.ThreadLocalRandom;
    +import java.util.function.BiFunction;
    +import java.util.function.Function;
    +
    +/**
    + * Tests for {@link InternalTimerHeap}.
    + */
    +public class InternalTimerHeapTest {
    +
    +	private static final KeyGroupRange KEY_GROUP_RANGE = new KeyGroupRange(0, 1);
    +
    +	private static void insertRandomTimers(
    +		InternalTimerHeap<Integer, VoidNamespace> timerPriorityQueue,
    +		int count) {
    +		insertRandomTimers(timerPriorityQueue, null, count);
    +	}
    +
    +	private static void insertRandomTimers(
    +		InternalTimerHeap<Integer, VoidNamespace> timerPriorityQueue,
    +		Set<TimerHeapInternalTimer<Integer, VoidNamespace>> checkSet,
    +		int count) {
    +
    +		ThreadLocalRandom localRandom = ThreadLocalRandom.current();
    +
    +		for (int i = 0; i < count; ++i) {
    +			TimerHeapInternalTimer<Integer, VoidNamespace> timer =
    +				new TimerHeapInternalTimer<>(localRandom.nextLong(), i, VoidNamespace.INSTANCE);
    +			if (checkSet != null) {
    +				Preconditions.checkState(checkSet.add(timer));
    +			}
    +			Assert.assertTrue(timerPriorityQueue.add(timer));
    +		}
    +	}
    +
    +	private static InternalTimerHeap<Integer, VoidNamespace> newPriorityQueue(int initialCapacity) {
    +		return new InternalTimerHeap<>(
    +			initialCapacity,
    +			KEY_GROUP_RANGE,
    +			KEY_GROUP_RANGE.getNumberOfKeyGroups());
    +	}
    +
    +	@Test
    +	public void testCombined() {
    --- End diff --
    
    A more descriptive test method name would be helpful.


---

[GitHub] flink issue #6062: [FLINK-9423][state] Implement efficient deletes for heap-...

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

    https://github.com/apache/flink/pull/6062
  
    Hi @StefanRRichter could I ask one more thing that I'm very interested in? From this PR I can feel that the `InternalTimerHeap` is really like a `HeapState` which scoped to the key group(not per key). If we pull this to a higher level abstraction, it seems that Timer Service is just a type of `HeapState` that with the de-duplication works, and this seems to also work for the timer that based on RocksDB. What do you think? Or do you already have any plan?


---

[GitHub] flink issue #6062: [FLINK-9423][state] Implement efficient deletes for heap-...

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

    https://github.com/apache/flink/pull/6062
  
    @sihuazhou thanks for the review. Addressed your comments, please take a look if you want.


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r191447784
  
    --- Diff: flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/InternalTimerHeapTest.java ---
    @@ -0,0 +1,470 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.runtime.state.KeyGroupRange;
    +import org.apache.flink.runtime.state.VoidNamespace;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.junit.Assert;
    +import org.junit.Test;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Set;
    +import java.util.concurrent.ThreadLocalRandom;
    +import java.util.function.BiFunction;
    +import java.util.function.Function;
    +
    +/**
    + * Tests for {@link InternalTimerHeap}.
    + */
    +public class InternalTimerHeapTest {
    +
    +	private static final KeyGroupRange KEY_GROUP_RANGE = new KeyGroupRange(0, 1);
    +
    +	private static void insertRandomTimers(
    +		InternalTimerHeap<Integer, VoidNamespace> timerPriorityQueue,
    +		int count) {
    +		insertRandomTimers(timerPriorityQueue, null, count);
    +	}
    +
    +	private static void insertRandomTimers(
    +		InternalTimerHeap<Integer, VoidNamespace> timerPriorityQueue,
    +		Set<TimerHeapInternalTimer<Integer, VoidNamespace>> checkSet,
    +		int count) {
    +
    +		ThreadLocalRandom localRandom = ThreadLocalRandom.current();
    +
    +		for (int i = 0; i < count; ++i) {
    +			TimerHeapInternalTimer<Integer, VoidNamespace> timer =
    +				new TimerHeapInternalTimer<>(localRandom.nextLong(), i, VoidNamespace.INSTANCE);
    +			if (checkSet != null) {
    +				Preconditions.checkState(checkSet.add(timer));
    +			}
    +			Assert.assertTrue(timerPriorityQueue.add(timer));
    +		}
    +	}
    +
    +	private static InternalTimerHeap<Integer, VoidNamespace> newPriorityQueue(int initialCapacity) {
    +		return new InternalTimerHeap<>(
    +			initialCapacity,
    +			KEY_GROUP_RANGE,
    +			KEY_GROUP_RANGE.getNumberOfKeyGroups());
    +	}
    +
    +	@Test
    +	public void testCombined() {
    +		final int initialCapacity = 4;
    +		final int testSize = 1000;
    +		InternalTimerHeap<Integer, VoidNamespace> timerPriorityQueue = newPriorityQueue(initialCapacity);
    +		HashSet<TimerHeapInternalTimer<Integer, VoidNamespace>> checkSet = new HashSet<>(testSize);
    +
    +		insertRandomTimers(timerPriorityQueue, checkSet, testSize);
    +
    +		long lastTimestamp = Long.MIN_VALUE;
    +		int lastSize = timerPriorityQueue.size();
    +		Assert.assertEquals(testSize, lastSize);
    +		TimerHeapInternalTimer<Integer, VoidNamespace> timer;
    +		while ((timer = timerPriorityQueue.peek()) != null) {
    +			Assert.assertFalse(timerPriorityQueue.isEmpty());
    +			Assert.assertEquals(lastSize, timerPriorityQueue.size());
    +			Assert.assertEquals(timer, timerPriorityQueue.poll());
    +			Assert.assertTrue(checkSet.remove(timer));
    +			Assert.assertTrue(timer.getTimestamp() >= lastTimestamp);
    +			lastTimestamp = timer.getTimestamp();
    +			--lastSize;
    +		}
    +
    +		Assert.assertTrue(timerPriorityQueue.isEmpty());
    +		Assert.assertEquals(0, timerPriorityQueue.size());
    +		Assert.assertEquals(0, checkSet.size());
    +	}
    +
    +	@Test
    +	public void testAdd() {
    +		testAddOfferCommon(InternalTimerHeap<Integer, VoidNamespace>::add);
    +	}
    +
    +	@Test
    +	public void testOffer() {
    +		testAddOfferCommon(InternalTimerHeap<Integer, VoidNamespace>::offer);
    --- End diff --
    
    What's the difference between `add` and `offer`? Aren't these effectively the same methods?


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r191732576
  
    --- Diff: flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/InternalTimerHeapTest.java ---
    @@ -0,0 +1,470 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.runtime.state.KeyGroupRange;
    +import org.apache.flink.runtime.state.VoidNamespace;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.junit.Assert;
    +import org.junit.Test;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Set;
    +import java.util.concurrent.ThreadLocalRandom;
    +import java.util.function.BiFunction;
    +import java.util.function.Function;
    +
    +/**
    + * Tests for {@link InternalTimerHeap}.
    + */
    +public class InternalTimerHeapTest {
    +
    +	private static final KeyGroupRange KEY_GROUP_RANGE = new KeyGroupRange(0, 1);
    +
    +	private static void insertRandomTimers(
    +		InternalTimerHeap<Integer, VoidNamespace> timerPriorityQueue,
    +		int count) {
    +		insertRandomTimers(timerPriorityQueue, null, count);
    +	}
    +
    +	private static void insertRandomTimers(
    +		InternalTimerHeap<Integer, VoidNamespace> timerPriorityQueue,
    +		Set<TimerHeapInternalTimer<Integer, VoidNamespace>> checkSet,
    +		int count) {
    +
    +		ThreadLocalRandom localRandom = ThreadLocalRandom.current();
    +
    +		for (int i = 0; i < count; ++i) {
    +			TimerHeapInternalTimer<Integer, VoidNamespace> timer =
    +				new TimerHeapInternalTimer<>(localRandom.nextLong(), i, VoidNamespace.INSTANCE);
    +			if (checkSet != null) {
    +				Preconditions.checkState(checkSet.add(timer));
    +			}
    +			Assert.assertTrue(timerPriorityQueue.add(timer));
    +		}
    +	}
    +
    +	private static InternalTimerHeap<Integer, VoidNamespace> newPriorityQueue(int initialCapacity) {
    +		return new InternalTimerHeap<>(
    +			initialCapacity,
    +			KEY_GROUP_RANGE,
    +			KEY_GROUP_RANGE.getNumberOfKeyGroups());
    +	}
    +
    +	@Test
    +	public void testCombined() {
    +		final int initialCapacity = 4;
    +		final int testSize = 1000;
    +		InternalTimerHeap<Integer, VoidNamespace> timerPriorityQueue = newPriorityQueue(initialCapacity);
    +		HashSet<TimerHeapInternalTimer<Integer, VoidNamespace>> checkSet = new HashSet<>(testSize);
    +
    +		insertRandomTimers(timerPriorityQueue, checkSet, testSize);
    +
    +		long lastTimestamp = Long.MIN_VALUE;
    +		int lastSize = timerPriorityQueue.size();
    +		Assert.assertEquals(testSize, lastSize);
    +		TimerHeapInternalTimer<Integer, VoidNamespace> timer;
    +		while ((timer = timerPriorityQueue.peek()) != null) {
    +			Assert.assertFalse(timerPriorityQueue.isEmpty());
    +			Assert.assertEquals(lastSize, timerPriorityQueue.size());
    +			Assert.assertEquals(timer, timerPriorityQueue.poll());
    +			Assert.assertTrue(checkSet.remove(timer));
    +			Assert.assertTrue(timer.getTimestamp() >= lastTimestamp);
    +			lastTimestamp = timer.getTimestamp();
    +			--lastSize;
    +		}
    +
    +		Assert.assertTrue(timerPriorityQueue.isEmpty());
    +		Assert.assertEquals(0, timerPriorityQueue.size());
    +		Assert.assertEquals(0, checkSet.size());
    +	}
    +
    +	@Test
    +	public void testAdd() {
    +		testAddOfferCommon(InternalTimerHeap<Integer, VoidNamespace>::add);
    +	}
    +
    +	@Test
    +	public void testOffer() {
    +		testAddOfferCommon(InternalTimerHeap<Integer, VoidNamespace>::offer);
    --- End diff --
    
    The difference is as defined in the queue interface, but for the tested class they are equal because there is no capacity limit.


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r191413234
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/HeapInternalTimerService.java ---
    @@ -301,114 +259,39 @@ public void advanceWatermark(long time) throws Exception {
     				keySerializer.snapshotConfiguration(),
     				namespaceSerializer,
     				namespaceSerializer.snapshotConfiguration(),
    -				getEventTimeTimerSetForKeyGroup(keyGroupIdx),
    -				getProcessingTimeTimerSetForKeyGroup(keyGroupIdx));
    +				eventTimeTimersQueue.getTimersForKeyGroup(keyGroupIdx),
    +				processingTimeTimersQueue.getTimersForKeyGroup(keyGroupIdx));
     	}
     
     	/**
     	 * Restore the timers (both processing and event time ones) for a given {@code keyGroupIdx}.
     	 *
    -	 * @param restoredTimersSnapshot the restored snapshot containing the key-group's timers,
    +	 * @param restoredSnapshot the restored snapshot containing the key-group's timers,
     	 *                       and the serializers that were used to write them
     	 * @param keyGroupIdx the id of the key-group to be put in the snapshot.
     	 */
     	@SuppressWarnings("unchecked")
    -	public void restoreTimersForKeyGroup(InternalTimersSnapshot<?, ?> restoredTimersSnapshot, int keyGroupIdx) throws IOException {
    -		this.restoredTimersSnapshot = (InternalTimersSnapshot<K, N>) restoredTimersSnapshot;
    +	public void restoreTimersForKeyGroup(InternalTimersSnapshot<?, ?> restoredSnapshot, int keyGroupIdx) {
    +		this.restoredTimersSnapshot = (InternalTimersSnapshot<K, N>) restoredSnapshot;
     
    -		if ((this.keyDeserializer != null && !this.keyDeserializer.equals(restoredTimersSnapshot.getKeySerializer())) ||
    -			(this.namespaceDeserializer != null && !this.namespaceDeserializer.equals(restoredTimersSnapshot.getNamespaceSerializer()))) {
    +		if ((this.keyDeserializer != null && !this.keyDeserializer.equals(restoredSnapshot.getKeySerializer())) ||
    --- End diff --
    
    This check could be factored out into a method with a meaningful and easy to understand name, e.g. `checkSerializerCompatibility`.


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r191426419
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerHeap.java ---
    @@ -0,0 +1,511 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.annotation.VisibleForTesting;
    +import org.apache.flink.runtime.state.KeyGroupRange;
    +import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
    +
    +import javax.annotation.Nonnegative;
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Queue;
    +import java.util.Set;
    +
    +import static org.apache.flink.util.Preconditions.checkArgument;
    +
    +/**
    + * A heap-based priority queue for internal timers. This heap is supported by hash sets for fast contains
    + * (de-duplication) and deletes. The heap implementation is a simple binary tree stored inside an array. Element indexes
    + * in the heap array start at 1 instead of 0 to make array index computations a bit simpler in the hot methods.
    + *
    + * <p>Possible future improvements:
    + * <ul>
    + *  <li>We could also implement shrinking for the heap and the deduplication maps.</li>
    + *  <li>We could replace the deduplication maps with more efficient custom implementations. In particular, a hash set
    + * would be enough if it could return existing elements on unsuccessful adding, etc..</li>
    + * </ul>
    + *
    + * @param <K> type of the key of the internal timers managed by this priority queue.
    + * @param <N> type of the namespace of the internal timers managed by this priority queue.
    + */
    +public class InternalTimerHeap<K, N> implements Queue<TimerHeapInternalTimer<K, N>>, Set<TimerHeapInternalTimer<K, N>> {
    +
    +	/**
    +	 * A safe maximum size for arrays in the JVM.
    +	 */
    +	private static final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8;
    +
    +	/**
    +	 * Comparator for {@link TimerHeapInternalTimer}, based on the timestamp in ascending order.
    +	 */
    +	private static final Comparator<TimerHeapInternalTimer<?, ?>> COMPARATOR =
    +		(o1, o2) -> Long.compare(o1.getTimestamp(), o2.getTimestamp());
    +
    +	/**
    +	 * This array contains one hash set per key-group. The sets are used for fast de-duplication and deletes of timers.
    +	 */
    +	private final HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>>[] deduplicationMapsByKeyGroup;
    +
    +	/**
    +	 * The array that represents the heap-organized priority queue.
    +	 */
    +	private TimerHeapInternalTimer<K, N>[] queue;
    +
    +	/**
    +	 * The current size of the priority queue.
    +	 */
    +	private int size;
    +
    +	/**
    +	 * The key-group range of timers that are managed by this queue.
    +	 */
    +	private final KeyGroupRange keyGroupRange;
    +
    +	/**
    +	 * The total number of key-groups of the job.
    +	 */
    +	private final int totalNumberOfKeyGroups;
    +
    +
    +	/**
    +	 * Creates an empty {@link InternalTimerHeap} with the requested initial capacity.
    +	 *
    +	 * @param minimumCapacity the minimum and initial capacity of this priority queue.
    +	 */
    +	@SuppressWarnings("unchecked")
    +	InternalTimerHeap(
    +		@Nonnegative int minimumCapacity,
    +		@Nonnull KeyGroupRange keyGroupRange,
    +		@Nonnegative int totalNumberOfKeyGroups) {
    +
    +		this.totalNumberOfKeyGroups = totalNumberOfKeyGroups;
    +		this.keyGroupRange = keyGroupRange;
    +
    +		final int keyGroupsInLocalRange = keyGroupRange.getNumberOfKeyGroups();
    +		final int deduplicationSetSize = 1 + minimumCapacity / keyGroupsInLocalRange;
    +		this.deduplicationMapsByKeyGroup = new HashMap[keyGroupsInLocalRange];
    +		for (int i = 0; i < keyGroupsInLocalRange; ++i) {
    +			deduplicationMapsByKeyGroup[i] = new HashMap<>(deduplicationSetSize);
    +		}
    +
    +		this.queue = new TimerHeapInternalTimer[1 + minimumCapacity];
    +	}
    +
    +	/**
    +	 * @see Set#add(Object)
    +	 */
    +	@Override
    +	public boolean add(@Nonnull TimerHeapInternalTimer<K, N> timer) {
    +
    +		if (getDedupMapForKeyGroup(timer).putIfAbsent(timer, timer) == null) {
    +			final int newSize = ++this.size;
    +			growIfRequired(newSize);
    +			moveElementToIdx(timer, newSize);
    +			siftUp(newSize);
    +			return true;
    +		} else {
    +			return false;
    +		}
    +	}
    +
    +	/**
    +	 * This behaves like {@link #add(TimerHeapInternalTimer)}.
    +	 */
    +	@Override
    +	public boolean offer(@Nonnull TimerHeapInternalTimer<K, N> k) {
    +		return add(k);
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> poll() {
    +		return size() > 0 ? removeElementAtIndex(1) : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> remove() {
    +		TimerHeapInternalTimer<K, N> pollResult = poll();
    +		if (pollResult != null) {
    +			return pollResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> peek() {
    +		return size() > 0 ? queue[1] : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> element() {
    +		TimerHeapInternalTimer<K, N> peekResult = peek();
    +		if (peekResult != null) {
    +			return peekResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Override
    +	public boolean isEmpty() {
    +		return size() == 0;
    +	}
    +
    +	@Override
    +	public boolean contains(@Nullable Object toCheck) {
    +		return (toCheck instanceof TimerHeapInternalTimer)
    +			&& getDedupMapForKeyGroup((TimerHeapInternalTimer<?, ?>) toCheck).containsKey(toCheck);
    +	}
    +
    +	@Override
    +	public boolean remove(@Nullable Object toRemove) {
    +		if (toRemove instanceof TimerHeapInternalTimer) {
    +			return removeInternal((TimerHeapInternalTimer<?, ?>) toRemove);
    +		}
    +		return false;
    +	}
    +
    +	@Override
    +	public boolean addAll(@Nullable Collection<? extends TimerHeapInternalTimer<K, N>> timers) {
    +
    +		if (timers == null) {
    +			return true;
    +		}
    +
    +		final int oldSize = size();
    +		resizeForBulkLoad(oldSize + timers.size());
    +
    +		for (TimerHeapInternalTimer<K, N> k : timers) {
    +			add(k);
    +		}
    +
    +		return oldSize != size();
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public Object[] toArray() {
    +		return Arrays.copyOfRange(queue, 1, size + 1);
    +	}
    +
    +	@SuppressWarnings({"unchecked", "SuspiciousSystemArraycopy"})
    +	@Nonnull
    +	@Override
    +	public <T> T[] toArray(@Nonnull T[] array) {
    +		if (array.length >= size) {
    +			System.arraycopy(queue, 1, array, 0, size);
    +			return array;
    +		}
    +		return (T[]) Arrays.copyOfRange(queue, 1, size + 1, array.getClass());
    +	}
    +
    +	@Override
    +	public boolean removeAll(@Nullable Collection<?> toRemove) {
    +
    +		if (toRemove == null) {
    +			return false;
    +		}
    +
    +		int oldSize = size();
    +		for (Object o : toRemove) {
    +			remove(o);
    +		}
    +		return oldSize != size();
    +	}
    +
    +	/**
    +	 * Returns an iterator over the elements in this queue. The iterator
    +	 * does not return the elements in any particular order.
    +	 *
    +	 * @return an iterator over the elements in this queue.
    +	 */
    +	@Nonnull
    +	@Override
    +	public Iterator<TimerHeapInternalTimer<K, N>> iterator() {
    +		return new InternalTimerPriorityQueueIterator();
    +	}
    +
    +	@Override
    +	public boolean containsAll(@Nullable Collection<?> toCheck) {
    +
    +		if (toCheck == null) {
    +			return true;
    +		}
    +
    +		for (Object o : toCheck) {
    +			if (!contains(o)) {
    +				return false;
    +			}
    +		}
    +
    +		return true;
    +	}
    +
    +	@Nonnegative
    +	@Override
    +	public int size() {
    +		return size;
    +	}
    +
    +	@Override
    +	public void clear() {
    +
    +		Arrays.fill(queue, null);
    +		for (HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> timerHashMap :
    +			deduplicationMapsByKeyGroup) {
    +			timerHashMap.clear();
    +		}
    +		size = 0;
    +	}
    +
    +	/**
    +	 * This method is currently not implemented.
    +	 */
    +	@Override
    +	public boolean retainAll(@Nullable Collection<?> toRetain) {
    +		throw new UnsupportedOperationException();
    +	}
    +
    +	/**
    +	 * Adds a new timer with the given timestamp, key, and namespace to the heap, if an identical timer was not yet
    +	 * registered.
    +	 *
    +	 * @param timestamp the timer timestamp.
    +	 * @param key the timer key.
    +	 * @param namespace the timer namespace.
    +	 * @return true iff a new timer with given timestamp, key, and namespace was added to the heap.
    +	 */
    +	boolean scheduleTimer(long timestamp, K key, N namespace) {
    +		return add(new TimerHeapInternalTimer<>(timestamp, key, namespace));
    +	}
    +
    +	/**
    +	 * Stops timer with the given timestamp, key, and namespace by removing it from the heap, if it exists on the heap.
    +	 *
    +	 * @param timestamp the timer timestamp.
    +	 * @param key the timer key.
    +	 * @param namespace the timer namespace.
    +	 * @return true iff a timer with given timestamp, key, and namespace was found and removed from the heap.
    +	 */
    +	boolean stopTimer(long timestamp, K key, N namespace) {
    +		return removeInternal(new TimerHeapInternalTimer<>(timestamp, key, namespace));
    +	}
    +
    +	/**
    +	 * This method adds all the given timers to the heap.
    +	 */
    +	void addRestoredTimers(Collection<? extends InternalTimer<K, N>> restoredTimers) {
    +
    +		if (restoredTimers == null) {
    +			return;
    +		}
    +
    +		resizeForBulkLoad(restoredTimers.size());
    +
    +		for (InternalTimer<K, N> timer : restoredTimers) {
    +			if (timer instanceof TimerHeapInternalTimer) {
    +				add((TimerHeapInternalTimer<K, N>) timer);
    +			} else {
    +				scheduleTimer(timer.getTimestamp(), timer.getKey(), timer.getNamespace());
    +			}
    +		}
    +	}
    +
    +	private boolean removeInternal(TimerHeapInternalTimer<?, ?> timerToRemove) {
    +
    +		TimerHeapInternalTimer<K, N> storedTimer = getDedupMapForKeyGroup(timerToRemove).remove(timerToRemove);
    +
    +		if (storedTimer != null) {
    +			removeElementAtIndex(storedTimer.getTimerHeapIndex());
    +			return true;
    +		}
    +
    +		return false;
    +	}
    +
    +	private TimerHeapInternalTimer<K, N> removeElementAtIndex(int removeIdx) {
    +		TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		TimerHeapInternalTimer<K, N> removedValue = heap[removeIdx];
    +
    +		assert removedValue.getTimerHeapIndex() == removeIdx;
    +
    +		final int oldSize = size;
    +
    +		if (removeIdx != oldSize) {
    +			TimerHeapInternalTimer<K, N> timer = heap[oldSize];
    +			moveElementToIdx(timer, removeIdx);
    +			siftDown(removeIdx);
    +			if (heap[removeIdx] == timer) {
    +				siftUp(removeIdx);
    +			}
    +		}
    +
    +		heap[oldSize] = null;
    +		getDedupMapForKeyGroup(removedValue).remove(removedValue);
    +
    +		--size;
    +		return removedValue;
    +	}
    +
    +	private void siftUp(int idx) {
    +		final TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		TimerHeapInternalTimer<K, N> currentTimer = heap[idx];
    +		int parentIdx = idx >>> 1;
    +
    +		while (parentIdx > 0 && COMPARATOR.compare(currentTimer, heap[parentIdx]) < 0) {
    +			moveElementToIdx(heap[parentIdx], idx);
    +			idx = parentIdx;
    +			parentIdx = parentIdx >>> 1;
    +		}
    +
    +		moveElementToIdx(currentTimer, idx);
    +	}
    +
    +	private void siftDown(int idx) {
    +		final TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		final int heapSize = this.size;
    +
    +		TimerHeapInternalTimer<K, N> currentTimer = heap[idx];
    +		int firstChildIdx = idx << 1;
    +		int secondChildIdx = firstChildIdx + 1;
    +
    +		if (secondChildIdx <= heapSize && COMPARATOR.compare(heap[secondChildIdx], heap[firstChildIdx]) < 0) {
    +			firstChildIdx = secondChildIdx;
    +		}
    +
    +		while (firstChildIdx <= heapSize && COMPARATOR.compare(heap[firstChildIdx], currentTimer) < 0) {
    +			moveElementToIdx(heap[firstChildIdx], idx);
    +			idx = firstChildIdx;
    +			firstChildIdx = idx << 1;
    +			secondChildIdx = firstChildIdx + 1;
    +
    +			if (secondChildIdx <= heapSize && COMPARATOR.compare(heap[secondChildIdx], heap[firstChildIdx]) < 0) {
    +				firstChildIdx = secondChildIdx;
    +			}
    --- End diff --
    
    This looks like a code duplication. I think this could be refactored into a private method.


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r190289953
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerHeap.java ---
    @@ -0,0 +1,504 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.annotation.VisibleForTesting;
    +import org.apache.flink.runtime.state.KeyGroupRange;
    +import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
    +
    +import javax.annotation.Nonnegative;
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Queue;
    +import java.util.Set;
    +
    +import static org.apache.flink.util.Preconditions.checkArgument;
    +
    +/**
    + * A heap-based priority queue for internal timers. This heap is supported by hash sets for fast contains
    + * (de-duplication) and deletes. The heap implementation is a simple binary tree stored inside an array. Element indexes
    + * in the heap array start at 1 instead of 0 to make array index computations a bit simpler in the hot methods.
    + *
    + * <p>Possible future improvements:
    + * <ul>
    + *  <li>We could also implement shrinking for the heap and the deduplication maps.</li>
    + *  <li>We could replace the deduplication maps with more efficient custom implementations. In particular, a hash set
    + * would be enough if it could return existing elements on unsuccessful adding, etc..</li>
    + * </ul>
    + *
    + * @param <K> type of the key of the internal timers managed by this priority queue.
    + * @param <N> type of the namespace of the internal timers managed by this priority queue.
    + */
    +public class InternalTimerHeap<K, N> implements Queue<TimerHeapInternalTimer<K, N>>, Set<TimerHeapInternalTimer<K, N>> {
    +
    +	/**
    +	 * A safe maximum size for arrays in the JVM.
    +	 */
    +	private static final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8;
    +
    +	/**
    +	 * Comparator for {@link TimerHeapInternalTimer}, based on the timestamp in ascending order.
    +	 */
    +	private static final Comparator<TimerHeapInternalTimer<?, ?>> COMPARATOR =
    +		(o1, o2) -> Long.compare(o1.getTimestamp(), o2.getTimestamp());
    +
    +	/**
    +	 * This array contains one hash set per key-group. The sets are used for fast de-duplication and deletes of timers.
    +	 */
    +	private final HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>>[] deduplicationMapsByKeyGroup;
    +
    +	/**
    +	 * The array that represents the heap-organized priority queue.
    +	 */
    +	private TimerHeapInternalTimer<K, N>[] queue;
    +
    +	/**
    +	 * The current size of the priority queue.
    +	 */
    +	private int size;
    +
    +	/**
    +	 * The key-group range of timers that are managed by this queue.
    +	 */
    +	private final KeyGroupRange keyGroupRange;
    +
    +	/**
    +	 * The total number of key-groups of the job.
    +	 */
    +	private final int totalNumberOfKeyGroups;
    +
    +
    +	/**
    +	 * Creates an empty {@link InternalTimerHeap} with the requested initial capacity.
    +	 *
    +	 * @param minimumCapacity the minimum and initial capacity of this priority queue.
    +	 */
    +	@SuppressWarnings("unchecked")
    +	InternalTimerHeap(
    +		@Nonnegative int minimumCapacity,
    +		@Nonnull KeyGroupRange keyGroupRange,
    +		@Nonnegative int totalNumberOfKeyGroups) {
    +
    +		this.totalNumberOfKeyGroups = totalNumberOfKeyGroups;
    +		this.keyGroupRange = keyGroupRange;
    +
    +		final int keyGroupsInLocalRange = keyGroupRange.getNumberOfKeyGroups();
    +		final int deduplicationSetSize = 1 + minimumCapacity / keyGroupsInLocalRange;
    +		this.deduplicationMapsByKeyGroup = new HashMap[keyGroupsInLocalRange];
    +		for (int i = 0; i < keyGroupsInLocalRange; ++i) {
    +			deduplicationMapsByKeyGroup[i] = new HashMap<>(deduplicationSetSize);
    +		}
    +
    +		this.queue = new TimerHeapInternalTimer[1 + minimumCapacity];
    +	}
    +
    +	/**
    +	 * @see Set#add(Object)
    +	 */
    +	@Override
    +	public boolean add(@Nonnull TimerHeapInternalTimer<K, N> timer) {
    +
    +		if (getDedupMapForKeyGroup(timer).putIfAbsent(timer, timer) == null) {
    +			final int newSize = ++this.size;
    +			checkCapacity(newSize);
    +			moveElementToIdx(timer, newSize);
    +			siftUp(newSize);
    +			return true;
    +		} else {
    +			return false;
    +		}
    +	}
    +
    +	/**
    +	 * This behaves like {@link #add(TimerHeapInternalTimer)}.
    +	 */
    +	@Override
    +	public boolean offer(@Nonnull TimerHeapInternalTimer<K, N> k) {
    +		return add(k);
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> poll() {
    +		return size() > 0 ? removeElementAtIndex(1) : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> remove() {
    +		TimerHeapInternalTimer<K, N> pollResult = poll();
    +		if (pollResult != null) {
    +			return pollResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> peek() {
    +		return size() > 0 ? queue[1] : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> element() {
    +		TimerHeapInternalTimer<K, N> peekResult = peek();
    +		if (peekResult != null) {
    +			return peekResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Override
    +	public boolean isEmpty() {
    +		return size() == 0;
    +	}
    +
    +	@Override
    +	public boolean contains(@Nullable Object o) {
    +		return (o instanceof TimerHeapInternalTimer)
    +			&& getDedupMapForKeyGroup((TimerHeapInternalTimer<?, ?>) o).containsKey(o);
    +	}
    +
    +	@Override
    +	public boolean remove(@Nullable Object o) {
    +		if (o instanceof TimerHeapInternalTimer) {
    +			return removeInternal((TimerHeapInternalTimer<?, ?>) o);
    +		}
    +		return false;
    +	}
    +
    +	@Override
    +	public boolean addAll(@Nullable Collection<? extends TimerHeapInternalTimer<K, N>> timers) {
    +
    +		if (timers == null) {
    +			return true;
    +		}
    +
    +		if (timers.size() > queue.length) {
    --- End diff --
    
    Good question, what the right choice here is because it might also be that all elements in timers are duplicates and already contained. Not sure what the optimal choice is here.


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r191414669
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerHeap.java ---
    @@ -0,0 +1,511 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.annotation.VisibleForTesting;
    +import org.apache.flink.runtime.state.KeyGroupRange;
    +import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
    +
    +import javax.annotation.Nonnegative;
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Queue;
    +import java.util.Set;
    +
    +import static org.apache.flink.util.Preconditions.checkArgument;
    +
    +/**
    + * A heap-based priority queue for internal timers. This heap is supported by hash sets for fast contains
    + * (de-duplication) and deletes. The heap implementation is a simple binary tree stored inside an array. Element indexes
    + * in the heap array start at 1 instead of 0 to make array index computations a bit simpler in the hot methods.
    + *
    + * <p>Possible future improvements:
    + * <ul>
    + *  <li>We could also implement shrinking for the heap and the deduplication maps.</li>
    + *  <li>We could replace the deduplication maps with more efficient custom implementations. In particular, a hash set
    + * would be enough if it could return existing elements on unsuccessful adding, etc..</li>
    + * </ul>
    + *
    + * @param <K> type of the key of the internal timers managed by this priority queue.
    + * @param <N> type of the namespace of the internal timers managed by this priority queue.
    + */
    +public class InternalTimerHeap<K, N> implements Queue<TimerHeapInternalTimer<K, N>>, Set<TimerHeapInternalTimer<K, N>> {
    +
    +	/**
    +	 * A safe maximum size for arrays in the JVM.
    +	 */
    +	private static final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8;
    +
    +	/**
    +	 * Comparator for {@link TimerHeapInternalTimer}, based on the timestamp in ascending order.
    +	 */
    +	private static final Comparator<TimerHeapInternalTimer<?, ?>> COMPARATOR =
    +		(o1, o2) -> Long.compare(o1.getTimestamp(), o2.getTimestamp());
    --- End diff --
    
    Could be replaced by `Comparator.comparingLong(TimerHeapInternalTimer::getTimestamp)`


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r191446801
  
    --- Diff: flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/InternalTimerHeapTest.java ---
    @@ -0,0 +1,470 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.runtime.state.KeyGroupRange;
    +import org.apache.flink.runtime.state.VoidNamespace;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.junit.Assert;
    +import org.junit.Test;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Set;
    +import java.util.concurrent.ThreadLocalRandom;
    +import java.util.function.BiFunction;
    +import java.util.function.Function;
    +
    +/**
    + * Tests for {@link InternalTimerHeap}.
    + */
    +public class InternalTimerHeapTest {
    +
    +	private static final KeyGroupRange KEY_GROUP_RANGE = new KeyGroupRange(0, 1);
    +
    +	private static void insertRandomTimers(
    +		InternalTimerHeap<Integer, VoidNamespace> timerPriorityQueue,
    +		int count) {
    +		insertRandomTimers(timerPriorityQueue, null, count);
    +	}
    +
    +	private static void insertRandomTimers(
    +		InternalTimerHeap<Integer, VoidNamespace> timerPriorityQueue,
    +		Set<TimerHeapInternalTimer<Integer, VoidNamespace>> checkSet,
    --- End diff --
    
    `@Nullable` missing


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r191432905
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerHeap.java ---
    @@ -0,0 +1,511 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.annotation.VisibleForTesting;
    +import org.apache.flink.runtime.state.KeyGroupRange;
    +import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
    +
    +import javax.annotation.Nonnegative;
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Queue;
    +import java.util.Set;
    +
    +import static org.apache.flink.util.Preconditions.checkArgument;
    +
    +/**
    + * A heap-based priority queue for internal timers. This heap is supported by hash sets for fast contains
    + * (de-duplication) and deletes. The heap implementation is a simple binary tree stored inside an array. Element indexes
    + * in the heap array start at 1 instead of 0 to make array index computations a bit simpler in the hot methods.
    + *
    + * <p>Possible future improvements:
    + * <ul>
    + *  <li>We could also implement shrinking for the heap and the deduplication maps.</li>
    + *  <li>We could replace the deduplication maps with more efficient custom implementations. In particular, a hash set
    + * would be enough if it could return existing elements on unsuccessful adding, etc..</li>
    + * </ul>
    + *
    + * @param <K> type of the key of the internal timers managed by this priority queue.
    + * @param <N> type of the namespace of the internal timers managed by this priority queue.
    + */
    +public class InternalTimerHeap<K, N> implements Queue<TimerHeapInternalTimer<K, N>>, Set<TimerHeapInternalTimer<K, N>> {
    +
    +	/**
    +	 * A safe maximum size for arrays in the JVM.
    +	 */
    +	private static final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8;
    +
    +	/**
    +	 * Comparator for {@link TimerHeapInternalTimer}, based on the timestamp in ascending order.
    +	 */
    +	private static final Comparator<TimerHeapInternalTimer<?, ?>> COMPARATOR =
    +		(o1, o2) -> Long.compare(o1.getTimestamp(), o2.getTimestamp());
    +
    +	/**
    +	 * This array contains one hash set per key-group. The sets are used for fast de-duplication and deletes of timers.
    +	 */
    +	private final HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>>[] deduplicationMapsByKeyGroup;
    +
    +	/**
    +	 * The array that represents the heap-organized priority queue.
    +	 */
    +	private TimerHeapInternalTimer<K, N>[] queue;
    +
    +	/**
    +	 * The current size of the priority queue.
    +	 */
    +	private int size;
    +
    +	/**
    +	 * The key-group range of timers that are managed by this queue.
    +	 */
    +	private final KeyGroupRange keyGroupRange;
    +
    +	/**
    +	 * The total number of key-groups of the job.
    +	 */
    +	private final int totalNumberOfKeyGroups;
    +
    +
    +	/**
    +	 * Creates an empty {@link InternalTimerHeap} with the requested initial capacity.
    +	 *
    +	 * @param minimumCapacity the minimum and initial capacity of this priority queue.
    +	 */
    +	@SuppressWarnings("unchecked")
    +	InternalTimerHeap(
    +		@Nonnegative int minimumCapacity,
    +		@Nonnull KeyGroupRange keyGroupRange,
    +		@Nonnegative int totalNumberOfKeyGroups) {
    +
    +		this.totalNumberOfKeyGroups = totalNumberOfKeyGroups;
    +		this.keyGroupRange = keyGroupRange;
    +
    +		final int keyGroupsInLocalRange = keyGroupRange.getNumberOfKeyGroups();
    +		final int deduplicationSetSize = 1 + minimumCapacity / keyGroupsInLocalRange;
    +		this.deduplicationMapsByKeyGroup = new HashMap[keyGroupsInLocalRange];
    +		for (int i = 0; i < keyGroupsInLocalRange; ++i) {
    +			deduplicationMapsByKeyGroup[i] = new HashMap<>(deduplicationSetSize);
    +		}
    +
    +		this.queue = new TimerHeapInternalTimer[1 + minimumCapacity];
    +	}
    +
    +	/**
    +	 * @see Set#add(Object)
    +	 */
    +	@Override
    +	public boolean add(@Nonnull TimerHeapInternalTimer<K, N> timer) {
    +
    +		if (getDedupMapForKeyGroup(timer).putIfAbsent(timer, timer) == null) {
    +			final int newSize = ++this.size;
    +			growIfRequired(newSize);
    +			moveElementToIdx(timer, newSize);
    +			siftUp(newSize);
    +			return true;
    +		} else {
    +			return false;
    +		}
    +	}
    +
    +	/**
    +	 * This behaves like {@link #add(TimerHeapInternalTimer)}.
    +	 */
    +	@Override
    +	public boolean offer(@Nonnull TimerHeapInternalTimer<K, N> k) {
    +		return add(k);
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> poll() {
    +		return size() > 0 ? removeElementAtIndex(1) : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> remove() {
    +		TimerHeapInternalTimer<K, N> pollResult = poll();
    +		if (pollResult != null) {
    +			return pollResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> peek() {
    +		return size() > 0 ? queue[1] : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> element() {
    +		TimerHeapInternalTimer<K, N> peekResult = peek();
    +		if (peekResult != null) {
    +			return peekResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Override
    +	public boolean isEmpty() {
    +		return size() == 0;
    +	}
    +
    +	@Override
    +	public boolean contains(@Nullable Object toCheck) {
    +		return (toCheck instanceof TimerHeapInternalTimer)
    +			&& getDedupMapForKeyGroup((TimerHeapInternalTimer<?, ?>) toCheck).containsKey(toCheck);
    +	}
    +
    +	@Override
    +	public boolean remove(@Nullable Object toRemove) {
    +		if (toRemove instanceof TimerHeapInternalTimer) {
    +			return removeInternal((TimerHeapInternalTimer<?, ?>) toRemove);
    +		}
    +		return false;
    +	}
    +
    +	@Override
    +	public boolean addAll(@Nullable Collection<? extends TimerHeapInternalTimer<K, N>> timers) {
    +
    +		if (timers == null) {
    +			return true;
    +		}
    +
    +		final int oldSize = size();
    +		resizeForBulkLoad(oldSize + timers.size());
    +
    +		for (TimerHeapInternalTimer<K, N> k : timers) {
    +			add(k);
    +		}
    +
    +		return oldSize != size();
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public Object[] toArray() {
    +		return Arrays.copyOfRange(queue, 1, size + 1);
    +	}
    +
    +	@SuppressWarnings({"unchecked", "SuspiciousSystemArraycopy"})
    +	@Nonnull
    +	@Override
    +	public <T> T[] toArray(@Nonnull T[] array) {
    +		if (array.length >= size) {
    +			System.arraycopy(queue, 1, array, 0, size);
    +			return array;
    +		}
    +		return (T[]) Arrays.copyOfRange(queue, 1, size + 1, array.getClass());
    +	}
    +
    +	@Override
    +	public boolean removeAll(@Nullable Collection<?> toRemove) {
    +
    +		if (toRemove == null) {
    +			return false;
    +		}
    +
    +		int oldSize = size();
    +		for (Object o : toRemove) {
    +			remove(o);
    +		}
    +		return oldSize != size();
    +	}
    +
    +	/**
    +	 * Returns an iterator over the elements in this queue. The iterator
    +	 * does not return the elements in any particular order.
    +	 *
    +	 * @return an iterator over the elements in this queue.
    +	 */
    +	@Nonnull
    +	@Override
    +	public Iterator<TimerHeapInternalTimer<K, N>> iterator() {
    +		return new InternalTimerPriorityQueueIterator();
    +	}
    +
    +	@Override
    +	public boolean containsAll(@Nullable Collection<?> toCheck) {
    +
    +		if (toCheck == null) {
    +			return true;
    +		}
    +
    +		for (Object o : toCheck) {
    +			if (!contains(o)) {
    +				return false;
    +			}
    +		}
    +
    +		return true;
    +	}
    +
    +	@Nonnegative
    +	@Override
    +	public int size() {
    +		return size;
    +	}
    +
    +	@Override
    +	public void clear() {
    +
    +		Arrays.fill(queue, null);
    +		for (HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> timerHashMap :
    +			deduplicationMapsByKeyGroup) {
    +			timerHashMap.clear();
    +		}
    +		size = 0;
    +	}
    +
    +	/**
    +	 * This method is currently not implemented.
    +	 */
    +	@Override
    +	public boolean retainAll(@Nullable Collection<?> toRetain) {
    +		throw new UnsupportedOperationException();
    +	}
    +
    +	/**
    +	 * Adds a new timer with the given timestamp, key, and namespace to the heap, if an identical timer was not yet
    +	 * registered.
    +	 *
    +	 * @param timestamp the timer timestamp.
    +	 * @param key the timer key.
    +	 * @param namespace the timer namespace.
    +	 * @return true iff a new timer with given timestamp, key, and namespace was added to the heap.
    +	 */
    +	boolean scheduleTimer(long timestamp, K key, N namespace) {
    +		return add(new TimerHeapInternalTimer<>(timestamp, key, namespace));
    +	}
    +
    +	/**
    +	 * Stops timer with the given timestamp, key, and namespace by removing it from the heap, if it exists on the heap.
    +	 *
    +	 * @param timestamp the timer timestamp.
    +	 * @param key the timer key.
    +	 * @param namespace the timer namespace.
    +	 * @return true iff a timer with given timestamp, key, and namespace was found and removed from the heap.
    +	 */
    +	boolean stopTimer(long timestamp, K key, N namespace) {
    +		return removeInternal(new TimerHeapInternalTimer<>(timestamp, key, namespace));
    +	}
    +
    +	/**
    +	 * This method adds all the given timers to the heap.
    +	 */
    +	void addRestoredTimers(Collection<? extends InternalTimer<K, N>> restoredTimers) {
    +
    +		if (restoredTimers == null) {
    +			return;
    +		}
    +
    +		resizeForBulkLoad(restoredTimers.size());
    +
    +		for (InternalTimer<K, N> timer : restoredTimers) {
    +			if (timer instanceof TimerHeapInternalTimer) {
    +				add((TimerHeapInternalTimer<K, N>) timer);
    +			} else {
    +				scheduleTimer(timer.getTimestamp(), timer.getKey(), timer.getNamespace());
    +			}
    +		}
    +	}
    +
    +	private boolean removeInternal(TimerHeapInternalTimer<?, ?> timerToRemove) {
    +
    +		TimerHeapInternalTimer<K, N> storedTimer = getDedupMapForKeyGroup(timerToRemove).remove(timerToRemove);
    +
    +		if (storedTimer != null) {
    +			removeElementAtIndex(storedTimer.getTimerHeapIndex());
    +			return true;
    +		}
    +
    +		return false;
    +	}
    +
    +	private TimerHeapInternalTimer<K, N> removeElementAtIndex(int removeIdx) {
    +		TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		TimerHeapInternalTimer<K, N> removedValue = heap[removeIdx];
    +
    +		assert removedValue.getTimerHeapIndex() == removeIdx;
    +
    +		final int oldSize = size;
    +
    +		if (removeIdx != oldSize) {
    +			TimerHeapInternalTimer<K, N> timer = heap[oldSize];
    +			moveElementToIdx(timer, removeIdx);
    +			siftDown(removeIdx);
    +			if (heap[removeIdx] == timer) {
    +				siftUp(removeIdx);
    +			}
    +		}
    +
    +		heap[oldSize] = null;
    +		getDedupMapForKeyGroup(removedValue).remove(removedValue);
    +
    +		--size;
    +		return removedValue;
    +	}
    +
    +	private void siftUp(int idx) {
    +		final TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		TimerHeapInternalTimer<K, N> currentTimer = heap[idx];
    +		int parentIdx = idx >>> 1;
    +
    +		while (parentIdx > 0 && COMPARATOR.compare(currentTimer, heap[parentIdx]) < 0) {
    +			moveElementToIdx(heap[parentIdx], idx);
    +			idx = parentIdx;
    +			parentIdx = parentIdx >>> 1;
    +		}
    +
    +		moveElementToIdx(currentTimer, idx);
    +	}
    +
    +	private void siftDown(int idx) {
    +		final TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		final int heapSize = this.size;
    +
    +		TimerHeapInternalTimer<K, N> currentTimer = heap[idx];
    +		int firstChildIdx = idx << 1;
    +		int secondChildIdx = firstChildIdx + 1;
    +
    +		if (secondChildIdx <= heapSize && COMPARATOR.compare(heap[secondChildIdx], heap[firstChildIdx]) < 0) {
    +			firstChildIdx = secondChildIdx;
    +		}
    +
    +		while (firstChildIdx <= heapSize && COMPARATOR.compare(heap[firstChildIdx], currentTimer) < 0) {
    +			moveElementToIdx(heap[firstChildIdx], idx);
    +			idx = firstChildIdx;
    +			firstChildIdx = idx << 1;
    +			secondChildIdx = firstChildIdx + 1;
    +
    +			if (secondChildIdx <= heapSize && COMPARATOR.compare(heap[secondChildIdx], heap[firstChildIdx]) < 0) {
    +				firstChildIdx = secondChildIdx;
    +			}
    +		}
    +
    +		moveElementToIdx(currentTimer, idx);
    +	}
    +
    +	/**
    +	 * Returns an unmodifiable set of all timers in the given key-group.
    +	 */
    +	Set<InternalTimer<K, N>> getTimersForKeyGroup(@Nonnegative int keyGroupIdx) {
    +		return Collections.unmodifiableSet(getDedupMapForKeyGroup(keyGroupIdx).keySet());
    +	}
    +
    +	private HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> getDedupMapForKeyGroup(
    +		@Nonnegative int keyGroupIdx) {
    +		return deduplicationMapsByKeyGroup[globalKeyGroupToLocalIndex(keyGroupIdx)];
    +	}
    +
    +	@VisibleForTesting
    +	@SuppressWarnings("unchecked")
    +	List<Set<InternalTimer<K, N>>> getTimersByKeyGroup() {
    +		List<Set<InternalTimer<K, N>>> result = new ArrayList<>(deduplicationMapsByKeyGroup.length);
    +		for (int i = 0; i < deduplicationMapsByKeyGroup.length; ++i) {
    +			result.add(i, Collections.unmodifiableSet(deduplicationMapsByKeyGroup[i].keySet()));
    +		}
    +		return result;
    +	}
    +
    +	private void moveElementToIdx(TimerHeapInternalTimer<K, N> element, int idx) {
    +		queue[idx] = element;
    +		element.setTimerHeapIndex(idx);
    +	}
    +
    +	private HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> getDedupMapForKeyGroup(
    +		TimerHeapInternalTimer<?, ?> timer) {
    +		int keyGroup = KeyGroupRangeAssignment.assignToKeyGroup(timer.getKey(), totalNumberOfKeyGroups);
    +		return getDedupMapForKeyGroup(keyGroup);
    +	}
    +
    +	private int globalKeyGroupToLocalIndex(int keyGroup) {
    +		checkArgument(keyGroupRange.contains(keyGroup));
    +		return keyGroup - keyGroupRange.getStartKeyGroup();
    +	}
    +
    +	private void growIfRequired(int requiredSize) {
    +		int oldArraySize = queue.length;
    +
    +		if (requiredSize >= oldArraySize) {
    +			final int grow = (oldArraySize < 64) ? oldArraySize + 2 : oldArraySize >> 1;
    +			resizeQueueArray(oldArraySize + grow);
    +		}
    --- End diff --
    
    If `requiredSize != oldArraySize + 1`, then this logic might fail. Maybe refactor to call it `growIfRequiredByOne` and do the changes to `size` within in this method. 


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r191441546
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TimerHeapInternalTimer.java ---
    @@ -0,0 +1,246 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.annotation.Internal;
    +import org.apache.flink.api.common.typeutils.CompatibilityResult;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
    +import org.apache.flink.api.common.typeutils.base.LongSerializer;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.core.memory.DataOutputView;
    +
    +import java.io.IOException;
    +
    +/**
    + * Implementation of {@link InternalTimer} for the {@link InternalTimerHeap}.
    + *
    + * @param <K> Type of the keys to which timers are scoped.
    + * @param <N> Type of the namespace to which timers are scoped.
    + */
    +@Internal
    +public final class TimerHeapInternalTimer<K, N> implements InternalTimer<K, N> {
    +
    +	/** The index that indicates that a tracked internal timer is not tracked. */
    +	private static final int NOT_MANAGED_BY_TIMER_QUEUE_INDEX = Integer.MIN_VALUE;
    +
    +	private final long timestamp;
    +
    +	private final K key;
    +
    +	private final N namespace;
    +
    +	/**
    +	 * This field holds the current physical index of this timer when it is managed by a timer heap so that we can
    +	 * support fast deletes.
    +	 */
    +	private transient int timerHeapIndex;
    +
    +	TimerHeapInternalTimer(long timestamp, K key, N namespace) {
    +		this.timestamp = timestamp;
    +		this.key = key;
    +		this.namespace = namespace;
    +		this.timerHeapIndex = NOT_MANAGED_BY_TIMER_QUEUE_INDEX;
    +	}
    +
    +	@Override
    +	public long getTimestamp() {
    +		return timestamp;
    +	}
    +
    +	@Override
    +	public K getKey() {
    +		return key;
    +	}
    +
    +	@Override
    +	public N getNamespace() {
    +		return namespace;
    +	}
    +
    +	@Override
    +	public boolean equals(Object o) {
    +		if (this == o) {
    +			return true;
    +		}
    +
    +		if (o instanceof InternalTimer) {
    +			InternalTimer<?, ?> timer = (InternalTimer<?, ?>) o;
    +			return timestamp == timer.getTimestamp()
    +				&& key.equals(timer.getKey())
    +				&& namespace.equals(timer.getNamespace());
    +		}
    +
    +		return false;
    +	}
    +
    +	/**
    +	 * Returns the current index of this timer in the owning timer heap.
    +	 */
    +	int getTimerHeapIndex() {
    +		return timerHeapIndex;
    +	}
    +
    +	/**
    +	 * Sets the current index of this timer in the owning timer heap and should only be called by the managing heap.
    +	 * @param timerHeapIndex the new index in the timer heap.
    +	 */
    +	void setTimerHeapIndex(int timerHeapIndex) {
    +		this.timerHeapIndex = timerHeapIndex;
    +	}
    +
    +	/**
    +	 * This method can be called to indicate that the timer is no longer managed be a timer heap, e.g. because it as
    +	 * removed.
    +	 */
    +	void removedFromTimerQueue() {
    +		setTimerHeapIndex(NOT_MANAGED_BY_TIMER_QUEUE_INDEX);
    +	}
    +
    +	@Override
    +	public int hashCode() {
    +		int result = (int) (timestamp ^ (timestamp >>> 32));
    +		result = 31 * result + key.hashCode();
    +		result = 31 * result + namespace.hashCode();
    +		return result;
    +	}
    +
    +	@Override
    +	public String toString() {
    +		return "Timer{" +
    +				"timestamp=" + timestamp +
    +				", key=" + key +
    +				", namespace=" + namespace +
    +				'}';
    +	}
    +
    +	/**
    +	 * A {@link TypeSerializer} used to serialize/deserialize a {@link TimerHeapInternalTimer}.
    +	 */
    +	public static class TimerSerializer<K, N> extends TypeSerializer<InternalTimer<K, N>> {
    +
    +		private static final long serialVersionUID = 1119562170939152304L;
    +
    +		private final TypeSerializer<K> keySerializer;
    +
    +		private final TypeSerializer<N> namespaceSerializer;
    +
    +		TimerSerializer(TypeSerializer<K> keySerializer, TypeSerializer<N> namespaceSerializer) {
    +			this.keySerializer = keySerializer;
    +			this.namespaceSerializer = namespaceSerializer;
    +		}
    +
    +		@Override
    +		public boolean isImmutableType() {
    +			return false;
    +		}
    +
    +		@Override
    +		public TypeSerializer<InternalTimer<K, N>> duplicate() {
    +
    +			final TypeSerializer<K> keySerializerDuplicate = keySerializer.duplicate();
    +			final TypeSerializer<N> namespaceSerializerDuplicate = namespaceSerializer.duplicate();
    +
    +			if (keySerializerDuplicate == keySerializer &&
    +				namespaceSerializerDuplicate == namespaceSerializer) {
    +				// all delegate serializers seem stateless, so this is also stateless.
    +				return this;
    +			} else {
    +				// at least one delegate serializer seems to be stateful, so we return a new instance.
    +				return new TimerSerializer<>(keySerializerDuplicate, namespaceSerializerDuplicate);
    +			}
    +		}
    +
    +		@Override
    +		public TimerHeapInternalTimer<K, N> createInstance() {
    +			return null;
    --- End diff --
    
    Maybe we should throw a `OperationNotSupportedException` with a proper exception message instead of risking a NPE somewhere else.


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r191446484
  
    --- Diff: flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/InternalTimerHeapTest.java ---
    @@ -0,0 +1,470 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.runtime.state.KeyGroupRange;
    +import org.apache.flink.runtime.state.VoidNamespace;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.junit.Assert;
    +import org.junit.Test;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Set;
    +import java.util.concurrent.ThreadLocalRandom;
    +import java.util.function.BiFunction;
    +import java.util.function.Function;
    +
    +/**
    + * Tests for {@link InternalTimerHeap}.
    + */
    +public class InternalTimerHeapTest {
    +
    +	private static final KeyGroupRange KEY_GROUP_RANGE = new KeyGroupRange(0, 1);
    +
    +	private static void insertRandomTimers(
    +		InternalTimerHeap<Integer, VoidNamespace> timerPriorityQueue,
    +		int count) {
    --- End diff --
    
    Double indentation for parameter list which are wrapped.


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r190517094
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerHeap.java ---
    @@ -458,24 +458,33 @@ private int globalKeyGroupToLocalIndex(int keyGroup) {
     		return keyGroup - keyGroupRange.getStartKeyGroup();
     	}
     
    -	private void checkCapacity(int requested) {
    +	private void growIfRequired(int requiredSize) {
     		int oldArraySize = queue.length;
     
    -		if (requested >= oldArraySize) {
    +		if (requiredSize >= oldArraySize) {
     			final int grow = (oldArraySize < 64) ? oldArraySize + 2 : oldArraySize >> 1;
    -			int newArraySize = oldArraySize + grow;
    -			if (newArraySize - MAX_ARRAY_SIZE > 0) {
    -				if (newArraySize < 0 || requested > MAX_ARRAY_SIZE) {
    -					throw new OutOfMemoryError("Required timer heap exceeds maximum size!");
    -				} else {
    -					newArraySize = MAX_ARRAY_SIZE;
    -				}
    -			}
    -			queue = Arrays.copyOf(queue, newArraySize);
    +			resizeQueueArray(oldArraySize + grow);
     		}
     		// TODO implement shrinking as well?
     	}
     
    +	private void resizeForBulkLoad(int maxTotalSize) {
    +		if (maxTotalSize > queue.length) {
    +			resizeQueueArray(maxTotalSize + (maxTotalSize / 8));
    --- End diff --
    
    `maxTotalSize  / 8` -> `maxTotalSize >>> 3`


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r191409829
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/HeapInternalTimerService.java ---
    @@ -96,14 +91,12 @@
     
     	public HeapInternalTimerService(
    --- End diff --
    
    Could be package private.


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r191423366
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerHeap.java ---
    @@ -0,0 +1,511 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.annotation.VisibleForTesting;
    +import org.apache.flink.runtime.state.KeyGroupRange;
    +import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
    +
    +import javax.annotation.Nonnegative;
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Queue;
    +import java.util.Set;
    +
    +import static org.apache.flink.util.Preconditions.checkArgument;
    +
    +/**
    + * A heap-based priority queue for internal timers. This heap is supported by hash sets for fast contains
    + * (de-duplication) and deletes. The heap implementation is a simple binary tree stored inside an array. Element indexes
    + * in the heap array start at 1 instead of 0 to make array index computations a bit simpler in the hot methods.
    + *
    + * <p>Possible future improvements:
    + * <ul>
    + *  <li>We could also implement shrinking for the heap and the deduplication maps.</li>
    + *  <li>We could replace the deduplication maps with more efficient custom implementations. In particular, a hash set
    + * would be enough if it could return existing elements on unsuccessful adding, etc..</li>
    + * </ul>
    + *
    + * @param <K> type of the key of the internal timers managed by this priority queue.
    + * @param <N> type of the namespace of the internal timers managed by this priority queue.
    + */
    +public class InternalTimerHeap<K, N> implements Queue<TimerHeapInternalTimer<K, N>>, Set<TimerHeapInternalTimer<K, N>> {
    --- End diff --
    
    That way we could also save us from implementing some methods like `toArray` or `iterator`.


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r191711210
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/HeapInternalTimerService.java ---
    @@ -199,17 +186,9 @@ public long currentWatermark() {
     
     	@Override
     	public void registerProcessingTimeTimer(N namespace, long time) {
    -		InternalTimer<K, N> timer = new InternalTimer<>(time, (K) keyContext.getCurrentKey(), namespace);
    -
    -		// make sure we only put one timer per key into the queue
    -		Set<InternalTimer<K, N>> timerSet = getProcessingTimeTimerSetForTimer(timer);
    -		if (timerSet.add(timer)) {
    -
    -			InternalTimer<K, N> oldHead = processingTimeTimersQueue.peek();
    +		InternalTimer<K, N> oldHead = processingTimeTimersQueue.peek();
    +		if (processingTimeTimersQueue.scheduleTimer(time, (K) keyContext.getCurrentKey(), namespace)) {
    --- End diff --
    
    Good point, I would suggest we do this in another PR.


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r191412983
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/HeapInternalTimerService.java ---
    @@ -199,17 +186,9 @@ public long currentWatermark() {
     
     	@Override
     	public void registerProcessingTimeTimer(N namespace, long time) {
    -		InternalTimer<K, N> timer = new InternalTimer<>(time, (K) keyContext.getCurrentKey(), namespace);
    -
    -		// make sure we only put one timer per key into the queue
    -		Set<InternalTimer<K, N>> timerSet = getProcessingTimeTimerSetForTimer(timer);
    -		if (timerSet.add(timer)) {
    -
    -			InternalTimer<K, N> oldHead = processingTimeTimersQueue.peek();
    +		InternalTimer<K, N> oldHead = processingTimeTimersQueue.peek();
    +		if (processingTimeTimersQueue.scheduleTimer(time, (K) keyContext.getCurrentKey(), namespace)) {
    --- End diff --
    
    I know this is out of scope, but I think we could get rid of the `KeyContext` by passing the current key to the `registerProcessingTimeTimer` method. Moreover, instead of calling `KeyContext#setCurrentKey` we could pass the key value to the `Triggerable#onEvent/ProcessingTime` method. Triggering side effects via the `KeyContext` before calling certain methods is imo very brittle.


---

[GitHub] flink issue #6062: [FLINK-9423][state] Implement efficient deletes for heap-...

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

    https://github.com/apache/flink/pull/6062
  
    Thanks for the reviews guys! I think I addressed your comments and will merge now.


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r191435370
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerHeap.java ---
    @@ -0,0 +1,511 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.annotation.VisibleForTesting;
    +import org.apache.flink.runtime.state.KeyGroupRange;
    +import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
    +
    +import javax.annotation.Nonnegative;
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Queue;
    +import java.util.Set;
    +
    +import static org.apache.flink.util.Preconditions.checkArgument;
    +
    +/**
    + * A heap-based priority queue for internal timers. This heap is supported by hash sets for fast contains
    + * (de-duplication) and deletes. The heap implementation is a simple binary tree stored inside an array. Element indexes
    + * in the heap array start at 1 instead of 0 to make array index computations a bit simpler in the hot methods.
    + *
    + * <p>Possible future improvements:
    + * <ul>
    + *  <li>We could also implement shrinking for the heap and the deduplication maps.</li>
    + *  <li>We could replace the deduplication maps with more efficient custom implementations. In particular, a hash set
    + * would be enough if it could return existing elements on unsuccessful adding, etc..</li>
    + * </ul>
    + *
    + * @param <K> type of the key of the internal timers managed by this priority queue.
    + * @param <N> type of the namespace of the internal timers managed by this priority queue.
    + */
    +public class InternalTimerHeap<K, N> implements Queue<TimerHeapInternalTimer<K, N>>, Set<TimerHeapInternalTimer<K, N>> {
    +
    +	/**
    +	 * A safe maximum size for arrays in the JVM.
    +	 */
    +	private static final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8;
    +
    +	/**
    +	 * Comparator for {@link TimerHeapInternalTimer}, based on the timestamp in ascending order.
    +	 */
    +	private static final Comparator<TimerHeapInternalTimer<?, ?>> COMPARATOR =
    +		(o1, o2) -> Long.compare(o1.getTimestamp(), o2.getTimestamp());
    +
    +	/**
    +	 * This array contains one hash set per key-group. The sets are used for fast de-duplication and deletes of timers.
    +	 */
    +	private final HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>>[] deduplicationMapsByKeyGroup;
    +
    +	/**
    +	 * The array that represents the heap-organized priority queue.
    +	 */
    +	private TimerHeapInternalTimer<K, N>[] queue;
    +
    +	/**
    +	 * The current size of the priority queue.
    +	 */
    +	private int size;
    +
    +	/**
    +	 * The key-group range of timers that are managed by this queue.
    +	 */
    +	private final KeyGroupRange keyGroupRange;
    +
    +	/**
    +	 * The total number of key-groups of the job.
    +	 */
    +	private final int totalNumberOfKeyGroups;
    +
    +
    +	/**
    +	 * Creates an empty {@link InternalTimerHeap} with the requested initial capacity.
    +	 *
    +	 * @param minimumCapacity the minimum and initial capacity of this priority queue.
    +	 */
    +	@SuppressWarnings("unchecked")
    +	InternalTimerHeap(
    +		@Nonnegative int minimumCapacity,
    +		@Nonnull KeyGroupRange keyGroupRange,
    +		@Nonnegative int totalNumberOfKeyGroups) {
    +
    +		this.totalNumberOfKeyGroups = totalNumberOfKeyGroups;
    +		this.keyGroupRange = keyGroupRange;
    +
    +		final int keyGroupsInLocalRange = keyGroupRange.getNumberOfKeyGroups();
    +		final int deduplicationSetSize = 1 + minimumCapacity / keyGroupsInLocalRange;
    +		this.deduplicationMapsByKeyGroup = new HashMap[keyGroupsInLocalRange];
    +		for (int i = 0; i < keyGroupsInLocalRange; ++i) {
    +			deduplicationMapsByKeyGroup[i] = new HashMap<>(deduplicationSetSize);
    +		}
    +
    +		this.queue = new TimerHeapInternalTimer[1 + minimumCapacity];
    +	}
    +
    +	/**
    +	 * @see Set#add(Object)
    +	 */
    +	@Override
    +	public boolean add(@Nonnull TimerHeapInternalTimer<K, N> timer) {
    +
    +		if (getDedupMapForKeyGroup(timer).putIfAbsent(timer, timer) == null) {
    +			final int newSize = ++this.size;
    +			growIfRequired(newSize);
    +			moveElementToIdx(timer, newSize);
    +			siftUp(newSize);
    +			return true;
    +		} else {
    +			return false;
    +		}
    +	}
    +
    +	/**
    +	 * This behaves like {@link #add(TimerHeapInternalTimer)}.
    +	 */
    +	@Override
    +	public boolean offer(@Nonnull TimerHeapInternalTimer<K, N> k) {
    +		return add(k);
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> poll() {
    +		return size() > 0 ? removeElementAtIndex(1) : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> remove() {
    +		TimerHeapInternalTimer<K, N> pollResult = poll();
    +		if (pollResult != null) {
    +			return pollResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> peek() {
    +		return size() > 0 ? queue[1] : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> element() {
    +		TimerHeapInternalTimer<K, N> peekResult = peek();
    +		if (peekResult != null) {
    +			return peekResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Override
    +	public boolean isEmpty() {
    +		return size() == 0;
    +	}
    +
    +	@Override
    +	public boolean contains(@Nullable Object toCheck) {
    +		return (toCheck instanceof TimerHeapInternalTimer)
    +			&& getDedupMapForKeyGroup((TimerHeapInternalTimer<?, ?>) toCheck).containsKey(toCheck);
    +	}
    +
    +	@Override
    +	public boolean remove(@Nullable Object toRemove) {
    +		if (toRemove instanceof TimerHeapInternalTimer) {
    +			return removeInternal((TimerHeapInternalTimer<?, ?>) toRemove);
    +		}
    +		return false;
    +	}
    +
    +	@Override
    +	public boolean addAll(@Nullable Collection<? extends TimerHeapInternalTimer<K, N>> timers) {
    +
    +		if (timers == null) {
    +			return true;
    +		}
    +
    +		final int oldSize = size();
    +		resizeForBulkLoad(oldSize + timers.size());
    +
    +		for (TimerHeapInternalTimer<K, N> k : timers) {
    +			add(k);
    +		}
    +
    +		return oldSize != size();
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public Object[] toArray() {
    +		return Arrays.copyOfRange(queue, 1, size + 1);
    +	}
    +
    +	@SuppressWarnings({"unchecked", "SuspiciousSystemArraycopy"})
    +	@Nonnull
    +	@Override
    +	public <T> T[] toArray(@Nonnull T[] array) {
    +		if (array.length >= size) {
    +			System.arraycopy(queue, 1, array, 0, size);
    +			return array;
    +		}
    +		return (T[]) Arrays.copyOfRange(queue, 1, size + 1, array.getClass());
    +	}
    +
    +	@Override
    +	public boolean removeAll(@Nullable Collection<?> toRemove) {
    +
    +		if (toRemove == null) {
    +			return false;
    +		}
    +
    +		int oldSize = size();
    +		for (Object o : toRemove) {
    +			remove(o);
    +		}
    +		return oldSize != size();
    +	}
    +
    +	/**
    +	 * Returns an iterator over the elements in this queue. The iterator
    +	 * does not return the elements in any particular order.
    +	 *
    +	 * @return an iterator over the elements in this queue.
    +	 */
    +	@Nonnull
    +	@Override
    +	public Iterator<TimerHeapInternalTimer<K, N>> iterator() {
    +		return new InternalTimerPriorityQueueIterator();
    +	}
    +
    +	@Override
    +	public boolean containsAll(@Nullable Collection<?> toCheck) {
    +
    +		if (toCheck == null) {
    +			return true;
    +		}
    +
    +		for (Object o : toCheck) {
    +			if (!contains(o)) {
    +				return false;
    +			}
    +		}
    +
    +		return true;
    +	}
    +
    +	@Nonnegative
    +	@Override
    +	public int size() {
    +		return size;
    +	}
    +
    +	@Override
    +	public void clear() {
    +
    +		Arrays.fill(queue, null);
    +		for (HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> timerHashMap :
    +			deduplicationMapsByKeyGroup) {
    +			timerHashMap.clear();
    +		}
    +		size = 0;
    +	}
    +
    +	/**
    +	 * This method is currently not implemented.
    +	 */
    +	@Override
    +	public boolean retainAll(@Nullable Collection<?> toRetain) {
    +		throw new UnsupportedOperationException();
    +	}
    +
    +	/**
    +	 * Adds a new timer with the given timestamp, key, and namespace to the heap, if an identical timer was not yet
    +	 * registered.
    +	 *
    +	 * @param timestamp the timer timestamp.
    +	 * @param key the timer key.
    +	 * @param namespace the timer namespace.
    +	 * @return true iff a new timer with given timestamp, key, and namespace was added to the heap.
    +	 */
    +	boolean scheduleTimer(long timestamp, K key, N namespace) {
    +		return add(new TimerHeapInternalTimer<>(timestamp, key, namespace));
    +	}
    +
    +	/**
    +	 * Stops timer with the given timestamp, key, and namespace by removing it from the heap, if it exists on the heap.
    +	 *
    +	 * @param timestamp the timer timestamp.
    +	 * @param key the timer key.
    +	 * @param namespace the timer namespace.
    +	 * @return true iff a timer with given timestamp, key, and namespace was found and removed from the heap.
    +	 */
    +	boolean stopTimer(long timestamp, K key, N namespace) {
    +		return removeInternal(new TimerHeapInternalTimer<>(timestamp, key, namespace));
    +	}
    +
    +	/**
    +	 * This method adds all the given timers to the heap.
    +	 */
    +	void addRestoredTimers(Collection<? extends InternalTimer<K, N>> restoredTimers) {
    +
    +		if (restoredTimers == null) {
    +			return;
    +		}
    +
    +		resizeForBulkLoad(restoredTimers.size());
    +
    +		for (InternalTimer<K, N> timer : restoredTimers) {
    +			if (timer instanceof TimerHeapInternalTimer) {
    +				add((TimerHeapInternalTimer<K, N>) timer);
    +			} else {
    +				scheduleTimer(timer.getTimestamp(), timer.getKey(), timer.getNamespace());
    +			}
    +		}
    +	}
    +
    +	private boolean removeInternal(TimerHeapInternalTimer<?, ?> timerToRemove) {
    +
    +		TimerHeapInternalTimer<K, N> storedTimer = getDedupMapForKeyGroup(timerToRemove).remove(timerToRemove);
    +
    +		if (storedTimer != null) {
    +			removeElementAtIndex(storedTimer.getTimerHeapIndex());
    +			return true;
    +		}
    +
    +		return false;
    +	}
    +
    +	private TimerHeapInternalTimer<K, N> removeElementAtIndex(int removeIdx) {
    +		TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		TimerHeapInternalTimer<K, N> removedValue = heap[removeIdx];
    +
    +		assert removedValue.getTimerHeapIndex() == removeIdx;
    +
    +		final int oldSize = size;
    +
    +		if (removeIdx != oldSize) {
    +			TimerHeapInternalTimer<K, N> timer = heap[oldSize];
    +			moveElementToIdx(timer, removeIdx);
    +			siftDown(removeIdx);
    +			if (heap[removeIdx] == timer) {
    +				siftUp(removeIdx);
    +			}
    +		}
    +
    +		heap[oldSize] = null;
    +		getDedupMapForKeyGroup(removedValue).remove(removedValue);
    +
    +		--size;
    +		return removedValue;
    +	}
    +
    +	private void siftUp(int idx) {
    +		final TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		TimerHeapInternalTimer<K, N> currentTimer = heap[idx];
    +		int parentIdx = idx >>> 1;
    +
    +		while (parentIdx > 0 && COMPARATOR.compare(currentTimer, heap[parentIdx]) < 0) {
    +			moveElementToIdx(heap[parentIdx], idx);
    +			idx = parentIdx;
    +			parentIdx = parentIdx >>> 1;
    +		}
    +
    +		moveElementToIdx(currentTimer, idx);
    +	}
    +
    +	private void siftDown(int idx) {
    +		final TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		final int heapSize = this.size;
    +
    +		TimerHeapInternalTimer<K, N> currentTimer = heap[idx];
    +		int firstChildIdx = idx << 1;
    +		int secondChildIdx = firstChildIdx + 1;
    +
    +		if (secondChildIdx <= heapSize && COMPARATOR.compare(heap[secondChildIdx], heap[firstChildIdx]) < 0) {
    +			firstChildIdx = secondChildIdx;
    +		}
    +
    +		while (firstChildIdx <= heapSize && COMPARATOR.compare(heap[firstChildIdx], currentTimer) < 0) {
    +			moveElementToIdx(heap[firstChildIdx], idx);
    +			idx = firstChildIdx;
    +			firstChildIdx = idx << 1;
    +			secondChildIdx = firstChildIdx + 1;
    +
    +			if (secondChildIdx <= heapSize && COMPARATOR.compare(heap[secondChildIdx], heap[firstChildIdx]) < 0) {
    +				firstChildIdx = secondChildIdx;
    +			}
    +		}
    +
    +		moveElementToIdx(currentTimer, idx);
    +	}
    +
    +	/**
    +	 * Returns an unmodifiable set of all timers in the given key-group.
    +	 */
    +	Set<InternalTimer<K, N>> getTimersForKeyGroup(@Nonnegative int keyGroupIdx) {
    +		return Collections.unmodifiableSet(getDedupMapForKeyGroup(keyGroupIdx).keySet());
    +	}
    +
    +	private HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> getDedupMapForKeyGroup(
    +		@Nonnegative int keyGroupIdx) {
    +		return deduplicationMapsByKeyGroup[globalKeyGroupToLocalIndex(keyGroupIdx)];
    +	}
    +
    +	@VisibleForTesting
    +	@SuppressWarnings("unchecked")
    +	List<Set<InternalTimer<K, N>>> getTimersByKeyGroup() {
    +		List<Set<InternalTimer<K, N>>> result = new ArrayList<>(deduplicationMapsByKeyGroup.length);
    +		for (int i = 0; i < deduplicationMapsByKeyGroup.length; ++i) {
    +			result.add(i, Collections.unmodifiableSet(deduplicationMapsByKeyGroup[i].keySet()));
    +		}
    +		return result;
    +	}
    +
    +	private void moveElementToIdx(TimerHeapInternalTimer<K, N> element, int idx) {
    +		queue[idx] = element;
    +		element.setTimerHeapIndex(idx);
    +	}
    +
    +	private HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> getDedupMapForKeyGroup(
    +		TimerHeapInternalTimer<?, ?> timer) {
    +		int keyGroup = KeyGroupRangeAssignment.assignToKeyGroup(timer.getKey(), totalNumberOfKeyGroups);
    +		return getDedupMapForKeyGroup(keyGroup);
    +	}
    +
    +	private int globalKeyGroupToLocalIndex(int keyGroup) {
    +		checkArgument(keyGroupRange.contains(keyGroup));
    +		return keyGroup - keyGroupRange.getStartKeyGroup();
    +	}
    +
    +	private void growIfRequired(int requiredSize) {
    +		int oldArraySize = queue.length;
    +
    +		if (requiredSize >= oldArraySize) {
    +			final int grow = (oldArraySize < 64) ? oldArraySize + 2 : oldArraySize >> 1;
    +			resizeQueueArray(oldArraySize + grow);
    +		}
    +		// TODO implement shrinking as well?
    +	}
    +
    +	private void resizeForBulkLoad(int maxTotalSize) {
    +		if (maxTotalSize > queue.length) {
    +			resizeQueueArray(maxTotalSize + (maxTotalSize >>> 3));
    +		}
    +	}
    +
    +	private void resizeQueueArray(int newArraySize) {
    +		if (newArraySize - MAX_ARRAY_SIZE > 0) {
    +			if (newArraySize < 0 || newArraySize > MAX_ARRAY_SIZE) {
    +				throw new OutOfMemoryError("Required timer heap exceeds maximum size!");
    +			} else {
    +				newArraySize = MAX_ARRAY_SIZE;
    +			}
    +		}
    --- End diff --
    
    Against what are we guarding here exactly? That `newArraySize` is not greater than `MAX_ARRAY_SIZE` and also not negative? Shouldn't it be enough to say `assert(newArraySize > 0 && newArraySize < MAX_ARRAY_SIZE)`?


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r190482258
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerHeap.java ---
    @@ -0,0 +1,504 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.annotation.VisibleForTesting;
    +import org.apache.flink.runtime.state.KeyGroupRange;
    +import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
    +
    +import javax.annotation.Nonnegative;
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Queue;
    +import java.util.Set;
    +
    +import static org.apache.flink.util.Preconditions.checkArgument;
    +
    +/**
    + * A heap-based priority queue for internal timers. This heap is supported by hash sets for fast contains
    + * (de-duplication) and deletes. The heap implementation is a simple binary tree stored inside an array. Element indexes
    + * in the heap array start at 1 instead of 0 to make array index computations a bit simpler in the hot methods.
    + *
    + * <p>Possible future improvements:
    + * <ul>
    + *  <li>We could also implement shrinking for the heap and the deduplication maps.</li>
    + *  <li>We could replace the deduplication maps with more efficient custom implementations. In particular, a hash set
    + * would be enough if it could return existing elements on unsuccessful adding, etc..</li>
    + * </ul>
    + *
    + * @param <K> type of the key of the internal timers managed by this priority queue.
    + * @param <N> type of the namespace of the internal timers managed by this priority queue.
    + */
    +public class InternalTimerHeap<K, N> implements Queue<TimerHeapInternalTimer<K, N>>, Set<TimerHeapInternalTimer<K, N>> {
    +
    +	/**
    +	 * A safe maximum size for arrays in the JVM.
    +	 */
    +	private static final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8;
    +
    +	/**
    +	 * Comparator for {@link TimerHeapInternalTimer}, based on the timestamp in ascending order.
    +	 */
    +	private static final Comparator<TimerHeapInternalTimer<?, ?>> COMPARATOR =
    +		(o1, o2) -> Long.compare(o1.getTimestamp(), o2.getTimestamp());
    +
    +	/**
    +	 * This array contains one hash set per key-group. The sets are used for fast de-duplication and deletes of timers.
    +	 */
    +	private final HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>>[] deduplicationMapsByKeyGroup;
    +
    +	/**
    +	 * The array that represents the heap-organized priority queue.
    +	 */
    +	private TimerHeapInternalTimer<K, N>[] queue;
    +
    +	/**
    +	 * The current size of the priority queue.
    +	 */
    +	private int size;
    +
    +	/**
    +	 * The key-group range of timers that are managed by this queue.
    +	 */
    +	private final KeyGroupRange keyGroupRange;
    +
    +	/**
    +	 * The total number of key-groups of the job.
    +	 */
    +	private final int totalNumberOfKeyGroups;
    +
    +
    +	/**
    +	 * Creates an empty {@link InternalTimerHeap} with the requested initial capacity.
    +	 *
    +	 * @param minimumCapacity the minimum and initial capacity of this priority queue.
    +	 */
    +	@SuppressWarnings("unchecked")
    +	InternalTimerHeap(
    +		@Nonnegative int minimumCapacity,
    +		@Nonnull KeyGroupRange keyGroupRange,
    +		@Nonnegative int totalNumberOfKeyGroups) {
    +
    +		this.totalNumberOfKeyGroups = totalNumberOfKeyGroups;
    +		this.keyGroupRange = keyGroupRange;
    +
    +		final int keyGroupsInLocalRange = keyGroupRange.getNumberOfKeyGroups();
    +		final int deduplicationSetSize = 1 + minimumCapacity / keyGroupsInLocalRange;
    +		this.deduplicationMapsByKeyGroup = new HashMap[keyGroupsInLocalRange];
    +		for (int i = 0; i < keyGroupsInLocalRange; ++i) {
    +			deduplicationMapsByKeyGroup[i] = new HashMap<>(deduplicationSetSize);
    +		}
    +
    +		this.queue = new TimerHeapInternalTimer[1 + minimumCapacity];
    +	}
    +
    +	/**
    +	 * @see Set#add(Object)
    +	 */
    +	@Override
    +	public boolean add(@Nonnull TimerHeapInternalTimer<K, N> timer) {
    +
    +		if (getDedupMapForKeyGroup(timer).putIfAbsent(timer, timer) == null) {
    +			final int newSize = ++this.size;
    +			checkCapacity(newSize);
    +			moveElementToIdx(timer, newSize);
    +			siftUp(newSize);
    +			return true;
    +		} else {
    +			return false;
    +		}
    +	}
    +
    +	/**
    +	 * This behaves like {@link #add(TimerHeapInternalTimer)}.
    +	 */
    +	@Override
    +	public boolean offer(@Nonnull TimerHeapInternalTimer<K, N> k) {
    +		return add(k);
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> poll() {
    +		return size() > 0 ? removeElementAtIndex(1) : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> remove() {
    +		TimerHeapInternalTimer<K, N> pollResult = poll();
    +		if (pollResult != null) {
    +			return pollResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> peek() {
    +		return size() > 0 ? queue[1] : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> element() {
    +		TimerHeapInternalTimer<K, N> peekResult = peek();
    +		if (peekResult != null) {
    +			return peekResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Override
    +	public boolean isEmpty() {
    +		return size() == 0;
    +	}
    +
    +	@Override
    +	public boolean contains(@Nullable Object o) {
    +		return (o instanceof TimerHeapInternalTimer)
    +			&& getDedupMapForKeyGroup((TimerHeapInternalTimer<?, ?>) o).containsKey(o);
    +	}
    +
    +	@Override
    +	public boolean remove(@Nullable Object o) {
    +		if (o instanceof TimerHeapInternalTimer) {
    +			return removeInternal((TimerHeapInternalTimer<?, ?>) o);
    +		}
    +		return false;
    +	}
    +
    +	@Override
    +	public boolean addAll(@Nullable Collection<? extends TimerHeapInternalTimer<K, N>> timers) {
    +
    +		if (timers == null) {
    +			return true;
    +		}
    +
    +		if (timers.size() > queue.length) {
    +			checkCapacity(timers.size());
    +		}
    +
    +		for (TimerHeapInternalTimer<K, N> k : timers) {
    +			add(k);
    +		}
    +
    +		return true;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public Object[] toArray() {
    +		return Arrays.copyOfRange(queue, 1, size + 1);
    +	}
    +
    +	@SuppressWarnings({"unchecked", "SuspiciousSystemArraycopy"})
    +	@Nonnull
    +	@Override
    +	public <T> T[] toArray(@Nonnull T[] array) {
    +		if (array.length >= size) {
    +			System.arraycopy(queue, 1, array, 0, size);
    +			return array;
    +		}
    +		return (T[]) Arrays.copyOfRange(queue, 1, size + 1, array.getClass());
    +	}
    +
    +	@Override
    +	public boolean removeAll(@Nullable Collection<?> toRemove) {
    +
    +		if (toRemove == null) {
    +			return false;
    +		}
    +
    +		int oldSize = size();
    +		for (Object o : toRemove) {
    +			remove(o);
    +		}
    +		return size() == oldSize;
    +	}
    +
    +	/**
    +	 * Returns an iterator over the elements in this queue. The iterator
    +	 * does not return the elements in any particular order.
    +	 *
    +	 * @return an iterator over the elements in this queue.
    +	 */
    +	@Nonnull
    +	@Override
    +	public Iterator<TimerHeapInternalTimer<K, N>> iterator() {
    +		return new InternalTimerPriorityQueueIterator();
    +	}
    +
    +	@Override
    +	public boolean containsAll(@Nullable Collection<?> toCheck) {
    +
    +		if (toCheck == null) {
    +			return true;
    +		}
    +
    +		for (Object o : toCheck) {
    +			if (!contains(o)) {
    +				return false;
    +			}
    +		}
    +
    +		return true;
    +	}
    +
    +	@Nonnegative
    +	@Override
    +	public int size() {
    +		return size;
    +	}
    +
    +	@Override
    +	public void clear() {
    +
    +		Arrays.fill(queue, null);
    +		for (HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> timerHashMap :
    +			deduplicationMapsByKeyGroup) {
    +			timerHashMap.clear();
    +		}
    +		size = 0;
    +	}
    +
    +	/**
    +	 * This method is currently not implemented.
    +	 */
    +	@Override
    +	public boolean retainAll(@Nullable Collection<?> toRetain) {
    +		throw new UnsupportedOperationException();
    +	}
    +
    +	/**
    +	 * Adds a new timer with the given timestamp, key, and namespace to the heap, if an identical timer was not yet
    +	 * registered.
    +	 *
    +	 * @param timestamp the timer timestamp.
    +	 * @param key the timer key.
    +	 * @param namespace the timer namespace.
    +	 * @return true iff a new timer with given timestamp, key, and namespace was added to the heap.
    +	 */
    +	boolean scheduleTimer(long timestamp, K key, N namespace) {
    +		return add(new TimerHeapInternalTimer<>(timestamp, key, namespace));
    +	}
    +
    +	/**
    +	 * Stops timer with the given timestamp, key, and namespace by removing it from the heap, if it exists on the heap.
    +	 *
    +	 * @param timestamp the timer timestamp.
    +	 * @param key the timer key.
    +	 * @param namespace the timer namespace.
    +	 * @return true iff a timer with given timestamp, key, and namespace was found and removed from the heap.
    +	 */
    +	boolean stopTimer(long timestamp, K key, N namespace) {
    +		return removeInternal(new TimerHeapInternalTimer<>(timestamp, key, namespace));
    +	}
    +
    +	/**
    +	 * This method adds all the given timers to the heap.
    +	 */
    +	void restoreTimers(Collection<? extends InternalTimer<K, N>> toAdd) {
    +		if (toAdd == null) {
    +			return;
    +		}
    +
    +		if (toAdd.size() > queue.length) {
    +			checkCapacity(toAdd.size());
    +		}
    +
    +		for (InternalTimer<K, N> k : toAdd) {
    +			if (k instanceof TimerHeapInternalTimer) {
    +				add((TimerHeapInternalTimer<K, N>) k);
    +			} else {
    +				scheduleTimer(k.getTimestamp(), k.getKey(), k.getNamespace());
    +			}
    +		}
    +	}
    +
    +	private boolean removeInternal(TimerHeapInternalTimer<?, ?> timerToRemove) {
    +
    +		TimerHeapInternalTimer<K, N> storedTimer = getDedupMapForKeyGroup(timerToRemove).remove(timerToRemove);
    +
    +		if (storedTimer != null) {
    +			removeElementAtIndex(storedTimer.getTimerHeapIndex());
    +			return true;
    +		}
    +
    +		return false;
    +	}
    +
    +	private TimerHeapInternalTimer<K, N> removeElementAtIndex(int removeIdx) {
    +		TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		TimerHeapInternalTimer<K, N> removedValue = heap[removeIdx];
    +
    +		assert removedValue.getTimerHeapIndex() == removeIdx;
    +
    +		final int oldSize = size;
    +
    +		if (removeIdx != oldSize) {
    +			TimerHeapInternalTimer<K, N> timer = heap[oldSize];
    +			moveElementToIdx(timer, removeIdx);
    +			siftDown(removeIdx);
    +			if (heap[removeIdx] == timer) {
    +				siftUp(removeIdx);
    +			}
    +		}
    +
    +		heap[oldSize] = null;
    +		getDedupMapForKeyGroup(removedValue).remove(removedValue);
    +
    +		--size;
    +		return removedValue;
    +	}
    +
    +	private void siftUp(int idx) {
    +		final TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		TimerHeapInternalTimer<K, N> currentTimer = heap[idx];
    +		int parentIdx = idx >>> 1;
    +
    +		while (parentIdx > 0 && COMPARATOR.compare(currentTimer, heap[parentIdx]) < 0) {
    +			moveElementToIdx(heap[parentIdx], idx);
    +			idx = parentIdx;
    +			parentIdx = parentIdx >>> 1;
    +		}
    +
    +		moveElementToIdx(currentTimer, idx);
    +	}
    +
    +	private void siftDown(int idx) {
    +		final TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		final int heapSize = this.size;
    +
    +		TimerHeapInternalTimer<K, N> currentTimer = heap[idx];
    +		int firstChildIdx = idx << 1;
    +		int secondChildIdx = firstChildIdx + 1;
    +
    +		if (secondChildIdx <= heapSize && COMPARATOR.compare(heap[secondChildIdx], heap[firstChildIdx]) < 0) {
    +			firstChildIdx = secondChildIdx;
    +		}
    +
    +		while (firstChildIdx <= heapSize && COMPARATOR.compare(heap[firstChildIdx], currentTimer) < 0) {
    +			moveElementToIdx(heap[firstChildIdx], idx);
    +			idx = firstChildIdx;
    +			firstChildIdx = idx << 1;
    +			secondChildIdx = firstChildIdx + 1;
    +
    +			if (secondChildIdx <= heapSize && COMPARATOR.compare(heap[secondChildIdx], heap[firstChildIdx]) < 0) {
    +				firstChildIdx = secondChildIdx;
    +			}
    +		}
    +
    +		moveElementToIdx(currentTimer, idx);
    +	}
    +
    +	/**
    +	 * Returns an unmodifiable set of all timers in the given key-group.
    +	 */
    +	Set<InternalTimer<K, N>> getTimersForKeyGroup(@Nonnegative int keyGroupIdx) {
    +		return Collections.unmodifiableSet(getDedupMapForKeyGroup(keyGroupIdx).keySet());
    +	}
    +
    +	private HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> getDedupMapForKeyGroup(
    +		@Nonnegative int keyGroupIdx) {
    +		return deduplicationMapsByKeyGroup[globalKeyGroupToLocalIndex(keyGroupIdx)];
    +	}
    +
    +	@VisibleForTesting
    +	@SuppressWarnings("unchecked")
    +	List<Set<InternalTimer<K, N>>> getTimersByKeyGroup() {
    +		List<Set<InternalTimer<K, N>>> result = new ArrayList<>(deduplicationMapsByKeyGroup.length);
    +		for (int i = 0; i < deduplicationMapsByKeyGroup.length; ++i) {
    +			result.add(i, Collections.unmodifiableSet(deduplicationMapsByKeyGroup[i].keySet()));
    +		}
    +		return result;
    +	}
    +
    +	private void moveElementToIdx(TimerHeapInternalTimer<K, N> element, int idx) {
    +		queue[idx] = element;
    +		element.setTimerHeapIndex(idx);
    +	}
    +
    +	private HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> getDedupMapForKeyGroup(
    +		TimerHeapInternalTimer<?, ?> timer) {
    +		int keyGroup = KeyGroupRangeAssignment.assignToKeyGroup(timer.getKey(), totalNumberOfKeyGroups);
    +		return getDedupMapForKeyGroup(keyGroup);
    +	}
    +
    +	private int globalKeyGroupToLocalIndex(int keyGroup) {
    +		checkArgument(keyGroupRange.contains(keyGroup));
    +		return keyGroup - keyGroupRange.getStartKeyGroup();
    +	}
    +
    +	private void checkCapacity(int requested) {
    +		int oldArraySize = queue.length;
    +
    +		if (requested >= oldArraySize) {
    +			final int grow = (oldArraySize < 64) ? oldArraySize + 2 : oldArraySize >> 1;
    +			int newArraySize = oldArraySize + grow;
    --- End diff --
    
    Right, the code was initially only used to handle growth in steps of 1 from the `add` method. We can adjust it a bit so that bulk inserts take less growth steps.


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r191731222
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TimerHeapInternalTimer.java ---
    @@ -0,0 +1,246 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.annotation.Internal;
    +import org.apache.flink.api.common.typeutils.CompatibilityResult;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
    +import org.apache.flink.api.common.typeutils.base.LongSerializer;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.core.memory.DataOutputView;
    +
    +import java.io.IOException;
    +
    +/**
    + * Implementation of {@link InternalTimer} for the {@link InternalTimerHeap}.
    + *
    + * @param <K> Type of the keys to which timers are scoped.
    + * @param <N> Type of the namespace to which timers are scoped.
    + */
    +@Internal
    +public final class TimerHeapInternalTimer<K, N> implements InternalTimer<K, N> {
    +
    +	/** The index that indicates that a tracked internal timer is not tracked. */
    +	private static final int NOT_MANAGED_BY_TIMER_QUEUE_INDEX = Integer.MIN_VALUE;
    +
    +	private final long timestamp;
    +
    +	private final K key;
    --- End diff --
    
    The basically this whole class already existed and is marked as addition from the "extract interface" refactoring. Nevertheless, it makes sense to include that as improvement.


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r190293752
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerHeap.java ---
    @@ -0,0 +1,504 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.annotation.VisibleForTesting;
    +import org.apache.flink.runtime.state.KeyGroupRange;
    +import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
    +
    +import javax.annotation.Nonnegative;
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Queue;
    +import java.util.Set;
    +
    +import static org.apache.flink.util.Preconditions.checkArgument;
    +
    +/**
    + * A heap-based priority queue for internal timers. This heap is supported by hash sets for fast contains
    + * (de-duplication) and deletes. The heap implementation is a simple binary tree stored inside an array. Element indexes
    + * in the heap array start at 1 instead of 0 to make array index computations a bit simpler in the hot methods.
    + *
    + * <p>Possible future improvements:
    + * <ul>
    + *  <li>We could also implement shrinking for the heap and the deduplication maps.</li>
    + *  <li>We could replace the deduplication maps with more efficient custom implementations. In particular, a hash set
    + * would be enough if it could return existing elements on unsuccessful adding, etc..</li>
    + * </ul>
    + *
    + * @param <K> type of the key of the internal timers managed by this priority queue.
    + * @param <N> type of the namespace of the internal timers managed by this priority queue.
    + */
    +public class InternalTimerHeap<K, N> implements Queue<TimerHeapInternalTimer<K, N>>, Set<TimerHeapInternalTimer<K, N>> {
    +
    +	/**
    +	 * A safe maximum size for arrays in the JVM.
    +	 */
    +	private static final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8;
    +
    +	/**
    +	 * Comparator for {@link TimerHeapInternalTimer}, based on the timestamp in ascending order.
    +	 */
    +	private static final Comparator<TimerHeapInternalTimer<?, ?>> COMPARATOR =
    +		(o1, o2) -> Long.compare(o1.getTimestamp(), o2.getTimestamp());
    +
    +	/**
    +	 * This array contains one hash set per key-group. The sets are used for fast de-duplication and deletes of timers.
    +	 */
    +	private final HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>>[] deduplicationMapsByKeyGroup;
    +
    +	/**
    +	 * The array that represents the heap-organized priority queue.
    +	 */
    +	private TimerHeapInternalTimer<K, N>[] queue;
    +
    +	/**
    +	 * The current size of the priority queue.
    +	 */
    +	private int size;
    +
    +	/**
    +	 * The key-group range of timers that are managed by this queue.
    +	 */
    +	private final KeyGroupRange keyGroupRange;
    +
    +	/**
    +	 * The total number of key-groups of the job.
    +	 */
    +	private final int totalNumberOfKeyGroups;
    +
    +
    +	/**
    +	 * Creates an empty {@link InternalTimerHeap} with the requested initial capacity.
    +	 *
    +	 * @param minimumCapacity the minimum and initial capacity of this priority queue.
    +	 */
    +	@SuppressWarnings("unchecked")
    +	InternalTimerHeap(
    +		@Nonnegative int minimumCapacity,
    +		@Nonnull KeyGroupRange keyGroupRange,
    +		@Nonnegative int totalNumberOfKeyGroups) {
    +
    +		this.totalNumberOfKeyGroups = totalNumberOfKeyGroups;
    +		this.keyGroupRange = keyGroupRange;
    +
    +		final int keyGroupsInLocalRange = keyGroupRange.getNumberOfKeyGroups();
    +		final int deduplicationSetSize = 1 + minimumCapacity / keyGroupsInLocalRange;
    +		this.deduplicationMapsByKeyGroup = new HashMap[keyGroupsInLocalRange];
    +		for (int i = 0; i < keyGroupsInLocalRange; ++i) {
    +			deduplicationMapsByKeyGroup[i] = new HashMap<>(deduplicationSetSize);
    +		}
    +
    +		this.queue = new TimerHeapInternalTimer[1 + minimumCapacity];
    +	}
    +
    +	/**
    +	 * @see Set#add(Object)
    +	 */
    +	@Override
    +	public boolean add(@Nonnull TimerHeapInternalTimer<K, N> timer) {
    +
    +		if (getDedupMapForKeyGroup(timer).putIfAbsent(timer, timer) == null) {
    +			final int newSize = ++this.size;
    +			checkCapacity(newSize);
    +			moveElementToIdx(timer, newSize);
    +			siftUp(newSize);
    +			return true;
    +		} else {
    +			return false;
    +		}
    +	}
    +
    +	/**
    +	 * This behaves like {@link #add(TimerHeapInternalTimer)}.
    +	 */
    +	@Override
    +	public boolean offer(@Nonnull TimerHeapInternalTimer<K, N> k) {
    +		return add(k);
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> poll() {
    +		return size() > 0 ? removeElementAtIndex(1) : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> remove() {
    +		TimerHeapInternalTimer<K, N> pollResult = poll();
    +		if (pollResult != null) {
    +			return pollResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> peek() {
    +		return size() > 0 ? queue[1] : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> element() {
    +		TimerHeapInternalTimer<K, N> peekResult = peek();
    +		if (peekResult != null) {
    +			return peekResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Override
    +	public boolean isEmpty() {
    +		return size() == 0;
    +	}
    +
    +	@Override
    +	public boolean contains(@Nullable Object o) {
    +		return (o instanceof TimerHeapInternalTimer)
    +			&& getDedupMapForKeyGroup((TimerHeapInternalTimer<?, ?>) o).containsKey(o);
    +	}
    +
    +	@Override
    +	public boolean remove(@Nullable Object o) {
    +		if (o instanceof TimerHeapInternalTimer) {
    +			return removeInternal((TimerHeapInternalTimer<?, ?>) o);
    +		}
    +		return false;
    +	}
    +
    +	@Override
    +	public boolean addAll(@Nullable Collection<? extends TimerHeapInternalTimer<K, N>> timers) {
    +
    +		if (timers == null) {
    +			return true;
    +		}
    +
    +		if (timers.size() > queue.length) {
    +			checkCapacity(timers.size());
    +		}
    +
    +		for (TimerHeapInternalTimer<K, N> k : timers) {
    +			add(k);
    +		}
    +
    +		return true;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public Object[] toArray() {
    +		return Arrays.copyOfRange(queue, 1, size + 1);
    +	}
    +
    +	@SuppressWarnings({"unchecked", "SuspiciousSystemArraycopy"})
    +	@Nonnull
    +	@Override
    +	public <T> T[] toArray(@Nonnull T[] array) {
    +		if (array.length >= size) {
    +			System.arraycopy(queue, 1, array, 0, size);
    +			return array;
    +		}
    +		return (T[]) Arrays.copyOfRange(queue, 1, size + 1, array.getClass());
    +	}
    +
    +	@Override
    +	public boolean removeAll(@Nullable Collection<?> toRemove) {
    +
    +		if (toRemove == null) {
    +			return false;
    +		}
    +
    +		int oldSize = size();
    +		for (Object o : toRemove) {
    +			remove(o);
    +		}
    +		return size() == oldSize;
    +	}
    +
    +	/**
    +	 * Returns an iterator over the elements in this queue. The iterator
    +	 * does not return the elements in any particular order.
    +	 *
    +	 * @return an iterator over the elements in this queue.
    +	 */
    +	@Nonnull
    +	@Override
    +	public Iterator<TimerHeapInternalTimer<K, N>> iterator() {
    +		return new InternalTimerPriorityQueueIterator();
    +	}
    +
    +	@Override
    +	public boolean containsAll(@Nullable Collection<?> toCheck) {
    +
    +		if (toCheck == null) {
    +			return true;
    +		}
    +
    +		for (Object o : toCheck) {
    +			if (!contains(o)) {
    +				return false;
    +			}
    +		}
    +
    +		return true;
    +	}
    +
    +	@Nonnegative
    +	@Override
    +	public int size() {
    +		return size;
    +	}
    +
    +	@Override
    +	public void clear() {
    +
    +		Arrays.fill(queue, null);
    +		for (HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> timerHashMap :
    +			deduplicationMapsByKeyGroup) {
    +			timerHashMap.clear();
    +		}
    +		size = 0;
    +	}
    +
    +	/**
    +	 * This method is currently not implemented.
    +	 */
    +	@Override
    +	public boolean retainAll(@Nullable Collection<?> toRetain) {
    +		throw new UnsupportedOperationException();
    +	}
    +
    +	/**
    +	 * Adds a new timer with the given timestamp, key, and namespace to the heap, if an identical timer was not yet
    +	 * registered.
    +	 *
    +	 * @param timestamp the timer timestamp.
    +	 * @param key the timer key.
    +	 * @param namespace the timer namespace.
    +	 * @return true iff a new timer with given timestamp, key, and namespace was added to the heap.
    +	 */
    +	boolean scheduleTimer(long timestamp, K key, N namespace) {
    +		return add(new TimerHeapInternalTimer<>(timestamp, key, namespace));
    +	}
    +
    +	/**
    +	 * Stops timer with the given timestamp, key, and namespace by removing it from the heap, if it exists on the heap.
    +	 *
    +	 * @param timestamp the timer timestamp.
    +	 * @param key the timer key.
    +	 * @param namespace the timer namespace.
    +	 * @return true iff a timer with given timestamp, key, and namespace was found and removed from the heap.
    +	 */
    +	boolean stopTimer(long timestamp, K key, N namespace) {
    +		return removeInternal(new TimerHeapInternalTimer<>(timestamp, key, namespace));
    +	}
    +
    +	/**
    +	 * This method adds all the given timers to the heap.
    +	 */
    +	void restoreTimers(Collection<? extends InternalTimer<K, N>> toAdd) {
    +		if (toAdd == null) {
    +			return;
    +		}
    +
    +		if (toAdd.size() > queue.length) {
    +			checkCapacity(toAdd.size());
    +		}
    +
    +		for (InternalTimer<K, N> k : toAdd) {
    +			if (k instanceof TimerHeapInternalTimer) {
    +				add((TimerHeapInternalTimer<K, N>) k);
    +			} else {
    +				scheduleTimer(k.getTimestamp(), k.getKey(), k.getNamespace());
    +			}
    +		}
    +	}
    +
    +	private boolean removeInternal(TimerHeapInternalTimer<?, ?> timerToRemove) {
    +
    +		TimerHeapInternalTimer<K, N> storedTimer = getDedupMapForKeyGroup(timerToRemove).remove(timerToRemove);
    +
    +		if (storedTimer != null) {
    +			removeElementAtIndex(storedTimer.getTimerHeapIndex());
    +			return true;
    +		}
    +
    +		return false;
    +	}
    +
    +	private TimerHeapInternalTimer<K, N> removeElementAtIndex(int removeIdx) {
    +		TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		TimerHeapInternalTimer<K, N> removedValue = heap[removeIdx];
    +
    +		assert removedValue.getTimerHeapIndex() == removeIdx;
    +
    +		final int oldSize = size;
    +
    +		if (removeIdx != oldSize) {
    +			TimerHeapInternalTimer<K, N> timer = heap[oldSize];
    +			moveElementToIdx(timer, removeIdx);
    +			siftDown(removeIdx);
    +			if (heap[removeIdx] == timer) {
    +				siftUp(removeIdx);
    +			}
    +		}
    +
    +		heap[oldSize] = null;
    +		getDedupMapForKeyGroup(removedValue).remove(removedValue);
    +
    +		--size;
    +		return removedValue;
    +	}
    +
    +	private void siftUp(int idx) {
    +		final TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		TimerHeapInternalTimer<K, N> currentTimer = heap[idx];
    +		int parentIdx = idx >>> 1;
    +
    +		while (parentIdx > 0 && COMPARATOR.compare(currentTimer, heap[parentIdx]) < 0) {
    +			moveElementToIdx(heap[parentIdx], idx);
    +			idx = parentIdx;
    +			parentIdx = parentIdx >>> 1;
    +		}
    +
    +		moveElementToIdx(currentTimer, idx);
    +	}
    +
    +	private void siftDown(int idx) {
    +		final TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		final int heapSize = this.size;
    +
    +		TimerHeapInternalTimer<K, N> currentTimer = heap[idx];
    +		int firstChildIdx = idx << 1;
    +		int secondChildIdx = firstChildIdx + 1;
    +
    +		if (secondChildIdx <= heapSize && COMPARATOR.compare(heap[secondChildIdx], heap[firstChildIdx]) < 0) {
    +			firstChildIdx = secondChildIdx;
    +		}
    +
    +		while (firstChildIdx <= heapSize && COMPARATOR.compare(heap[firstChildIdx], currentTimer) < 0) {
    +			moveElementToIdx(heap[firstChildIdx], idx);
    +			idx = firstChildIdx;
    +			firstChildIdx = idx << 1;
    +			secondChildIdx = firstChildIdx + 1;
    +
    +			if (secondChildIdx <= heapSize && COMPARATOR.compare(heap[secondChildIdx], heap[firstChildIdx]) < 0) {
    +				firstChildIdx = secondChildIdx;
    +			}
    +		}
    +
    +		moveElementToIdx(currentTimer, idx);
    +	}
    +
    +	/**
    +	 * Returns an unmodifiable set of all timers in the given key-group.
    +	 */
    +	Set<InternalTimer<K, N>> getTimersForKeyGroup(@Nonnegative int keyGroupIdx) {
    +		return Collections.unmodifiableSet(getDedupMapForKeyGroup(keyGroupIdx).keySet());
    +	}
    +
    +	private HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> getDedupMapForKeyGroup(
    +		@Nonnegative int keyGroupIdx) {
    +		return deduplicationMapsByKeyGroup[globalKeyGroupToLocalIndex(keyGroupIdx)];
    +	}
    +
    +	@VisibleForTesting
    +	@SuppressWarnings("unchecked")
    +	List<Set<InternalTimer<K, N>>> getTimersByKeyGroup() {
    +		List<Set<InternalTimer<K, N>>> result = new ArrayList<>(deduplicationMapsByKeyGroup.length);
    +		for (int i = 0; i < deduplicationMapsByKeyGroup.length; ++i) {
    +			result.add(i, Collections.unmodifiableSet(deduplicationMapsByKeyGroup[i].keySet()));
    +		}
    +		return result;
    +	}
    +
    +	private void moveElementToIdx(TimerHeapInternalTimer<K, N> element, int idx) {
    +		queue[idx] = element;
    +		element.setTimerHeapIndex(idx);
    +	}
    +
    +	private HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> getDedupMapForKeyGroup(
    +		TimerHeapInternalTimer<?, ?> timer) {
    +		int keyGroup = KeyGroupRangeAssignment.assignToKeyGroup(timer.getKey(), totalNumberOfKeyGroups);
    +		return getDedupMapForKeyGroup(keyGroup);
    +	}
    +
    +	private int globalKeyGroupToLocalIndex(int keyGroup) {
    +		checkArgument(keyGroupRange.contains(keyGroup));
    +		return keyGroup - keyGroupRange.getStartKeyGroup();
    +	}
    +
    +	private void checkCapacity(int requested) {
    +		int oldArraySize = queue.length;
    +
    +		if (requested >= oldArraySize) {
    --- End diff --
    
    I think `requested >= oldArraySize` is correct. I agree that the `if` check can is redundant with the bulk method, but not e.g. with `add`.


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r191732272
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimersSnapshotReaderWriters.java ---
    @@ -96,7 +96,7 @@ public AbstractInternalTimersSnapshotWriter(InternalTimersSnapshot<K, N> timersS
     		public final void writeTimersSnapshot(DataOutputView out) throws IOException {
     			writeKeyAndNamespaceSerializers(out);
     
    -			InternalTimer.TimerSerializer<K, N> timerSerializer = new InternalTimer.TimerSerializer<>(
    +			TimerHeapInternalTimer.TimerSerializer<K, N> timerSerializer = new TimerHeapInternalTimer.TimerSerializer<>(
    --- End diff --
    
    I think this is ok for the moment and we can adjust in case there will ever be another implementation of `InternalTimer `


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r191445752
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimersSnapshotReaderWriters.java ---
    @@ -96,7 +96,7 @@ public AbstractInternalTimersSnapshotWriter(InternalTimersSnapshot<K, N> timersS
     		public final void writeTimersSnapshot(DataOutputView out) throws IOException {
     			writeKeyAndNamespaceSerializers(out);
     
    -			InternalTimer.TimerSerializer<K, N> timerSerializer = new InternalTimer.TimerSerializer<>(
    +			TimerHeapInternalTimer.TimerSerializer<K, N> timerSerializer = new TimerHeapInternalTimer.TimerSerializer<>(
    --- End diff --
    
    Isn't this a bit problematic if we add a new `TimerService` implementation other than the `HeapInternalTimerService`? The `InternalTimersSnapshot` is independent of the underlying timer service implementation and so should the timer serializer be.


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r191422654
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerHeap.java ---
    @@ -0,0 +1,511 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.annotation.VisibleForTesting;
    +import org.apache.flink.runtime.state.KeyGroupRange;
    +import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
    +
    +import javax.annotation.Nonnegative;
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Queue;
    +import java.util.Set;
    +
    +import static org.apache.flink.util.Preconditions.checkArgument;
    +
    +/**
    + * A heap-based priority queue for internal timers. This heap is supported by hash sets for fast contains
    + * (de-duplication) and deletes. The heap implementation is a simple binary tree stored inside an array. Element indexes
    + * in the heap array start at 1 instead of 0 to make array index computations a bit simpler in the hot methods.
    + *
    + * <p>Possible future improvements:
    + * <ul>
    + *  <li>We could also implement shrinking for the heap and the deduplication maps.</li>
    + *  <li>We could replace the deduplication maps with more efficient custom implementations. In particular, a hash set
    + * would be enough if it could return existing elements on unsuccessful adding, etc..</li>
    + * </ul>
    + *
    + * @param <K> type of the key of the internal timers managed by this priority queue.
    + * @param <N> type of the namespace of the internal timers managed by this priority queue.
    + */
    +public class InternalTimerHeap<K, N> implements Queue<TimerHeapInternalTimer<K, N>>, Set<TimerHeapInternalTimer<K, N>> {
    --- End diff --
    
    Why does this class implement the `Queue` and `Set` interface? Is it intended to be used as a `Queue` or `Set` somewhere?


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r191446041
  
    --- Diff: flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/InternalTimerHeapTest.java ---
    @@ -0,0 +1,470 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.runtime.state.KeyGroupRange;
    +import org.apache.flink.runtime.state.VoidNamespace;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.junit.Assert;
    +import org.junit.Test;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Set;
    +import java.util.concurrent.ThreadLocalRandom;
    +import java.util.function.BiFunction;
    +import java.util.function.Function;
    +
    +/**
    + * Tests for {@link InternalTimerHeap}.
    + */
    +public class InternalTimerHeapTest {
    --- End diff --
    
    Should extend `TestLogger`.


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r191440434
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TimerHeapInternalTimer.java ---
    @@ -0,0 +1,246 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.annotation.Internal;
    +import org.apache.flink.api.common.typeutils.CompatibilityResult;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
    +import org.apache.flink.api.common.typeutils.base.LongSerializer;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.core.memory.DataOutputView;
    +
    +import java.io.IOException;
    +
    +/**
    + * Implementation of {@link InternalTimer} for the {@link InternalTimerHeap}.
    + *
    + * @param <K> Type of the keys to which timers are scoped.
    + * @param <N> Type of the namespace to which timers are scoped.
    + */
    +@Internal
    +public final class TimerHeapInternalTimer<K, N> implements InternalTimer<K, N> {
    +
    +	/** The index that indicates that a tracked internal timer is not tracked. */
    +	private static final int NOT_MANAGED_BY_TIMER_QUEUE_INDEX = Integer.MIN_VALUE;
    +
    +	private final long timestamp;
    +
    +	private final K key;
    +
    +	private final N namespace;
    +
    +	/**
    +	 * This field holds the current physical index of this timer when it is managed by a timer heap so that we can
    +	 * support fast deletes.
    +	 */
    +	private transient int timerHeapIndex;
    +
    +	TimerHeapInternalTimer(long timestamp, K key, N namespace) {
    +		this.timestamp = timestamp;
    +		this.key = key;
    +		this.namespace = namespace;
    +		this.timerHeapIndex = NOT_MANAGED_BY_TIMER_QUEUE_INDEX;
    +	}
    +
    +	@Override
    +	public long getTimestamp() {
    +		return timestamp;
    +	}
    +
    +	@Override
    +	public K getKey() {
    +		return key;
    +	}
    +
    +	@Override
    +	public N getNamespace() {
    +		return namespace;
    +	}
    +
    +	@Override
    +	public boolean equals(Object o) {
    +		if (this == o) {
    +			return true;
    +		}
    +
    +		if (o instanceof InternalTimer) {
    +			InternalTimer<?, ?> timer = (InternalTimer<?, ?>) o;
    +			return timestamp == timer.getTimestamp()
    +				&& key.equals(timer.getKey())
    +				&& namespace.equals(timer.getNamespace());
    +		}
    +
    +		return false;
    +	}
    +
    +	/**
    +	 * Returns the current index of this timer in the owning timer heap.
    +	 */
    +	int getTimerHeapIndex() {
    +		return timerHeapIndex;
    +	}
    +
    +	/**
    +	 * Sets the current index of this timer in the owning timer heap and should only be called by the managing heap.
    +	 * @param timerHeapIndex the new index in the timer heap.
    +	 */
    +	void setTimerHeapIndex(int timerHeapIndex) {
    +		this.timerHeapIndex = timerHeapIndex;
    +	}
    +
    +	/**
    +	 * This method can be called to indicate that the timer is no longer managed be a timer heap, e.g. because it as
    +	 * removed.
    +	 */
    +	void removedFromTimerQueue() {
    +		setTimerHeapIndex(NOT_MANAGED_BY_TIMER_QUEUE_INDEX);
    +	}
    +
    +	@Override
    +	public int hashCode() {
    +		int result = (int) (timestamp ^ (timestamp >>> 32));
    +		result = 31 * result + key.hashCode();
    +		result = 31 * result + namespace.hashCode();
    +		return result;
    +	}
    +
    +	@Override
    +	public String toString() {
    +		return "Timer{" +
    +				"timestamp=" + timestamp +
    +				", key=" + key +
    +				", namespace=" + namespace +
    +				'}';
    +	}
    +
    +	/**
    +	 * A {@link TypeSerializer} used to serialize/deserialize a {@link TimerHeapInternalTimer}.
    +	 */
    +	public static class TimerSerializer<K, N> extends TypeSerializer<InternalTimer<K, N>> {
    +
    +		private static final long serialVersionUID = 1119562170939152304L;
    +
    +		private final TypeSerializer<K> keySerializer;
    +
    +		private final TypeSerializer<N> namespaceSerializer;
    --- End diff --
    
    Can these serializers be `null`? If not, then let's mark them as `@Nonnull`


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r190517336
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimer.java ---
    @@ -23,8 +23,8 @@
     /**
      * Internal interface for in-flight timers.
      *
    - * @param <K> type of the timer key.
    - * @param <N> type of the timer namespace.
    + * @param <K> Type of the keys to which timers are scoped.	+ * @param <K> type of the timer key.
    --- End diff --
    
    Does this comment  mean `@param <K> Type of the keys to which timers are scoped.`?


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r191440142
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TimerHeapInternalTimer.java ---
    @@ -0,0 +1,246 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.annotation.Internal;
    +import org.apache.flink.api.common.typeutils.CompatibilityResult;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
    +import org.apache.flink.api.common.typeutils.base.LongSerializer;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.core.memory.DataOutputView;
    +
    +import java.io.IOException;
    +
    +/**
    + * Implementation of {@link InternalTimer} for the {@link InternalTimerHeap}.
    + *
    + * @param <K> Type of the keys to which timers are scoped.
    + * @param <N> Type of the namespace to which timers are scoped.
    + */
    +@Internal
    +public final class TimerHeapInternalTimer<K, N> implements InternalTimer<K, N> {
    +
    +	/** The index that indicates that a tracked internal timer is not tracked. */
    +	private static final int NOT_MANAGED_BY_TIMER_QUEUE_INDEX = Integer.MIN_VALUE;
    +
    +	private final long timestamp;
    +
    +	private final K key;
    --- End diff --
    
    Can this be `null`?


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r191436243
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TimerHeapInternalTimer.java ---
    @@ -0,0 +1,246 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.annotation.Internal;
    +import org.apache.flink.api.common.typeutils.CompatibilityResult;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
    +import org.apache.flink.api.common.typeutils.base.LongSerializer;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.core.memory.DataOutputView;
    +
    +import java.io.IOException;
    +
    +/**
    + * Implementation of {@link InternalTimer} for the {@link InternalTimerHeap}.
    + *
    + * @param <K> Type of the keys to which timers are scoped.
    + * @param <N> Type of the namespace to which timers are scoped.
    + */
    +@Internal
    +public final class TimerHeapInternalTimer<K, N> implements InternalTimer<K, N> {
    +
    +	/** The index that indicates that a tracked internal timer is not tracked. */
    +	private static final int NOT_MANAGED_BY_TIMER_QUEUE_INDEX = Integer.MIN_VALUE;
    +
    +	private final long timestamp;
    +
    +	private final K key;
    +
    +	private final N namespace;
    +
    +	/**
    +	 * This field holds the current physical index of this timer when it is managed by a timer heap so that we can
    +	 * support fast deletes.
    +	 */
    +	private transient int timerHeapIndex;
    --- End diff --
    
    `TimerHeapInternalTimer` is non serializable. Thus, the `transient` keyword should not be needed.


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r190285625
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TimerHeapInternalTimer.java ---
    @@ -0,0 +1,235 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.annotation.Internal;
    +import org.apache.flink.api.common.typeutils.CompatibilityResult;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
    +import org.apache.flink.api.common.typeutils.base.LongSerializer;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.core.memory.DataOutputView;
    +
    +import java.io.IOException;
    +
    +/**
    + * Implementation of {@link InternalTimer} for the {@link InternalTimerHeap}.
    + *
    + * @param <K> Type of the keys to which timers are scoped.
    + * @param <N> Type of the namespace to which timers are scoped.
    + */
    +@Internal
    +public final class TimerHeapInternalTimer<K, N> implements InternalTimer<K, N> {
    +
    +	/** The index that indicates that a tracked internal timer is not tracked. */
    +	private static final int NOT_MANAGED_BY_TIMER_QUEUE_INDEX = Integer.MIN_VALUE;
    +
    +	private final long timestamp;
    +
    +	private final K key;
    +
    +	private final N namespace;
    +
    +	/**
    +	 * This field holds the current physical index if this timer when it is managed by a timer heap so that we can
    --- End diff --
    
    typo: `if` -> `of`.


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r190296235
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerHeap.java ---
    @@ -0,0 +1,504 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.annotation.VisibleForTesting;
    +import org.apache.flink.runtime.state.KeyGroupRange;
    +import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
    +
    +import javax.annotation.Nonnegative;
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Queue;
    +import java.util.Set;
    +
    +import static org.apache.flink.util.Preconditions.checkArgument;
    +
    +/**
    + * A heap-based priority queue for internal timers. This heap is supported by hash sets for fast contains
    + * (de-duplication) and deletes. The heap implementation is a simple binary tree stored inside an array. Element indexes
    + * in the heap array start at 1 instead of 0 to make array index computations a bit simpler in the hot methods.
    + *
    + * <p>Possible future improvements:
    + * <ul>
    + *  <li>We could also implement shrinking for the heap and the deduplication maps.</li>
    + *  <li>We could replace the deduplication maps with more efficient custom implementations. In particular, a hash set
    + * would be enough if it could return existing elements on unsuccessful adding, etc..</li>
    + * </ul>
    + *
    + * @param <K> type of the key of the internal timers managed by this priority queue.
    + * @param <N> type of the namespace of the internal timers managed by this priority queue.
    + */
    +public class InternalTimerHeap<K, N> implements Queue<TimerHeapInternalTimer<K, N>>, Set<TimerHeapInternalTimer<K, N>> {
    +
    +	/**
    +	 * A safe maximum size for arrays in the JVM.
    +	 */
    +	private static final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8;
    +
    +	/**
    +	 * Comparator for {@link TimerHeapInternalTimer}, based on the timestamp in ascending order.
    +	 */
    +	private static final Comparator<TimerHeapInternalTimer<?, ?>> COMPARATOR =
    +		(o1, o2) -> Long.compare(o1.getTimestamp(), o2.getTimestamp());
    +
    +	/**
    +	 * This array contains one hash set per key-group. The sets are used for fast de-duplication and deletes of timers.
    +	 */
    +	private final HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>>[] deduplicationMapsByKeyGroup;
    +
    +	/**
    +	 * The array that represents the heap-organized priority queue.
    +	 */
    +	private TimerHeapInternalTimer<K, N>[] queue;
    +
    +	/**
    +	 * The current size of the priority queue.
    +	 */
    +	private int size;
    +
    +	/**
    +	 * The key-group range of timers that are managed by this queue.
    +	 */
    +	private final KeyGroupRange keyGroupRange;
    +
    +	/**
    +	 * The total number of key-groups of the job.
    +	 */
    +	private final int totalNumberOfKeyGroups;
    +
    +
    +	/**
    +	 * Creates an empty {@link InternalTimerHeap} with the requested initial capacity.
    +	 *
    +	 * @param minimumCapacity the minimum and initial capacity of this priority queue.
    +	 */
    +	@SuppressWarnings("unchecked")
    +	InternalTimerHeap(
    +		@Nonnegative int minimumCapacity,
    +		@Nonnull KeyGroupRange keyGroupRange,
    +		@Nonnegative int totalNumberOfKeyGroups) {
    +
    +		this.totalNumberOfKeyGroups = totalNumberOfKeyGroups;
    +		this.keyGroupRange = keyGroupRange;
    +
    +		final int keyGroupsInLocalRange = keyGroupRange.getNumberOfKeyGroups();
    +		final int deduplicationSetSize = 1 + minimumCapacity / keyGroupsInLocalRange;
    +		this.deduplicationMapsByKeyGroup = new HashMap[keyGroupsInLocalRange];
    +		for (int i = 0; i < keyGroupsInLocalRange; ++i) {
    +			deduplicationMapsByKeyGroup[i] = new HashMap<>(deduplicationSetSize);
    +		}
    +
    +		this.queue = new TimerHeapInternalTimer[1 + minimumCapacity];
    +	}
    +
    +	/**
    +	 * @see Set#add(Object)
    +	 */
    +	@Override
    +	public boolean add(@Nonnull TimerHeapInternalTimer<K, N> timer) {
    +
    +		if (getDedupMapForKeyGroup(timer).putIfAbsent(timer, timer) == null) {
    +			final int newSize = ++this.size;
    +			checkCapacity(newSize);
    +			moveElementToIdx(timer, newSize);
    +			siftUp(newSize);
    +			return true;
    +		} else {
    +			return false;
    +		}
    +	}
    +
    +	/**
    +	 * This behaves like {@link #add(TimerHeapInternalTimer)}.
    +	 */
    +	@Override
    +	public boolean offer(@Nonnull TimerHeapInternalTimer<K, N> k) {
    +		return add(k);
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> poll() {
    +		return size() > 0 ? removeElementAtIndex(1) : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> remove() {
    +		TimerHeapInternalTimer<K, N> pollResult = poll();
    +		if (pollResult != null) {
    +			return pollResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> peek() {
    +		return size() > 0 ? queue[1] : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> element() {
    +		TimerHeapInternalTimer<K, N> peekResult = peek();
    +		if (peekResult != null) {
    +			return peekResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Override
    +	public boolean isEmpty() {
    +		return size() == 0;
    +	}
    +
    +	@Override
    +	public boolean contains(@Nullable Object o) {
    +		return (o instanceof TimerHeapInternalTimer)
    +			&& getDedupMapForKeyGroup((TimerHeapInternalTimer<?, ?>) o).containsKey(o);
    +	}
    +
    +	@Override
    +	public boolean remove(@Nullable Object o) {
    +		if (o instanceof TimerHeapInternalTimer) {
    +			return removeInternal((TimerHeapInternalTimer<?, ?>) o);
    +		}
    +		return false;
    +	}
    +
    +	@Override
    +	public boolean addAll(@Nullable Collection<? extends TimerHeapInternalTimer<K, N>> timers) {
    +
    +		if (timers == null) {
    +			return true;
    +		}
    +
    +		if (timers.size() > queue.length) {
    --- End diff --
    
    Yes, the downside is, if all elements are already contained we waste the space. Anyways, in practise this method is currently only used to bulk load empty heaps, so we can also go for your suggestion.


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r191768328
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerHeap.java ---
    @@ -0,0 +1,511 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.annotation.VisibleForTesting;
    +import org.apache.flink.runtime.state.KeyGroupRange;
    +import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
    +
    +import javax.annotation.Nonnegative;
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Queue;
    +import java.util.Set;
    +
    +import static org.apache.flink.util.Preconditions.checkArgument;
    +
    +/**
    + * A heap-based priority queue for internal timers. This heap is supported by hash sets for fast contains
    + * (de-duplication) and deletes. The heap implementation is a simple binary tree stored inside an array. Element indexes
    + * in the heap array start at 1 instead of 0 to make array index computations a bit simpler in the hot methods.
    + *
    + * <p>Possible future improvements:
    + * <ul>
    + *  <li>We could also implement shrinking for the heap and the deduplication maps.</li>
    + *  <li>We could replace the deduplication maps with more efficient custom implementations. In particular, a hash set
    + * would be enough if it could return existing elements on unsuccessful adding, etc..</li>
    + * </ul>
    + *
    + * @param <K> type of the key of the internal timers managed by this priority queue.
    + * @param <N> type of the namespace of the internal timers managed by this priority queue.
    + */
    +public class InternalTimerHeap<K, N> implements Queue<TimerHeapInternalTimer<K, N>>, Set<TimerHeapInternalTimer<K, N>> {
    +
    +	/**
    +	 * A safe maximum size for arrays in the JVM.
    +	 */
    +	private static final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8;
    +
    +	/**
    +	 * Comparator for {@link TimerHeapInternalTimer}, based on the timestamp in ascending order.
    +	 */
    +	private static final Comparator<TimerHeapInternalTimer<?, ?>> COMPARATOR =
    +		(o1, o2) -> Long.compare(o1.getTimestamp(), o2.getTimestamp());
    +
    +	/**
    +	 * This array contains one hash set per key-group. The sets are used for fast de-duplication and deletes of timers.
    +	 */
    +	private final HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>>[] deduplicationMapsByKeyGroup;
    +
    +	/**
    +	 * The array that represents the heap-organized priority queue.
    +	 */
    +	private TimerHeapInternalTimer<K, N>[] queue;
    +
    +	/**
    +	 * The current size of the priority queue.
    +	 */
    +	private int size;
    +
    +	/**
    +	 * The key-group range of timers that are managed by this queue.
    +	 */
    +	private final KeyGroupRange keyGroupRange;
    +
    +	/**
    +	 * The total number of key-groups of the job.
    +	 */
    +	private final int totalNumberOfKeyGroups;
    +
    +
    +	/**
    +	 * Creates an empty {@link InternalTimerHeap} with the requested initial capacity.
    +	 *
    +	 * @param minimumCapacity the minimum and initial capacity of this priority queue.
    +	 */
    +	@SuppressWarnings("unchecked")
    +	InternalTimerHeap(
    +		@Nonnegative int minimumCapacity,
    +		@Nonnull KeyGroupRange keyGroupRange,
    +		@Nonnegative int totalNumberOfKeyGroups) {
    +
    +		this.totalNumberOfKeyGroups = totalNumberOfKeyGroups;
    +		this.keyGroupRange = keyGroupRange;
    +
    +		final int keyGroupsInLocalRange = keyGroupRange.getNumberOfKeyGroups();
    +		final int deduplicationSetSize = 1 + minimumCapacity / keyGroupsInLocalRange;
    +		this.deduplicationMapsByKeyGroup = new HashMap[keyGroupsInLocalRange];
    +		for (int i = 0; i < keyGroupsInLocalRange; ++i) {
    +			deduplicationMapsByKeyGroup[i] = new HashMap<>(deduplicationSetSize);
    +		}
    +
    +		this.queue = new TimerHeapInternalTimer[1 + minimumCapacity];
    +	}
    +
    +	/**
    +	 * @see Set#add(Object)
    +	 */
    +	@Override
    +	public boolean add(@Nonnull TimerHeapInternalTimer<K, N> timer) {
    +
    +		if (getDedupMapForKeyGroup(timer).putIfAbsent(timer, timer) == null) {
    +			final int newSize = ++this.size;
    +			growIfRequired(newSize);
    +			moveElementToIdx(timer, newSize);
    +			siftUp(newSize);
    +			return true;
    +		} else {
    +			return false;
    +		}
    +	}
    +
    +	/**
    +	 * This behaves like {@link #add(TimerHeapInternalTimer)}.
    +	 */
    +	@Override
    +	public boolean offer(@Nonnull TimerHeapInternalTimer<K, N> k) {
    +		return add(k);
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> poll() {
    +		return size() > 0 ? removeElementAtIndex(1) : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> remove() {
    +		TimerHeapInternalTimer<K, N> pollResult = poll();
    +		if (pollResult != null) {
    +			return pollResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> peek() {
    +		return size() > 0 ? queue[1] : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> element() {
    +		TimerHeapInternalTimer<K, N> peekResult = peek();
    +		if (peekResult != null) {
    +			return peekResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Override
    +	public boolean isEmpty() {
    +		return size() == 0;
    +	}
    +
    +	@Override
    +	public boolean contains(@Nullable Object toCheck) {
    +		return (toCheck instanceof TimerHeapInternalTimer)
    +			&& getDedupMapForKeyGroup((TimerHeapInternalTimer<?, ?>) toCheck).containsKey(toCheck);
    +	}
    +
    +	@Override
    +	public boolean remove(@Nullable Object toRemove) {
    +		if (toRemove instanceof TimerHeapInternalTimer) {
    +			return removeInternal((TimerHeapInternalTimer<?, ?>) toRemove);
    +		}
    +		return false;
    +	}
    +
    +	@Override
    +	public boolean addAll(@Nullable Collection<? extends TimerHeapInternalTimer<K, N>> timers) {
    +
    +		if (timers == null) {
    +			return true;
    +		}
    +
    +		final int oldSize = size();
    +		resizeForBulkLoad(oldSize + timers.size());
    +
    +		for (TimerHeapInternalTimer<K, N> k : timers) {
    +			add(k);
    +		}
    +
    +		return oldSize != size();
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public Object[] toArray() {
    +		return Arrays.copyOfRange(queue, 1, size + 1);
    +	}
    +
    +	@SuppressWarnings({"unchecked", "SuspiciousSystemArraycopy"})
    +	@Nonnull
    +	@Override
    +	public <T> T[] toArray(@Nonnull T[] array) {
    +		if (array.length >= size) {
    +			System.arraycopy(queue, 1, array, 0, size);
    +			return array;
    +		}
    +		return (T[]) Arrays.copyOfRange(queue, 1, size + 1, array.getClass());
    +	}
    +
    +	@Override
    +	public boolean removeAll(@Nullable Collection<?> toRemove) {
    +
    +		if (toRemove == null) {
    +			return false;
    +		}
    +
    +		int oldSize = size();
    +		for (Object o : toRemove) {
    +			remove(o);
    +		}
    +		return oldSize != size();
    +	}
    +
    +	/**
    +	 * Returns an iterator over the elements in this queue. The iterator
    +	 * does not return the elements in any particular order.
    +	 *
    +	 * @return an iterator over the elements in this queue.
    +	 */
    +	@Nonnull
    +	@Override
    +	public Iterator<TimerHeapInternalTimer<K, N>> iterator() {
    +		return new InternalTimerPriorityQueueIterator();
    +	}
    +
    +	@Override
    +	public boolean containsAll(@Nullable Collection<?> toCheck) {
    +
    +		if (toCheck == null) {
    +			return true;
    +		}
    +
    +		for (Object o : toCheck) {
    +			if (!contains(o)) {
    +				return false;
    +			}
    +		}
    +
    +		return true;
    +	}
    +
    +	@Nonnegative
    +	@Override
    +	public int size() {
    +		return size;
    +	}
    +
    +	@Override
    +	public void clear() {
    +
    +		Arrays.fill(queue, null);
    +		for (HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> timerHashMap :
    +			deduplicationMapsByKeyGroup) {
    +			timerHashMap.clear();
    +		}
    +		size = 0;
    +	}
    +
    +	/**
    +	 * This method is currently not implemented.
    +	 */
    +	@Override
    +	public boolean retainAll(@Nullable Collection<?> toRetain) {
    +		throw new UnsupportedOperationException();
    +	}
    +
    +	/**
    +	 * Adds a new timer with the given timestamp, key, and namespace to the heap, if an identical timer was not yet
    +	 * registered.
    +	 *
    +	 * @param timestamp the timer timestamp.
    +	 * @param key the timer key.
    +	 * @param namespace the timer namespace.
    +	 * @return true iff a new timer with given timestamp, key, and namespace was added to the heap.
    +	 */
    +	boolean scheduleTimer(long timestamp, K key, N namespace) {
    +		return add(new TimerHeapInternalTimer<>(timestamp, key, namespace));
    +	}
    +
    +	/**
    +	 * Stops timer with the given timestamp, key, and namespace by removing it from the heap, if it exists on the heap.
    +	 *
    +	 * @param timestamp the timer timestamp.
    +	 * @param key the timer key.
    +	 * @param namespace the timer namespace.
    +	 * @return true iff a timer with given timestamp, key, and namespace was found and removed from the heap.
    +	 */
    +	boolean stopTimer(long timestamp, K key, N namespace) {
    +		return removeInternal(new TimerHeapInternalTimer<>(timestamp, key, namespace));
    +	}
    +
    +	/**
    +	 * This method adds all the given timers to the heap.
    +	 */
    +	void addRestoredTimers(Collection<? extends InternalTimer<K, N>> restoredTimers) {
    +
    +		if (restoredTimers == null) {
    +			return;
    +		}
    +
    +		resizeForBulkLoad(restoredTimers.size());
    +
    +		for (InternalTimer<K, N> timer : restoredTimers) {
    +			if (timer instanceof TimerHeapInternalTimer) {
    +				add((TimerHeapInternalTimer<K, N>) timer);
    +			} else {
    +				scheduleTimer(timer.getTimestamp(), timer.getKey(), timer.getNamespace());
    +			}
    +		}
    +	}
    +
    +	private boolean removeInternal(TimerHeapInternalTimer<?, ?> timerToRemove) {
    +
    +		TimerHeapInternalTimer<K, N> storedTimer = getDedupMapForKeyGroup(timerToRemove).remove(timerToRemove);
    +
    +		if (storedTimer != null) {
    +			removeElementAtIndex(storedTimer.getTimerHeapIndex());
    +			return true;
    +		}
    +
    +		return false;
    +	}
    +
    +	private TimerHeapInternalTimer<K, N> removeElementAtIndex(int removeIdx) {
    +		TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		TimerHeapInternalTimer<K, N> removedValue = heap[removeIdx];
    +
    +		assert removedValue.getTimerHeapIndex() == removeIdx;
    +
    +		final int oldSize = size;
    +
    +		if (removeIdx != oldSize) {
    +			TimerHeapInternalTimer<K, N> timer = heap[oldSize];
    +			moveElementToIdx(timer, removeIdx);
    +			siftDown(removeIdx);
    +			if (heap[removeIdx] == timer) {
    +				siftUp(removeIdx);
    +			}
    +		}
    +
    +		heap[oldSize] = null;
    +		getDedupMapForKeyGroup(removedValue).remove(removedValue);
    +
    +		--size;
    +		return removedValue;
    +	}
    +
    +	private void siftUp(int idx) {
    +		final TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		TimerHeapInternalTimer<K, N> currentTimer = heap[idx];
    +		int parentIdx = idx >>> 1;
    +
    +		while (parentIdx > 0 && COMPARATOR.compare(currentTimer, heap[parentIdx]) < 0) {
    +			moveElementToIdx(heap[parentIdx], idx);
    +			idx = parentIdx;
    +			parentIdx = parentIdx >>> 1;
    +		}
    +
    +		moveElementToIdx(currentTimer, idx);
    +	}
    +
    +	private void siftDown(int idx) {
    +		final TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		final int heapSize = this.size;
    +
    +		TimerHeapInternalTimer<K, N> currentTimer = heap[idx];
    +		int firstChildIdx = idx << 1;
    +		int secondChildIdx = firstChildIdx + 1;
    +
    +		if (secondChildIdx <= heapSize && COMPARATOR.compare(heap[secondChildIdx], heap[firstChildIdx]) < 0) {
    +			firstChildIdx = secondChildIdx;
    +		}
    +
    +		while (firstChildIdx <= heapSize && COMPARATOR.compare(heap[firstChildIdx], currentTimer) < 0) {
    +			moveElementToIdx(heap[firstChildIdx], idx);
    +			idx = firstChildIdx;
    +			firstChildIdx = idx << 1;
    +			secondChildIdx = firstChildIdx + 1;
    +
    +			if (secondChildIdx <= heapSize && COMPARATOR.compare(heap[secondChildIdx], heap[firstChildIdx]) < 0) {
    +				firstChildIdx = secondChildIdx;
    +			}
    +		}
    +
    +		moveElementToIdx(currentTimer, idx);
    +	}
    +
    +	/**
    +	 * Returns an unmodifiable set of all timers in the given key-group.
    +	 */
    +	Set<InternalTimer<K, N>> getTimersForKeyGroup(@Nonnegative int keyGroupIdx) {
    +		return Collections.unmodifiableSet(getDedupMapForKeyGroup(keyGroupIdx).keySet());
    +	}
    +
    +	private HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> getDedupMapForKeyGroup(
    +		@Nonnegative int keyGroupIdx) {
    +		return deduplicationMapsByKeyGroup[globalKeyGroupToLocalIndex(keyGroupIdx)];
    +	}
    +
    +	@VisibleForTesting
    +	@SuppressWarnings("unchecked")
    +	List<Set<InternalTimer<K, N>>> getTimersByKeyGroup() {
    +		List<Set<InternalTimer<K, N>>> result = new ArrayList<>(deduplicationMapsByKeyGroup.length);
    +		for (int i = 0; i < deduplicationMapsByKeyGroup.length; ++i) {
    +			result.add(i, Collections.unmodifiableSet(deduplicationMapsByKeyGroup[i].keySet()));
    +		}
    +		return result;
    +	}
    +
    +	private void moveElementToIdx(TimerHeapInternalTimer<K, N> element, int idx) {
    +		queue[idx] = element;
    +		element.setTimerHeapIndex(idx);
    +	}
    +
    +	private HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> getDedupMapForKeyGroup(
    +		TimerHeapInternalTimer<?, ?> timer) {
    +		int keyGroup = KeyGroupRangeAssignment.assignToKeyGroup(timer.getKey(), totalNumberOfKeyGroups);
    +		return getDedupMapForKeyGroup(keyGroup);
    +	}
    +
    +	private int globalKeyGroupToLocalIndex(int keyGroup) {
    +		checkArgument(keyGroupRange.contains(keyGroup));
    +		return keyGroup - keyGroupRange.getStartKeyGroup();
    +	}
    +
    +	private void growIfRequired(int requiredSize) {
    +		int oldArraySize = queue.length;
    +
    +		if (requiredSize >= oldArraySize) {
    +			final int grow = (oldArraySize < 64) ? oldArraySize + 2 : oldArraySize >> 1;
    +			resizeQueueArray(oldArraySize + grow);
    +		}
    +		// TODO implement shrinking as well?
    +	}
    +
    +	private void resizeForBulkLoad(int maxTotalSize) {
    +		if (maxTotalSize > queue.length) {
    +			resizeQueueArray(maxTotalSize + (maxTotalSize >>> 3));
    +		}
    +	}
    +
    +	private void resizeQueueArray(int newArraySize) {
    +		if (newArraySize - MAX_ARRAY_SIZE > 0) {
    +			if (newArraySize < 0 || newArraySize > MAX_ARRAY_SIZE) {
    +				throw new OutOfMemoryError("Required timer heap exceeds maximum size!");
    +			} else {
    +				newArraySize = MAX_ARRAY_SIZE;
    +			}
    +		}
    --- End diff --
    
    `newArraySize` can become negative via `final int grow = (oldArraySize < 64) ? oldArraySize + 2 : oldArraySize >> 1;`


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r191772629
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerHeap.java ---
    @@ -0,0 +1,511 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.annotation.VisibleForTesting;
    +import org.apache.flink.runtime.state.KeyGroupRange;
    +import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
    +
    +import javax.annotation.Nonnegative;
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Queue;
    +import java.util.Set;
    +
    +import static org.apache.flink.util.Preconditions.checkArgument;
    +
    +/**
    + * A heap-based priority queue for internal timers. This heap is supported by hash sets for fast contains
    + * (de-duplication) and deletes. The heap implementation is a simple binary tree stored inside an array. Element indexes
    + * in the heap array start at 1 instead of 0 to make array index computations a bit simpler in the hot methods.
    + *
    + * <p>Possible future improvements:
    + * <ul>
    + *  <li>We could also implement shrinking for the heap and the deduplication maps.</li>
    + *  <li>We could replace the deduplication maps with more efficient custom implementations. In particular, a hash set
    + * would be enough if it could return existing elements on unsuccessful adding, etc..</li>
    + * </ul>
    + *
    + * @param <K> type of the key of the internal timers managed by this priority queue.
    + * @param <N> type of the namespace of the internal timers managed by this priority queue.
    + */
    +public class InternalTimerHeap<K, N> implements Queue<TimerHeapInternalTimer<K, N>>, Set<TimerHeapInternalTimer<K, N>> {
    +
    +	/**
    +	 * A safe maximum size for arrays in the JVM.
    +	 */
    +	private static final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8;
    +
    +	/**
    +	 * Comparator for {@link TimerHeapInternalTimer}, based on the timestamp in ascending order.
    +	 */
    +	private static final Comparator<TimerHeapInternalTimer<?, ?>> COMPARATOR =
    +		(o1, o2) -> Long.compare(o1.getTimestamp(), o2.getTimestamp());
    +
    +	/**
    +	 * This array contains one hash set per key-group. The sets are used for fast de-duplication and deletes of timers.
    +	 */
    +	private final HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>>[] deduplicationMapsByKeyGroup;
    +
    +	/**
    +	 * The array that represents the heap-organized priority queue.
    +	 */
    +	private TimerHeapInternalTimer<K, N>[] queue;
    +
    +	/**
    +	 * The current size of the priority queue.
    +	 */
    +	private int size;
    +
    +	/**
    +	 * The key-group range of timers that are managed by this queue.
    +	 */
    +	private final KeyGroupRange keyGroupRange;
    +
    +	/**
    +	 * The total number of key-groups of the job.
    +	 */
    +	private final int totalNumberOfKeyGroups;
    +
    +
    +	/**
    +	 * Creates an empty {@link InternalTimerHeap} with the requested initial capacity.
    +	 *
    +	 * @param minimumCapacity the minimum and initial capacity of this priority queue.
    +	 */
    +	@SuppressWarnings("unchecked")
    +	InternalTimerHeap(
    +		@Nonnegative int minimumCapacity,
    +		@Nonnull KeyGroupRange keyGroupRange,
    +		@Nonnegative int totalNumberOfKeyGroups) {
    +
    +		this.totalNumberOfKeyGroups = totalNumberOfKeyGroups;
    +		this.keyGroupRange = keyGroupRange;
    +
    +		final int keyGroupsInLocalRange = keyGroupRange.getNumberOfKeyGroups();
    +		final int deduplicationSetSize = 1 + minimumCapacity / keyGroupsInLocalRange;
    +		this.deduplicationMapsByKeyGroup = new HashMap[keyGroupsInLocalRange];
    +		for (int i = 0; i < keyGroupsInLocalRange; ++i) {
    +			deduplicationMapsByKeyGroup[i] = new HashMap<>(deduplicationSetSize);
    +		}
    +
    +		this.queue = new TimerHeapInternalTimer[1 + minimumCapacity];
    +	}
    +
    +	/**
    +	 * @see Set#add(Object)
    +	 */
    +	@Override
    +	public boolean add(@Nonnull TimerHeapInternalTimer<K, N> timer) {
    +
    +		if (getDedupMapForKeyGroup(timer).putIfAbsent(timer, timer) == null) {
    +			final int newSize = ++this.size;
    +			growIfRequired(newSize);
    +			moveElementToIdx(timer, newSize);
    +			siftUp(newSize);
    +			return true;
    +		} else {
    +			return false;
    +		}
    +	}
    +
    +	/**
    +	 * This behaves like {@link #add(TimerHeapInternalTimer)}.
    +	 */
    +	@Override
    +	public boolean offer(@Nonnull TimerHeapInternalTimer<K, N> k) {
    +		return add(k);
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> poll() {
    +		return size() > 0 ? removeElementAtIndex(1) : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> remove() {
    +		TimerHeapInternalTimer<K, N> pollResult = poll();
    +		if (pollResult != null) {
    +			return pollResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> peek() {
    +		return size() > 0 ? queue[1] : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> element() {
    +		TimerHeapInternalTimer<K, N> peekResult = peek();
    +		if (peekResult != null) {
    +			return peekResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Override
    +	public boolean isEmpty() {
    +		return size() == 0;
    +	}
    +
    +	@Override
    +	public boolean contains(@Nullable Object toCheck) {
    +		return (toCheck instanceof TimerHeapInternalTimer)
    +			&& getDedupMapForKeyGroup((TimerHeapInternalTimer<?, ?>) toCheck).containsKey(toCheck);
    +	}
    +
    +	@Override
    +	public boolean remove(@Nullable Object toRemove) {
    +		if (toRemove instanceof TimerHeapInternalTimer) {
    +			return removeInternal((TimerHeapInternalTimer<?, ?>) toRemove);
    +		}
    +		return false;
    +	}
    +
    +	@Override
    +	public boolean addAll(@Nullable Collection<? extends TimerHeapInternalTimer<K, N>> timers) {
    +
    +		if (timers == null) {
    +			return true;
    +		}
    +
    +		final int oldSize = size();
    +		resizeForBulkLoad(oldSize + timers.size());
    +
    +		for (TimerHeapInternalTimer<K, N> k : timers) {
    +			add(k);
    +		}
    +
    +		return oldSize != size();
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public Object[] toArray() {
    +		return Arrays.copyOfRange(queue, 1, size + 1);
    +	}
    +
    +	@SuppressWarnings({"unchecked", "SuspiciousSystemArraycopy"})
    +	@Nonnull
    +	@Override
    +	public <T> T[] toArray(@Nonnull T[] array) {
    +		if (array.length >= size) {
    +			System.arraycopy(queue, 1, array, 0, size);
    +			return array;
    +		}
    +		return (T[]) Arrays.copyOfRange(queue, 1, size + 1, array.getClass());
    +	}
    +
    +	@Override
    +	public boolean removeAll(@Nullable Collection<?> toRemove) {
    +
    +		if (toRemove == null) {
    +			return false;
    +		}
    +
    +		int oldSize = size();
    +		for (Object o : toRemove) {
    +			remove(o);
    +		}
    +		return oldSize != size();
    +	}
    +
    +	/**
    +	 * Returns an iterator over the elements in this queue. The iterator
    +	 * does not return the elements in any particular order.
    +	 *
    +	 * @return an iterator over the elements in this queue.
    +	 */
    +	@Nonnull
    +	@Override
    +	public Iterator<TimerHeapInternalTimer<K, N>> iterator() {
    +		return new InternalTimerPriorityQueueIterator();
    +	}
    +
    +	@Override
    +	public boolean containsAll(@Nullable Collection<?> toCheck) {
    +
    +		if (toCheck == null) {
    +			return true;
    +		}
    +
    +		for (Object o : toCheck) {
    +			if (!contains(o)) {
    +				return false;
    +			}
    +		}
    +
    +		return true;
    +	}
    +
    +	@Nonnegative
    +	@Override
    +	public int size() {
    +		return size;
    +	}
    +
    +	@Override
    +	public void clear() {
    +
    +		Arrays.fill(queue, null);
    +		for (HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> timerHashMap :
    +			deduplicationMapsByKeyGroup) {
    +			timerHashMap.clear();
    +		}
    +		size = 0;
    +	}
    +
    +	/**
    +	 * This method is currently not implemented.
    +	 */
    +	@Override
    +	public boolean retainAll(@Nullable Collection<?> toRetain) {
    +		throw new UnsupportedOperationException();
    +	}
    +
    +	/**
    +	 * Adds a new timer with the given timestamp, key, and namespace to the heap, if an identical timer was not yet
    +	 * registered.
    +	 *
    +	 * @param timestamp the timer timestamp.
    +	 * @param key the timer key.
    +	 * @param namespace the timer namespace.
    +	 * @return true iff a new timer with given timestamp, key, and namespace was added to the heap.
    +	 */
    +	boolean scheduleTimer(long timestamp, K key, N namespace) {
    +		return add(new TimerHeapInternalTimer<>(timestamp, key, namespace));
    +	}
    +
    +	/**
    +	 * Stops timer with the given timestamp, key, and namespace by removing it from the heap, if it exists on the heap.
    +	 *
    +	 * @param timestamp the timer timestamp.
    +	 * @param key the timer key.
    +	 * @param namespace the timer namespace.
    +	 * @return true iff a timer with given timestamp, key, and namespace was found and removed from the heap.
    +	 */
    +	boolean stopTimer(long timestamp, K key, N namespace) {
    +		return removeInternal(new TimerHeapInternalTimer<>(timestamp, key, namespace));
    +	}
    +
    +	/**
    +	 * This method adds all the given timers to the heap.
    +	 */
    +	void addRestoredTimers(Collection<? extends InternalTimer<K, N>> restoredTimers) {
    +
    +		if (restoredTimers == null) {
    +			return;
    +		}
    +
    +		resizeForBulkLoad(restoredTimers.size());
    +
    +		for (InternalTimer<K, N> timer : restoredTimers) {
    +			if (timer instanceof TimerHeapInternalTimer) {
    +				add((TimerHeapInternalTimer<K, N>) timer);
    +			} else {
    +				scheduleTimer(timer.getTimestamp(), timer.getKey(), timer.getNamespace());
    +			}
    +		}
    +	}
    +
    +	private boolean removeInternal(TimerHeapInternalTimer<?, ?> timerToRemove) {
    +
    +		TimerHeapInternalTimer<K, N> storedTimer = getDedupMapForKeyGroup(timerToRemove).remove(timerToRemove);
    +
    +		if (storedTimer != null) {
    +			removeElementAtIndex(storedTimer.getTimerHeapIndex());
    +			return true;
    +		}
    +
    +		return false;
    +	}
    +
    +	private TimerHeapInternalTimer<K, N> removeElementAtIndex(int removeIdx) {
    +		TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		TimerHeapInternalTimer<K, N> removedValue = heap[removeIdx];
    +
    +		assert removedValue.getTimerHeapIndex() == removeIdx;
    +
    +		final int oldSize = size;
    +
    +		if (removeIdx != oldSize) {
    +			TimerHeapInternalTimer<K, N> timer = heap[oldSize];
    +			moveElementToIdx(timer, removeIdx);
    +			siftDown(removeIdx);
    +			if (heap[removeIdx] == timer) {
    +				siftUp(removeIdx);
    +			}
    +		}
    +
    +		heap[oldSize] = null;
    +		getDedupMapForKeyGroup(removedValue).remove(removedValue);
    +
    +		--size;
    +		return removedValue;
    +	}
    +
    +	private void siftUp(int idx) {
    +		final TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		TimerHeapInternalTimer<K, N> currentTimer = heap[idx];
    +		int parentIdx = idx >>> 1;
    +
    +		while (parentIdx > 0 && COMPARATOR.compare(currentTimer, heap[parentIdx]) < 0) {
    +			moveElementToIdx(heap[parentIdx], idx);
    +			idx = parentIdx;
    +			parentIdx = parentIdx >>> 1;
    +		}
    +
    +		moveElementToIdx(currentTimer, idx);
    +	}
    +
    +	private void siftDown(int idx) {
    +		final TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		final int heapSize = this.size;
    +
    +		TimerHeapInternalTimer<K, N> currentTimer = heap[idx];
    +		int firstChildIdx = idx << 1;
    +		int secondChildIdx = firstChildIdx + 1;
    +
    +		if (secondChildIdx <= heapSize && COMPARATOR.compare(heap[secondChildIdx], heap[firstChildIdx]) < 0) {
    +			firstChildIdx = secondChildIdx;
    +		}
    +
    +		while (firstChildIdx <= heapSize && COMPARATOR.compare(heap[firstChildIdx], currentTimer) < 0) {
    +			moveElementToIdx(heap[firstChildIdx], idx);
    +			idx = firstChildIdx;
    +			firstChildIdx = idx << 1;
    +			secondChildIdx = firstChildIdx + 1;
    +
    +			if (secondChildIdx <= heapSize && COMPARATOR.compare(heap[secondChildIdx], heap[firstChildIdx]) < 0) {
    +				firstChildIdx = secondChildIdx;
    +			}
    +		}
    +
    +		moveElementToIdx(currentTimer, idx);
    +	}
    +
    +	/**
    +	 * Returns an unmodifiable set of all timers in the given key-group.
    +	 */
    +	Set<InternalTimer<K, N>> getTimersForKeyGroup(@Nonnegative int keyGroupIdx) {
    +		return Collections.unmodifiableSet(getDedupMapForKeyGroup(keyGroupIdx).keySet());
    +	}
    +
    +	private HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> getDedupMapForKeyGroup(
    +		@Nonnegative int keyGroupIdx) {
    +		return deduplicationMapsByKeyGroup[globalKeyGroupToLocalIndex(keyGroupIdx)];
    +	}
    +
    +	@VisibleForTesting
    +	@SuppressWarnings("unchecked")
    +	List<Set<InternalTimer<K, N>>> getTimersByKeyGroup() {
    +		List<Set<InternalTimer<K, N>>> result = new ArrayList<>(deduplicationMapsByKeyGroup.length);
    +		for (int i = 0; i < deduplicationMapsByKeyGroup.length; ++i) {
    +			result.add(i, Collections.unmodifiableSet(deduplicationMapsByKeyGroup[i].keySet()));
    +		}
    +		return result;
    +	}
    +
    +	private void moveElementToIdx(TimerHeapInternalTimer<K, N> element, int idx) {
    +		queue[idx] = element;
    +		element.setTimerHeapIndex(idx);
    +	}
    +
    +	private HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> getDedupMapForKeyGroup(
    +		TimerHeapInternalTimer<?, ?> timer) {
    +		int keyGroup = KeyGroupRangeAssignment.assignToKeyGroup(timer.getKey(), totalNumberOfKeyGroups);
    +		return getDedupMapForKeyGroup(keyGroup);
    +	}
    +
    +	private int globalKeyGroupToLocalIndex(int keyGroup) {
    +		checkArgument(keyGroupRange.contains(keyGroup));
    +		return keyGroup - keyGroupRange.getStartKeyGroup();
    +	}
    +
    +	private void growIfRequired(int requiredSize) {
    +		int oldArraySize = queue.length;
    +
    +		if (requiredSize >= oldArraySize) {
    +			final int grow = (oldArraySize < 64) ? oldArraySize + 2 : oldArraySize >> 1;
    +			resizeQueueArray(oldArraySize + grow);
    +		}
    +		// TODO implement shrinking as well?
    +	}
    +
    +	private void resizeForBulkLoad(int maxTotalSize) {
    +		if (maxTotalSize > queue.length) {
    +			resizeQueueArray(maxTotalSize + (maxTotalSize >>> 3));
    +		}
    +	}
    +
    +	private void resizeQueueArray(int newArraySize) {
    +		if (newArraySize - MAX_ARRAY_SIZE > 0) {
    +			if (newArraySize < 0 || newArraySize > MAX_ARRAY_SIZE) {
    +				throw new OutOfMemoryError("Required timer heap exceeds maximum size!");
    +			} else {
    +				newArraySize = MAX_ARRAY_SIZE;
    +			}
    +		}
    --- End diff --
    
    Agreed, but wouldn't it make sense to guard it without the special cases? Simply checking whether `newArraySize` is positive? This would make it easier to understand as well.


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r191454466
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerHeap.java ---
    @@ -0,0 +1,511 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.annotation.VisibleForTesting;
    +import org.apache.flink.runtime.state.KeyGroupRange;
    +import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
    +
    +import javax.annotation.Nonnegative;
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Queue;
    +import java.util.Set;
    +
    +import static org.apache.flink.util.Preconditions.checkArgument;
    +
    +/**
    + * A heap-based priority queue for internal timers. This heap is supported by hash sets for fast contains
    + * (de-duplication) and deletes. The heap implementation is a simple binary tree stored inside an array. Element indexes
    + * in the heap array start at 1 instead of 0 to make array index computations a bit simpler in the hot methods.
    + *
    + * <p>Possible future improvements:
    + * <ul>
    + *  <li>We could also implement shrinking for the heap and the deduplication maps.</li>
    + *  <li>We could replace the deduplication maps with more efficient custom implementations. In particular, a hash set
    + * would be enough if it could return existing elements on unsuccessful adding, etc..</li>
    + * </ul>
    + *
    + * @param <K> type of the key of the internal timers managed by this priority queue.
    + * @param <N> type of the namespace of the internal timers managed by this priority queue.
    + */
    +public class InternalTimerHeap<K, N> implements Queue<TimerHeapInternalTimer<K, N>>, Set<TimerHeapInternalTimer<K, N>> {
    +
    +	/**
    +	 * A safe maximum size for arrays in the JVM.
    +	 */
    +	private static final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8;
    +
    +	/**
    +	 * Comparator for {@link TimerHeapInternalTimer}, based on the timestamp in ascending order.
    +	 */
    +	private static final Comparator<TimerHeapInternalTimer<?, ?>> COMPARATOR =
    +		(o1, o2) -> Long.compare(o1.getTimestamp(), o2.getTimestamp());
    --- End diff --
    
    I know, was not quiet sure if this has performance implications.


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r190294307
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerHeap.java ---
    @@ -0,0 +1,504 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.annotation.VisibleForTesting;
    +import org.apache.flink.runtime.state.KeyGroupRange;
    +import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
    +
    +import javax.annotation.Nonnegative;
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Queue;
    +import java.util.Set;
    +
    +import static org.apache.flink.util.Preconditions.checkArgument;
    +
    +/**
    + * A heap-based priority queue for internal timers. This heap is supported by hash sets for fast contains
    + * (de-duplication) and deletes. The heap implementation is a simple binary tree stored inside an array. Element indexes
    + * in the heap array start at 1 instead of 0 to make array index computations a bit simpler in the hot methods.
    + *
    + * <p>Possible future improvements:
    + * <ul>
    + *  <li>We could also implement shrinking for the heap and the deduplication maps.</li>
    + *  <li>We could replace the deduplication maps with more efficient custom implementations. In particular, a hash set
    + * would be enough if it could return existing elements on unsuccessful adding, etc..</li>
    + * </ul>
    + *
    + * @param <K> type of the key of the internal timers managed by this priority queue.
    + * @param <N> type of the namespace of the internal timers managed by this priority queue.
    + */
    +public class InternalTimerHeap<K, N> implements Queue<TimerHeapInternalTimer<K, N>>, Set<TimerHeapInternalTimer<K, N>> {
    +
    +	/**
    +	 * A safe maximum size for arrays in the JVM.
    +	 */
    +	private static final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8;
    +
    +	/**
    +	 * Comparator for {@link TimerHeapInternalTimer}, based on the timestamp in ascending order.
    +	 */
    +	private static final Comparator<TimerHeapInternalTimer<?, ?>> COMPARATOR =
    +		(o1, o2) -> Long.compare(o1.getTimestamp(), o2.getTimestamp());
    +
    +	/**
    +	 * This array contains one hash set per key-group. The sets are used for fast de-duplication and deletes of timers.
    +	 */
    +	private final HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>>[] deduplicationMapsByKeyGroup;
    +
    +	/**
    +	 * The array that represents the heap-organized priority queue.
    +	 */
    +	private TimerHeapInternalTimer<K, N>[] queue;
    +
    +	/**
    +	 * The current size of the priority queue.
    +	 */
    +	private int size;
    +
    +	/**
    +	 * The key-group range of timers that are managed by this queue.
    +	 */
    +	private final KeyGroupRange keyGroupRange;
    +
    +	/**
    +	 * The total number of key-groups of the job.
    +	 */
    +	private final int totalNumberOfKeyGroups;
    +
    +
    +	/**
    +	 * Creates an empty {@link InternalTimerHeap} with the requested initial capacity.
    +	 *
    +	 * @param minimumCapacity the minimum and initial capacity of this priority queue.
    +	 */
    +	@SuppressWarnings("unchecked")
    +	InternalTimerHeap(
    +		@Nonnegative int minimumCapacity,
    +		@Nonnull KeyGroupRange keyGroupRange,
    +		@Nonnegative int totalNumberOfKeyGroups) {
    +
    +		this.totalNumberOfKeyGroups = totalNumberOfKeyGroups;
    +		this.keyGroupRange = keyGroupRange;
    +
    +		final int keyGroupsInLocalRange = keyGroupRange.getNumberOfKeyGroups();
    +		final int deduplicationSetSize = 1 + minimumCapacity / keyGroupsInLocalRange;
    +		this.deduplicationMapsByKeyGroup = new HashMap[keyGroupsInLocalRange];
    +		for (int i = 0; i < keyGroupsInLocalRange; ++i) {
    +			deduplicationMapsByKeyGroup[i] = new HashMap<>(deduplicationSetSize);
    +		}
    +
    +		this.queue = new TimerHeapInternalTimer[1 + minimumCapacity];
    +	}
    +
    +	/**
    +	 * @see Set#add(Object)
    +	 */
    +	@Override
    +	public boolean add(@Nonnull TimerHeapInternalTimer<K, N> timer) {
    +
    +		if (getDedupMapForKeyGroup(timer).putIfAbsent(timer, timer) == null) {
    +			final int newSize = ++this.size;
    +			checkCapacity(newSize);
    +			moveElementToIdx(timer, newSize);
    +			siftUp(newSize);
    +			return true;
    +		} else {
    +			return false;
    +		}
    +	}
    +
    +	/**
    +	 * This behaves like {@link #add(TimerHeapInternalTimer)}.
    +	 */
    +	@Override
    +	public boolean offer(@Nonnull TimerHeapInternalTimer<K, N> k) {
    +		return add(k);
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> poll() {
    +		return size() > 0 ? removeElementAtIndex(1) : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> remove() {
    +		TimerHeapInternalTimer<K, N> pollResult = poll();
    +		if (pollResult != null) {
    +			return pollResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> peek() {
    +		return size() > 0 ? queue[1] : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> element() {
    +		TimerHeapInternalTimer<K, N> peekResult = peek();
    +		if (peekResult != null) {
    +			return peekResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Override
    +	public boolean isEmpty() {
    +		return size() == 0;
    +	}
    +
    +	@Override
    +	public boolean contains(@Nullable Object o) {
    +		return (o instanceof TimerHeapInternalTimer)
    +			&& getDedupMapForKeyGroup((TimerHeapInternalTimer<?, ?>) o).containsKey(o);
    +	}
    +
    +	@Override
    +	public boolean remove(@Nullable Object o) {
    +		if (o instanceof TimerHeapInternalTimer) {
    +			return removeInternal((TimerHeapInternalTimer<?, ?>) o);
    +		}
    +		return false;
    +	}
    +
    +	@Override
    +	public boolean addAll(@Nullable Collection<? extends TimerHeapInternalTimer<K, N>> timers) {
    +
    +		if (timers == null) {
    +			return true;
    +		}
    +
    +		if (timers.size() > queue.length) {
    +			checkCapacity(timers.size());
    +		}
    +
    +		for (TimerHeapInternalTimer<K, N> k : timers) {
    +			add(k);
    +		}
    +
    +		return true;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public Object[] toArray() {
    +		return Arrays.copyOfRange(queue, 1, size + 1);
    +	}
    +
    +	@SuppressWarnings({"unchecked", "SuspiciousSystemArraycopy"})
    +	@Nonnull
    +	@Override
    +	public <T> T[] toArray(@Nonnull T[] array) {
    +		if (array.length >= size) {
    +			System.arraycopy(queue, 1, array, 0, size);
    +			return array;
    +		}
    +		return (T[]) Arrays.copyOfRange(queue, 1, size + 1, array.getClass());
    +	}
    +
    +	@Override
    +	public boolean removeAll(@Nullable Collection<?> toRemove) {
    +
    +		if (toRemove == null) {
    +			return false;
    +		}
    +
    +		int oldSize = size();
    +		for (Object o : toRemove) {
    +			remove(o);
    +		}
    +		return size() == oldSize;
    +	}
    +
    +	/**
    +	 * Returns an iterator over the elements in this queue. The iterator
    +	 * does not return the elements in any particular order.
    +	 *
    +	 * @return an iterator over the elements in this queue.
    +	 */
    +	@Nonnull
    +	@Override
    +	public Iterator<TimerHeapInternalTimer<K, N>> iterator() {
    +		return new InternalTimerPriorityQueueIterator();
    +	}
    +
    +	@Override
    +	public boolean containsAll(@Nullable Collection<?> toCheck) {
    +
    +		if (toCheck == null) {
    +			return true;
    +		}
    +
    +		for (Object o : toCheck) {
    +			if (!contains(o)) {
    +				return false;
    +			}
    +		}
    +
    +		return true;
    +	}
    +
    +	@Nonnegative
    +	@Override
    +	public int size() {
    +		return size;
    +	}
    +
    +	@Override
    +	public void clear() {
    +
    +		Arrays.fill(queue, null);
    +		for (HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> timerHashMap :
    +			deduplicationMapsByKeyGroup) {
    +			timerHashMap.clear();
    +		}
    +		size = 0;
    +	}
    +
    +	/**
    +	 * This method is currently not implemented.
    +	 */
    +	@Override
    +	public boolean retainAll(@Nullable Collection<?> toRetain) {
    +		throw new UnsupportedOperationException();
    +	}
    +
    +	/**
    +	 * Adds a new timer with the given timestamp, key, and namespace to the heap, if an identical timer was not yet
    +	 * registered.
    +	 *
    +	 * @param timestamp the timer timestamp.
    +	 * @param key the timer key.
    +	 * @param namespace the timer namespace.
    +	 * @return true iff a new timer with given timestamp, key, and namespace was added to the heap.
    +	 */
    +	boolean scheduleTimer(long timestamp, K key, N namespace) {
    +		return add(new TimerHeapInternalTimer<>(timestamp, key, namespace));
    +	}
    +
    +	/**
    +	 * Stops timer with the given timestamp, key, and namespace by removing it from the heap, if it exists on the heap.
    +	 *
    +	 * @param timestamp the timer timestamp.
    +	 * @param key the timer key.
    +	 * @param namespace the timer namespace.
    +	 * @return true iff a timer with given timestamp, key, and namespace was found and removed from the heap.
    +	 */
    +	boolean stopTimer(long timestamp, K key, N namespace) {
    +		return removeInternal(new TimerHeapInternalTimer<>(timestamp, key, namespace));
    +	}
    +
    +	/**
    +	 * This method adds all the given timers to the heap.
    +	 */
    +	void restoreTimers(Collection<? extends InternalTimer<K, N>> toAdd) {
    +		if (toAdd == null) {
    +			return;
    +		}
    +
    +		if (toAdd.size() > queue.length) {
    +			checkCapacity(toAdd.size());
    +		}
    +
    +		for (InternalTimer<K, N> k : toAdd) {
    +			if (k instanceof TimerHeapInternalTimer) {
    +				add((TimerHeapInternalTimer<K, N>) k);
    +			} else {
    +				scheduleTimer(k.getTimestamp(), k.getKey(), k.getNamespace());
    +			}
    +		}
    +	}
    +
    +	private boolean removeInternal(TimerHeapInternalTimer<?, ?> timerToRemove) {
    +
    +		TimerHeapInternalTimer<K, N> storedTimer = getDedupMapForKeyGroup(timerToRemove).remove(timerToRemove);
    +
    +		if (storedTimer != null) {
    +			removeElementAtIndex(storedTimer.getTimerHeapIndex());
    +			return true;
    +		}
    +
    +		return false;
    +	}
    +
    +	private TimerHeapInternalTimer<K, N> removeElementAtIndex(int removeIdx) {
    +		TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		TimerHeapInternalTimer<K, N> removedValue = heap[removeIdx];
    +
    +		assert removedValue.getTimerHeapIndex() == removeIdx;
    +
    +		final int oldSize = size;
    +
    +		if (removeIdx != oldSize) {
    +			TimerHeapInternalTimer<K, N> timer = heap[oldSize];
    +			moveElementToIdx(timer, removeIdx);
    +			siftDown(removeIdx);
    +			if (heap[removeIdx] == timer) {
    +				siftUp(removeIdx);
    +			}
    +		}
    +
    +		heap[oldSize] = null;
    +		getDedupMapForKeyGroup(removedValue).remove(removedValue);
    +
    +		--size;
    +		return removedValue;
    +	}
    +
    +	private void siftUp(int idx) {
    +		final TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		TimerHeapInternalTimer<K, N> currentTimer = heap[idx];
    +		int parentIdx = idx >>> 1;
    +
    +		while (parentIdx > 0 && COMPARATOR.compare(currentTimer, heap[parentIdx]) < 0) {
    +			moveElementToIdx(heap[parentIdx], idx);
    +			idx = parentIdx;
    +			parentIdx = parentIdx >>> 1;
    +		}
    +
    +		moveElementToIdx(currentTimer, idx);
    +	}
    +
    +	private void siftDown(int idx) {
    +		final TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		final int heapSize = this.size;
    +
    +		TimerHeapInternalTimer<K, N> currentTimer = heap[idx];
    +		int firstChildIdx = idx << 1;
    +		int secondChildIdx = firstChildIdx + 1;
    +
    +		if (secondChildIdx <= heapSize && COMPARATOR.compare(heap[secondChildIdx], heap[firstChildIdx]) < 0) {
    +			firstChildIdx = secondChildIdx;
    +		}
    +
    +		while (firstChildIdx <= heapSize && COMPARATOR.compare(heap[firstChildIdx], currentTimer) < 0) {
    +			moveElementToIdx(heap[firstChildIdx], idx);
    +			idx = firstChildIdx;
    +			firstChildIdx = idx << 1;
    +			secondChildIdx = firstChildIdx + 1;
    +
    +			if (secondChildIdx <= heapSize && COMPARATOR.compare(heap[secondChildIdx], heap[firstChildIdx]) < 0) {
    +				firstChildIdx = secondChildIdx;
    +			}
    +		}
    +
    +		moveElementToIdx(currentTimer, idx);
    +	}
    +
    +	/**
    +	 * Returns an unmodifiable set of all timers in the given key-group.
    +	 */
    +	Set<InternalTimer<K, N>> getTimersForKeyGroup(@Nonnegative int keyGroupIdx) {
    +		return Collections.unmodifiableSet(getDedupMapForKeyGroup(keyGroupIdx).keySet());
    +	}
    +
    +	private HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> getDedupMapForKeyGroup(
    +		@Nonnegative int keyGroupIdx) {
    +		return deduplicationMapsByKeyGroup[globalKeyGroupToLocalIndex(keyGroupIdx)];
    +	}
    +
    +	@VisibleForTesting
    +	@SuppressWarnings("unchecked")
    +	List<Set<InternalTimer<K, N>>> getTimersByKeyGroup() {
    +		List<Set<InternalTimer<K, N>>> result = new ArrayList<>(deduplicationMapsByKeyGroup.length);
    +		for (int i = 0; i < deduplicationMapsByKeyGroup.length; ++i) {
    +			result.add(i, Collections.unmodifiableSet(deduplicationMapsByKeyGroup[i].keySet()));
    +		}
    +		return result;
    +	}
    +
    +	private void moveElementToIdx(TimerHeapInternalTimer<K, N> element, int idx) {
    +		queue[idx] = element;
    +		element.setTimerHeapIndex(idx);
    +	}
    +
    +	private HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> getDedupMapForKeyGroup(
    +		TimerHeapInternalTimer<?, ?> timer) {
    +		int keyGroup = KeyGroupRangeAssignment.assignToKeyGroup(timer.getKey(), totalNumberOfKeyGroups);
    +		return getDedupMapForKeyGroup(keyGroup);
    +	}
    +
    +	private int globalKeyGroupToLocalIndex(int keyGroup) {
    +		checkArgument(keyGroupRange.contains(keyGroup));
    +		return keyGroup - keyGroupRange.getStartKeyGroup();
    +	}
    +
    +	private void checkCapacity(int requested) {
    +		int oldArraySize = queue.length;
    +
    +		if (requested >= oldArraySize) {
    --- End diff --
    
    You are right 👍 


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r190295347
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerHeap.java ---
    @@ -0,0 +1,504 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.annotation.VisibleForTesting;
    +import org.apache.flink.runtime.state.KeyGroupRange;
    +import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
    +
    +import javax.annotation.Nonnegative;
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Queue;
    +import java.util.Set;
    +
    +import static org.apache.flink.util.Preconditions.checkArgument;
    +
    +/**
    + * A heap-based priority queue for internal timers. This heap is supported by hash sets for fast contains
    + * (de-duplication) and deletes. The heap implementation is a simple binary tree stored inside an array. Element indexes
    + * in the heap array start at 1 instead of 0 to make array index computations a bit simpler in the hot methods.
    + *
    + * <p>Possible future improvements:
    + * <ul>
    + *  <li>We could also implement shrinking for the heap and the deduplication maps.</li>
    + *  <li>We could replace the deduplication maps with more efficient custom implementations. In particular, a hash set
    + * would be enough if it could return existing elements on unsuccessful adding, etc..</li>
    + * </ul>
    + *
    + * @param <K> type of the key of the internal timers managed by this priority queue.
    + * @param <N> type of the namespace of the internal timers managed by this priority queue.
    + */
    +public class InternalTimerHeap<K, N> implements Queue<TimerHeapInternalTimer<K, N>>, Set<TimerHeapInternalTimer<K, N>> {
    +
    +	/**
    +	 * A safe maximum size for arrays in the JVM.
    +	 */
    +	private static final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8;
    +
    +	/**
    +	 * Comparator for {@link TimerHeapInternalTimer}, based on the timestamp in ascending order.
    +	 */
    +	private static final Comparator<TimerHeapInternalTimer<?, ?>> COMPARATOR =
    +		(o1, o2) -> Long.compare(o1.getTimestamp(), o2.getTimestamp());
    +
    +	/**
    +	 * This array contains one hash set per key-group. The sets are used for fast de-duplication and deletes of timers.
    +	 */
    +	private final HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>>[] deduplicationMapsByKeyGroup;
    +
    +	/**
    +	 * The array that represents the heap-organized priority queue.
    +	 */
    +	private TimerHeapInternalTimer<K, N>[] queue;
    +
    +	/**
    +	 * The current size of the priority queue.
    +	 */
    +	private int size;
    +
    +	/**
    +	 * The key-group range of timers that are managed by this queue.
    +	 */
    +	private final KeyGroupRange keyGroupRange;
    +
    +	/**
    +	 * The total number of key-groups of the job.
    +	 */
    +	private final int totalNumberOfKeyGroups;
    +
    +
    +	/**
    +	 * Creates an empty {@link InternalTimerHeap} with the requested initial capacity.
    +	 *
    +	 * @param minimumCapacity the minimum and initial capacity of this priority queue.
    +	 */
    +	@SuppressWarnings("unchecked")
    +	InternalTimerHeap(
    +		@Nonnegative int minimumCapacity,
    +		@Nonnull KeyGroupRange keyGroupRange,
    +		@Nonnegative int totalNumberOfKeyGroups) {
    +
    +		this.totalNumberOfKeyGroups = totalNumberOfKeyGroups;
    +		this.keyGroupRange = keyGroupRange;
    +
    +		final int keyGroupsInLocalRange = keyGroupRange.getNumberOfKeyGroups();
    +		final int deduplicationSetSize = 1 + minimumCapacity / keyGroupsInLocalRange;
    +		this.deduplicationMapsByKeyGroup = new HashMap[keyGroupsInLocalRange];
    +		for (int i = 0; i < keyGroupsInLocalRange; ++i) {
    +			deduplicationMapsByKeyGroup[i] = new HashMap<>(deduplicationSetSize);
    +		}
    +
    +		this.queue = new TimerHeapInternalTimer[1 + minimumCapacity];
    +	}
    +
    +	/**
    +	 * @see Set#add(Object)
    +	 */
    +	@Override
    +	public boolean add(@Nonnull TimerHeapInternalTimer<K, N> timer) {
    +
    +		if (getDedupMapForKeyGroup(timer).putIfAbsent(timer, timer) == null) {
    +			final int newSize = ++this.size;
    +			checkCapacity(newSize);
    +			moveElementToIdx(timer, newSize);
    +			siftUp(newSize);
    +			return true;
    +		} else {
    +			return false;
    +		}
    +	}
    +
    +	/**
    +	 * This behaves like {@link #add(TimerHeapInternalTimer)}.
    +	 */
    +	@Override
    +	public boolean offer(@Nonnull TimerHeapInternalTimer<K, N> k) {
    +		return add(k);
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> poll() {
    +		return size() > 0 ? removeElementAtIndex(1) : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> remove() {
    +		TimerHeapInternalTimer<K, N> pollResult = poll();
    +		if (pollResult != null) {
    +			return pollResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> peek() {
    +		return size() > 0 ? queue[1] : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> element() {
    +		TimerHeapInternalTimer<K, N> peekResult = peek();
    +		if (peekResult != null) {
    +			return peekResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Override
    +	public boolean isEmpty() {
    +		return size() == 0;
    +	}
    +
    +	@Override
    +	public boolean contains(@Nullable Object o) {
    +		return (o instanceof TimerHeapInternalTimer)
    +			&& getDedupMapForKeyGroup((TimerHeapInternalTimer<?, ?>) o).containsKey(o);
    +	}
    +
    +	@Override
    +	public boolean remove(@Nullable Object o) {
    +		if (o instanceof TimerHeapInternalTimer) {
    +			return removeInternal((TimerHeapInternalTimer<?, ?>) o);
    +		}
    +		return false;
    +	}
    +
    +	@Override
    +	public boolean addAll(@Nullable Collection<? extends TimerHeapInternalTimer<K, N>> timers) {
    +
    +		if (timers == null) {
    +			return true;
    +		}
    +
    +		if (timers.size() > queue.length) {
    --- End diff --
    
    I think maybe `if (time.size() + size() > queue.length)` is better, because we can ensure that we only need to rescale the array once with this approach. With the current version, we may need to rescale it multi times, what do you think?


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r191427611
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerHeap.java ---
    @@ -0,0 +1,511 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.annotation.VisibleForTesting;
    +import org.apache.flink.runtime.state.KeyGroupRange;
    +import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
    +
    +import javax.annotation.Nonnegative;
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Queue;
    +import java.util.Set;
    +
    +import static org.apache.flink.util.Preconditions.checkArgument;
    +
    +/**
    + * A heap-based priority queue for internal timers. This heap is supported by hash sets for fast contains
    + * (de-duplication) and deletes. The heap implementation is a simple binary tree stored inside an array. Element indexes
    + * in the heap array start at 1 instead of 0 to make array index computations a bit simpler in the hot methods.
    + *
    + * <p>Possible future improvements:
    + * <ul>
    + *  <li>We could also implement shrinking for the heap and the deduplication maps.</li>
    + *  <li>We could replace the deduplication maps with more efficient custom implementations. In particular, a hash set
    + * would be enough if it could return existing elements on unsuccessful adding, etc..</li>
    + * </ul>
    + *
    + * @param <K> type of the key of the internal timers managed by this priority queue.
    + * @param <N> type of the namespace of the internal timers managed by this priority queue.
    + */
    +public class InternalTimerHeap<K, N> implements Queue<TimerHeapInternalTimer<K, N>>, Set<TimerHeapInternalTimer<K, N>> {
    +
    +	/**
    +	 * A safe maximum size for arrays in the JVM.
    +	 */
    +	private static final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8;
    +
    +	/**
    +	 * Comparator for {@link TimerHeapInternalTimer}, based on the timestamp in ascending order.
    +	 */
    +	private static final Comparator<TimerHeapInternalTimer<?, ?>> COMPARATOR =
    +		(o1, o2) -> Long.compare(o1.getTimestamp(), o2.getTimestamp());
    +
    +	/**
    +	 * This array contains one hash set per key-group. The sets are used for fast de-duplication and deletes of timers.
    +	 */
    +	private final HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>>[] deduplicationMapsByKeyGroup;
    +
    +	/**
    +	 * The array that represents the heap-organized priority queue.
    +	 */
    +	private TimerHeapInternalTimer<K, N>[] queue;
    +
    +	/**
    +	 * The current size of the priority queue.
    +	 */
    +	private int size;
    +
    +	/**
    +	 * The key-group range of timers that are managed by this queue.
    +	 */
    +	private final KeyGroupRange keyGroupRange;
    +
    +	/**
    +	 * The total number of key-groups of the job.
    +	 */
    +	private final int totalNumberOfKeyGroups;
    +
    +
    +	/**
    +	 * Creates an empty {@link InternalTimerHeap} with the requested initial capacity.
    +	 *
    +	 * @param minimumCapacity the minimum and initial capacity of this priority queue.
    +	 */
    +	@SuppressWarnings("unchecked")
    +	InternalTimerHeap(
    +		@Nonnegative int minimumCapacity,
    +		@Nonnull KeyGroupRange keyGroupRange,
    +		@Nonnegative int totalNumberOfKeyGroups) {
    +
    +		this.totalNumberOfKeyGroups = totalNumberOfKeyGroups;
    +		this.keyGroupRange = keyGroupRange;
    +
    +		final int keyGroupsInLocalRange = keyGroupRange.getNumberOfKeyGroups();
    +		final int deduplicationSetSize = 1 + minimumCapacity / keyGroupsInLocalRange;
    +		this.deduplicationMapsByKeyGroup = new HashMap[keyGroupsInLocalRange];
    +		for (int i = 0; i < keyGroupsInLocalRange; ++i) {
    +			deduplicationMapsByKeyGroup[i] = new HashMap<>(deduplicationSetSize);
    +		}
    +
    +		this.queue = new TimerHeapInternalTimer[1 + minimumCapacity];
    +	}
    +
    +	/**
    +	 * @see Set#add(Object)
    +	 */
    +	@Override
    +	public boolean add(@Nonnull TimerHeapInternalTimer<K, N> timer) {
    +
    +		if (getDedupMapForKeyGroup(timer).putIfAbsent(timer, timer) == null) {
    +			final int newSize = ++this.size;
    +			growIfRequired(newSize);
    +			moveElementToIdx(timer, newSize);
    +			siftUp(newSize);
    +			return true;
    +		} else {
    +			return false;
    +		}
    +	}
    +
    +	/**
    +	 * This behaves like {@link #add(TimerHeapInternalTimer)}.
    +	 */
    +	@Override
    +	public boolean offer(@Nonnull TimerHeapInternalTimer<K, N> k) {
    +		return add(k);
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> poll() {
    +		return size() > 0 ? removeElementAtIndex(1) : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> remove() {
    +		TimerHeapInternalTimer<K, N> pollResult = poll();
    +		if (pollResult != null) {
    +			return pollResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> peek() {
    +		return size() > 0 ? queue[1] : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> element() {
    +		TimerHeapInternalTimer<K, N> peekResult = peek();
    +		if (peekResult != null) {
    +			return peekResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Override
    +	public boolean isEmpty() {
    +		return size() == 0;
    +	}
    +
    +	@Override
    +	public boolean contains(@Nullable Object toCheck) {
    +		return (toCheck instanceof TimerHeapInternalTimer)
    +			&& getDedupMapForKeyGroup((TimerHeapInternalTimer<?, ?>) toCheck).containsKey(toCheck);
    +	}
    +
    +	@Override
    +	public boolean remove(@Nullable Object toRemove) {
    +		if (toRemove instanceof TimerHeapInternalTimer) {
    +			return removeInternal((TimerHeapInternalTimer<?, ?>) toRemove);
    +		}
    +		return false;
    +	}
    +
    +	@Override
    +	public boolean addAll(@Nullable Collection<? extends TimerHeapInternalTimer<K, N>> timers) {
    +
    +		if (timers == null) {
    +			return true;
    +		}
    +
    +		final int oldSize = size();
    +		resizeForBulkLoad(oldSize + timers.size());
    +
    +		for (TimerHeapInternalTimer<K, N> k : timers) {
    +			add(k);
    +		}
    +
    +		return oldSize != size();
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public Object[] toArray() {
    +		return Arrays.copyOfRange(queue, 1, size + 1);
    +	}
    +
    +	@SuppressWarnings({"unchecked", "SuspiciousSystemArraycopy"})
    +	@Nonnull
    +	@Override
    +	public <T> T[] toArray(@Nonnull T[] array) {
    +		if (array.length >= size) {
    +			System.arraycopy(queue, 1, array, 0, size);
    +			return array;
    +		}
    +		return (T[]) Arrays.copyOfRange(queue, 1, size + 1, array.getClass());
    +	}
    +
    +	@Override
    +	public boolean removeAll(@Nullable Collection<?> toRemove) {
    +
    +		if (toRemove == null) {
    +			return false;
    +		}
    +
    +		int oldSize = size();
    +		for (Object o : toRemove) {
    +			remove(o);
    +		}
    +		return oldSize != size();
    +	}
    +
    +	/**
    +	 * Returns an iterator over the elements in this queue. The iterator
    +	 * does not return the elements in any particular order.
    +	 *
    +	 * @return an iterator over the elements in this queue.
    +	 */
    +	@Nonnull
    +	@Override
    +	public Iterator<TimerHeapInternalTimer<K, N>> iterator() {
    +		return new InternalTimerPriorityQueueIterator();
    +	}
    +
    +	@Override
    +	public boolean containsAll(@Nullable Collection<?> toCheck) {
    +
    +		if (toCheck == null) {
    +			return true;
    +		}
    +
    +		for (Object o : toCheck) {
    +			if (!contains(o)) {
    +				return false;
    +			}
    +		}
    +
    +		return true;
    +	}
    +
    +	@Nonnegative
    +	@Override
    +	public int size() {
    +		return size;
    +	}
    +
    +	@Override
    +	public void clear() {
    +
    +		Arrays.fill(queue, null);
    +		for (HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> timerHashMap :
    +			deduplicationMapsByKeyGroup) {
    +			timerHashMap.clear();
    +		}
    +		size = 0;
    +	}
    +
    +	/**
    +	 * This method is currently not implemented.
    +	 */
    +	@Override
    +	public boolean retainAll(@Nullable Collection<?> toRetain) {
    +		throw new UnsupportedOperationException();
    +	}
    +
    +	/**
    +	 * Adds a new timer with the given timestamp, key, and namespace to the heap, if an identical timer was not yet
    +	 * registered.
    +	 *
    +	 * @param timestamp the timer timestamp.
    +	 * @param key the timer key.
    +	 * @param namespace the timer namespace.
    +	 * @return true iff a new timer with given timestamp, key, and namespace was added to the heap.
    +	 */
    +	boolean scheduleTimer(long timestamp, K key, N namespace) {
    +		return add(new TimerHeapInternalTimer<>(timestamp, key, namespace));
    +	}
    +
    +	/**
    +	 * Stops timer with the given timestamp, key, and namespace by removing it from the heap, if it exists on the heap.
    +	 *
    +	 * @param timestamp the timer timestamp.
    +	 * @param key the timer key.
    +	 * @param namespace the timer namespace.
    +	 * @return true iff a timer with given timestamp, key, and namespace was found and removed from the heap.
    +	 */
    +	boolean stopTimer(long timestamp, K key, N namespace) {
    +		return removeInternal(new TimerHeapInternalTimer<>(timestamp, key, namespace));
    +	}
    +
    +	/**
    +	 * This method adds all the given timers to the heap.
    +	 */
    +	void addRestoredTimers(Collection<? extends InternalTimer<K, N>> restoredTimers) {
    +
    +		if (restoredTimers == null) {
    +			return;
    +		}
    +
    +		resizeForBulkLoad(restoredTimers.size());
    +
    +		for (InternalTimer<K, N> timer : restoredTimers) {
    +			if (timer instanceof TimerHeapInternalTimer) {
    +				add((TimerHeapInternalTimer<K, N>) timer);
    +			} else {
    +				scheduleTimer(timer.getTimestamp(), timer.getKey(), timer.getNamespace());
    +			}
    +		}
    +	}
    +
    +	private boolean removeInternal(TimerHeapInternalTimer<?, ?> timerToRemove) {
    +
    +		TimerHeapInternalTimer<K, N> storedTimer = getDedupMapForKeyGroup(timerToRemove).remove(timerToRemove);
    +
    +		if (storedTimer != null) {
    +			removeElementAtIndex(storedTimer.getTimerHeapIndex());
    +			return true;
    +		}
    +
    +		return false;
    +	}
    +
    +	private TimerHeapInternalTimer<K, N> removeElementAtIndex(int removeIdx) {
    +		TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		TimerHeapInternalTimer<K, N> removedValue = heap[removeIdx];
    +
    +		assert removedValue.getTimerHeapIndex() == removeIdx;
    +
    +		final int oldSize = size;
    +
    +		if (removeIdx != oldSize) {
    +			TimerHeapInternalTimer<K, N> timer = heap[oldSize];
    +			moveElementToIdx(timer, removeIdx);
    +			siftDown(removeIdx);
    +			if (heap[removeIdx] == timer) {
    +				siftUp(removeIdx);
    +			}
    +		}
    +
    +		heap[oldSize] = null;
    +		getDedupMapForKeyGroup(removedValue).remove(removedValue);
    +
    +		--size;
    +		return removedValue;
    +	}
    +
    +	private void siftUp(int idx) {
    +		final TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		TimerHeapInternalTimer<K, N> currentTimer = heap[idx];
    +		int parentIdx = idx >>> 1;
    +
    +		while (parentIdx > 0 && COMPARATOR.compare(currentTimer, heap[parentIdx]) < 0) {
    +			moveElementToIdx(heap[parentIdx], idx);
    +			idx = parentIdx;
    +			parentIdx = parentIdx >>> 1;
    +		}
    +
    +		moveElementToIdx(currentTimer, idx);
    +	}
    +
    +	private void siftDown(int idx) {
    +		final TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		final int heapSize = this.size;
    +
    +		TimerHeapInternalTimer<K, N> currentTimer = heap[idx];
    +		int firstChildIdx = idx << 1;
    +		int secondChildIdx = firstChildIdx + 1;
    +
    +		if (secondChildIdx <= heapSize && COMPARATOR.compare(heap[secondChildIdx], heap[firstChildIdx]) < 0) {
    +			firstChildIdx = secondChildIdx;
    +		}
    +
    +		while (firstChildIdx <= heapSize && COMPARATOR.compare(heap[firstChildIdx], currentTimer) < 0) {
    +			moveElementToIdx(heap[firstChildIdx], idx);
    +			idx = firstChildIdx;
    +			firstChildIdx = idx << 1;
    +			secondChildIdx = firstChildIdx + 1;
    +
    +			if (secondChildIdx <= heapSize && COMPARATOR.compare(heap[secondChildIdx], heap[firstChildIdx]) < 0) {
    +				firstChildIdx = secondChildIdx;
    +			}
    +		}
    +
    +		moveElementToIdx(currentTimer, idx);
    +	}
    +
    +	/**
    +	 * Returns an unmodifiable set of all timers in the given key-group.
    +	 */
    +	Set<InternalTimer<K, N>> getTimersForKeyGroup(@Nonnegative int keyGroupIdx) {
    +		return Collections.unmodifiableSet(getDedupMapForKeyGroup(keyGroupIdx).keySet());
    +	}
    +
    +	private HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> getDedupMapForKeyGroup(
    +		@Nonnegative int keyGroupIdx) {
    +		return deduplicationMapsByKeyGroup[globalKeyGroupToLocalIndex(keyGroupIdx)];
    +	}
    +
    +	@VisibleForTesting
    +	@SuppressWarnings("unchecked")
    +	List<Set<InternalTimer<K, N>>> getTimersByKeyGroup() {
    +		List<Set<InternalTimer<K, N>>> result = new ArrayList<>(deduplicationMapsByKeyGroup.length);
    +		for (int i = 0; i < deduplicationMapsByKeyGroup.length; ++i) {
    +			result.add(i, Collections.unmodifiableSet(deduplicationMapsByKeyGroup[i].keySet()));
    +		}
    +		return result;
    +	}
    +
    +	private void moveElementToIdx(TimerHeapInternalTimer<K, N> element, int idx) {
    +		queue[idx] = element;
    +		element.setTimerHeapIndex(idx);
    +	}
    +
    +	private HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> getDedupMapForKeyGroup(
    --- End diff --
    
    I think a better name would be `getDedupMapForTimer` because we pass in a `TimerHeapInternalTimer` instead of a key group index.


---

[GitHub] flink issue #6062: [FLINK-9423][state] Implement efficient deletes for heap-...

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

    https://github.com/apache/flink/pull/6062
  
    @StefanRRichter Thanks for your reply, I think I should also have a look at https://github.com/apache/flink/pull/3359 to supplement my knowledge on this. Looking forward your design document!


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r190288320
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerHeap.java ---
    @@ -0,0 +1,504 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.annotation.VisibleForTesting;
    +import org.apache.flink.runtime.state.KeyGroupRange;
    +import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
    +
    +import javax.annotation.Nonnegative;
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Queue;
    +import java.util.Set;
    +
    +import static org.apache.flink.util.Preconditions.checkArgument;
    +
    +/**
    + * A heap-based priority queue for internal timers. This heap is supported by hash sets for fast contains
    + * (de-duplication) and deletes. The heap implementation is a simple binary tree stored inside an array. Element indexes
    + * in the heap array start at 1 instead of 0 to make array index computations a bit simpler in the hot methods.
    + *
    + * <p>Possible future improvements:
    + * <ul>
    + *  <li>We could also implement shrinking for the heap and the deduplication maps.</li>
    + *  <li>We could replace the deduplication maps with more efficient custom implementations. In particular, a hash set
    + * would be enough if it could return existing elements on unsuccessful adding, etc..</li>
    + * </ul>
    + *
    + * @param <K> type of the key of the internal timers managed by this priority queue.
    + * @param <N> type of the namespace of the internal timers managed by this priority queue.
    + */
    +public class InternalTimerHeap<K, N> implements Queue<TimerHeapInternalTimer<K, N>>, Set<TimerHeapInternalTimer<K, N>> {
    +
    +	/**
    +	 * A safe maximum size for arrays in the JVM.
    +	 */
    +	private static final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8;
    +
    +	/**
    +	 * Comparator for {@link TimerHeapInternalTimer}, based on the timestamp in ascending order.
    +	 */
    +	private static final Comparator<TimerHeapInternalTimer<?, ?>> COMPARATOR =
    +		(o1, o2) -> Long.compare(o1.getTimestamp(), o2.getTimestamp());
    +
    +	/**
    +	 * This array contains one hash set per key-group. The sets are used for fast de-duplication and deletes of timers.
    +	 */
    +	private final HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>>[] deduplicationMapsByKeyGroup;
    +
    +	/**
    +	 * The array that represents the heap-organized priority queue.
    +	 */
    +	private TimerHeapInternalTimer<K, N>[] queue;
    +
    +	/**
    +	 * The current size of the priority queue.
    +	 */
    +	private int size;
    +
    +	/**
    +	 * The key-group range of timers that are managed by this queue.
    +	 */
    +	private final KeyGroupRange keyGroupRange;
    +
    +	/**
    +	 * The total number of key-groups of the job.
    +	 */
    +	private final int totalNumberOfKeyGroups;
    +
    +
    +	/**
    +	 * Creates an empty {@link InternalTimerHeap} with the requested initial capacity.
    +	 *
    +	 * @param minimumCapacity the minimum and initial capacity of this priority queue.
    +	 */
    +	@SuppressWarnings("unchecked")
    +	InternalTimerHeap(
    +		@Nonnegative int minimumCapacity,
    +		@Nonnull KeyGroupRange keyGroupRange,
    +		@Nonnegative int totalNumberOfKeyGroups) {
    +
    +		this.totalNumberOfKeyGroups = totalNumberOfKeyGroups;
    +		this.keyGroupRange = keyGroupRange;
    +
    +		final int keyGroupsInLocalRange = keyGroupRange.getNumberOfKeyGroups();
    +		final int deduplicationSetSize = 1 + minimumCapacity / keyGroupsInLocalRange;
    +		this.deduplicationMapsByKeyGroup = new HashMap[keyGroupsInLocalRange];
    +		for (int i = 0; i < keyGroupsInLocalRange; ++i) {
    +			deduplicationMapsByKeyGroup[i] = new HashMap<>(deduplicationSetSize);
    +		}
    +
    +		this.queue = new TimerHeapInternalTimer[1 + minimumCapacity];
    +	}
    +
    +	/**
    +	 * @see Set#add(Object)
    +	 */
    +	@Override
    +	public boolean add(@Nonnull TimerHeapInternalTimer<K, N> timer) {
    +
    +		if (getDedupMapForKeyGroup(timer).putIfAbsent(timer, timer) == null) {
    +			final int newSize = ++this.size;
    +			checkCapacity(newSize);
    +			moveElementToIdx(timer, newSize);
    +			siftUp(newSize);
    +			return true;
    +		} else {
    +			return false;
    +		}
    +	}
    +
    +	/**
    +	 * This behaves like {@link #add(TimerHeapInternalTimer)}.
    +	 */
    +	@Override
    +	public boolean offer(@Nonnull TimerHeapInternalTimer<K, N> k) {
    +		return add(k);
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> poll() {
    +		return size() > 0 ? removeElementAtIndex(1) : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> remove() {
    +		TimerHeapInternalTimer<K, N> pollResult = poll();
    +		if (pollResult != null) {
    +			return pollResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> peek() {
    +		return size() > 0 ? queue[1] : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> element() {
    +		TimerHeapInternalTimer<K, N> peekResult = peek();
    +		if (peekResult != null) {
    +			return peekResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Override
    +	public boolean isEmpty() {
    +		return size() == 0;
    +	}
    +
    +	@Override
    +	public boolean contains(@Nullable Object o) {
    +		return (o instanceof TimerHeapInternalTimer)
    +			&& getDedupMapForKeyGroup((TimerHeapInternalTimer<?, ?>) o).containsKey(o);
    +	}
    +
    +	@Override
    +	public boolean remove(@Nullable Object o) {
    +		if (o instanceof TimerHeapInternalTimer) {
    +			return removeInternal((TimerHeapInternalTimer<?, ?>) o);
    +		}
    +		return false;
    +	}
    +
    +	@Override
    +	public boolean addAll(@Nullable Collection<? extends TimerHeapInternalTimer<K, N>> timers) {
    +
    +		if (timers == null) {
    +			return true;
    +		}
    +
    +		if (timers.size() > queue.length) {
    --- End diff --
    
    Should this be `if (time.size() + size() > queue.length)`?


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r190293256
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerHeap.java ---
    @@ -0,0 +1,504 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.annotation.VisibleForTesting;
    +import org.apache.flink.runtime.state.KeyGroupRange;
    +import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
    +
    +import javax.annotation.Nonnegative;
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Queue;
    +import java.util.Set;
    +
    +import static org.apache.flink.util.Preconditions.checkArgument;
    +
    +/**
    + * A heap-based priority queue for internal timers. This heap is supported by hash sets for fast contains
    + * (de-duplication) and deletes. The heap implementation is a simple binary tree stored inside an array. Element indexes
    + * in the heap array start at 1 instead of 0 to make array index computations a bit simpler in the hot methods.
    + *
    + * <p>Possible future improvements:
    + * <ul>
    + *  <li>We could also implement shrinking for the heap and the deduplication maps.</li>
    + *  <li>We could replace the deduplication maps with more efficient custom implementations. In particular, a hash set
    + * would be enough if it could return existing elements on unsuccessful adding, etc..</li>
    + * </ul>
    + *
    + * @param <K> type of the key of the internal timers managed by this priority queue.
    + * @param <N> type of the namespace of the internal timers managed by this priority queue.
    + */
    +public class InternalTimerHeap<K, N> implements Queue<TimerHeapInternalTimer<K, N>>, Set<TimerHeapInternalTimer<K, N>> {
    +
    +	/**
    +	 * A safe maximum size for arrays in the JVM.
    +	 */
    +	private static final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8;
    +
    +	/**
    +	 * Comparator for {@link TimerHeapInternalTimer}, based on the timestamp in ascending order.
    +	 */
    +	private static final Comparator<TimerHeapInternalTimer<?, ?>> COMPARATOR =
    +		(o1, o2) -> Long.compare(o1.getTimestamp(), o2.getTimestamp());
    +
    +	/**
    +	 * This array contains one hash set per key-group. The sets are used for fast de-duplication and deletes of timers.
    +	 */
    +	private final HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>>[] deduplicationMapsByKeyGroup;
    +
    +	/**
    +	 * The array that represents the heap-organized priority queue.
    +	 */
    +	private TimerHeapInternalTimer<K, N>[] queue;
    +
    +	/**
    +	 * The current size of the priority queue.
    +	 */
    +	private int size;
    +
    +	/**
    +	 * The key-group range of timers that are managed by this queue.
    +	 */
    +	private final KeyGroupRange keyGroupRange;
    +
    +	/**
    +	 * The total number of key-groups of the job.
    +	 */
    +	private final int totalNumberOfKeyGroups;
    +
    +
    +	/**
    +	 * Creates an empty {@link InternalTimerHeap} with the requested initial capacity.
    +	 *
    +	 * @param minimumCapacity the minimum and initial capacity of this priority queue.
    +	 */
    +	@SuppressWarnings("unchecked")
    +	InternalTimerHeap(
    +		@Nonnegative int minimumCapacity,
    +		@Nonnull KeyGroupRange keyGroupRange,
    +		@Nonnegative int totalNumberOfKeyGroups) {
    +
    +		this.totalNumberOfKeyGroups = totalNumberOfKeyGroups;
    +		this.keyGroupRange = keyGroupRange;
    +
    +		final int keyGroupsInLocalRange = keyGroupRange.getNumberOfKeyGroups();
    +		final int deduplicationSetSize = 1 + minimumCapacity / keyGroupsInLocalRange;
    +		this.deduplicationMapsByKeyGroup = new HashMap[keyGroupsInLocalRange];
    +		for (int i = 0; i < keyGroupsInLocalRange; ++i) {
    +			deduplicationMapsByKeyGroup[i] = new HashMap<>(deduplicationSetSize);
    +		}
    +
    +		this.queue = new TimerHeapInternalTimer[1 + minimumCapacity];
    +	}
    +
    +	/**
    +	 * @see Set#add(Object)
    +	 */
    +	@Override
    +	public boolean add(@Nonnull TimerHeapInternalTimer<K, N> timer) {
    +
    +		if (getDedupMapForKeyGroup(timer).putIfAbsent(timer, timer) == null) {
    +			final int newSize = ++this.size;
    +			checkCapacity(newSize);
    +			moveElementToIdx(timer, newSize);
    +			siftUp(newSize);
    +			return true;
    +		} else {
    +			return false;
    +		}
    +	}
    +
    +	/**
    +	 * This behaves like {@link #add(TimerHeapInternalTimer)}.
    +	 */
    +	@Override
    +	public boolean offer(@Nonnull TimerHeapInternalTimer<K, N> k) {
    +		return add(k);
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> poll() {
    +		return size() > 0 ? removeElementAtIndex(1) : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> remove() {
    +		TimerHeapInternalTimer<K, N> pollResult = poll();
    +		if (pollResult != null) {
    +			return pollResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> peek() {
    +		return size() > 0 ? queue[1] : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> element() {
    +		TimerHeapInternalTimer<K, N> peekResult = peek();
    +		if (peekResult != null) {
    +			return peekResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Override
    +	public boolean isEmpty() {
    +		return size() == 0;
    +	}
    +
    +	@Override
    +	public boolean contains(@Nullable Object o) {
    +		return (o instanceof TimerHeapInternalTimer)
    +			&& getDedupMapForKeyGroup((TimerHeapInternalTimer<?, ?>) o).containsKey(o);
    +	}
    +
    +	@Override
    +	public boolean remove(@Nullable Object o) {
    +		if (o instanceof TimerHeapInternalTimer) {
    +			return removeInternal((TimerHeapInternalTimer<?, ?>) o);
    +		}
    +		return false;
    +	}
    +
    +	@Override
    +	public boolean addAll(@Nullable Collection<? extends TimerHeapInternalTimer<K, N>> timers) {
    +
    +		if (timers == null) {
    +			return true;
    +		}
    +
    +		if (timers.size() > queue.length) {
    +			checkCapacity(timers.size());
    +		}
    +
    +		for (TimerHeapInternalTimer<K, N> k : timers) {
    +			add(k);
    +		}
    +
    +		return true;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public Object[] toArray() {
    +		return Arrays.copyOfRange(queue, 1, size + 1);
    +	}
    +
    +	@SuppressWarnings({"unchecked", "SuspiciousSystemArraycopy"})
    +	@Nonnull
    +	@Override
    +	public <T> T[] toArray(@Nonnull T[] array) {
    +		if (array.length >= size) {
    +			System.arraycopy(queue, 1, array, 0, size);
    +			return array;
    +		}
    +		return (T[]) Arrays.copyOfRange(queue, 1, size + 1, array.getClass());
    +	}
    +
    +	@Override
    +	public boolean removeAll(@Nullable Collection<?> toRemove) {
    +
    +		if (toRemove == null) {
    +			return false;
    +		}
    +
    +		int oldSize = size();
    +		for (Object o : toRemove) {
    +			remove(o);
    +		}
    +		return size() == oldSize;
    +	}
    +
    +	/**
    +	 * Returns an iterator over the elements in this queue. The iterator
    +	 * does not return the elements in any particular order.
    +	 *
    +	 * @return an iterator over the elements in this queue.
    +	 */
    +	@Nonnull
    +	@Override
    +	public Iterator<TimerHeapInternalTimer<K, N>> iterator() {
    +		return new InternalTimerPriorityQueueIterator();
    +	}
    +
    +	@Override
    +	public boolean containsAll(@Nullable Collection<?> toCheck) {
    +
    +		if (toCheck == null) {
    +			return true;
    +		}
    +
    +		for (Object o : toCheck) {
    +			if (!contains(o)) {
    +				return false;
    +			}
    +		}
    +
    +		return true;
    +	}
    +
    +	@Nonnegative
    +	@Override
    +	public int size() {
    +		return size;
    +	}
    +
    +	@Override
    +	public void clear() {
    +
    +		Arrays.fill(queue, null);
    +		for (HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> timerHashMap :
    +			deduplicationMapsByKeyGroup) {
    +			timerHashMap.clear();
    +		}
    +		size = 0;
    +	}
    +
    +	/**
    +	 * This method is currently not implemented.
    +	 */
    +	@Override
    +	public boolean retainAll(@Nullable Collection<?> toRetain) {
    +		throw new UnsupportedOperationException();
    +	}
    +
    +	/**
    +	 * Adds a new timer with the given timestamp, key, and namespace to the heap, if an identical timer was not yet
    +	 * registered.
    +	 *
    +	 * @param timestamp the timer timestamp.
    +	 * @param key the timer key.
    +	 * @param namespace the timer namespace.
    +	 * @return true iff a new timer with given timestamp, key, and namespace was added to the heap.
    +	 */
    +	boolean scheduleTimer(long timestamp, K key, N namespace) {
    +		return add(new TimerHeapInternalTimer<>(timestamp, key, namespace));
    +	}
    +
    +	/**
    +	 * Stops timer with the given timestamp, key, and namespace by removing it from the heap, if it exists on the heap.
    +	 *
    +	 * @param timestamp the timer timestamp.
    +	 * @param key the timer key.
    +	 * @param namespace the timer namespace.
    +	 * @return true iff a timer with given timestamp, key, and namespace was found and removed from the heap.
    +	 */
    +	boolean stopTimer(long timestamp, K key, N namespace) {
    +		return removeInternal(new TimerHeapInternalTimer<>(timestamp, key, namespace));
    +	}
    +
    +	/**
    +	 * This method adds all the given timers to the heap.
    +	 */
    +	void restoreTimers(Collection<? extends InternalTimer<K, N>> toAdd) {
    +		if (toAdd == null) {
    +			return;
    +		}
    +
    +		if (toAdd.size() > queue.length) {
    +			checkCapacity(toAdd.size());
    +		}
    +
    +		for (InternalTimer<K, N> k : toAdd) {
    +			if (k instanceof TimerHeapInternalTimer) {
    +				add((TimerHeapInternalTimer<K, N>) k);
    +			} else {
    +				scheduleTimer(k.getTimestamp(), k.getKey(), k.getNamespace());
    +			}
    +		}
    +	}
    +
    +	private boolean removeInternal(TimerHeapInternalTimer<?, ?> timerToRemove) {
    +
    +		TimerHeapInternalTimer<K, N> storedTimer = getDedupMapForKeyGroup(timerToRemove).remove(timerToRemove);
    +
    +		if (storedTimer != null) {
    +			removeElementAtIndex(storedTimer.getTimerHeapIndex());
    +			return true;
    +		}
    +
    +		return false;
    +	}
    +
    +	private TimerHeapInternalTimer<K, N> removeElementAtIndex(int removeIdx) {
    +		TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		TimerHeapInternalTimer<K, N> removedValue = heap[removeIdx];
    +
    +		assert removedValue.getTimerHeapIndex() == removeIdx;
    +
    +		final int oldSize = size;
    +
    +		if (removeIdx != oldSize) {
    +			TimerHeapInternalTimer<K, N> timer = heap[oldSize];
    +			moveElementToIdx(timer, removeIdx);
    +			siftDown(removeIdx);
    +			if (heap[removeIdx] == timer) {
    +				siftUp(removeIdx);
    +			}
    +		}
    +
    +		heap[oldSize] = null;
    +		getDedupMapForKeyGroup(removedValue).remove(removedValue);
    +
    +		--size;
    +		return removedValue;
    +	}
    +
    +	private void siftUp(int idx) {
    +		final TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		TimerHeapInternalTimer<K, N> currentTimer = heap[idx];
    +		int parentIdx = idx >>> 1;
    +
    +		while (parentIdx > 0 && COMPARATOR.compare(currentTimer, heap[parentIdx]) < 0) {
    +			moveElementToIdx(heap[parentIdx], idx);
    +			idx = parentIdx;
    +			parentIdx = parentIdx >>> 1;
    +		}
    +
    +		moveElementToIdx(currentTimer, idx);
    +	}
    +
    +	private void siftDown(int idx) {
    +		final TimerHeapInternalTimer<K, N>[] heap = this.queue;
    +		final int heapSize = this.size;
    +
    +		TimerHeapInternalTimer<K, N> currentTimer = heap[idx];
    +		int firstChildIdx = idx << 1;
    +		int secondChildIdx = firstChildIdx + 1;
    +
    +		if (secondChildIdx <= heapSize && COMPARATOR.compare(heap[secondChildIdx], heap[firstChildIdx]) < 0) {
    +			firstChildIdx = secondChildIdx;
    +		}
    +
    +		while (firstChildIdx <= heapSize && COMPARATOR.compare(heap[firstChildIdx], currentTimer) < 0) {
    +			moveElementToIdx(heap[firstChildIdx], idx);
    +			idx = firstChildIdx;
    +			firstChildIdx = idx << 1;
    +			secondChildIdx = firstChildIdx + 1;
    +
    +			if (secondChildIdx <= heapSize && COMPARATOR.compare(heap[secondChildIdx], heap[firstChildIdx]) < 0) {
    +				firstChildIdx = secondChildIdx;
    +			}
    +		}
    +
    +		moveElementToIdx(currentTimer, idx);
    +	}
    +
    +	/**
    +	 * Returns an unmodifiable set of all timers in the given key-group.
    +	 */
    +	Set<InternalTimer<K, N>> getTimersForKeyGroup(@Nonnegative int keyGroupIdx) {
    +		return Collections.unmodifiableSet(getDedupMapForKeyGroup(keyGroupIdx).keySet());
    +	}
    +
    +	private HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> getDedupMapForKeyGroup(
    +		@Nonnegative int keyGroupIdx) {
    +		return deduplicationMapsByKeyGroup[globalKeyGroupToLocalIndex(keyGroupIdx)];
    +	}
    +
    +	@VisibleForTesting
    +	@SuppressWarnings("unchecked")
    +	List<Set<InternalTimer<K, N>>> getTimersByKeyGroup() {
    +		List<Set<InternalTimer<K, N>>> result = new ArrayList<>(deduplicationMapsByKeyGroup.length);
    +		for (int i = 0; i < deduplicationMapsByKeyGroup.length; ++i) {
    +			result.add(i, Collections.unmodifiableSet(deduplicationMapsByKeyGroup[i].keySet()));
    +		}
    +		return result;
    +	}
    +
    +	private void moveElementToIdx(TimerHeapInternalTimer<K, N> element, int idx) {
    +		queue[idx] = element;
    +		element.setTimerHeapIndex(idx);
    +	}
    +
    +	private HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>> getDedupMapForKeyGroup(
    +		TimerHeapInternalTimer<?, ?> timer) {
    +		int keyGroup = KeyGroupRangeAssignment.assignToKeyGroup(timer.getKey(), totalNumberOfKeyGroups);
    +		return getDedupMapForKeyGroup(keyGroup);
    +	}
    +
    +	private int globalKeyGroupToLocalIndex(int keyGroup) {
    +		checkArgument(keyGroupRange.contains(keyGroup));
    +		return keyGroup - keyGroupRange.getStartKeyGroup();
    +	}
    +
    +	private void checkCapacity(int requested) {
    +		int oldArraySize = queue.length;
    +
    +		if (requested >= oldArraySize) {
    --- End diff --
    
    Sorry my bad, I think I misunderstand here, plz ignore this...


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r190279440
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerHeap.java ---
    @@ -0,0 +1,504 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.annotation.VisibleForTesting;
    +import org.apache.flink.runtime.state.KeyGroupRange;
    +import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
    +
    +import javax.annotation.Nonnegative;
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Queue;
    +import java.util.Set;
    +
    +import static org.apache.flink.util.Preconditions.checkArgument;
    +
    +/**
    + * A heap-based priority queue for internal timers. This heap is supported by hash sets for fast contains
    + * (de-duplication) and deletes. The heap implementation is a simple binary tree stored inside an array. Element indexes
    + * in the heap array start at 1 instead of 0 to make array index computations a bit simpler in the hot methods.
    + *
    + * <p>Possible future improvements:
    + * <ul>
    + *  <li>We could also implement shrinking for the heap and the deduplication maps.</li>
    + *  <li>We could replace the deduplication maps with more efficient custom implementations. In particular, a hash set
    + * would be enough if it could return existing elements on unsuccessful adding, etc..</li>
    + * </ul>
    + *
    + * @param <K> type of the key of the internal timers managed by this priority queue.
    + * @param <N> type of the namespace of the internal timers managed by this priority queue.
    + */
    +public class InternalTimerHeap<K, N> implements Queue<TimerHeapInternalTimer<K, N>>, Set<TimerHeapInternalTimer<K, N>> {
    +
    +	/**
    +	 * A safe maximum size for arrays in the JVM.
    +	 */
    +	private static final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8;
    +
    +	/**
    +	 * Comparator for {@link TimerHeapInternalTimer}, based on the timestamp in ascending order.
    +	 */
    +	private static final Comparator<TimerHeapInternalTimer<?, ?>> COMPARATOR =
    +		(o1, o2) -> Long.compare(o1.getTimestamp(), o2.getTimestamp());
    +
    +	/**
    +	 * This array contains one hash set per key-group. The sets are used for fast de-duplication and deletes of timers.
    +	 */
    +	private final HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>>[] deduplicationMapsByKeyGroup;
    +
    +	/**
    +	 * The array that represents the heap-organized priority queue.
    +	 */
    +	private TimerHeapInternalTimer<K, N>[] queue;
    +
    +	/**
    +	 * The current size of the priority queue.
    +	 */
    +	private int size;
    +
    +	/**
    +	 * The key-group range of timers that are managed by this queue.
    +	 */
    +	private final KeyGroupRange keyGroupRange;
    +
    +	/**
    +	 * The total number of key-groups of the job.
    +	 */
    +	private final int totalNumberOfKeyGroups;
    +
    +
    +	/**
    +	 * Creates an empty {@link InternalTimerHeap} with the requested initial capacity.
    +	 *
    +	 * @param minimumCapacity the minimum and initial capacity of this priority queue.
    +	 */
    +	@SuppressWarnings("unchecked")
    +	InternalTimerHeap(
    +		@Nonnegative int minimumCapacity,
    +		@Nonnull KeyGroupRange keyGroupRange,
    +		@Nonnegative int totalNumberOfKeyGroups) {
    +
    +		this.totalNumberOfKeyGroups = totalNumberOfKeyGroups;
    +		this.keyGroupRange = keyGroupRange;
    +
    +		final int keyGroupsInLocalRange = keyGroupRange.getNumberOfKeyGroups();
    +		final int deduplicationSetSize = 1 + minimumCapacity / keyGroupsInLocalRange;
    +		this.deduplicationMapsByKeyGroup = new HashMap[keyGroupsInLocalRange];
    +		for (int i = 0; i < keyGroupsInLocalRange; ++i) {
    +			deduplicationMapsByKeyGroup[i] = new HashMap<>(deduplicationSetSize);
    +		}
    +
    +		this.queue = new TimerHeapInternalTimer[1 + minimumCapacity];
    +	}
    +
    +	/**
    +	 * @see Set#add(Object)
    +	 */
    +	@Override
    +	public boolean add(@Nonnull TimerHeapInternalTimer<K, N> timer) {
    +
    +		if (getDedupMapForKeyGroup(timer).putIfAbsent(timer, timer) == null) {
    +			final int newSize = ++this.size;
    +			checkCapacity(newSize);
    +			moveElementToIdx(timer, newSize);
    +			siftUp(newSize);
    +			return true;
    +		} else {
    +			return false;
    +		}
    +	}
    +
    +	/**
    +	 * This behaves like {@link #add(TimerHeapInternalTimer)}.
    +	 */
    +	@Override
    +	public boolean offer(@Nonnull TimerHeapInternalTimer<K, N> k) {
    +		return add(k);
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> poll() {
    +		return size() > 0 ? removeElementAtIndex(1) : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> remove() {
    +		TimerHeapInternalTimer<K, N> pollResult = poll();
    +		if (pollResult != null) {
    +			return pollResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Nullable
    +	@Override
    +	public TimerHeapInternalTimer<K, N> peek() {
    +		return size() > 0 ? queue[1] : null;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public TimerHeapInternalTimer<K, N> element() {
    +		TimerHeapInternalTimer<K, N> peekResult = peek();
    +		if (peekResult != null) {
    +			return peekResult;
    +		} else {
    +			throw new NoSuchElementException("InternalTimerPriorityQueue is empty.");
    +		}
    +	}
    +
    +	@Override
    +	public boolean isEmpty() {
    +		return size() == 0;
    +	}
    +
    +	@Override
    +	public boolean contains(@Nullable Object o) {
    +		return (o instanceof TimerHeapInternalTimer)
    +			&& getDedupMapForKeyGroup((TimerHeapInternalTimer<?, ?>) o).containsKey(o);
    +	}
    +
    +	@Override
    +	public boolean remove(@Nullable Object o) {
    +		if (o instanceof TimerHeapInternalTimer) {
    +			return removeInternal((TimerHeapInternalTimer<?, ?>) o);
    +		}
    +		return false;
    +	}
    +
    +	@Override
    +	public boolean addAll(@Nullable Collection<? extends TimerHeapInternalTimer<K, N>> timers) {
    +
    +		if (timers == null) {
    +			return true;
    +		}
    +
    +		if (timers.size() > queue.length) {
    +			checkCapacity(timers.size());
    +		}
    +
    +		for (TimerHeapInternalTimer<K, N> k : timers) {
    +			add(k);
    +		}
    +
    +		return true;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public Object[] toArray() {
    +		return Arrays.copyOfRange(queue, 1, size + 1);
    +	}
    +
    +	@SuppressWarnings({"unchecked", "SuspiciousSystemArraycopy"})
    +	@Nonnull
    +	@Override
    +	public <T> T[] toArray(@Nonnull T[] array) {
    +		if (array.length >= size) {
    +			System.arraycopy(queue, 1, array, 0, size);
    +			return array;
    +		}
    +		return (T[]) Arrays.copyOfRange(queue, 1, size + 1, array.getClass());
    +	}
    +
    +	@Override
    +	public boolean removeAll(@Nullable Collection<?> toRemove) {
    +
    +		if (toRemove == null) {
    +			return false;
    +		}
    +
    +		int oldSize = size();
    +		for (Object o : toRemove) {
    +			remove(o);
    +		}
    +		return size() == oldSize;
    --- End diff --
    
    Should this be `size() != oldSize`?


---

[GitHub] flink issue #6062: [FLINK-9423][state] Implement efficient deletes for heap-...

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

    https://github.com/apache/flink/pull/6062
  
    Ok, I also think might still be some room for optimizations of the implementation for a `RocksDBTimerHeap` over the existing code.


---

[GitHub] flink issue #6062: [FLINK-9423][state] Implement efficient deletes for heap-...

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

    https://github.com/apache/flink/pull/6062
  
    @sihuazhou This is exactly my plan on how introduce a "timer state" for the keyed state backends. In a nutshell, time timer service can just register and operator on those states, where the basic operations that we expose are somewhat along the lines of `scheduleTimer`, `stopTimer`, `peekNextTimer`, `pollNextTimer`. I am currently working on a small design document to outline this integration. For RocksDB we can implementation of the state can be loosely based on some of the ideas from the class `RocksDBTimerHeap` of PR #3359.


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r191738639
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerHeap.java ---
    @@ -0,0 +1,511 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.annotation.VisibleForTesting;
    +import org.apache.flink.runtime.state.KeyGroupRange;
    +import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
    +
    +import javax.annotation.Nonnegative;
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Queue;
    +import java.util.Set;
    +
    +import static org.apache.flink.util.Preconditions.checkArgument;
    +
    +/**
    + * A heap-based priority queue for internal timers. This heap is supported by hash sets for fast contains
    + * (de-duplication) and deletes. The heap implementation is a simple binary tree stored inside an array. Element indexes
    + * in the heap array start at 1 instead of 0 to make array index computations a bit simpler in the hot methods.
    + *
    + * <p>Possible future improvements:
    + * <ul>
    + *  <li>We could also implement shrinking for the heap and the deduplication maps.</li>
    + *  <li>We could replace the deduplication maps with more efficient custom implementations. In particular, a hash set
    + * would be enough if it could return existing elements on unsuccessful adding, etc..</li>
    + * </ul>
    + *
    + * @param <K> type of the key of the internal timers managed by this priority queue.
    + * @param <N> type of the namespace of the internal timers managed by this priority queue.
    + */
    +public class InternalTimerHeap<K, N> implements Queue<TimerHeapInternalTimer<K, N>>, Set<TimerHeapInternalTimer<K, N>> {
    --- End diff --
    
    I think `toArray` (for later snapshots) and `iterator` (for tests and in general) are actually useful. Will consider removing the interfaces.


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r190463723
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TimerHeapInternalTimer.java ---
    @@ -0,0 +1,235 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.annotation.Internal;
    +import org.apache.flink.api.common.typeutils.CompatibilityResult;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
    +import org.apache.flink.api.common.typeutils.base.LongSerializer;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.core.memory.DataOutputView;
    +
    +import java.io.IOException;
    +
    +/**
    + * Implementation of {@link InternalTimer} for the {@link InternalTimerHeap}.
    + *
    + * @param <K> Type of the keys to which timers are scoped.
    + * @param <N> Type of the namespace to which timers are scoped.
    + */
    +@Internal
    +public final class TimerHeapInternalTimer<K, N> implements InternalTimer<K, N> {
    +
    +	/** The index that indicates that a tracked internal timer is not tracked. */
    +	private static final int NOT_MANAGED_BY_TIMER_QUEUE_INDEX = Integer.MIN_VALUE;
    +
    +	private final long timestamp;
    +
    +	private final K key;
    +
    +	private final N namespace;
    +
    +	/**
    +	 * This field holds the current physical index if this timer when it is managed by a timer heap so that we can
    +	 * support fast deletes.
    +	 */
    +	private transient int timerHeapIndex;
    +
    +	TimerHeapInternalTimer(long timestamp, K key, N namespace) {
    +		this.timestamp = timestamp;
    +		this.key = key;
    +		this.namespace = namespace;
    +		this.timerHeapIndex = NOT_MANAGED_BY_TIMER_QUEUE_INDEX;
    +	}
    +
    +	@Override
    +	public long getTimestamp() {
    +		return timestamp;
    +	}
    +
    +	@Override
    +	public K getKey() {
    +		return key;
    +	}
    +
    +	@Override
    +	public N getNamespace() {
    +		return namespace;
    +	}
    +
    +	@Override
    +	public boolean equals(Object o) {
    +		if (this == o) {
    +			return true;
    +		}
    +
    +		if (o instanceof InternalTimer) {
    +			InternalTimer<?, ?> timer = (InternalTimer<?, ?>) o;
    +			return timestamp == timer.getTimestamp()
    +				&& key.equals(timer.getKey())
    +				&& namespace.equals(timer.getNamespace());
    +		}
    +
    +		return false;
    +	}
    +
    +	/**
    +	 * Returns the current index of this timer in the owning timer heap.
    +	 */
    +	int getTimerHeapIndex() {
    +		return timerHeapIndex;
    +	}
    +
    +	/**
    +	 * Sets the current index of this timer in the owning timer heap and should only be called by the managing heap.
    +	 * @param timerHeapIndex the new index in the timer heap.
    +	 */
    +	void setTimerHeapIndex(int timerHeapIndex) {
    +		this.timerHeapIndex = timerHeapIndex;
    +	}
    +
    +	/**
    +	 * This method can be called to indicate that the timer is no longer managed be a timer heap, e.g. because it as
    +	 * removed.
    +	 */
    +	void removedFromTimerQueue() {
    +		setTimerHeapIndex(NOT_MANAGED_BY_TIMER_QUEUE_INDEX);
    +	}
    +
    +	@Override
    +	public int hashCode() {
    +		int result = (int) (timestamp ^ (timestamp >>> 32));
    +		result = 31 * result + key.hashCode();
    +		result = 31 * result + namespace.hashCode();
    +		return result;
    +	}
    +
    +	@Override
    +	public String toString() {
    +		return "Timer{" +
    +				"timestamp=" + timestamp +
    +				", key=" + key +
    +				", namespace=" + namespace +
    +				'}';
    +	}
    +
    +	/**
    +	 * A {@link TypeSerializer} used to serialize/deserialize a {@link TimerHeapInternalTimer}.
    +	 */
    +	public static class TimerSerializer<K, N> extends TypeSerializer<InternalTimer<K, N>> {
    +
    +		private static final long serialVersionUID = 1119562170939152304L;
    +
    +		private final TypeSerializer<K> keySerializer;
    +
    +		private final TypeSerializer<N> namespaceSerializer;
    +
    +		TimerSerializer(TypeSerializer<K> keySerializer, TypeSerializer<N> namespaceSerializer) {
    +			this.keySerializer = keySerializer;
    +			this.namespaceSerializer = namespaceSerializer;
    +		}
    +
    +		@Override
    +		public boolean isImmutableType() {
    +			return false;
    +		}
    +
    +		@Override
    +		public TypeSerializer<InternalTimer<K, N>> duplicate() {
    +			return this;
    --- End diff --
    
    This means that the result of `duplicate()` is not a "duplication" if `keySerializer` or `namespaceSerializer` is stateful.


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r191419645
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerHeap.java ---
    @@ -0,0 +1,511 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.annotation.VisibleForTesting;
    +import org.apache.flink.runtime.state.KeyGroupRange;
    +import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
    +
    +import javax.annotation.Nonnegative;
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.NoSuchElementException;
    +import java.util.Queue;
    +import java.util.Set;
    +
    +import static org.apache.flink.util.Preconditions.checkArgument;
    +
    +/**
    + * A heap-based priority queue for internal timers. This heap is supported by hash sets for fast contains
    + * (de-duplication) and deletes. The heap implementation is a simple binary tree stored inside an array. Element indexes
    + * in the heap array start at 1 instead of 0 to make array index computations a bit simpler in the hot methods.
    + *
    + * <p>Possible future improvements:
    + * <ul>
    + *  <li>We could also implement shrinking for the heap and the deduplication maps.</li>
    + *  <li>We could replace the deduplication maps with more efficient custom implementations. In particular, a hash set
    + * would be enough if it could return existing elements on unsuccessful adding, etc..</li>
    + * </ul>
    + *
    + * @param <K> type of the key of the internal timers managed by this priority queue.
    + * @param <N> type of the namespace of the internal timers managed by this priority queue.
    + */
    +public class InternalTimerHeap<K, N> implements Queue<TimerHeapInternalTimer<K, N>>, Set<TimerHeapInternalTimer<K, N>> {
    +
    +	/**
    +	 * A safe maximum size for arrays in the JVM.
    +	 */
    +	private static final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8;
    +
    +	/**
    +	 * Comparator for {@link TimerHeapInternalTimer}, based on the timestamp in ascending order.
    +	 */
    +	private static final Comparator<TimerHeapInternalTimer<?, ?>> COMPARATOR =
    +		(o1, o2) -> Long.compare(o1.getTimestamp(), o2.getTimestamp());
    +
    +	/**
    +	 * This array contains one hash set per key-group. The sets are used for fast de-duplication and deletes of timers.
    +	 */
    +	private final HashMap<TimerHeapInternalTimer<K, N>, TimerHeapInternalTimer<K, N>>[] deduplicationMapsByKeyGroup;
    +
    +	/**
    +	 * The array that represents the heap-organized priority queue.
    +	 */
    +	private TimerHeapInternalTimer<K, N>[] queue;
    +
    +	/**
    +	 * The current size of the priority queue.
    +	 */
    +	private int size;
    +
    +	/**
    +	 * The key-group range of timers that are managed by this queue.
    +	 */
    +	private final KeyGroupRange keyGroupRange;
    +
    +	/**
    +	 * The total number of key-groups of the job.
    +	 */
    +	private final int totalNumberOfKeyGroups;
    +
    +
    +	/**
    +	 * Creates an empty {@link InternalTimerHeap} with the requested initial capacity.
    +	 *
    +	 * @param minimumCapacity the minimum and initial capacity of this priority queue.
    +	 */
    +	@SuppressWarnings("unchecked")
    +	InternalTimerHeap(
    +		@Nonnegative int minimumCapacity,
    +		@Nonnull KeyGroupRange keyGroupRange,
    +		@Nonnegative int totalNumberOfKeyGroups) {
    +
    +		this.totalNumberOfKeyGroups = totalNumberOfKeyGroups;
    +		this.keyGroupRange = keyGroupRange;
    +
    +		final int keyGroupsInLocalRange = keyGroupRange.getNumberOfKeyGroups();
    +		final int deduplicationSetSize = 1 + minimumCapacity / keyGroupsInLocalRange;
    +		this.deduplicationMapsByKeyGroup = new HashMap[keyGroupsInLocalRange];
    +		for (int i = 0; i < keyGroupsInLocalRange; ++i) {
    +			deduplicationMapsByKeyGroup[i] = new HashMap<>(deduplicationSetSize);
    +		}
    +
    +		this.queue = new TimerHeapInternalTimer[1 + minimumCapacity];
    +	}
    +
    +	/**
    +	 * @see Set#add(Object)
    +	 */
    +	@Override
    +	public boolean add(@Nonnull TimerHeapInternalTimer<K, N> timer) {
    +
    +		if (getDedupMapForKeyGroup(timer).putIfAbsent(timer, timer) == null) {
    +			final int newSize = ++this.size;
    +			growIfRequired(newSize);
    +			moveElementToIdx(timer, newSize);
    +			siftUp(newSize);
    +			return true;
    +		} else {
    +			return false;
    +		}
    +	}
    +
    +	/**
    +	 * This behaves like {@link #add(TimerHeapInternalTimer)}.
    +	 */
    +	@Override
    +	public boolean offer(@Nonnull TimerHeapInternalTimer<K, N> k) {
    --- End diff --
    
    let's give the parameter a proper name like `timer`


---

[GitHub] flink pull request #6062: [FLINK-9423][state] Implement efficient deletes fo...

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

    https://github.com/apache/flink/pull/6062#discussion_r190481987
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TimerHeapInternalTimer.java ---
    @@ -0,0 +1,235 @@
    +/*
    + * 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.flink.streaming.api.operators;
    +
    +import org.apache.flink.annotation.Internal;
    +import org.apache.flink.api.common.typeutils.CompatibilityResult;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
    +import org.apache.flink.api.common.typeutils.base.LongSerializer;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.core.memory.DataOutputView;
    +
    +import java.io.IOException;
    +
    +/**
    + * Implementation of {@link InternalTimer} for the {@link InternalTimerHeap}.
    + *
    + * @param <K> Type of the keys to which timers are scoped.
    + * @param <N> Type of the namespace to which timers are scoped.
    + */
    +@Internal
    +public final class TimerHeapInternalTimer<K, N> implements InternalTimer<K, N> {
    +
    +	/** The index that indicates that a tracked internal timer is not tracked. */
    +	private static final int NOT_MANAGED_BY_TIMER_QUEUE_INDEX = Integer.MIN_VALUE;
    +
    +	private final long timestamp;
    +
    +	private final K key;
    +
    +	private final N namespace;
    +
    +	/**
    +	 * This field holds the current physical index if this timer when it is managed by a timer heap so that we can
    +	 * support fast deletes.
    +	 */
    +	private transient int timerHeapIndex;
    +
    +	TimerHeapInternalTimer(long timestamp, K key, N namespace) {
    +		this.timestamp = timestamp;
    +		this.key = key;
    +		this.namespace = namespace;
    +		this.timerHeapIndex = NOT_MANAGED_BY_TIMER_QUEUE_INDEX;
    +	}
    +
    +	@Override
    +	public long getTimestamp() {
    +		return timestamp;
    +	}
    +
    +	@Override
    +	public K getKey() {
    +		return key;
    +	}
    +
    +	@Override
    +	public N getNamespace() {
    +		return namespace;
    +	}
    +
    +	@Override
    +	public boolean equals(Object o) {
    +		if (this == o) {
    +			return true;
    +		}
    +
    +		if (o instanceof InternalTimer) {
    +			InternalTimer<?, ?> timer = (InternalTimer<?, ?>) o;
    +			return timestamp == timer.getTimestamp()
    +				&& key.equals(timer.getKey())
    +				&& namespace.equals(timer.getNamespace());
    +		}
    +
    +		return false;
    +	}
    +
    +	/**
    +	 * Returns the current index of this timer in the owning timer heap.
    +	 */
    +	int getTimerHeapIndex() {
    +		return timerHeapIndex;
    +	}
    +
    +	/**
    +	 * Sets the current index of this timer in the owning timer heap and should only be called by the managing heap.
    +	 * @param timerHeapIndex the new index in the timer heap.
    +	 */
    +	void setTimerHeapIndex(int timerHeapIndex) {
    +		this.timerHeapIndex = timerHeapIndex;
    +	}
    +
    +	/**
    +	 * This method can be called to indicate that the timer is no longer managed be a timer heap, e.g. because it as
    +	 * removed.
    +	 */
    +	void removedFromTimerQueue() {
    +		setTimerHeapIndex(NOT_MANAGED_BY_TIMER_QUEUE_INDEX);
    +	}
    +
    +	@Override
    +	public int hashCode() {
    +		int result = (int) (timestamp ^ (timestamp >>> 32));
    +		result = 31 * result + key.hashCode();
    +		result = 31 * result + namespace.hashCode();
    +		return result;
    +	}
    +
    +	@Override
    +	public String toString() {
    +		return "Timer{" +
    +				"timestamp=" + timestamp +
    +				", key=" + key +
    +				", namespace=" + namespace +
    +				'}';
    +	}
    +
    +	/**
    +	 * A {@link TypeSerializer} used to serialize/deserialize a {@link TimerHeapInternalTimer}.
    +	 */
    +	public static class TimerSerializer<K, N> extends TypeSerializer<InternalTimer<K, N>> {
    +
    +		private static final long serialVersionUID = 1119562170939152304L;
    +
    +		private final TypeSerializer<K> keySerializer;
    +
    +		private final TypeSerializer<N> namespaceSerializer;
    +
    +		TimerSerializer(TypeSerializer<K> keySerializer, TypeSerializer<N> namespaceSerializer) {
    +			this.keySerializer = keySerializer;
    +			this.namespaceSerializer = namespaceSerializer;
    +		}
    +
    +		@Override
    +		public boolean isImmutableType() {
    +			return false;
    +		}
    +
    +		@Override
    +		public TypeSerializer<InternalTimer<K, N>> duplicate() {
    +			return this;
    --- End diff --
    
    That is true, but this was already the case before this PR and it was never an issue because the timer service snapshots are not yet asynchronous. Nevertheless, it should be improved 👍 


---