carp84 commented on a change in pull request #9501: [FLINK-12697] [State 
Backends] Support on-disk state storage for spill-able heap backend
URL: https://github.com/apache/flink/pull/9501#discussion_r325986596
 
 

 ##########
 File path: 
flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteSkipListStateMap.java
 ##########
 @@ -0,0 +1,1527 @@
+/*
+ * 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.runtime.state.heap;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.core.memory.ByteBufferUtils;
+import org.apache.flink.runtime.state.StateEntry;
+import org.apache.flink.runtime.state.StateTransformationFunction;
+import org.apache.flink.runtime.state.heap.space.Allocator;
+import org.apache.flink.runtime.state.heap.space.Chunk;
+import org.apache.flink.runtime.state.heap.space.SpaceUtils;
+import org.apache.flink.runtime.state.internal.InternalKvState;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.ResourceGuard;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.Spliterators;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+
+import static org.apache.flink.runtime.state.heap.SkipListUtils.HEAD_NODE;
+import static org.apache.flink.runtime.state.heap.SkipListUtils.NIL_NODE;
+import static 
org.apache.flink.runtime.state.heap.SkipListUtils.NIL_VALUE_POINTER;
+
+/**
+ * Implementation of state map which is based on skip list with copy-on-write 
support. states will
+ * be serialized to bytes and stored in the space allocated with the given 
allocator.
+ *
+ * @param <K> type of key
+ * @param <N> type of namespace
+ * @param <S> type of state
+ */
+public class CopyOnWriteSkipListStateMap<K, N, S> extends StateMap<K, N, S> 
implements AutoCloseable {
+
+       private static final Logger LOG = 
LoggerFactory.getLogger(CopyOnWriteSkipListStateMap.class);
+
+       /**
+        * Default max number of logically-removed keys to delete one time.
+        */
+       private static final int DEFAULT_MAX_KEYS_TO_DELETE_ONE_TIME = 3;
+
+       /**
+        * Default ratio of the logically-removed keys to trigger deletion when 
snapshot.
+        */
+       private static final float DEFAULT_LOGICAL_REMOVED_KEYS_RATIO = 0.2f;
+
+       /**
+        * The serializer used to serialize the key and namespace to bytes 
stored in skip list.
+        */
+       private final SkipListKeySerializer<K, N> skipListKeySerializer;
+
+       /**
+        * The serializer used to serialize the state to bytes stored in skip 
list.
+        */
+       private final SkipListValueSerializer<S> skipListValueSerializer;
+
+       /**
+        * Space allocator.
+        */
+       private final Allocator spaceAllocator;
+
+       /**
+        * The level index header.
+        */
+       private final LevelIndexHeader levelIndexHeader;
+
+       /**
+        * Seed to generate random index level.
+        */
+       private int randomSeed;
+
+       /**
+        * The current version of this map. Used for copy-on-write mechanics.
+        */
+       private int stateMapVersion;
+
+       /**
+        * The highest version of this map that is still required by any 
unreleased snapshot.
+        */
+       private int highestRequiredSnapshotVersion;
+
+       /**
+        * Snapshots no more than this version must have been finished, but 
there may be some
+        * snapshots more than this version are still running.
+        */
+       private volatile int highestFinishedSnapshotVersion;
+
+       /**
+        * Maintains an ordered set of version ids that are still used by 
unreleased snapshots.
+        */
+       private final TreeSet<Integer> snapshotVersions;
+
+       /**
+        * The size of skip list which includes the logical removed keys.
+        */
+       private int totalSize;
+
+       /**
+        * Number of requests for this skip list.
+        */
+       private int requestCount;
+
+       /**
+        * Set of logical removed nodes.
+        */
+       private final Set<Long> logicallyRemovedNodes;
+
+       /**
+        * Number of keys to remove physically one time.
+        */
+       private int numKeysToDeleteOneTime;
+
+       /**
+        * Ratio of the logically-removed keys to trigger deletion when 
snapshot.
+        */
+       private float logicalRemovedKeysRatio;
+
+       /**
+        * Set of nodes whose values are being pruned by snapshots.
+        */
+       private final Set<Long> pruningValueNodes;
+
+       /**
+        * Whether this map has been closed.
+        */
+       private final AtomicBoolean closed;
+
+       /**
+        * Guards for the free of space when state map is closed. This is mainly
+        * used to synchronize with snapshots.
+        */
+       private final ResourceGuard resourceGuard;
+
+       public CopyOnWriteSkipListStateMap(
+                       @Nonnull TypeSerializer<K> keySerializer,
+                       @Nonnull TypeSerializer<N> namespaceSerializer,
+                       @Nonnull TypeSerializer<S> stateSerializer,
+                       @Nonnull Allocator spaceAllocator) {
+               this(keySerializer, namespaceSerializer, stateSerializer, 
spaceAllocator,
+                       DEFAULT_MAX_KEYS_TO_DELETE_ONE_TIME, 
DEFAULT_LOGICAL_REMOVED_KEYS_RATIO);
+       }
+
+       public CopyOnWriteSkipListStateMap(
+                       @Nonnull TypeSerializer<K> keySerializer,
+                       @Nonnull TypeSerializer<N> namespaceSerializer,
+                       @Nonnull TypeSerializer<S> stateSerializer,
+                       @Nonnull Allocator spaceAllocator,
+                       int numKeysToDeleteOneTime,
+                       float logicalRemovedKeysRatio) {
+               this.skipListKeySerializer = new 
SkipListKeySerializer<>(keySerializer, namespaceSerializer);
+               this.skipListValueSerializer = new 
SkipListValueSerializer<>(stateSerializer);
+               this.spaceAllocator = spaceAllocator;
+               Preconditions.checkArgument(numKeysToDeleteOneTime >= 0,
+                       "numKeysToDeleteOneTime should be non-negative, but is 
"  + numKeysToDeleteOneTime);
+               this.numKeysToDeleteOneTime = numKeysToDeleteOneTime;
+               Preconditions.checkArgument(logicalRemovedKeysRatio >= 0 && 
logicalRemovedKeysRatio <= 1,
+                       "logicalRemovedKeysRatio should be in [0, 1], but is " 
+ logicalRemovedKeysRatio);
+               this.logicalRemovedKeysRatio = logicalRemovedKeysRatio;
+
+               this.levelIndexHeader = new OnHeapLevelIndexHeader();
+               this.randomSeed = ThreadLocalRandom.current().nextInt() | 
0x0100;
+
+               this.stateMapVersion = 0;
+               this.highestRequiredSnapshotVersion = 0;
+               this.highestFinishedSnapshotVersion = 0;
+               this.snapshotVersions = new TreeSet<>();
+
+               this.totalSize = 0;
+               this.requestCount = 0;
+
+               this.logicallyRemovedNodes = new HashSet<>();
+               this.pruningValueNodes = ConcurrentHashMap.newKeySet();
+
+               this.closed = new AtomicBoolean(false);
+               this.resourceGuard = new ResourceGuard();
+       }
+
+       @Override
+       public int size() {
+               return totalSize - logicallyRemovedNodes.size();
+       }
+
+       /**
+        * Returns total size of this map, including logically removed state.
+        */
+       public int totalSize() {
+               return totalSize;
+       }
+
+       public int getRequestCount() {
+               return requestCount;
+       }
+
+       @Override
+       public S get(K key, N namespace) {
+               updateStat();
+
+               long node = getNodeInternal(key, namespace);
+
+               if (node == NIL_NODE) {
+                       return null;
+               }
+
+               return helpGetNodeState(node);
+       }
+
+       @Override
+       public boolean containsKey(K key, N namespace) {
+               updateStat();
+
+               long node = getNodeInternal(key, namespace);
+
+               return node != NIL_NODE;
+       }
+
+       @Override
+       public void put(K key, N namespace, S state) {
+               updateStat();
+               ByteBuffer keyByteBuffer = getKeyByteBuffer(key, namespace);
+               int keyLen = keyByteBuffer.limit();
+               byte[] value = skipListValueSerializer.serialize(state);
+
+               putNode(keyByteBuffer, 0, keyLen, value, false);
+       }
+
+       @Override
+       public S putAndGetOld(K key, N namespace, S state) {
+               updateStat();
+               ByteBuffer keyByteBuffer = getKeyByteBuffer(key, namespace);
+               int keyLen = keyByteBuffer.limit();
+               byte[] value = skipListValueSerializer.serialize(state);
+
+               return putNode(keyByteBuffer, 0, keyLen, value, true);
+       }
+
+       @Override
+       public void remove(K key, N namespace) {
+               updateStat();
+               ByteBuffer keyByteBuffer = getKeyByteBuffer(key, namespace);
+               int keyLen = keyByteBuffer.limit();
+
+               removeNode(keyByteBuffer, 0, keyLen, false);
+       }
+
+       @Override
+       public S removeAndGetOld(K key, N namespace) {
+               updateStat();
+               ByteBuffer keyByteBuffer = getKeyByteBuffer(key, namespace);
+               int keyLen = keyByteBuffer.limit();
+
+               return removeNode(keyByteBuffer, 0, keyLen, true);
+       }
+
+       @Override
+       public <T> void transform(
+               K key,
+               N namespace,
+               T value,
+               StateTransformationFunction<S, T> transformation) throws 
Exception {
+               updateStat();
+               ByteBuffer keyByteBuffer = getKeyByteBuffer(key, namespace);
+               int keyLen = keyByteBuffer.limit();
+
+               long node = getNode(keyByteBuffer, 0, keyLen);
+               S oldState = node == NIL_NODE ? null : helpGetNodeState(node);
+               S newState = transformation.apply(oldState, value);
+               byte[] stateBytes = skipListValueSerializer.serialize(newState);
+               putNode(keyByteBuffer, 0, keyLen, stateBytes, false);
+       }
+
+       // Detail implementation methods 
---------------------------------------------------------------
+
+       /**
+        * Find the node containing the given key.
+        *
+        * @param keyByteBuffer byte buffer storing the key.
+        * @param keyOffset     offset of the key.
+        * @param keyLen        length of the key.
+        * @return id of the node. NIL_NODE will be returned if key does no 
exist.
+        */
+       @VisibleForTesting
+       long getNode(ByteBuffer keyByteBuffer, int keyOffset, int keyLen) {
+               int deleteCount = 0;
+               long prevNode = findPredecessor(keyByteBuffer, keyOffset, 1);
+               long currentNode  = helpGetNextNode(prevNode, 0);
+               long nextNode;
+
+               int c;
+               while (currentNode != NIL_NODE) {
+                       nextNode = helpGetNextNode(currentNode, 0);
+
+                       boolean isRemoved = isNodeRemoved(currentNode);
+                       if (isRemoved) {
+                               // remove the node physically when there is no 
snapshot running
+                               if (highestRequiredSnapshotVersion == 0 && 
deleteCount < numKeysToDeleteOneTime) {
+                                       doPhysicalRemove(currentNode, prevNode, 
nextNode);
+                                       
logicallyRemovedNodes.remove(currentNode);
+                                       totalSize--;
+                                       deleteCount++;
+                               } else {
+                                       prevNode = currentNode;
+                               }
+                               currentNode = nextNode;
+                               continue;
+                       }
+
+                       c = compareByteBufferAndNode(keyByteBuffer, keyOffset, 
keyLen, currentNode);
+
+                       // find the key
+                       if (c == 0) {
+                               return currentNode;
+                       }
+
+                       // the key is less than the current node, and nodes 
after current
+                       // node can not be equal to the key.
+                       if (c < 0) {
+                               break;
+                       }
+
+                       prevNode = currentNode;
+                       currentNode = helpGetNextNode(currentNode, 0);
+               }
+
+               return NIL_NODE;
+       }
+
+       /**
+        * Put the key into the skip list. If the key does not exist before, a 
new node
+        * will be created. If the key exists before, the old state will be 
returned.
+        *
+        * @param keyByteBuffer  byte buffer storing the key.
+        * @param keyOffset      offset of the key.
+        * @param keyLen         length of the key.
+        * @param value          the value.
+        * @param returnOldState whether to return old state.
+        * @return the old state. Null will be returned if key does not exist 
or returnOldState is false.
+        */
+       @VisibleForTesting
+       S putNode(ByteBuffer keyByteBuffer, int keyOffset, int keyLen, byte[] 
value, boolean returnOldState) {
+               int deleteCount = 0;
+               long prevNode = findPredecessor(keyByteBuffer, keyOffset, 1);
+               long currentNode = helpGetNextNode(prevNode, 0);
+               long nextNode;
+
+               int c;
+               for ( ; ; ) {
+                       if (currentNode != NIL_NODE) {
+                               nextNode = helpGetNextNode(currentNode, 0);
+
+                               boolean isRemoved = isNodeRemoved(currentNode);
+
+                               // note that the key may be put again before 
the node is physically removed,
+                               // so the node still need to compare with the 
key although they can not be
+                               // physically removed here
 
 Review comment:
   Nice catch! This is something we plan to improve but forget to do, will make 
it in next commit.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to