RussellSpitzer commented on a change in pull request #2361:
URL: https://github.com/apache/iceberg/pull/2361#discussion_r599979276



##########
File path: 
spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteManifestsSparkAction.java
##########
@@ -0,0 +1,369 @@
+/*
+ * 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.spark.actions;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.ManifestFiles;
+import org.apache.iceberg.ManifestWriter;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.actions.BaseRewriteManifestsActionResult;
+import org.apache.iceberg.actions.RewriteManifests;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.JobGroupInfo;
+import org.apache.iceberg.spark.SparkDataFile;
+import org.apache.iceberg.spark.SparkUtil;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.api.java.function.MapFunction;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoder;
+import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.MetadataTableType.ENTRIES;
+
+/**
+ * An action that rewrites manifests in a distributed manner and co-locates 
metadata for partitions.
+ * <p>
+ * By default, this action rewrites all manifests for the current partition 
spec and writes the result
+ * to the metadata folder. The behavior can be modified by passing a custom 
predicate to {@link #rewriteIf(Predicate)}
+ * and a custom spec id to {@link #specId(int)}. In addition, there is a way 
to configure a custom location
+ * for new manifests via {@link #stagingLocation}.
+ */
+public class BaseRewriteManifestsSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteManifests, 
RewriteManifests.Result>
+    implements RewriteManifests {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(BaseRewriteManifestsSparkAction.class);
+
+  private final Encoder<ManifestFile> manifestEncoder;
+  private final Table table;
+  private final int formatVersion;
+  private final FileIO fileIO;
+  private final long targetManifestSizeBytes;
+
+  private PartitionSpec spec = null;
+  private Predicate<ManifestFile> predicate = manifest -> true;
+  private String stagingLocation = null;
+
+  public BaseRewriteManifestsSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.manifestEncoder = Encoders.javaSerialization(ManifestFile.class);
+    this.table = table;
+    this.spec = table.spec();
+    this.targetManifestSizeBytes = PropertyUtil.propertyAsLong(
+        table.properties(),
+        TableProperties.MANIFEST_TARGET_SIZE_BYTES,
+        TableProperties.MANIFEST_TARGET_SIZE_BYTES_DEFAULT);
+    this.fileIO = SparkUtil.serializableFileIO(table);
+
+    // default the staging location to the metadata location
+    TableOperations ops = ((HasTableOperations) table).operations();
+    Path metadataFilePath = new Path(ops.metadataFileLocation("file"));
+    this.stagingLocation = metadataFilePath.getParent().toString();
+
+    // use the current table format version for new manifests
+    this.formatVersion = ops.current().formatVersion();
+  }
+
+  @Override
+  protected RewriteManifests self() {
+    return this;
+  }
+
+  public RewriteManifests specId(int specId) {
+    Preconditions.checkArgument(table.specs().containsKey(specId), "Invalid 
spec id %d", specId);
+    this.spec = table.specs().get(specId);
+    return this;
+  }
+
+  /**
+   * Rewrites only manifests that match the given predicate.
+   *
+   * @param newPredicate a predicate
+   * @return this for method chaining
+   */
+  public RewriteManifests rewriteIf(Predicate<ManifestFile> newPredicate) {
+    this.predicate = newPredicate;
+    return this;
+  }
+
+  /**
+   * Passes a location where the manifests should be written.
+   *
+   * @param newStagingLocation a staging location
+   * @return this for method chaining
+   */
+  public RewriteManifests stagingLocation(String newStagingLocation) {
+    this.stagingLocation = newStagingLocation;
+    return this;
+  }
+
+  @Override
+  public RewriteManifests.Result execute() {
+    JobGroupInfo info = newJobGroupInfo("REWRITE-MANIFESTS", 
"REWRITE-MANIFESTS");
+    return withJobGroupInfo(info, this::doExecute);
+  }
+
+  private RewriteManifests.Result doExecute() {
+    List<ManifestFile> matchingManifests = findMatchingManifests();
+    if (matchingManifests.isEmpty()) {
+      return BaseRewriteManifestsActionResult.empty();
+    }
+
+    long totalSizeBytes = 0L;
+    int numEntries = 0;
+
+    for (ManifestFile manifest : matchingManifests) {
+      ValidationException.check(hasFileCounts(manifest), "No file counts in 
manifest: %s", manifest.path());
+
+      totalSizeBytes += manifest.length();
+      numEntries += manifest.addedFilesCount() + manifest.existingFilesCount() 
+ manifest.deletedFilesCount();
+    }
+
+    int targetNumManifests = targetNumManifests(totalSizeBytes);
+    int targetNumManifestEntries = targetNumManifestEntries(numEntries, 
targetNumManifests);
+
+    Dataset<Row> manifestEntryDF = buildManifestEntryDF(matchingManifests);
+
+    List<ManifestFile> newManifests;
+    if (spec.fields().size() < 1) {
+      newManifests = writeManifestsForUnpartitionedTable(manifestEntryDF, 
targetNumManifests);
+    } else {
+      newManifests = writeManifestsForPartitionedTable(manifestEntryDF, 
targetNumManifests, targetNumManifestEntries);
+    }
+
+    replaceManifests(matchingManifests, newManifests);
+
+    return new BaseRewriteManifestsActionResult(matchingManifests, 
newManifests);
+  }
+
+  private Dataset<Row> buildManifestEntryDF(List<ManifestFile> manifests) {
+    Dataset<Row> manifestDF = spark()
+        .createDataset(Lists.transform(manifests, ManifestFile::path), 
Encoders.STRING())
+        .toDF("manifest");
+
+    Dataset<Row> manifestEntryDF = loadMetadataTable(table, ENTRIES)
+        .filter("status < 2") // select only live entries

Review comment:
       We should probably make this less magic in the future, also may as well 
use a programmatic api here instead of passing in a raw string, not a priority 
for this PR, just thinking out loud.




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



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to