aokolnychyi commented on a change in pull request #875: [WIP] Spark: Implement an action to rewrite manifests URL: https://github.com/apache/incubator-iceberg/pull/875#discussion_r405052410
########## File path: spark/src/main/java/org/apache/iceberg/RewriteManifestsAction.java ########## @@ -0,0 +1,490 @@ +/* + * 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; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import java.io.Serializable; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.function.Predicate; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.util.BinPacking; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.Tasks; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.api.java.function.FlatMapGroupsFunction; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.broadcast.Broadcast; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoder; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.TypedColumn; +import org.apache.spark.sql.expressions.Aggregator; +import org.apache.spark.util.SerializableConfiguration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +// TODO: concurrent modification of snapshotIdInheritanceEnabled or specs? +public class RewriteManifestsAction + implements SnapshotUpdateAction<RewriteManifestsAction, RewriteManifestsActionResult> { + + private static final Logger LOG = LoggerFactory.getLogger(RewriteManifestsAction.class); + + private final SparkSession spark; + private final JavaSparkContext sparkContext; + private final Table table; + private final FileIO fileIO; + private final Map<Integer, PartitionSpec> specs; + private final Map<String, String> summary; + private final int defaultParallelism; + private final boolean snapshotIdInheritanceEnabled; + private final long targetManifestSizeBytes; + + private final Encoder<ManifestFile> manifestEncoder = Encoders.javaSerialization(ManifestFile.class); + private final Encoder<Entry> entryEncoder = Encoders.javaSerialization(Entry.class); + private final Encoder<Bin> binEncoder = Encoders.bean(Bin.class); + + private Predicate<ManifestFile> predicate = manifest -> true; + private String stagingLocation = null; + + RewriteManifestsAction(SparkSession spark, Table table) { + this.spark = spark; + this.sparkContext = new JavaSparkContext(spark.sparkContext()); + this.table = table; + this.specs = table.specs(); + this.summary = Maps.newHashMap(); + this.defaultParallelism = Integer.parseInt( + spark.conf().get("spark.default.parallelism", "200")); + this.snapshotIdInheritanceEnabled = PropertyUtil.propertyAsBoolean( + table.properties(), + TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, + TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED_DEFAULT); + this.targetManifestSizeBytes = PropertyUtil.propertyAsLong( + table.properties(), + TableProperties.MANIFEST_TARGET_SIZE_BYTES, + TableProperties.MANIFEST_TARGET_SIZE_BYTES_DEFAULT); + + if (table.io() instanceof HadoopFileIO) { + // we need to use Spark's SerializableConfiguration to avoid issues with Kryo serialization + SerializableConfiguration conf = new SerializableConfiguration(((HadoopFileIO) table.io()).conf()); + fileIO = new HadoopFileIO(conf::value); + } else { + fileIO = table.io(); + } + } + + public RewriteManifestsAction rewriteIf(Predicate<ManifestFile> newPredicate) { + this.predicate = newPredicate; + return this; + } + + public RewriteManifestsAction stagingLocation(String newStagingLocation) { + this.stagingLocation = newStagingLocation; + return this; + } + + @Override + public RewriteManifestsAction set(String property, String value) { + summary.put(property, value); + return this; + } + + @Override + public RewriteManifestsActionResult execute() { + Preconditions.checkArgument(stagingLocation != null, "Staging location must be set"); + + List<ManifestFile> matchingManifests = findMatchingManifests(); + if (matchingManifests.isEmpty()) { + return null; + } + + Broadcast<FileIO> io = sparkContext.broadcast(fileIO); + + int parallelism = Math.min(matchingManifests.size(), defaultParallelism); + JavaRDD<ManifestFile> manifestRDD = sparkContext.parallelize(matchingManifests, parallelism); + Dataset<ManifestFile> manifestDS = spark.createDataset(manifestRDD.rdd(), manifestEncoder); + Dataset<Entry> manifestEntryDS = manifestDS.flatMap(toEntries(io, specs), entryEncoder); + + try { + manifestEntryDS.cache(); + + long manifestEntrySizeBytes = computeManifestEntrySizeBytes(matchingManifests); + Map<Integer, List<PartitionMetadata>> metadataSizeSummary = computeMetadataSizeSummary( + manifestEntryDS, + manifestEntrySizeBytes); Review comment: I mention a couple of challenges in the TODO part of the PR description. I think we can try to map Spark's `Row` into `DataFile` or `StructLike` manually if we fix processing to one partition spec at a time. Any other ideas there, @rdblue? ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: [email protected] With regards, Apache Git Services --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
