FANNG1 commented on code in PR #8980:
URL: https://github.com/apache/gravitino/pull/8980#discussion_r2579389633


##########
iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/ScanPlanCache.java:
##########
@@ -0,0 +1,243 @@
+/*
+ * 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.gravitino.iceberg.service;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.Scheduler;
+import com.google.common.base.Objects;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.rest.requests.PlanTableScanRequest;
+import org.apache.iceberg.rest.responses.PlanTableScanResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Scan plan cache. */
+public class ScanPlanCache implements Closeable {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ScanPlanCache.class);
+
+  private final Cache<ScanPlanCacheKey, PlanTableScanResponse> scanPlanCache;
+  private final ScheduledExecutorService cleanupExecutor;
+
+  public ScanPlanCache(int capacity, int expireMinutes) {
+    LOG.info(
+        "Initializing ScanPlanCache with capacity: {}, expireAfterAccess: {} 
minutes",
+        capacity,
+        expireMinutes);
+
+    // Create a scheduled executor for periodic cleanup
+    this.cleanupExecutor =
+        new ScheduledThreadPoolExecutor(
+            1,
+            new ThreadFactoryBuilder()
+                .setDaemon(true)
+                .setNameFormat("scan-plan-cache-cleanup-%d")
+                .build());
+
+    this.scanPlanCache =
+        Caffeine.newBuilder()
+            .maximumSize(capacity)
+            .expireAfterAccess(expireMinutes, TimeUnit.MINUTES)
+            .scheduler(Scheduler.forScheduledExecutorService(cleanupExecutor))

Review Comment:
   Caffeine will evict the cache entity asynchronously if using a schedule to 
remove entries; this may increase memory usage for large requests. how about 
using the sync evict like 
https://github.com/apache/gravitino/blob/3f06131a41590e5d3d44f5853fed552e31cbbf7b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/cache/LocalTableMetadataCache.java#L49-L50



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