stevenzwu commented on code in PR #11144:
URL: https://github.com/apache/iceberg/pull/11144#discussion_r1777664609


##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.iceberg.flink.maintenance.operator;
+
+import java.util.Set;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.io.BulkDeletionFailureException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.SupportsBulkOperations;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Delete the files using the {@link FileIO} which implements {@link 
SupportsBulkOperations}. */
+@Internal
+public class DeleteFilesProcessor extends AbstractStreamOperator<Void>
+    implements OneInputStreamOperator<String, Void> {
+  private static final Logger LOG = 
LoggerFactory.getLogger(DeleteFilesProcessor.class);
+
+  private final String name;
+  private final SupportsBulkOperations io;
+  private final String tableName;
+  private final Set<String> filesToDelete = Sets.newHashSet();
+  private final int batchSize;
+
+  private transient Counter failedCounter;
+  private transient Counter succeededCounter;
+
+  public DeleteFilesProcessor(String name, TableLoader tableLoader, int 
batchSize) {
+    Preconditions.checkNotNull(name, "Name should no be null");
+    Preconditions.checkNotNull(tableLoader, "Table loader should no be null");
+
+    tableLoader.open();

Review Comment:
   I meant `MaintenanceTaskBuilder.append(...)` can pass in both `TableLoader` 
and `SerializableTable`.
   
   > Calling TableLoader.clone every time when we pass it as a parameter seems 
excessive to me.
   
   I meant clone when the lifecycle management needs to be different.
   
   here is an example from `ContinuousSplitPlannerImpl`
   ````
     public ContinuousSplitPlannerImpl(
         TableLoader tableLoader, ScanContext scanContext, String threadName) {
       this.tableLoader = tableLoader.clone();
       this.tableLoader.open();
       this.table = this.tableLoader.loadTable();
   ...
     }
   
     @Override
     public void close() throws IOException {
   ...
       tableLoader.close();
     }
   ```
   
   another example from `IcebergSource`
   ```
     private List<IcebergSourceSplit> planSplitsForBatch(String threadName) {
   ...
       try (TableLoader loader = tableLoader.clone()) {
         loader.open();
   ...
     }
   ```



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

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org

Reply via email to