rkhachatryan commented on code in PR #21362:
URL: https://github.com/apache/flink/pull/21362#discussion_r1034045588


##########
flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/InternalKeyContextImpl.java:
##########
@@ -72,6 +73,10 @@ public void setCurrentKey(@Nonnull K currentKey) {
 
     @Override
     public void setCurrentKeyGroupIndex(int currentKeyGroupIndex) {
+        if (!keyGroupRange.contains(currentKeyGroupIndex)) {
+            throw KeyGroupRangeOffsets.newIllegalKeyGroupException(
+                    currentKeyGroupIndex, keyGroupRange);
+        }

Review Comment:
   > BTW, apart from changing the base class, we can also only change the 
RocksDBKeyedStateBackend#setCurrentKey to avoid the impact on 
HashMapStateBackend, as the HashMapStateBackend will always check the key group 
when accessing state tables.
   
   How about the opposite: removing the check from HashMapStateBackend 
(StateTable) and relying on `InternalKeyContext`? So that it works for any 
backend.
   As a side benefit, that would be a bit faster because the check would be 
done once per input record, not per state access.



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

Reply via email to