Github user tillrohrmann commented on a diff in the pull request: https://github.com/apache/flink/pull/5239#discussion_r169726050 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/state/DirectoryKeyedStateHandle.java --- @@ -0,0 +1,105 @@ +/* + * 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; + +import javax.annotation.Nonnull; + +/** + * This class is a keyed state handle based on a directory. It combines a {@link DirectoryStateHandle} and a + * {@link KeyGroupRange}. + */ +public class DirectoryKeyedStateHandle implements KeyedStateHandle { + + @Nonnull + private final DirectoryStateHandle directoryStateHandle; + + @Nonnull + private final KeyGroupRange keyGroupRange; + + public DirectoryKeyedStateHandle( + @Nonnull DirectoryStateHandle directoryStateHandle, + @Nonnull KeyGroupRange keyGroupRange) { + + this.directoryStateHandle = directoryStateHandle; + this.keyGroupRange = keyGroupRange; + } + + @Nonnull + public DirectoryStateHandle getDirectoryStateHandle() { + return directoryStateHandle; + } + + @Nonnull + @Override + public KeyGroupRange getKeyGroupRange() { + return keyGroupRange; + } + + @Override + public void discardState() throws Exception { + directoryStateHandle.discardState(); + } + + @Override + public long getStateSize() { + return directoryStateHandle.getStateSize(); + } + + @Override + public KeyedStateHandle getIntersection(KeyGroupRange keyGroupRange) { + return keyGroupRange.getIntersection(keyGroupRange).getNumberOfKeyGroups() > 0 ? this : null; --- End diff -- This looks like a bug. `keyGroupRange.getIntersection(keyGroupRange)` should be `keyGroupRange`. I think in general it is a good idea to not shadow local fields by function parameters.
---