dawidwys commented on a change in pull request #13550:
URL: https://github.com/apache/flink/pull/13550#discussion_r502368225



##########
File path: 
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/benchmark/StateBackendBenchmarkUtils.java
##########
@@ -74,6 +75,24 @@
                        case ROCKSDB:
                                rootDir = prepareDirectory(rootDirName, null);
                                return createRocksDBKeyedStateBackend(rootDir);
+                       case SINGLE_KEY:
+                               try {
+                                       return new 
SingleKeyStateBackend().createKeyedStateBackend(

Review comment:
       Actually I did it and forgot to force push :facepalm: 

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/sorted/state/SingleKeyKeyedStateBackend.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.sorted.state;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.state.AggregatingStateDescriptor;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.MapStateDescriptor;
+import org.apache.flink.api.common.state.ReducingStateDescriptor;
+import org.apache.flink.api.common.state.State;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.state.CheckpointStreamFactory;
+import org.apache.flink.runtime.state.CheckpointableKeyedStateBackend;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.KeyGroupedInternalPriorityQueue;
+import org.apache.flink.runtime.state.Keyed;
+import org.apache.flink.runtime.state.KeyedStateFunction;
+import org.apache.flink.runtime.state.KeyedStateHandle;
+import org.apache.flink.runtime.state.PriorityComparable;
+import org.apache.flink.runtime.state.PriorityComparator;
+import org.apache.flink.runtime.state.SnapshotResult;
+import org.apache.flink.runtime.state.StateSnapshotTransformer;
+import org.apache.flink.runtime.state.heap.HeapPriorityQueueElement;
+import org.apache.flink.runtime.state.internal.InternalKvState;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import javax.annotation.Nonnull;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.RunnableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A simple {@link CheckpointableKeyedStateBackend} which keeps values for a 
single key at a time.
+ *
+ * <p><b>IMPORTANT:</b> Requires the incoming records to be sorted/grouped by 
the key. Used in a BATCH style execution.
+ */
+class SingleKeyKeyedStateBackend<K> implements 
CheckpointableKeyedStateBackend<K> {
+       @SuppressWarnings("rawtypes")
+       private static final Map<Class<? extends StateDescriptor>, 
StateFactory> STATE_FACTORIES =
+               Stream.of(
+                       Tuple2.of(ValueStateDescriptor.class, (StateFactory) 
SingleKeyValueState::create),
+                       Tuple2.of(ListStateDescriptor.class, (StateFactory) 
SingleKeyListState::create),
+                       Tuple2.of(MapStateDescriptor.class, (StateFactory) 
SingleKeyMapState::create),
+                       Tuple2.of(AggregatingStateDescriptor.class, 
(StateFactory) SingleKeyAggregatingState::create),
+                       Tuple2.of(ReducingStateDescriptor.class, (StateFactory) 
SingleKeyReducingState::create)
+               ).collect(Collectors.toMap(t -> t.f0, t -> t.f1));
+
+       private K currentKey = null;
+       private final TypeSerializer<K> keySerializer;
+       private final List<KeySelectionListener<K>> keySelectionListeners = new 
ArrayList<>();
+       private final Map<String, State> states = new HashMap<>();
+       private final Map<String, KeyGroupedInternalPriorityQueue<?>> 
priorityQueues = new HashMap<>();
+       private final KeyGroupRange keyGroupRange;
+
+       public SingleKeyKeyedStateBackend(
+                       TypeSerializer<K> keySerializer,
+                       KeyGroupRange keyGroupRange) {
+               this.keySerializer = keySerializer;
+               this.keyGroupRange = keyGroupRange;
+       }
+
+       @Override
+       public void setCurrentKey(K newKey) {
+               if (!Objects.equals(newKey, currentKey)) {
+                       notifyKeySelected(newKey);
+                       for (State value : states.values()) {
+                               ((AbstractSingleKeyState<?, ?, ?>) 
value).clearAllNamespaces();
+                       }
+                       for (KeyGroupedInternalPriorityQueue<?> value : 
priorityQueues.values()) {
+                               while (value.poll() != null) {
+                                       // remove everything for the key
+                               }
+                       }
+                       this.currentKey = newKey;
+               }
+       }
+
+       @Override
+       public K getCurrentKey() {
+               return currentKey;
+       }
+
+       @Override
+       public TypeSerializer<K> getKeySerializer() {
+               return keySerializer;
+       }
+
+       @Override
+       public <N, S extends State, T> void applyToAllKeys(
+                       N namespace,
+                       TypeSerializer<N> namespaceSerializer,
+                       StateDescriptor<S, T> stateDescriptor,
+                       KeyedStateFunction<K, S> function) {
+               throw new UnsupportedOperationException("applyToAllKeys() is 
not supported in BATCH execution mode.");

Review comment:
       Personally I can't think of other scenarios...

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/sorted/state/SingleKeyKeyGroupedInternalPriorityQueue.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.sorted.state;
+
+import org.apache.flink.runtime.state.KeyGroupedInternalPriorityQueue;
+import org.apache.flink.runtime.state.PriorityComparator;
+import org.apache.flink.runtime.state.heap.HeapPriorityQueue;
+import org.apache.flink.runtime.state.heap.HeapPriorityQueueElement;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Very similar implementation to {@link 
org.apache.flink.runtime.state.heap.HeapPriorityQueueSet}. The only difference
+ * is it keeps track of elements for a single key at a time.
+ */
+class SingleKeyKeyGroupedInternalPriorityQueue<T extends 
HeapPriorityQueueElement>
+               extends HeapPriorityQueue<T>
+               implements KeyGroupedInternalPriorityQueue<T> {
+
+       private final Map<T, T> dedupMap = new HashMap<>();

Review comment:
       Good question.
   
   The reason is that we actually always use the `*PriorityQueue` as a `Set`. 
We use the `PriorityQueue` for storing timers and that's why we need Set 
semantics. We want to fire only a single timer for a timestamp. BTW, the logic 
is copied over from the `HeapPriorityQueueSet`. 
   
   However to better address it I will rename the class to 
`BatchExecutionInternalPriorityQueueSet`.

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/sorted/state/SingleKeyKeyedStateBackend.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.sorted.state;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.state.AggregatingStateDescriptor;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.MapStateDescriptor;
+import org.apache.flink.api.common.state.ReducingStateDescriptor;
+import org.apache.flink.api.common.state.State;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.state.CheckpointStreamFactory;
+import org.apache.flink.runtime.state.CheckpointableKeyedStateBackend;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.KeyGroupedInternalPriorityQueue;
+import org.apache.flink.runtime.state.Keyed;
+import org.apache.flink.runtime.state.KeyedStateFunction;
+import org.apache.flink.runtime.state.KeyedStateHandle;
+import org.apache.flink.runtime.state.PriorityComparable;
+import org.apache.flink.runtime.state.PriorityComparator;
+import org.apache.flink.runtime.state.SnapshotResult;
+import org.apache.flink.runtime.state.StateSnapshotTransformer;
+import org.apache.flink.runtime.state.heap.HeapPriorityQueueElement;
+import org.apache.flink.runtime.state.internal.InternalKvState;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import javax.annotation.Nonnull;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.RunnableFuture;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A simple {@link CheckpointableKeyedStateBackend} which keeps values for a 
single key at a time.
+ *
+ * <p><b>IMPORTANT:</b> Requires the incoming records to be sorted/grouped by 
the key. Used in a BATCH style execution.
+ */
+class SingleKeyKeyedStateBackend<K> implements 
CheckpointableKeyedStateBackend<K> {
+       @SuppressWarnings("rawtypes")
+       private static final Map<Class<? extends StateDescriptor>, 
StateFactory> STATE_FACTORIES =
+               Stream.of(
+                       Tuple2.of(ValueStateDescriptor.class, (StateFactory) 
SingleKeyValueState::create),
+                       Tuple2.of(ListStateDescriptor.class, (StateFactory) 
SingleKeyListState::create),
+                       Tuple2.of(MapStateDescriptor.class, (StateFactory) 
SingleKeyMapState::create),
+                       Tuple2.of(AggregatingStateDescriptor.class, 
(StateFactory) SingleKeyAggregatingState::create),
+                       Tuple2.of(ReducingStateDescriptor.class, (StateFactory) 
SingleKeyReducingState::create)
+               ).collect(Collectors.toMap(t -> t.f0, t -> t.f1));
+
+       private K currentKey = null;
+       private final TypeSerializer<K> keySerializer;
+       private final List<KeySelectionListener<K>> keySelectionListeners = new 
ArrayList<>();
+       private final Map<String, State> states = new HashMap<>();
+       private final Map<String, KeyGroupedInternalPriorityQueue<?>> 
priorityQueues = new HashMap<>();
+       private final KeyGroupRange keyGroupRange;
+
+       public SingleKeyKeyedStateBackend(
+                       TypeSerializer<K> keySerializer,
+                       KeyGroupRange keyGroupRange) {
+               this.keySerializer = keySerializer;
+               this.keyGroupRange = keyGroupRange;
+       }
+
+       @Override
+       public void setCurrentKey(K newKey) {
+               if (!Objects.equals(newKey, currentKey)) {
+                       notifyKeySelected(newKey);
+                       for (State value : states.values()) {
+                               ((AbstractSingleKeyState<?, ?, ?>) 
value).clearAllNamespaces();
+                       }
+                       for (KeyGroupedInternalPriorityQueue<?> value : 
priorityQueues.values()) {
+                               while (value.poll() != null) {
+                                       // remove everything for the key
+                               }
+                       }
+                       this.currentKey = newKey;
+               }
+       }
+
+       @Override
+       public K getCurrentKey() {
+               return currentKey;
+       }
+
+       @Override
+       public TypeSerializer<K> getKeySerializer() {
+               return keySerializer;
+       }
+
+       @Override
+       public <N, S extends State, T> void applyToAllKeys(
+                       N namespace,
+                       TypeSerializer<N> namespaceSerializer,
+                       StateDescriptor<S, T> stateDescriptor,
+                       KeyedStateFunction<K, S> function) {
+               throw new UnsupportedOperationException("applyToAllKeys() is 
not supported in BATCH execution mode.");

Review comment:
       Will rename the classes.




----------------------------------------------------------------
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:
[email protected]


Reply via email to