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_r409751463
 
 

 ##########
 File path: 
spark/src/main/java/org/apache/iceberg/actions/RewriteManifestsAction.java
 ##########
 @@ -0,0 +1,342 @@
+/*
+ * 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.actions;
+
+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 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.ManifestFile;
+import org.apache.iceberg.ManifestFiles;
+import org.apache.iceberg.ManifestWriter;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.RewriteManifests;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+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.spark.SparkDataFile;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.api.java.JavaSparkContext;
+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.util.SerializableConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class RewriteManifestsAction
+    extends BaseSnapshotUpdateAction<RewriteManifestsAction, 
RewriteManifestsActionResult> {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(RewriteManifestsAction.class);
+
+  private final SparkSession spark;
+  private final JavaSparkContext sparkContext;
+  private final Encoder<ManifestFile> manifestEncoder;
+  private final Table table;
+  private final FileIO fileIO;
+  private final long targetManifestSizeBytes;
+
+  private PartitionSpec spec = null;
+  private Predicate<ManifestFile> predicate = manifest -> true;
+  private String stagingLocation = null;
+  private boolean useCaching = true;
+
+  RewriteManifestsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.sparkContext = new JavaSparkContext(spark.sparkContext());
+    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);
+
+    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());
+      this.fileIO = new HadoopFileIO(conf::value);
+    } else {
+      this.fileIO = table.io();
+    }
+  }
+
+  @Override
+  protected RewriteManifestsAction self() {
+    return this;
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  public RewriteManifestsAction specId(int specId) {
+    Preconditions.checkArgument(table.specs().containsKey(specId), "Invalid 
spec id %d", specId);
+    this.spec = table.specs().get(specId);
+    return this;
+  }
+
+  public RewriteManifestsAction rewriteIf(Predicate<ManifestFile> 
newPredicate) {
+    this.predicate = newPredicate;
+    return this;
+  }
+
+  public RewriteManifestsAction stagingLocation(String newStagingLocation) {
+    this.stagingLocation = newStagingLocation;
+    return this;
+  }
+
+  public RewriteManifestsAction useCaching(boolean newUseCaching) {
+    this.useCaching = newUseCaching;
+    return this;
+  }
+
+  @Override
+  public RewriteManifestsActionResult execute() {
+    Preconditions.checkArgument(stagingLocation != null, "Staging location 
must be set");
+
+    List<ManifestFile> matchingManifests = findMatchingManifests();
+    if (matchingManifests.isEmpty()) {
+      return RewriteManifestsActionResult.empty();
+    }
+
+    long totalSizeBytes = 0L;
+    int numEntries = 0;
+
+    for (ManifestFile manifest : matchingManifests) {
+      ValidationException.check(hasFileCounts(manifest), "No file counts in 
manifest: " + 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 RewriteManifestsActionResult(matchingManifests, newManifests);
+  }
+
+  private Dataset<Row> buildManifestEntryDF(List<ManifestFile> manifests) {
 
 Review comment:
   This part requires further discussion. Right now, we accept a Java 
`Predicate` that we run on manifests locally and then we have a broadcast join 
between the matching manifest names and all manifest entries. The problem is 
that we read all manifest entries no matter what. However, this does not seem 
to be a huge problem as the metadata size is small. In addition, the current 
snapshot might change between the time we iterate through manifests locally and 
the time when we query the metadata table. It should work correctly, though, as 
the commit would fail if a manifest we are trying to replace is missing.
   
   As an alternative, we could accept a Spark filter instead and run it on top 
of the metadata table. In that case, we would need to run one more job to 
collect matching manifest file names and construct `GenericManifestFile`s from 
those names so that we can pass them to the commit operation.

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