Skip to content
Open
Show file tree
Hide file tree
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 @@ -122,6 +122,21 @@ public SnapshotResultSupplier<OperatorStateHandle> asyncSnapshot(
Map<String, BackendWritableBroadcastState<?, ?>> registeredBroadcastStatesDeepCopies =
syncPartResource.getRegisteredBroadcastStatesDeepCopies();

if (registeredBroadcastStatesDeepCopies.isEmpty()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It is strange that this if does
registeredOperatorStatesDeepCopies.isEmpty()
and the first thing the method you have introduced does is:
registeredOperatorStatesDeepCopies.isEmpty()

I suggest doing everything in the method

&& hasOnlyEmptyOperatorListStates(registeredOperatorStatesDeepCopies)) {
if (streamFactory instanceof FsMergingCheckpointStorageLocation) {
FsMergingCheckpointStorageLocation location =
(FsMergingCheckpointStorageLocation) streamFactory;
return snapshotCloseableRegistry ->
SnapshotResult.of(
EmptyFileMergingOperatorStreamStateHandle.create(
location.getExclusiveStateHandle(),
location.getSharedStateHandle()));
} else {
return snapshotCloseableRegistry -> SnapshotResult.empty();
}
}

if (registeredBroadcastStatesDeepCopies.isEmpty()
&& registeredOperatorStatesDeepCopies.isEmpty()) {
if (streamFactory instanceof FsMergingCheckpointStorageLocation) {
Expand Down Expand Up @@ -236,6 +251,20 @@ public SnapshotResultSupplier<OperatorStateHandle> asyncSnapshot(
};
}

private boolean hasOnlyEmptyOperatorListStates(
Map<String, PartitionableListState<?>> registeredOperatorStatesDeepCopies) {
if (registeredOperatorStatesDeepCopies.isEmpty()) {
return false;
}

for (PartitionableListState<?> listState : registeredOperatorStatesDeepCopies.values()) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I see the method is called private boolean hasOnlyEmptyOperatorListStates(
I am not sure what the test for emptiness is. The only thing I see is listState == null,
but if listState == null then we return false.

What am I missing?

if (listState == null || listState.get().iterator().hasNext()) {
return false;
}
}
return true;
}

static class DefaultOperatorStateBackendSnapshotResources implements SnapshotResources {

private final Map<String, PartitionableListState<?>> registeredOperatorStatesDeepCopies;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@
import org.apache.flink.runtime.checkpoint.filemerging.FileMergingType;
import org.apache.flink.runtime.clusterframework.types.ResourceID;
import org.apache.flink.runtime.execution.Environment;
import org.apache.flink.runtime.state.filemerging.EmptyFileMergingOperatorStreamStateHandle;
import org.apache.flink.runtime.state.filemerging.FileMergingOperatorStreamStateHandle;
import org.apache.flink.runtime.state.filesystem.AbstractFsCheckpointStorageAccess;
import org.apache.flink.runtime.state.filesystem.FsMergingCheckpointStorageLocation;
Expand Down Expand Up @@ -489,6 +490,124 @@ void testFileMergingSnapshotEmpty(@TempDir File tmpFolder) throws Exception {
assertThat(stateHandle).isInstanceOf(FileMergingOperatorStreamStateHandle.class);
}

@Test
void testSnapshotWithEmptyRegisteredOperatorState() throws Exception {
final AbstractStateBackend abstractStateBackend = new HashMapStateBackend();
CloseableRegistry cancelStreamRegistry = new CloseableRegistry();

Environment env = createMockEnvironment();
final OperatorStateBackend operatorStateBackend =
abstractStateBackend.createOperatorStateBackend(
new OperatorStateBackendParametersImpl(
env, "testOperator", emptyStateHandles, cancelStreamRegistry));

ListStateDescriptor<Integer> stateDescriptor =
new ListStateDescriptor<>("test-empty-list-state", IntSerializer.INSTANCE);
try {
ListState<Integer> listState = operatorStateBackend.getListState(stateDescriptor);
assertThat(listState.get()).isEmpty();

CheckpointStreamFactory streamFactory = new MemCheckpointStreamFactory(4096);

RunnableFuture<SnapshotResult<OperatorStateHandle>> snapshot =
operatorStateBackend.snapshot(
0L,
0L,
streamFactory,
CheckpointOptions.forCheckpointWithDefaultLocation());

SnapshotResult<OperatorStateHandle> snapshotResult =
FutureUtils.runIfNotDoneAndGet(snapshot);
OperatorStateHandle stateHandle = snapshotResult.getJobManagerOwnedSnapshot();
assertThat(stateHandle).isNull();
} finally {
operatorStateBackend.close();
operatorStateBackend.dispose();
}
}

@Test
void testFileMergingSnapshotRestoreWithEmptyRegisteredUnionState(@TempDir File tmpFolder)
throws Exception {
final AbstractStateBackend abstractStateBackend = new HashMapStateBackend();
CloseableRegistry cancelStreamRegistry = new CloseableRegistry();

Environment env = createMockEnvironment();
OperatorStateBackend operatorStateBackend =
abstractStateBackend.createOperatorStateBackend(
new OperatorStateBackendParametersImpl(
env, "testOperator", emptyStateHandles, cancelStreamRegistry));

ListStateDescriptor<Integer> stateDescriptor =
new ListStateDescriptor<>("test-empty-union-state", IntSerializer.INSTANCE);
ListState<Integer> unionState = operatorStateBackend.getUnionListState(stateDescriptor);
assertThat(unionState.get()).isEmpty();

Path checkpointBaseDir = new Path(tmpFolder.toString());
Path sharedStateDir =
new Path(
checkpointBaseDir,
AbstractFsCheckpointStorageAccess.CHECKPOINT_SHARED_STATE_DIR);
Path taskOwnedStateDir =
new Path(
checkpointBaseDir,
AbstractFsCheckpointStorageAccess.CHECKPOINT_TASK_OWNED_STATE_DIR);

JobID jobId = JobID.generate();
final FileMergingSnapshotManager.SubtaskKey subtaskKey =
new FileMergingSnapshotManager.SubtaskKey(jobId.toHexString(), "opId", 1, 1);
LocalFileSystem fs = getSharedInstance();
CheckpointStorageLocationReference cslReference =
AbstractFsCheckpointStorageAccess.encodePathAsReference(
fromLocalFile(fs.pathToFile(checkpointBaseDir)));
FileMergingSnapshotManager snapshotManager =
createFileMergingSnapshotManager(
checkpointBaseDir, sharedStateDir, taskOwnedStateDir, subtaskKey);
CheckpointStreamFactory streamFactory =
new FsMergingCheckpointStorageLocation(
subtaskKey,
fs,
checkpointBaseDir,
sharedStateDir,
taskOwnedStateDir,
cslReference,
1024,
1024,
snapshotManager,
0);

OperatorStateHandle stateHandle = null;
try {
RunnableFuture<SnapshotResult<OperatorStateHandle>> snapshot =
operatorStateBackend.snapshot(
0L,
0L,
streamFactory,
CheckpointOptions.forCheckpointWithDefaultLocation());

SnapshotResult<OperatorStateHandle> snapshotResult =
FutureUtils.runIfNotDoneAndGet(snapshot);
stateHandle = snapshotResult.getJobManagerOwnedSnapshot();

assertThat(stateHandle).isInstanceOf(EmptyFileMergingOperatorStreamStateHandle.class);

operatorStateBackend =
recreateOperatorStateBackend(
operatorStateBackend,
abstractStateBackend,
StateObjectCollection.singleton(stateHandle));

unionState = operatorStateBackend.getUnionListState(stateDescriptor);
assertThat(unionState.get()).isEmpty();
} finally {
operatorStateBackend.close();
operatorStateBackend.dispose();
if (stateHandle != null) {
stateHandle.discardState();
}
}
}

@Test
void testSnapshotBroadcastStateWithEmptyOperatorState() throws Exception {
final AbstractStateBackend abstractStateBackend = new HashMapStateBackend();
Expand Down