ahmedabu98 commented on code in PR #33504: URL: https://github.com/apache/beam/pull/33504#discussion_r2004411158
########## sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadUtils.java: ########## @@ -0,0 +1,192 @@ +/* + * 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.beam.sdk.io.iceberg; + +import static org.apache.iceberg.util.SnapshotUtil.ancestorsOf; + +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.BiFunction; +import java.util.stream.Collectors; +import org.apache.beam.sdk.io.iceberg.IcebergIO.ReadRows.StartingStrategy; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.IdentityPartitionConverters; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.parquet.GenericParquetReaders; +import org.apache.iceberg.encryption.EncryptedFiles; +import org.apache.iceberg.encryption.EncryptedInputFile; +import org.apache.iceberg.hadoop.HadoopInputFile; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.mapping.NameMappingParser; +import org.apache.iceberg.parquet.ParquetReader; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.PartitionUtil; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.parquet.HadoopReadOptions; +import org.apache.parquet.ParquetReadOptions; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** Helper class for source operations. */ +public class ReadUtils { + // default is 8MB. keep this low to avoid overwhelming memory + static final int MAX_FILE_BUFFER_SIZE = 1 << 18; // 256KB Review Comment: I'd rather we wait until there's a need to expose it ########## sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/TableCache.java: ########## @@ -0,0 +1,60 @@ +/* + * 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.beam.sdk.io.iceberg; + +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.Cache; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheBuilder; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** Utility to fetch and cache Iceberg {@link Table}s. */ +class TableCache { + private static final Cache<TableIdentifier, Table> CACHE = + CacheBuilder.newBuilder().expireAfterWrite(3, TimeUnit.MINUTES).build(); + + static Table get(TableIdentifier identifier, Catalog catalog) { + try { + return CACHE.get(identifier, () -> catalog.loadTable(identifier)); + } catch (ExecutionException e) { + throw new RuntimeException( + "Encountered a problem fetching table " + identifier + " from cache.", e); + } + } + + static Table get(String identifier, Catalog catalog) { + return get(TableIdentifier.parse(identifier), catalog); + } + + static Table getRefreshed(TableIdentifier identifier, Catalog catalog) { + @Nullable Table table = CACHE.getIfPresent(identifier); + if (table == null) { + return get(identifier, catalog); + } + table.refresh(); Review Comment: Ahhh that looks useful! thanks for the reference. It doesn't take much to refresh and I think it's fine refreshing from multiple threads (it just retrieves metadata from the catalog). However, I do like letting the cache automatically refresh periodically. I think It'll be useful for the sink to use as well (will make a separate change for that) -- 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]
