You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/10/26 12:58:11 UTC

[GitHub] [flink] lirui-apache commented on a change in pull request #13744: [FLINK-19766][table-runtime] Introduce File streaming compaction operators

lirui-apache commented on a change in pull request #13744:
URL: https://github.com/apache/flink/pull/13744#discussion_r511773377



##########
File path: flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/stream/compact/CompactMessages.java
##########
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.filesystem.stream.compact;
+
+import org.apache.flink.core.fs.Path;
+
+import java.io.Serializable;
+import java.net.URI;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Util class for all compaction messages.
+ *
+ * <p>The compaction operator graph is:
+ * TempFileWriter|parallel ---(InputFile&EndInputFile)---> CompactCoordinator|non-parallel
+ * ---(CompactionUnit&EndCompaction)--->CompactOperator|parallel---(PartitionCommitInfo)--->
+ * PartitionCommitter|non-parallel
+ *
+ * <p>Because the end message is a kind of barrier of record messages, they can only be transmitted
+ * in the way of full broadcast in the link from coordinator to compact operator.
+ */
+public class CompactMessages {
+	private CompactMessages() {}
+
+	/**
+	 * The input of compact coordinator.
+	 */
+	public interface CoordinatorInput extends Serializable {}
+
+	/**
+	 * A partitioned input file.
+	 */
+	public static class InputFile implements CoordinatorInput {

Review comment:
       Missing serialVersionUID

##########
File path: flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/stream/compact/CompactMessages.java
##########
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.filesystem.stream.compact;
+
+import org.apache.flink.core.fs.Path;
+
+import java.io.Serializable;
+import java.net.URI;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Util class for all compaction messages.
+ *
+ * <p>The compaction operator graph is:
+ * TempFileWriter|parallel ---(InputFile&EndInputFile)---> CompactCoordinator|non-parallel
+ * ---(CompactionUnit&EndCompaction)--->CompactOperator|parallel---(PartitionCommitInfo)--->
+ * PartitionCommitter|non-parallel
+ *
+ * <p>Because the end message is a kind of barrier of record messages, they can only be transmitted
+ * in the way of full broadcast in the link from coordinator to compact operator.
+ */
+public class CompactMessages {
+	private CompactMessages() {}
+
+	/**
+	 * The input of compact coordinator.
+	 */
+	public interface CoordinatorInput extends Serializable {}
+
+	/**
+	 * A partitioned input file.
+	 */
+	public static class InputFile implements CoordinatorInput {
+		private final String partition;
+		private final Path file;
+
+		public InputFile(String partition, Path file) {
+			this.partition = partition;
+			this.file = file;
+		}
+
+		public String getPartition() {
+			return partition;
+		}
+
+		public Path getFile() {
+			return file;
+		}
+	}
+
+	/**
+	 * A flag to end file input.
+	 */
+	public static class EndInputFile implements CoordinatorInput {

Review comment:
       Missing serialVersionUID

##########
File path: flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/filesystem/stream/compact/CompactCoordinatorTest.java
##########
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.filesystem.stream.compact;
+
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
+import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
+import org.apache.flink.table.filesystem.stream.compact.CompactMessages.CompactionUnit;
+import org.apache.flink.table.filesystem.stream.compact.CompactMessages.CoordinatorInput;
+import org.apache.flink.table.filesystem.stream.compact.CompactMessages.CoordinatorOutput;
+import org.apache.flink.table.filesystem.stream.compact.CompactMessages.EndCompaction;
+import org.apache.flink.table.filesystem.stream.compact.CompactMessages.EndInputFile;
+import org.apache.flink.table.filesystem.stream.compact.CompactMessages.InputFile;
+import org.apache.flink.util.function.ThrowingConsumer;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+/**
+ * Test for {@link CompactCoordinator}.
+ */
+public class CompactCoordinatorTest extends AbstractCompactTestBase {
+
+	@Test
+	public void testCoordinatorCrossCheckpoints() throws Exception {
+		AtomicReference<OperatorSubtaskState> state = new AtomicReference<>();
+		runCoordinator(harness -> {
+			harness.setup();
+			harness.open();
+
+			harness.processElement(new InputFile("p0", newFile("f0", 3)), 0);
+			harness.processElement(new InputFile("p0", newFile("f1", 2)), 0);
+
+			harness.processElement(new InputFile("p1", newFile("f2", 2)), 0);
+
+			harness.processElement(new InputFile("p0", newFile("f3", 5)), 0);
+			harness.processElement(new InputFile("p0", newFile("f4", 1)), 0);
+
+			harness.processElement(new InputFile("p1", newFile("f5", 5)), 0);
+			harness.processElement(new InputFile("p1", newFile("f6", 4)), 0);
+
+			state.set(harness.snapshot(1, 0));
+		});
+
+		runCoordinator(harness -> {
+			harness.setup();
+			harness.initializeState(state.get());
+			harness.open();
+
+			harness.processElement(new InputFile("p0", newFile("f7", 3)), 0);
+			harness.processElement(new InputFile("p0", newFile("f8", 2)), 0);
+
+			state.set(harness.snapshot(2, 0));
+		});
+
+		runCoordinator(harness -> {
+			harness.setup();
+			harness.initializeState(state.get());
+			harness.open();
+
+			harness.processElement(new EndInputFile(2, 0, 1), 0);
+
+			List<CoordinatorOutput> outputs = harness.extractOutputValues();
+
+			Assert.assertEquals(7, outputs.size());
+
+			assertUnit(outputs.get(0), 0, "p0", Arrays.asList("f0", "f1", "f4"));

Review comment:
       I think `CompactCoordinator` doesn't guarantee to generate `CompactionUnit` in any specific order of partitions, right?

##########
File path: flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/stream/compact/CompactCoordinator.java
##########
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.filesystem.stream.compact;
+
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeutils.base.ListSerializer;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.api.common.typeutils.base.MapSerializer;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.filesystem.stream.TaskTracker;
+import org.apache.flink.table.filesystem.stream.compact.CompactMessages.CompactionUnit;
+import org.apache.flink.table.filesystem.stream.compact.CompactMessages.CoordinatorInput;
+import org.apache.flink.table.filesystem.stream.compact.CompactMessages.CoordinatorOutput;
+import org.apache.flink.table.filesystem.stream.compact.CompactMessages.EndCompaction;
+import org.apache.flink.table.filesystem.stream.compact.CompactMessages.EndInputFile;
+import org.apache.flink.table.filesystem.stream.compact.CompactMessages.InputFile;
+import org.apache.flink.table.runtime.util.BinPacking;
+import org.apache.flink.util.function.SupplierWithException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.function.Function;
+
+/**
+ * This is the single (non-parallel) monitoring task which coordinate input files to compaction units.
+ * - Receives in-flight input files inside checkpoint.
+ * - Receives all upstream end input messages after the checkpoint completes successfully,
+ *   starts coordination.
+ *
+ * <p>NOTE: The coordination is a stable algorithm, which can ensure that the downstream can
+ *          perform compaction at any time without worrying about fail over.
+ *
+ * <p>STATE: This operator stores input files in state, after the checkpoint completes successfully,
+ *           input files are taken out from the state for coordination.
+ */
+public class CompactCoordinator extends AbstractStreamOperator<CoordinatorOutput> implements
+		OneInputStreamOperator<CoordinatorInput, CoordinatorOutput> {
+
+	private static final long serialVersionUID = 1L;
+
+	private static final Logger LOG = LoggerFactory.getLogger(CompactCoordinator.class);
+
+	private final SupplierWithException<FileSystem, IOException> fsFactory;
+	private final long targetFileSize;
+
+	private transient FileSystem fileSystem;
+
+	private transient ListState<Map<Long, Map<String, List<Path>>>> inputFilesState;
+	private transient TreeMap<Long, Map<String, List<Path>>> inputFiles;
+	private transient Map<String, List<Path>> currentInputFiles;
+
+	private transient TaskTracker inputTaskTracker;
+
+	public CompactCoordinator(
+			SupplierWithException<FileSystem, IOException> fsFactory,
+			long targetFileSize) {
+		this.fsFactory = fsFactory;
+		this.targetFileSize = targetFileSize;
+	}
+
+	@Override
+	public void initializeState(StateInitializationContext context) throws Exception {
+		super.initializeState(context);
+
+		fileSystem = fsFactory.get();
+
+		ListStateDescriptor<Map<Long, Map<String, List<Path>>>> filesDescriptor =
+				new ListStateDescriptor<>("files-state", new MapSerializer<>(
+						LongSerializer.INSTANCE,
+						new MapSerializer<>(
+								StringSerializer.INSTANCE,
+								new ListSerializer<>(
+										new KryoSerializer<>(Path.class, getExecutionConfig())))));
+		inputFilesState = context.getOperatorStateStore().getListState(filesDescriptor);
+		inputFiles = new TreeMap<>();
+		currentInputFiles = new HashMap<>();
+		if (context.isRestored()) {
+			inputFiles.putAll(inputFilesState.get().iterator().next());
+		}
+	}
+
+	@Override
+	public void processElement(StreamRecord<CoordinatorInput> element) throws Exception {
+		CoordinatorInput value = element.getValue();
+		if (value instanceof InputFile) {
+			InputFile file = (InputFile) value;
+			currentInputFiles.computeIfAbsent(file.getPartition(), k -> new ArrayList<>()).add(file.getFile());
+		} else if (value instanceof EndInputFile) {
+			EndInputFile endInputFile = (EndInputFile) value;
+			if (inputTaskTracker == null) {
+				inputTaskTracker = new TaskTracker(endInputFile.getNumberOfTasks());
+			}
+
+			// ensure all files are ready to be compacted.
+			boolean triggerCommit = inputTaskTracker.add(
+					endInputFile.getCheckpointId(), endInputFile.getTaskId());
+			if (triggerCommit) {
+				commitUpToCheckpoint(endInputFile.getCheckpointId());
+			}
+		}

Review comment:
       Throw an exception for unknown elements?

##########
File path: flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/stream/compact/CompactMessages.java
##########
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.filesystem.stream.compact;
+
+import org.apache.flink.core.fs.Path;
+
+import java.io.Serializable;
+import java.net.URI;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Util class for all compaction messages.
+ *
+ * <p>The compaction operator graph is:
+ * TempFileWriter|parallel ---(InputFile&EndInputFile)---> CompactCoordinator|non-parallel
+ * ---(CompactionUnit&EndCompaction)--->CompactOperator|parallel---(PartitionCommitInfo)--->
+ * PartitionCommitter|non-parallel
+ *
+ * <p>Because the end message is a kind of barrier of record messages, they can only be transmitted
+ * in the way of full broadcast in the link from coordinator to compact operator.
+ */
+public class CompactMessages {
+	private CompactMessages() {}
+
+	/**
+	 * The input of compact coordinator.
+	 */
+	public interface CoordinatorInput extends Serializable {}
+
+	/**
+	 * A partitioned input file.
+	 */
+	public static class InputFile implements CoordinatorInput {
+		private final String partition;
+		private final Path file;
+
+		public InputFile(String partition, Path file) {
+			this.partition = partition;
+			this.file = file;
+		}
+
+		public String getPartition() {
+			return partition;
+		}
+
+		public Path getFile() {
+			return file;
+		}
+	}
+
+	/**
+	 * A flag to end file input.
+	 */
+	public static class EndInputFile implements CoordinatorInput {
+		private final long checkpointId;
+		private final int taskId;
+		private final int numberOfTasks;
+
+		public EndInputFile(long checkpointId, int taskId, int numberOfTasks) {
+			this.checkpointId = checkpointId;
+			this.taskId = taskId;
+			this.numberOfTasks = numberOfTasks;
+		}
+
+		public long getCheckpointId() {
+			return checkpointId;
+		}
+
+		public int getTaskId() {
+			return taskId;
+		}
+
+		public int getNumberOfTasks() {
+			return numberOfTasks;
+		}
+	}
+
+	/**
+	 * The output of compact coordinator.
+	 */
+	public interface CoordinatorOutput extends Serializable {}
+
+	/**
+	 * The unit of a single compaction.
+	 */
+	public static class CompactionUnit implements CoordinatorOutput {
+
+		private final int unitId;
+		private final String partition;
+
+		// Store strings to improve serialization performance.
+		private final String[] pathStrings;
+
+		public CompactionUnit(int unitId, String partition, List<Path> unit) {
+			this.unitId = unitId;
+			this.partition = partition;
+			this.pathStrings = unit.stream()
+					.map(Path::toUri)
+					.map(URI::toString)
+					.toArray(String[]::new);
+		}
+
+		public boolean isTaskMessage(int taskId) {

Review comment:
       What's the purpose of this method?

##########
File path: flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/stream/compact/CompactMessages.java
##########
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.filesystem.stream.compact;
+
+import org.apache.flink.core.fs.Path;
+
+import java.io.Serializable;
+import java.net.URI;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Util class for all compaction messages.
+ *
+ * <p>The compaction operator graph is:
+ * TempFileWriter|parallel ---(InputFile&EndInputFile)---> CompactCoordinator|non-parallel
+ * ---(CompactionUnit&EndCompaction)--->CompactOperator|parallel---(PartitionCommitInfo)--->
+ * PartitionCommitter|non-parallel
+ *
+ * <p>Because the end message is a kind of barrier of record messages, they can only be transmitted
+ * in the way of full broadcast in the link from coordinator to compact operator.
+ */
+public class CompactMessages {
+	private CompactMessages() {}
+
+	/**
+	 * The input of compact coordinator.
+	 */
+	public interface CoordinatorInput extends Serializable {}
+
+	/**
+	 * A partitioned input file.
+	 */
+	public static class InputFile implements CoordinatorInput {
+		private final String partition;
+		private final Path file;
+
+		public InputFile(String partition, Path file) {
+			this.partition = partition;
+			this.file = file;
+		}
+
+		public String getPartition() {
+			return partition;
+		}
+
+		public Path getFile() {
+			return file;
+		}
+	}
+
+	/**
+	 * A flag to end file input.

Review comment:
       It seems more of a flag to end checkpoint rather than file input?

##########
File path: flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/stream/compact/CompactOperator.java
##########
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.flink.table.filesystem.stream.compact;
+
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeutils.base.ListSerializer;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.api.common.typeutils.base.MapSerializer;
+import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.FileSystemKind;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.fs.RecoverableFsDataOutputStream;
+import org.apache.flink.core.fs.RecoverableWriter;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.functions.sink.filesystem.BucketWriter;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.filesystem.stream.PartitionCommitInfo;
+import org.apache.flink.table.filesystem.stream.compact.CompactMessages.CompactionUnit;
+import org.apache.flink.table.filesystem.stream.compact.CompactMessages.CoordinatorOutput;
+import org.apache.flink.table.filesystem.stream.compact.CompactMessages.EndCompaction;
+import org.apache.flink.util.IOUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.SupplierWithException;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Set;
+import java.util.TreeMap;
+
+/**
+ * Receives compaction units to do compaction. Send partition commit information after
+ * compaction finished.
+ *
+ * <p>Use {@link BulkFormat} to read and use {@link BucketWriter} to write.
+ *
+ * <p>STATE: This operator stores expired files in state, after the checkpoint completes successfully,
+ *           We can ensure that these files will not be used again and they can be deleted from the
+ *           file system.
+ */
+public class CompactOperator<T> extends AbstractStreamOperator<PartitionCommitInfo>
+		implements OneInputStreamOperator<CoordinatorOutput, PartitionCommitInfo>, BoundedOneInput {
+
+	private static final long serialVersionUID = 1L;
+
+	private static final String UNCOMPACTED_PREFIX = ".uncompacted-";
+
+	private static final String COMPACTED_PREFIX = "compacted-";
+
+	private final SupplierWithException<FileSystem, IOException> fsFactory;
+	private final CompactReader.Factory<T> readerFactory;
+	private final CompactWriter.Factory<T> writerFactory;
+
+	private transient FileSystem fileSystem;
+
+	private transient ListState<Map<Long, List<Path>>> expiredFilesState;
+	private transient TreeMap<Long, List<Path>> expiredFiles;
+	private transient List<Path> currentExpiredFiles;
+
+	private transient Set<String> partitions;
+
+	public CompactOperator(
+			SupplierWithException<FileSystem, IOException> fsFactory,
+			CompactReader.Factory<T> readerFactory,
+			CompactWriter.Factory<T> writerFactory) {
+		this.fsFactory = fsFactory;
+		this.readerFactory = readerFactory;
+		this.writerFactory = writerFactory;
+	}
+
+	@Override
+	public void initializeState(StateInitializationContext context) throws Exception {
+		super.initializeState(context);
+		this.partitions = new HashSet<>();
+		this.fileSystem = fsFactory.get();
+
+		ListStateDescriptor<Map<Long, List<Path>>> metaDescriptor =
+				new ListStateDescriptor<>("expired-files", new MapSerializer<>(
+						LongSerializer.INSTANCE,
+						new ListSerializer<>(new KryoSerializer<>(Path.class, getExecutionConfig()))
+				));
+		this.expiredFilesState = context.getOperatorStateStore().getListState(metaDescriptor);
+		this.expiredFiles = new TreeMap<>();
+		this.currentExpiredFiles = new ArrayList<>();
+
+		if (context.isRestored()) {
+			this.expiredFiles.putAll(this.expiredFilesState.get().iterator().next());
+		}
+	}
+
+	@Override
+	public void processElement(StreamRecord<CoordinatorOutput> element) throws Exception {
+		CoordinatorOutput value = element.getValue();
+		if (value instanceof CompactionUnit) {
+			CompactionUnit unit = (CompactionUnit) value;
+			if (unit.isTaskMessage(getRuntimeContext().getNumberOfParallelSubtasks())) {
+				String partition = unit.getPartition();
+				List<Path> paths = unit.getPaths();
+
+				doCompact(paths);
+				this.partitions.add(partition);
+
+				// Only after the current checkpoint is successfully executed can delete
+				// the expired files, so as to ensure the existence of the files.
+				this.currentExpiredFiles.addAll(paths);
+			}
+		} else if (value instanceof EndCompaction) {
+			endCompaction(((EndCompaction) value).getCheckpointId());
+		}
+	}
+
+	private void endCompaction(long checkpoint) {
+		this.output.collect(new StreamRecord<>(new PartitionCommitInfo(
+				checkpoint,
+				getRuntimeContext().getIndexOfThisSubtask(),
+				getRuntimeContext().getNumberOfParallelSubtasks(),
+				new ArrayList<>(this.partitions))));
+		this.partitions.clear();
+	}
+
+	@Override
+	public void snapshotState(StateSnapshotContext context) throws Exception {
+		super.snapshotState(context);
+
+		expiredFilesState.clear();
+		expiredFiles.put(context.getCheckpointId(), new ArrayList<>(currentExpiredFiles));
+		expiredFilesState.add(expiredFiles);
+		currentExpiredFiles.clear();
+	}
+
+	@Override
+	public void notifyCheckpointComplete(long checkpointId) throws Exception {

Review comment:
       If something goes wrong in this method and the job fails over, will this method be called again for the same checkpointId?




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org