You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2022/05/03 03:26:37 UTC

[GitHub] [iceberg] rizaon commented on a diff in pull request #4518: core: Provide mechanism to cache manifest file content

rizaon commented on code in PR #4518:
URL: https://github.com/apache/iceberg/pull/4518#discussion_r863388111


##########
core/src/main/java/org/apache/iceberg/io/CachingFileIO.java:
##########
@@ -0,0 +1,355 @@
+/*
+ * 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.io;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.Weigher;
+import com.github.benmanes.caffeine.cache.stats.CacheStats;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.PropertyUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implementation of FileIO that adds metadata content caching features.
+ * <p>
+ * CachingFileIO intend to speedup scan planning by caching most of the table metadata files in memory and reduce
+ * remote reads. It is not meant to be dynamically loaded. Instead, set
+ * {@link CatalogProperties#IO_CACHE_ENABLED} to true and use
+ * {@link org.apache.iceberg.CatalogUtil#loadFileIO(String, Map, Object)} to wrap another FileIO with this
+ * CachingFileIO.
+ */
+public class CachingFileIO implements FileIO {
+  private static final Logger LOG = LoggerFactory.getLogger(CachingFileIO.class);
+  private static final int BUFFER_CHUNK_SIZE = 4 * 1024 * 1024; // 4MB
+  private static ContentCache sharedCache;
+
+  private static ContentCache createCache(Map<String, String> properties) {
+    long durationMs = PropertyUtil.propertyAsLong(properties,
+        CatalogProperties.IO_CACHE_EXPIRATION_INTERVAL_MS,
+        CatalogProperties.IO_CACHE_EXPIRATION_INTERVAL_MS_DEFAULT);
+    long totalBytes = PropertyUtil.propertyAsLong(properties,
+        CatalogProperties.IO_CACHE_MAX_TOTAL_BYTES,
+        CatalogProperties.IO_CACHE_MAX_TOTAL_BYTES_DEFAULT);
+    long contentLength = PropertyUtil.propertyAsLong(properties,
+        CatalogProperties.IO_CACHE_MAX_CONTENT_LENGTH,
+        CatalogProperties.IO_CACHE_MAX_CONTENT_LENGTH_DEFAULT);
+
+    if (durationMs >= 0 && totalBytes > 0) {
+      return new ContentCache(durationMs, totalBytes, contentLength);
+    } else {
+      return null;
+    }
+  }
+
+  public static synchronized ContentCache getOrCreate(Map<String, String> properties) {
+    if (sharedCache == null) {
+      sharedCache = createCache(properties);
+    }
+    return sharedCache;
+  }
+
+  public static boolean ioCacheEnabled(Map<String, String> properties) {
+    boolean enabled = PropertyUtil.propertyAsBoolean(properties,
+        CatalogProperties.IO_CACHE_ENABLED,
+        CatalogProperties.IO_CACHE_ENABLED_DEFAULT);
+    long durationMs = PropertyUtil.propertyAsLong(properties,
+        CatalogProperties.IO_CACHE_EXPIRATION_INTERVAL_MS,
+        CatalogProperties.IO_CACHE_EXPIRATION_INTERVAL_MS_DEFAULT);
+    long totalBytes = PropertyUtil.propertyAsLong(properties,
+        CatalogProperties.IO_CACHE_MAX_TOTAL_BYTES,
+        CatalogProperties.IO_CACHE_MAX_TOTAL_BYTES_DEFAULT);
+
+    return enabled && durationMs >= 0 && totalBytes > 0;
+  }
+
+  private ContentCache fileContentCache;
+  private Map<String, String> properties;
+  private FileIO wrappedIO;
+
+  /**
+   * CachingFileIO is not meant to be dynamically loaded.
+   */
+  private CachingFileIO() {
+  }
+
+  public CachingFileIO(FileIO io) {
+    assignIO(io);
+  }
+
+  public void assignIO(FileIO io) {
+    this.wrappedIO = io;
+  }
+
+  /**
+   * Initialize File IO from catalog properties.
+   * <p>
+   * It assumes that the inner FileIO is already initialized and therefore does not reinitialize it.
+   * If this is a reinitialization, it is the caller responsibility to call {@link #close()} and
+   * {@link #assignIO(FileIO)} prior to reinitialization to close the previously assigned FileIO and reassign a new one.
+   * Otherwise, previously assigned FileIO will be kept in use.
+   * <p>
+   * Reinitialization will discard previous content cache unless shared cache is being used
+   * ({@link CatalogProperties#IO_CACHE_SHARED} stays true).
+   * @param newProperties catalog properties
+   */
+  @Override
+  public void initialize(Map<String, String> newProperties) {
+    this.properties = newProperties;
+    initCache();
+  }
+
+  private void initCache() {
+    boolean isSharedCache = PropertyUtil.propertyAsBoolean(properties,
+        CatalogProperties.IO_CACHE_SHARED,
+        CatalogProperties.IO_CACHE_SHARED_DEFAULT);
+    if (isSharedCache) {
+      this.fileContentCache = getOrCreate(properties);
+      LOG.info("CachingFileIO created with shared cache. {} ", this.fileContentCache);
+    } else {
+      this.fileContentCache = createCache(properties);
+      LOG.info("CachingFileIO created. {} ", this.fileContentCache);
+    }
+  }
+
+  @Override
+  public void close() {
+    if (wrappedIO != null) {
+      wrappedIO.close();
+      wrappedIO = null;
+    }
+  }
+
+  @Override
+  public InputFile newInputFile(String path) {
+    InputFile inFile = wrappedIO.newInputFile(path);
+    if (fileContentCache != null && path.contains("/metadata/")) {

Review Comment:
   Hi @danielcweeks , thank you for your feedback!
   
   I can't seem to find an easy way to pass catalog properties (where we define caching related knobs) to ManifestFiles.java. How about we let the CachingFileIO wrapping just like now, but create new method, say `newMetadataInputFile()`, that specially used by `ManifestReader.read()`?
   
   Maybe modification like this in ManifestFiles.java:85
   ```
   InputFile file = (io instanceof CachingFileIO)
       ? ((CachingFileIO) io).newMetadataInputFile(manifest.path())
       : io.newInputFile(manifest.path());
   ```
   
   For the `newInputFile()` implementation by CachingFileIO, we can just pass-through it to the inner FileIO. Thus, it ensure that caching only applies to metadata files.



-- 
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-unsubscribe@iceberg.apache.org

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


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