diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapFunction.java b/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapFunction.java index 4bb703e10..d39ab7c39 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapFunction.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapFunction.java @@ -41,16 +41,9 @@ import org.apache.hudi.util.StreamerUtil; import org.apache.avro.Schema; import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.api.common.state.CheckpointListener; -import org.apache.flink.api.common.state.ListState; -import org.apache.flink.api.common.state.ListStateDescriptor; -import org.apache.flink.api.scala.typeutils.Types; import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.state.FunctionInitializationContext; -import org.apache.flink.runtime.state.FunctionSnapshotContext; import org.apache.flink.runtime.state.KeyGroupRangeAssignment; import org.apache.flink.runtime.taskexecutor.GlobalAggregateManager; -import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; import org.apache.flink.streaming.api.functions.ProcessFunction; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.util.Collector; @@ -73,8 +66,7 @@ import static java.util.stream.Collectors.toList; *
The output records should then shuffle by the recordKey and thus do scalable write.
*/
public class BootstrapFunction
- extends ProcessFunction
- implements CheckpointedFunction, CheckpointListener {
+ extends ProcessFunction {
private static final Logger LOG = LoggerFactory.getLogger(BootstrapFunction.class);
@@ -86,7 +78,6 @@ public class BootstrapFunction
private transient HoodieWriteConfig writeConfig;
private GlobalAggregateManager aggregateManager;
- private ListState