Skip to content

[FLINK-37241][runtime] Remove Mockito usage from StateBackendTestBase. #26094

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 1 commit into from
Feb 13, 2025
Merged
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,7 @@
import org.apache.flink.runtime.checkpoint.StateAssignmentOperation;
import org.apache.flink.runtime.execution.Environment;
import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
import org.apache.flink.runtime.jobgraph.JobVertexID;
import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
import org.apache.flink.runtime.operators.testutils.MockEnvironment;
import org.apache.flink.runtime.query.KvStateRegistry;
Expand Down Expand Up @@ -125,11 +126,6 @@
import static org.assertj.core.api.Assertions.assertThatThrownBy;
import static org.assertj.core.api.Assertions.fail;
import static org.assertj.core.api.Assumptions.assumeThat;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;

/**
* Tests for the {@link KeyedStateBackend} and {@link OperatorStateBackend} as produced by various
Expand Down Expand Up @@ -4542,7 +4538,7 @@ void testQueryableStateRegistration() throws Exception {
try {
KeyGroupRange expectedKeyGroupRange = backend.getKeyGroupRange();

KvStateRegistryListener listener = mock(KvStateRegistryListener.class);
final TestingKvStateRegistryListener listener = new TestingKvStateRegistryListener();
registry.registerListener(HighAvailabilityServices.DEFAULT_JOB_ID, listener);

ValueStateDescriptor<Integer> desc =
Expand All @@ -4553,13 +4549,13 @@ void testQueryableStateRegistration() throws Exception {
VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, desc);

// Verify registered
verify(listener, times(1))
.notifyKvStateRegistered(
eq(env.getJobID()),
eq(env.getJobVertexId()),
eq(expectedKeyGroupRange),
eq("banana"),
any(KvStateID.class));
assertThat(
listener.isRegistered(
env.getJobID(),
env.getJobVertexId(),
expectedKeyGroupRange,
"banana"))
.isTrue();

KeyedStateHandle snapshot =
runSnapshot(
Expand All @@ -4572,13 +4568,15 @@ void testQueryableStateRegistration() throws Exception {

backend.dispose();

verify(listener, times(1))
.notifyKvStateUnregistered(
eq(env.getJobID()),
eq(env.getJobVertexId()),
eq(expectedKeyGroupRange),
eq("banana"));
assertThat(
listener.isRegistered(
env.getJobID(),
env.getJobVertexId(),
expectedKeyGroupRange,
"banana"))
.isFalse();
backend.dispose();

// Initialize again
backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot, env);
if (snapshot != null) {
Expand All @@ -4589,13 +4587,13 @@ void testQueryableStateRegistration() throws Exception {
VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, desc);

// Verify registered again
verify(listener, times(2))
.notifyKvStateRegistered(
eq(env.getJobID()),
eq(env.getJobVertexId()),
eq(expectedKeyGroupRange),
eq("banana"),
any(KvStateID.class));
assertThat(
listener.isRegistered(
env.getJobID(),
env.getJobVertexId(),
expectedKeyGroupRange,
"banana"))
.isTrue();
} finally {
IOUtils.closeQuietly(backend);
backend.dispose();
Expand Down Expand Up @@ -5696,6 +5694,50 @@ public static final class MutableLong {
long value;
}

private static class TestingKvStateRegistryListener implements KvStateRegistryListener {

private final Map<String, KvStateID> registeredStates = new HashMap<>();

private String createKey(
JobID jobId,
JobVertexID jobVertexId,
KeyGroupRange keyGroupRange,
String registrationName) {
return String.format(
"%s-%s-%s-%s",
jobId, jobVertexId, keyGroupRange.prettyPrintInterval(), registrationName);
}

private boolean isRegistered(
JobID jobId,
JobVertexID jobVertexId,
KeyGroupRange keyGroupRange,
String registrationName) {
return registeredStates.containsKey(
createKey(jobId, jobVertexId, keyGroupRange, registrationName));
}

@Override
public void notifyKvStateRegistered(
JobID jobId,
JobVertexID jobVertexId,
KeyGroupRange keyGroupRange,
String registrationName,
KvStateID kvStateId) {
registeredStates.put(
createKey(jobId, jobVertexId, keyGroupRange, registrationName), kvStateId);
}

@Override
public void notifyKvStateUnregistered(
JobID jobId,
JobVertexID jobVertexId,
KeyGroupRange keyGroupRange,
String registrationName) {
registeredStates.remove(createKey(jobId, jobVertexId, keyGroupRange, registrationName));
}
}

private MockEnvironment buildMockEnv() throws Exception {
MockEnvironment mockEnvironment =
MockEnvironment.builder().setTaskStateManager(getTestTaskStateManager()).build();
Expand Down