Re: [PR] [FLINK-35153][State] Internal async list/map state and corresponding state descriptor [flink]
Zakelly closed pull request #24781: [FLINK-35153][State] Internal async list/map state and corresponding state descriptor URL: https://github.com/apache/flink/pull/24781 -- 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
Re: [PR] [FLINK-35153][State] Internal async list/map state and corresponding state descriptor [flink]
masteryhx commented on code in PR #24781: URL: https://github.com/apache/flink/pull/24781#discussion_r1599562251 ## flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java: ## @@ -256,6 +256,20 @@ public org.apache.flink.api.common.state.v2.ValueState getValueState( return keyedStateStoreV2.getValueState(stateProperties); } +public org.apache.flink.api.common.state.v2.ListState getListState( Review Comment: Since `RuntimeContext` is a public API, We should expose it carefully. Considering `StateManager` will be exposed in FLIP-410. I think we could consider it after that. (Maybe exposing to `StateManager` is enough) -- 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
Re: [PR] [FLINK-35153][State] Internal async list/map state and corresponding state descriptor [flink]
Zakelly commented on code in PR #24781: URL: https://github.com/apache/flink/pull/24781#discussion_r1599528939 ## flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java: ## @@ -256,6 +256,20 @@ public org.apache.flink.api.common.state.v2.ValueState getValueState( return keyedStateStoreV2.getValueState(stateProperties); } +public org.apache.flink.api.common.state.v2.ListState getListState( Review Comment: Ah... I almost forgot, currently we provide `StateDeclaration` for users to leverage state v2. The `StateDescriptor` need to be moved only if we provide this in DS V1. -- 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
Re: [PR] [FLINK-35153][State] Internal async list/map state and corresponding state descriptor [flink]
Zakelly commented on code in PR #24781: URL: https://github.com/apache/flink/pull/24781#discussion_r1599526237 ## flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java: ## @@ -256,6 +256,20 @@ public org.apache.flink.api.common.state.v2.ValueState getValueState( return keyedStateStoreV2.getValueState(stateProperties); } +public org.apache.flink.api.common.state.v2.ListState getListState( Review Comment: Whether to expose State v2 to `RuntimeContext` needs more discussion. It is a mistake that the `StateDescriptor` in runtime module. It should be under `flink-core-api`, I'll move this. -- 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
Re: [PR] [FLINK-35153][State] Internal async list/map state and corresponding state descriptor [flink]
Zakelly commented on code in PR #24781: URL: https://github.com/apache/flink/pull/24781#discussion_r1599523493 ## 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 The type of the user key for this map state. + * @param The type of the values that the map state can hold. + */ +public class MapStateDescriptor extends StateDescriptor { + +/** The serializer for the user key. */ +@Nonnull private final TypeSerializer 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 userKeyTypeInfo, +TypeInformation 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 userKeyTypeInfo, +TypeInformation userValueTypeInfo, +SerializerConfig serializerConfig) { +super(stateId, userValueTypeInfo, serializerConfig); +this.userKeySerializer = userKeyTypeInfo.createSerializer(serializerConfig); +} + +@Nonnull +public TypeSerializer getUserKeySerializer() { +return userKeySerializer.duplicate(); +} + +@Override +public Type getType() { +return Type.MAP; +} + +@Override +@SuppressWarnings("unchecked") +public final boolean equals(Object o) { Review Comment: It is necessary. The base descriptor for state v1 has an unified serializer, while in v2, we will add new serializer for map state. -- 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
Re: [PR] [FLINK-35153][State] Internal async list/map state and corresponding state descriptor [flink]
fredia commented on code in PR #24781: URL: https://github.com/apache/flink/pull/24781#discussion_r1599505408 ## flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java: ## @@ -256,6 +256,20 @@ public org.apache.flink.api.common.state.v2.ValueState getValueState( return keyedStateStoreV2.getValueState(stateProperties); } +public org.apache.flink.api.common.state.v2.ListState getListState( Review Comment: Should `getValueState/getListState/getMapState` be added to `org.apache.flink.api.common.functions.RuntimeContext`? BTW, I found that `org.apache.flink.runtime.state.v2.StateDescriptor` is in the `runtime` module, developers can only create async-states by `StreamingRuntimeContext` now. -- 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
Re: [PR] [FLINK-35153][State] Internal async list/map state and corresponding state descriptor [flink]
fredia commented on code in PR #24781: URL: https://github.com/apache/flink/pull/24781#discussion_r1599505408 ## flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java: ## @@ -256,6 +256,20 @@ public org.apache.flink.api.common.state.v2.ValueState getValueState( return keyedStateStoreV2.getValueState(stateProperties); } +public org.apache.flink.api.common.state.v2.ListState getListState( Review Comment: Should `getValueState/getListState/getMapState` be added to `org.apache.flink.api.common.functions.RuntimeContext`? -- 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
Re: [PR] [FLINK-35153][State] Internal async list/map state and corresponding state descriptor [flink]
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 The type of the user key for this map state. + * @param The type of the values that the map state can hold. + */ +public class MapStateDescriptor extends StateDescriptor { + +/** The serializer for the user key. */ +@Nonnull private final TypeSerializer 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 userKeyTypeInfo, +TypeInformation 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 userKeyTypeInfo, +TypeInformation userValueTypeInfo, +SerializerConfig serializerConfig) { +super(stateId, userValueTypeInfo, serializerConfig); +this.userKeySerializer = userKeyTypeInfo.createSerializer(serializerConfig); +} + +@Nonnull +public TypeSerializer 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. */ ValueState getValueState(@Nonnull ValueStateDescriptor 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
Re: [PR] [FLINK-35153][State] Internal async list/map state and corresponding state descriptor [flink]
flinkbot commented on PR #24781: URL: https://github.com/apache/flink/pull/24781#issuecomment-2109314883 ## CI report: * 2f5f8598627c98bfa1bf88e7a6e0de51319748f2 UNKNOWN Bot commands The @flinkbot bot supports the following commands: - `@flinkbot run azure` re-run the last Azure build -- 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