masteryhx commented on code in PR #24781:
URL: https://github.com/apache/flink/pull/24781#discussion_r1599461043


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/v2/MapStateDescriptor.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.v2;
+
+import org.apache.flink.api.common.serialization.SerializerConfig;
+import org.apache.flink.api.common.serialization.SerializerConfigImpl;
+import org.apache.flink.api.common.state.v2.MapState;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+
+import javax.annotation.Nonnull;
+
+/**
+ * {@link StateDescriptor} for {@link MapState}. This can be used to create 
partitioned map state
+ * internally.
+ *
+ * @param <UK> The type of the user key for this map state.
+ * @param <UV> The type of the values that the map state can hold.
+ */
+public class MapStateDescriptor<UK, UV> extends StateDescriptor<UV> {
+
+    /** The serializer for the user key. */
+    @Nonnull private final TypeSerializer<UK> userKeySerializer;
+
+    /**
+     * Creates a new {@code MapStateDescriptor} with the given stateId and 
type.
+     *
+     * @param stateId The (unique) stateId for the state.
+     * @param userKeyTypeInfo The type of the user keys in the state.
+     * @param userValueTypeInfo The type of the values in the state.
+     */
+    public MapStateDescriptor(
+            String stateId,
+            TypeInformation<UK> userKeyTypeInfo,
+            TypeInformation<UV> userValueTypeInfo) {
+        this(stateId, userKeyTypeInfo, userValueTypeInfo, new 
SerializerConfigImpl());
+    }
+
+    /**
+     * Creates a new {@code MapStateDescriptor} with the given stateId and 
type.
+     *
+     * @param stateId The (unique) stateId for the state.
+     * @param userKeyTypeInfo The type of the user keys in the state.
+     * @param userValueTypeInfo The type of the values in the state.
+     * @param serializerConfig The serializer related config used to generate 
{@code
+     *     TypeSerializer}.
+     */
+    public MapStateDescriptor(
+            String stateId,
+            TypeInformation<UK> userKeyTypeInfo,
+            TypeInformation<UV> userValueTypeInfo,
+            SerializerConfig serializerConfig) {
+        super(stateId, userValueTypeInfo, serializerConfig);
+        this.userKeySerializer = 
userKeyTypeInfo.createSerializer(serializerConfig);
+    }
+
+    @Nonnull
+    public TypeSerializer<UK> getUserKeySerializer() {
+        return userKeySerializer.duplicate();
+    }
+
+    @Override
+    public Type getType() {
+        return Type.MAP;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public final boolean equals(Object o) {

Review Comment:
   Is it necessary ?
   Could we just judge equals by state id similar to before Descriptor?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/v2/KeyedStateStoreV2.java:
##########
@@ -43,4 +45,30 @@ public interface KeyedStateStoreV2 {
      * @return The partitioned state object.
      */
     <T> ValueState<T> getValueState(@Nonnull ValueStateDescriptor<T> 
stateProperties);
+
+    /**
+     * Gets a handle to the system's key / value list state. This state is 
optimized for state that
+     * holds lists. One can adds elements to the list, or retrieve the list as 
a whole. This state
+     * is only accessible if the function is executed on a KeyedStream.
+     *
+     * @param stateProperties The descriptor defining the properties of the 
stats.

Review Comment:
   ```suggestion
        * @param stateProperties The descriptor defining the properties of the 
state.
   ```



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/v2/InternalListState.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.v2;
+
+import org.apache.flink.api.common.state.v2.ListState;
+import org.apache.flink.api.common.state.v2.StateFuture;
+import org.apache.flink.api.common.state.v2.StateIterator;
+import org.apache.flink.runtime.asyncprocessing.AsyncExecutionController;
+import org.apache.flink.runtime.asyncprocessing.StateRequestType;
+
+import java.util.List;
+
+/**
+ * A default implementation of {@link ListState} which delegates all async 
requests to {@link
+ * AsyncExecutionController}.
+ *
+ * @param <K> The type of key the state is associated to.
+ * @param <V> The type of values kept internally in state.
+ */
+public class InternalListState<K, V> extends InternalKeyedState<K, V> 
implements ListState<V> {
+
+    public InternalListState(
+            AsyncExecutionController<K> asyncExecutionController,
+            ListStateDescriptor<V> valueStateDescriptor) {

Review Comment:
   ```suggestion
               ListStateDescriptor<V> listStateDescriptor) {
   ```



##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/v2/InternalListState.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.v2;
+
+import org.apache.flink.api.common.state.v2.ListState;
+import org.apache.flink.api.common.state.v2.StateFuture;
+import org.apache.flink.api.common.state.v2.StateIterator;
+import org.apache.flink.runtime.asyncprocessing.AsyncExecutionController;
+import org.apache.flink.runtime.asyncprocessing.StateRequestType;
+
+import java.util.List;
+
+/**
+ * A default implementation of {@link ListState} which delegates all async 
requests to {@link
+ * AsyncExecutionController}.
+ *
+ * @param <K> The type of key the state is associated to.
+ * @param <V> The type of values kept internally in state.
+ */
+public class InternalListState<K, V> extends InternalKeyedState<K, V> 
implements ListState<V> {
+
+    public InternalListState(
+            AsyncExecutionController<K> asyncExecutionController,

Review Comment:
   ```suggestion
               StateRequestHandler stateRequestHandler,
   ```



-- 
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.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to