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_r405069393
 
 

 ##########
 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);
+
+      Map<Integer, Map<StructLike, Integer>> bins = 
computeBins(metadataSizeSummary);
+
+      // the size of bins is estimated to be roughly targetManifestSizeBytes
+      // we allow the actual size of manifests to be 5% higher to avoid closing
+      // manifests near the end of bins if the estimation is not precise enough
+      // it is better to have slightly bigger manifests rather than manifests 
with a couple of entries
+      long manifestSizeBytes = (long) (1.05 * targetManifestSizeBytes);
+      List<ManifestFile> newManifests = manifestEntryDS
+          .groupByKey(toBin(bins), binEncoder)
+          .flatMapGroups(toManifest(io, manifestSizeBytes, stagingLocation, 
specs), manifestEncoder)
+          .collectAsList();
+
+      replaceManifests(matchingManifests, newManifests);
+
+      return new RewriteManifestsActionResult(matchingManifests, newManifests);
+    } finally {
+      manifestEntryDS.unpersist(false);
+    }
+  }
+
+  private List<ManifestFile> findMatchingManifests() {
+    Snapshot snapshot = table.currentSnapshot();
+    if (snapshot == null) {
+      return ImmutableList.of();
+    }
+    return 
snapshot.manifests().stream().filter(predicate).collect(Collectors.toList());
+  }
+
+  // computes the average manifest entry size based on available stats for 
manifests
+  private Long computeManifestEntrySizeBytes(List<ManifestFile> manifests) {
+    long totalSize = 0L;
+    int numEntries = 0;
+
+    for (ManifestFile m : manifests) {
+      ValidationException.check(
+          m.addedFilesCount() != null && m.existingFilesCount() != null && 
m.deletedFilesCount() != null,
+          "No file counts in manifest: " + m.path());
+
+      totalSize += m.length();
+      numEntries += m.addedFilesCount() + m.existingFilesCount() + 
m.deletedFilesCount();
+    }
+
+    ValidationException.check(totalSize > 0L, "Total size of manifests must be 
greater than 0");
+    ValidationException.check(numEntries > 0, "Number of manifest entries must 
be greater than 0");
+
+    return totalSize / numEntries;
+  }
+
+  // computes the estimated metadata size per spec and partition
+  private Map<Integer, List<PartitionMetadata>> computeMetadataSizeSummary(
+      Dataset<Entry> manifestEntryDS,
+      long manifestEntrySizeBytes) {
+
+    MetadataSizeAggregator agg = new MetadataSizeAggregator(specs, 
manifestEntrySizeBytes);
+    TypedColumn<Entry, Map<Integer, List<PartitionMetadata>>> column = 
agg.toColumn().name("result");
+    return manifestEntryDS.select(column).collectAsList().get(0);
+  }
+
+  // groups smaller partitions into bins of the target size
+  private Map<Integer, Map<StructLike, Integer>> computeBins(Map<Integer, 
List<PartitionMetadata>> sizeSummary) {
+    Map<Integer, Map<StructLike, Integer>> binMap = Maps.newHashMap();
+
+    sizeSummary.forEach((specId, sizes) -> {
+      BinPacking.ListPacker<PartitionMetadata> packer = new 
BinPacking.ListPacker<>(targetManifestSizeBytes, 1, false);
+      List<List<PartitionMetadata>> bins = packer.pack(sizes, 
PartitionMetadata::getMetadataSizeBytes);
+
+      for (int binIndex = 0; binIndex < bins.size(); binIndex++) {
+        List<PartitionMetadata> bin = bins.get(binIndex);
+
+        for (PartitionMetadata binEntry : bin) {
+          Map<StructLike, Integer> map = 
binMap.computeIfAbsent(binEntry.specId, key -> Maps.newHashMap());
+          map.put(binEntry.partition, binIndex);
+        }
+      }
+    });
+
+    return binMap;
+  }
+
+  private void replaceManifests(Iterable<ManifestFile> deletedManifests, 
Iterable<ManifestFile> addedManifests) {
+    try {
+      RewriteManifests rewriteManifests = table.rewriteManifests();
+      deletedManifests.forEach(rewriteManifests::deleteManifest);
+      addedManifests.forEach(rewriteManifests::addManifest);
+      summary.forEach(rewriteManifests::set);
+      rewriteManifests.commit();
+
+      if (!snapshotIdInheritanceEnabled) {
 
 Review comment:
   I also don't want to overcomplicate this. The only problem is that the 
clean-up logic of staged manifests in this action depends on the value before 
the commit while the actual commit logic that decides whether to rewrite 
manifests will depend on calling `ops.current().properties()` in 
`BaseRewriteManifests`. If snapshot id inheritance is enabled concurrently, 
there can be a case when `BaseRewriteManifests` appends manifests without 
rewriting them but we still delete them after the commit. The probability of 
this is low as we need to reuse the same `Table` instance to enable snapshot id 
inheritance. It is just a point I wanted to bring attention to.

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

Reply via email to