XComp commented on a change in pull request #18901: URL: https://github.com/apache/flink/pull/18901#discussion_r815933181
########## File path: flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStoreTest.java ########## @@ -272,6 +327,101 @@ public void testReplaceWithKeyNotExist() throws Exception { }; } + @Test + public void testReplaceWithDeletingKey() throws Exception { + new Context() { + { + runTest( + () -> { + leaderCallbackGrantLeadership(); + + final TestingLongStateHandleHelper.LongStateHandle oldState = + addDeletingEntry(getLeaderConfigMap(), key, 1337); + + final KubernetesStateHandleStore< + TestingLongStateHandleHelper.LongStateHandle> + store = + new KubernetesStateHandleStore<>( + flinkKubeClient, + LEADER_CONFIGMAP_NAME, + longStateStorage, + filter, + LOCK_IDENTITY); + final TestingLongStateHandleHelper.LongStateHandle newState = + new TestingLongStateHandleHelper.LongStateHandle(23456L); + + assertThat(store.exists(key), is(StringResourceVersion.notExisting())); + assertThrows( + StateHandleStore.NotExistException.class, + () -> + store.replace( + key, + StringResourceVersion.notExisting(), + newState)); + + // Both handles should have been stored and discarded. + assertThat(TestingLongStateHandleHelper.getGlobalStorageSize(), is(2)); + assertThat(oldState.isDiscarded(), is(true)); + assertThat(newState.isDiscarded(), is(true)); + }); + } + }; + } + + @Test + public void testReplaceWithDeletingKeyWithFailingDiscard() throws Exception { + new Context() { + { + runTest( + () -> { + leaderCallbackGrantLeadership(); + + final Exception discardException = new Exception("Unable to discard."); + final TestingLongStateHandleHelper.LongStateHandle oldState = + addDeletingEntry( + getLeaderConfigMap(), + key, + new TestingLongStateHandleHelper.LongStateHandle( + 1337, + discardIdx -> { + throw discardException; + })); + + final KubernetesStateHandleStore< + TestingLongStateHandleHelper.LongStateHandle> + store = + new KubernetesStateHandleStore<>( + flinkKubeClient, + LEADER_CONFIGMAP_NAME, + longStateStorage, + filter, + LOCK_IDENTITY); + final TestingLongStateHandleHelper.LongStateHandle newState = + new TestingLongStateHandleHelper.LongStateHandle(23456L); + + assertThat(store.exists(key), is(StringResourceVersion.notExisting())); + final StateHandleStore.NotExistException exception = + assertThrows( + StateHandleStore.NotExistException.class, + () -> + store.replace( + key, + StringResourceVersion.notExisting(), + newState)); + assertThat(exception.getSuppressed().length, is(1)); + assertThat( + exception.getSuppressed()[0], + FlinkMatchers.containsCause(discardException)); + + // Only the new handle should have been stored and discarded. Review comment: ```suggestion // Only the new handle should have been discarded. ``` The comment doesn't make sense because also the old state is stored, still ########## File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStore.java ########## @@ -469,7 +574,17 @@ public void releaseAndTryRemoveAll() throws Exception { "Could not properly remove all state handles.", exception)); } + // Commit the "removal transaction" by removing the entries from the + // ConfigMap. + return updateConfigMap( Review comment: Was this issue ignored on purposed or just missed? ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/persistence/TestingLongStateHandleHelper.java ########## @@ -106,17 +114,26 @@ public long getValue() { @Override public void discardState() { - preDiscardCallback.run(value); - - numberOfDiscardCalls++; + try { + preDiscardCallback.run(numberOfDiscardCalls); + numberOfSuccessfulDiscardCalls++; + } catch (Exception e) { Review comment: Out of curiosity: What do we gain out of it to make the callback throw a checked `Exception` and translating it here into a `RuntimeException` instead of just letting `PrecDiscardCallback` throw the `RuntimeException` rightaway? 🤔 ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStoreTest.java ########## @@ -271,7 +268,7 @@ public void testRepeatableCleanupWithLockOnNode() throws Exception { assertEquals( "discardState shouldn't have been called, yet.", 0, - stateHandle.getNumberOfDiscardCalls()); + stateHandle.getNumberOfSuccessfulDiscardCalls()); Review comment: I guess, here we can even check `stateHandle.getNumberOfDiscardCalls()`, because we're not expecting `discardState` to be called at all, yet -- 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