-
Notifications
You must be signed in to change notification settings - Fork 5.5k
[Iceberg] Add manifest file caching for HMS-based deployments #24481
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,119 @@ | ||
| /* | ||
| * Licensed 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 com.facebook.presto.iceberg; | ||
|
|
||
| import com.facebook.airlift.log.Logger; | ||
| import org.apache.iceberg.io.ByteBufferInputStream; | ||
| import org.apache.iceberg.io.InputFile; | ||
| import org.apache.iceberg.io.SeekableInputStream; | ||
|
|
||
| import java.io.IOException; | ||
| import java.nio.ByteBuffer; | ||
| import java.util.ArrayList; | ||
| import java.util.List; | ||
|
|
||
| import static java.lang.String.format; | ||
| import static java.util.Objects.requireNonNull; | ||
| import static org.apache.iceberg.io.IOUtil.readRemaining; | ||
|
|
||
| public class HdfsCachedInputFile | ||
| implements InputFile | ||
| { | ||
| private static final Logger LOG = Logger.get(HdfsCachedInputFile.class); | ||
|
|
||
| private final InputFile delegate; | ||
| private final ManifestFileCacheKey cacheKey; | ||
| private final ManifestFileCache cache; | ||
|
|
||
| public HdfsCachedInputFile(InputFile delegate, ManifestFileCacheKey cacheKey, ManifestFileCache cache) | ||
| { | ||
| this.delegate = requireNonNull(delegate, "delegate is null"); | ||
| this.cacheKey = requireNonNull(cacheKey, "cacheKey is null"); | ||
| this.cache = requireNonNull(cache, "cache is null"); | ||
| } | ||
|
|
||
| @Override | ||
| public long getLength() | ||
| { | ||
| ManifestFileCachedContent cachedContent = cache.getIfPresent(cacheKey); | ||
| if (cachedContent != null) { | ||
| return cachedContent.getLength(); | ||
| } | ||
| return delegate.getLength(); | ||
| } | ||
|
|
||
| @Override | ||
| public SeekableInputStream newStream() | ||
| { | ||
| ManifestFileCachedContent cachedContent = cache.getIfPresent(cacheKey); | ||
| if (cachedContent != null) { | ||
| return ByteBufferInputStream.wrap(cachedContent.getData()); | ||
| } | ||
|
|
||
| long fileLength = delegate.getLength(); | ||
| if (cache.isEnabled() && fileLength <= cache.getMaxFileLength()) { | ||
| try { | ||
| ManifestFileCachedContent content = readFully(delegate, fileLength, cache.getBufferChunkSize()); | ||
| cache.put(cacheKey, content); | ||
| cache.recordFileSize(content.getLength()); | ||
| return ByteBufferInputStream.wrap(content.getData()); | ||
| } | ||
| catch (IOException e) { | ||
| LOG.warn("Failed to cache input file: {}. Falling back to direct read.", delegate.location(), e); | ||
| } | ||
| } | ||
|
|
||
| return delegate.newStream(); | ||
| } | ||
|
|
||
| @Override | ||
| public String location() | ||
| { | ||
| return delegate.location(); | ||
| } | ||
|
|
||
| @Override | ||
| public boolean exists() | ||
| { | ||
| return cache.getIfPresent(cacheKey) != null || delegate.exists(); | ||
| } | ||
|
|
||
| private static ManifestFileCachedContent readFully(InputFile input, long fileLength, long chunkSize) | ||
| throws IOException | ||
| { | ||
| try (SeekableInputStream stream = input.newStream()) { | ||
| long totalBytesToRead = fileLength; | ||
| List<ByteBuffer> buffers = new ArrayList<>( | ||
| ((int) (fileLength / chunkSize)) + | ||
| (fileLength % chunkSize == 0 ? 0 : 1)); | ||
|
|
||
| while (totalBytesToRead > 0) { | ||
| int bytesToRead = (int) Math.min(chunkSize, totalBytesToRead); | ||
| byte[] buf = new byte[bytesToRead]; | ||
| int bytesRead = readRemaining(stream, buf, 0, bytesToRead); | ||
| totalBytesToRead -= bytesRead; | ||
|
|
||
| if (bytesRead < bytesToRead) { | ||
| throw new IOException( | ||
| format("Failed to read %d bytes from file %s : %d bytes read.", | ||
| fileLength, input.location(), fileLength - totalBytesToRead)); | ||
| } | ||
| else { | ||
| buffers.add(ByteBuffer.wrap(buf)); | ||
| } | ||
| } | ||
| return new ManifestFileCachedContent(buffers, fileLength); | ||
| } | ||
| } | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -93,6 +93,8 @@ | |
|
|
||
| import javax.inject.Singleton; | ||
|
|
||
| import java.nio.ByteBuffer; | ||
| import java.time.Duration; | ||
| import java.util.Optional; | ||
| import java.util.concurrent.ExecutorService; | ||
|
|
||
|
|
@@ -213,6 +215,26 @@ public StatisticsFileCache createStatisticsFileCache(IcebergConfig config, MBean | |
| return statisticsFileCache; | ||
| } | ||
|
|
||
| @Singleton | ||
| @Provides | ||
| public ManifestFileCache createManifestFileCache(IcebergConfig config, MBeanExporter exporter) | ||
| { | ||
| CacheBuilder<ManifestFileCacheKey, ManifestFileCachedContent> delegate = CacheBuilder.newBuilder() | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Two Questions here -
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I opted to create a new type for the cache key even though it's just the file in case we need to modify it more easily in the future As for caffeine, I spoke about this with Anant, but it's more complicated to use them due to a lack of utility classes and existing methods for Jmx metric support. It's a large enough effort that moving to Caffeine should be a separate PR. I have a draft up already for this. However, it still needs a little bit of work before it's ready to review #24608 |
||
| .maximumWeight(config.getMaxManifestCacheSize()) | ||
| .<ManifestFileCacheKey, ManifestFileCachedContent>weigher((key, entry) -> (int) entry.getData().stream().mapToLong(ByteBuffer::capacity).sum()) | ||
| .recordStats(); | ||
| if (config.getManifestCacheExpireDuration() > 0) { | ||
| delegate.expireAfterWrite(Duration.ofMillis(config.getManifestCacheExpireDuration())); | ||
| } | ||
| ManifestFileCache manifestFileCache = new ManifestFileCache( | ||
| delegate.build(), | ||
| config.getManifestCachingEnabled(), | ||
| config.getManifestCacheMaxContentLength(), | ||
| config.getManifestCacheMaxChunkSize().toBytes()); | ||
| exporter.export(generatedNameOf(ManifestFileCache.class, connectorId), manifestFileCache); | ||
| return manifestFileCache; | ||
| } | ||
|
|
||
| @ForCachingHiveMetastore | ||
| @Singleton | ||
| @Provides | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -68,10 +68,11 @@ public class IcebergConfig | |
|
|
||
| private EnumSet<ColumnStatisticType> hiveStatisticsMergeFlags = EnumSet.noneOf(ColumnStatisticType.class); | ||
| private String fileIOImpl = HadoopFileIO.class.getName(); | ||
| private boolean manifestCachingEnabled; | ||
| private boolean manifestCachingEnabled = true; | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is this intended?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes this is intentional. Performance is significantly worse with it disabled, and I don't think there are any known downsides to making this enabled by default other than an increased memory footprint |
||
| private long maxManifestCacheSize = IO_MANIFEST_CACHE_MAX_TOTAL_BYTES_DEFAULT; | ||
| private long manifestCacheExpireDuration = IO_MANIFEST_CACHE_EXPIRATION_INTERVAL_MS_DEFAULT; | ||
| private long manifestCacheMaxContentLength = IO_MANIFEST_CACHE_MAX_CONTENT_LENGTH_DEFAULT; | ||
| private DataSize manifestCacheMaxChunkSize = succinctDataSize(2, MEGABYTE); | ||
| private int splitManagerThreads = Runtime.getRuntime().availableProcessors(); | ||
| private DataSize maxStatisticsFileCacheSize = succinctDataSize(256, MEGABYTE); | ||
|
|
||
|
|
@@ -362,6 +363,20 @@ public IcebergConfig setManifestCacheMaxContentLength(long manifestCacheMaxConte | |
| return this; | ||
| } | ||
|
|
||
| public DataSize getManifestCacheMaxChunkSize() | ||
| { | ||
| return manifestCacheMaxChunkSize; | ||
| } | ||
|
|
||
| @Min(1024) | ||
| @Config("iceberg.io.manifest.cache.max-chunk-size") | ||
| @ConfigDescription("Maximum length of a buffer used to cache manifest file content. Only applicable to HIVE catalog.") | ||
| public IcebergConfig setManifestCacheMaxChunkSize(DataSize manifestCacheMaxChunkSize) | ||
| { | ||
| this.manifestCacheMaxChunkSize = manifestCacheMaxChunkSize; | ||
| return this; | ||
| } | ||
|
|
||
| @Min(0) | ||
| public int getSplitManagerThreads() | ||
| { | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.