Skip to content

Commit

Permalink
[FLINK-37301][state/forst] Fix wrong flag returned by `ForStStateBack…
Browse files Browse the repository at this point in the history
…end#supportsSavepointFormat` (#26143)
  • Loading branch information
Zakelly authored Feb 12, 2025
1 parent e299645 commit 62c0899
Show file tree
Hide file tree
Showing 2 changed files with 17 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import org.apache.flink.configuration.ReadableConfig;
import org.apache.flink.configuration.description.InlineElement;
import org.apache.flink.core.execution.RecoveryClaimMode;
import org.apache.flink.core.execution.SavepointFormatType;
import org.apache.flink.core.fs.Path;
import org.apache.flink.metrics.MetricGroup;
import org.apache.flink.runtime.execution.Environment;
Expand Down Expand Up @@ -636,11 +637,18 @@ private ForStOptionsFactory configureOptionsFactory(
return optionsFactory;
}

/** Both ForStSyncKeyedStateBackend and ForStKeyedStateBackend support no claim mode. */
@Override
public boolean supportsNoClaimRestoreMode() {
// Both ForStSyncKeyedStateBackend and ForStKeyedStateBackend support no claim mode.
return true;
}

@Override
public boolean supportsSavepointFormat(SavepointFormatType formatType) {
// We only support native format for now.
return formatType == SavepointFormatType.NATIVE;
}

// ------------------------------------------------------------------------
// Parameters
// ------------------------------------------------------------------------
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import org.apache.flink.configuration.CoreOptions;
import org.apache.flink.configuration.MemorySize;
import org.apache.flink.configuration.ReadableConfig;
import org.apache.flink.core.execution.SavepointFormatType;
import org.apache.flink.core.fs.CloseableRegistry;
import org.apache.flink.core.fs.FileSystem;
import org.apache.flink.core.fs.Path;
Expand Down Expand Up @@ -771,6 +772,13 @@ public void testPrimaryDirectory() throws Exception {
}
}

@Test
public void testSupportSavepoint() {
ForStStateBackend forStStateBackend = new ForStStateBackend();
assertFalse(forStStateBackend.supportsSavepointFormat(SavepointFormatType.CANONICAL));
assertTrue(forStStateBackend.supportsSavepointFormat(SavepointFormatType.NATIVE));
}

private void verifySetParameter(Runnable setter) {
try {
setter.run();
Expand Down

0 comments on commit 62c0899

Please sign in to comment.