rkhachatryan commented on code in PR #20217: URL: https://github.com/apache/flink/pull/20217#discussion_r937925172
########## flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/ChangelogStateBackendLocalHandle.java: ########## @@ -0,0 +1,116 @@ +/* + * 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.changelog; + +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.KeyedStateHandle; +import org.apache.flink.runtime.state.SharedStateRegistry; +import org.apache.flink.runtime.state.StateHandleID; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.util.List; + +public class ChangelogStateBackendLocalHandle implements ChangelogStateBackendHandle { + private static final long serialVersionUID = 1L; + private static final Logger LOG = + LoggerFactory.getLogger(ChangelogStateBackendLocalHandle.class); + private final List<KeyedStateHandle> localMaterialized; + private final List<ChangelogStateHandle> localNonMaterialized; + private final ChangelogStateBackendHandleImpl remoteHandle; + + public ChangelogStateBackendLocalHandle( + List<KeyedStateHandle> localMaterialized, + List<ChangelogStateHandle> localNonMaterialized, + ChangelogStateBackendHandleImpl remoteHandle) { + this.localMaterialized = localMaterialized; + this.localNonMaterialized = localNonMaterialized; + this.remoteHandle = remoteHandle; + } + + @Override + public List<KeyedStateHandle> getMaterializedStateHandles() { + return localMaterialized; + } + + @Override + public List<ChangelogStateHandle> getNonMaterializedStateHandles() { + return localNonMaterialized; + } + + @Override + public long getMaterializationID() { + return remoteHandle.getMaterializationID(); + } + + @Override + public ChangelogStateBackendHandle rebound(long checkpointId) { + return remoteHandle.rebound(checkpointId); + } + + public List<KeyedStateHandle> getRemoteMaterializedStateHandles() { + return remoteHandle.getMaterializedStateHandles(); + } + + public List<ChangelogStateHandle> getRemoteNonMaterializedStateHandles() { + return remoteHandle.getNonMaterializedStateHandles(); + } + + @Override + public long getCheckpointId() { + return remoteHandle.getCheckpointId(); + } + + @Override + public void registerSharedStates(SharedStateRegistry stateRegistry, long checkpointID) { + remoteHandle.registerSharedStates(stateRegistry, checkpointID); + } + + @Override + public long getCheckpointedSize() { + return remoteHandle.getCheckpointedSize(); + } + + @Override + public KeyGroupRange getKeyGroupRange() { + return remoteHandle.getKeyGroupRange(); + } + + @Nullable + @Override + public KeyedStateHandle getIntersection(KeyGroupRange keyGroupRange) { + throw new UnsupportedOperationException( + "This is a local state handle for the TM side only."); + } + + @Override + public StateHandleID getStateHandleId() { + return remoteHandle.getStateHandleId(); + } + + @Override + public void discardState() throws Exception {} Review Comment: Thanks! I've commented on the updated code. > Regardless of the deletion of local dstl, do you know how remote dstl handles this case? if one StateChangeSet is uploaded to /dstl, will it be removed on abort? Good question. For the remote state, TM can not make any assumptions whether it's in use or not, so this is managed by JM. The class responsible for that is checkpoint store, e.g. [`DefaultCompletedCheckpointStore.shutdown()`](https://github.com/apache/flink/blob/5405239dec0884dff746129c73955c90f455c465/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/DefaultCompletedCheckpointStore.java#L208). -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
