Zakelly commented on code in PR #25233: URL: https://github.com/apache/flink/pull/25233#discussion_r1732498188
########## flink-runtime/src/main/java/org/apache/flink/runtime/state/v2/StateDescriptorUtils.java: ########## @@ -0,0 +1,53 @@ +/* + * 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; + Review Comment: Please provide some javadoc here. Same for other new introduced classes. ########## flink-runtime/src/main/java/org/apache/flink/runtime/state/v2/StateDescriptorUtils.java: ########## @@ -0,0 +1,53 @@ +/* + * 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; + +public class StateDescriptorUtils { + private StateDescriptorUtils() {} + + public static org.apache.flink.api.common.state.StateDescriptor transformFromV2ToV1( + StateDescriptor stateDescriptorV2) { + org.apache.flink.api.common.state.StateDescriptor stateDescriptorV1 = + new org.apache.flink.api.common.state.StateDescriptor( + "", stateDescriptorV2.getSerializer(), null) { + @Override + public Type getType() { + StateDescriptor.Type typeV2 = stateDescriptorV2.getType(); + switch (typeV2) { + case VALUE: + return org.apache.flink.api.common.state.StateDescriptor.Type.VALUE; + case LIST: + return org.apache.flink.api.common.state.StateDescriptor.Type.LIST; + case REDUCING: + return org.apache.flink.api.common.state.StateDescriptor.Type + .REDUCING; + case AGGREGATING: + return org.apache.flink.api.common.state.StateDescriptor.Type + .AGGREGATING; + case MAP: + return org.apache.flink.api.common.state.StateDescriptor.Type.MAP; + default: + throw new IllegalArgumentException( + "Unsupported state type: " + typeV2); + } + } + }; + return stateDescriptorV1; Review Comment: We should not use anonymous class here. A preferred way: ``` @SuppressWarnings({"unchecked", "rawtypes"}) public static <S extends State, T> org.apache.flink.api.common.state.StateDescriptor<S, T> transformFromV2ToV1( StateDescriptor stateDescriptorV2) { switch (stateDescriptorV2.getType()) { case VALUE: return new org.apache.flink.api.common.state.ValueStateDescriptor(stateDescriptorV2.getStateId(), stateDescriptorV2.getSerializer()); case MAP: //xxx case LIST: //xxx case REDUCING: //xxx case AGGREGATING: //xxx default: throw new IllegalArgumentException( "Unsupported state type: " + stateDescriptorV2.getType()); } } ``` ########## flink-runtime/src/main/java/org/apache/flink/runtime/state/v2/adaptor/ValueStateV2Wrapper.java: ########## @@ -0,0 +1,77 @@ +/* + * 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.adaptor; + +import org.apache.flink.api.common.state.v2.StateFuture; +import org.apache.flink.api.common.state.v2.ValueState; +import org.apache.flink.core.state.StateFutureUtils; + +import java.io.IOException; + +public class ValueStateV2Wrapper<V> implements ValueState<V> { + private final org.apache.flink.api.common.state.ValueState<V> valueStateV1; Review Comment: Shall we just use `valueState` as name? `V1` is not intuitive. ########## flink-runtime/src/main/java/org/apache/flink/runtime/state/v2/adaptor/AsyncKeyedStateBackendAdaptor.java: ########## @@ -0,0 +1,74 @@ +/* + * 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.adaptor; + +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.v2.State; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.asyncprocessing.StateExecutor; +import org.apache.flink.runtime.asyncprocessing.StateRequestHandler; +import org.apache.flink.runtime.state.AsyncKeyedStateBackend; +import org.apache.flink.runtime.state.CheckpointableKeyedStateBackend; +import org.apache.flink.runtime.state.v2.StateDescriptor; +import org.apache.flink.runtime.state.v2.StateDescriptorUtils; + +import javax.annotation.Nonnull; + +import java.io.IOException; + +public class AsyncKeyedStateBackendAdaptor<K> implements AsyncKeyedStateBackend { + private final CheckpointableKeyedStateBackend<K> StateBackendV1; + + public AsyncKeyedStateBackendAdaptor(CheckpointableKeyedStateBackend<K> stateBackendV1) { + StateBackendV1 = stateBackendV1; + } + + @Override + public void setup(@Nonnull StateRequestHandler stateRequestHandler) {} + + @Nonnull + @Override + public <N, S extends State, SV> S createState( + @Nonnull N defaultNamespace, + @Nonnull TypeSerializer<N> namespaceSerializer, + @Nonnull StateDescriptor<SV> stateDesc) + throws Exception { + org.apache.flink.api.common.state.StateDescriptor stateDescV1 = + StateDescriptorUtils.transformFromV2ToV1(stateDesc); + org.apache.flink.api.common.state.State stateV1 = + StateBackendV1.getOrCreateKeyedState(namespaceSerializer, stateDescV1); + if (stateDescV1.getType() == org.apache.flink.api.common.state.StateDescriptor.Type.VALUE) { + return (S) new ValueStateV2Wrapper((ValueState) stateV1); + } + throw new UnsupportedOperationException( + String.format("Unsupported state type: %s", stateDesc.getType())); Review Comment: Shall we make this cleaner, such as: ``` org.apache.flink.api.common.state.State rawState = keyedStateBackend.getOrCreateKeyedState(namespaceSerializer, StateDescriptorUtils.transformFromV2ToV1(stateDesc)); switch (stateDesc.getType()) { case VALUE: return (S) new ValueStateV2Wrapper<>((org.apache.flink.api.common.state.ValueState<SV>) rawState); default: throw new UnsupportedOperationException( String.format("Unsupported state type: %s", stateDesc.getType())); } ``` ########## flink-runtime/src/main/java/org/apache/flink/runtime/state/v2/adaptor/AsyncKeyedStateBackendAdaptor.java: ########## @@ -0,0 +1,74 @@ +/* + * 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.adaptor; + +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.v2.State; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.asyncprocessing.StateExecutor; +import org.apache.flink.runtime.asyncprocessing.StateRequestHandler; +import org.apache.flink.runtime.state.AsyncKeyedStateBackend; +import org.apache.flink.runtime.state.CheckpointableKeyedStateBackend; +import org.apache.flink.runtime.state.v2.StateDescriptor; +import org.apache.flink.runtime.state.v2.StateDescriptorUtils; + +import javax.annotation.Nonnull; + +import java.io.IOException; + +public class AsyncKeyedStateBackendAdaptor<K> implements AsyncKeyedStateBackend { + private final CheckpointableKeyedStateBackend<K> StateBackendV1; Review Comment: Please use lower camel case here. And I suggest use `KeyedStateBackend<K>` here, naming it `keyedStateBackend` -- 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