JingsongLi commented on code in PR #1823:
URL: https://github.com/apache/incubator-paimon/pull/1823#discussion_r1306875796


##########
paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java:
##########
@@ -60,6 +63,8 @@ public abstract class AbstractFileStore<T> implements 
FileStore<T> {
 
     @Nullable private final SegmentsCache<String> writeManifestCache;
 
+    private ExecutorService lazyIoExecutor;

Review Comment:
   This `IoExecutor` is created in here, but passed to new classes. And closed 
in new classes.
   Its lifetime is weird...
   
   Can we just use `FileUtils.COMMON_IO_FORK_JOIN_POOL`?



##########
paimon-common/src/main/java/org/apache/paimon/CoreOptions.java:
##########
@@ -194,6 +194,26 @@ public class CoreOptions implements Serializable {
                     .defaultValue(Duration.ofHours(1))
                     .withDescription("The maximum time of completed snapshots 
to retain.");
 
+    public static final ConfigOption<ExpireExecutionMode> 
SNAPSHOT_EXPIRE_EXECUTION_MODE =
+            key("snapshot.expire.execution-mode")

Review Comment:
   I think about this again. Maybe it is not good to provide async mode now. 
Maybe there will be some thread conflicts.
   
   The performance maybe good enough now, so we don't need to provide async 
mode.



##########
paimon-core/src/test/java/org/apache/paimon/utils/ManuallyTriggeredScheduledExecutorService.java:
##########
@@ -0,0 +1,333 @@
+/*
+ * 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.paimon.utils;
+
+import javax.annotation.Nonnull;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/**
+ * Simple {@link ScheduledExecutorService} implementation for testing 
purposes. It spawns no
+ * threads, but lets you trigger the execution of tasks manually.
+ *
+ * <p>This class is helpful when implementing tests tasks synchronous and 
control when they run,
+ * which would otherwise asynchronous and require complex triggers and latches 
to test.
+ *
+ * <p>Copy from Flink.
+ */
+public class ManuallyTriggeredScheduledExecutorService implements 
ScheduledExecutorService {

Review Comment:
   Do we really need this? This is only for testing.
   I think maybe we can just get rid of these two classes.



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

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to