stevenzwu commented on code in PR #11144: URL: https://github.com/apache/iceberg/pull/11144#discussion_r1775665663
########## flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/stream/ExpireSnapshots.java: ########## @@ -0,0 +1,161 @@ +/* + * 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.stream; + +import java.time.Duration; +import java.util.concurrent.TimeUnit; +import org.apache.flink.streaming.api.datastream.AsyncDataStream; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.functions.async.AsyncRetryStrategy; +import org.apache.flink.streaming.util.retryable.AsyncRetryStrategies; +import org.apache.iceberg.flink.maintenance.operator.AsyncDeleteFiles; +import org.apache.iceberg.flink.maintenance.operator.ExpireSnapshotsProcessor; +import org.apache.iceberg.flink.maintenance.operator.TaskResult; +import org.apache.iceberg.flink.maintenance.operator.Trigger; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +public class ExpireSnapshots { + private static final long DELETE_INITIAL_DELAY_MS = 10L; + private static final long DELETE_MAX_RETRY_DELAY_MS = 1000L; + private static final double DELETE_BACKOFF_MULTIPLIER = 1.5; + private static final long DELETE_TIMEOUT_MS = 10000L; + private static final int DELETE_PLANNING_WORKER_POOL_SIZE_DEFAULT = 10; + private static final int DELETE_ATTEMPT_NUM = 10; + private static final int DELETE_WORKER_POOL_SIZE_DEFAULT = 10; + private static final String EXECUTOR_TASK_NAME = "ES Executor"; + @VisibleForTesting static final String DELETE_FILES_TASK_NAME = "Delete file"; + + private ExpireSnapshots() { + // Do not instantiate directly + } + + /** Creates the builder for creating a stream which expires snapshots for the table. */ + public static Builder builder() { + return new Builder(); + } + + public static class Builder extends MaintenanceTaskBuilder<ExpireSnapshots.Builder> { + private Duration minAge = null; + private Integer retainLast = null; + private int planningWorkerPoolSize = DELETE_PLANNING_WORKER_POOL_SIZE_DEFAULT; + private int deleteAttemptNum = DELETE_ATTEMPT_NUM; + private int deleteWorkerPoolSize = DELETE_WORKER_POOL_SIZE_DEFAULT; + + /** + * The snapshots newer than this age will not be removed. + * + * @param newMinAge of the files to be removed + * @return for chained calls + */ + public Builder minAge(Duration newMinAge) { + this.minAge = newMinAge; + return this; + } + + /** + * The minimum {@link org.apache.iceberg.Snapshot}s to retain. For more details description see + * {@link org.apache.iceberg.ExpireSnapshots#retainLast(int)}. + * + * @param newRetainLast number of snapshots to retain + * @return for chained calls + */ + public Builder retainLast(int newRetainLast) { + this.retainLast = newRetainLast; + return this; + } + + /** + * The worker pool size used to calculate the files to delete. + * + * @param newPlanningWorkerPoolSize for planning files to delete + * @return for chained calls + */ + public Builder planningWorkerPoolSize(int newPlanningWorkerPoolSize) { Review Comment: for IcebergSource, there are two paths (1) batch: a separate pool is created and closed after the usage in the source constructor. this should be executed in the main thread of JM. so it has no concern. But there is a separate memory leak bug with incorrect usage of thread pools with shutdown hook that was discussed in the dev thread (2) streaming. the `ContinuousSplitPlannerImpl` code does support the shared pool although it is not actually used. ``` public ContinuousSplitPlannerImpl( TableLoader tableLoader, ScanContext scanContext, String threadName) { ... this.isSharedPool = threadName == null; this.workerPool = isSharedPool ? ThreadPools.getWorkerPool() : ThreadPools.newWorkerPool( "iceberg-plan-worker-pool-" + threadName, scanContext.planParallelism()); } ``` My main question is if we should use the shared pools (worker and delete) for maintenance tasks. ``` public static ExecutorService getWorkerPool() { return WORKER_POOL; } ``` One option is that if `planningWorkerPoolSize` is not set, it is null and default to the shared pool. -- 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