JingsongLi commented on a change in pull request #13744:
URL: https://github.com/apache/flink/pull/13744#discussion_r512378183



##########
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:
       No, then will wait for next checkpoint notify.




----------------------------------------------------------------
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:
[email protected]


Reply via email to