You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2019/06/05 20:40:47 UTC

[GitHub] [incubator-iceberg] danielcweeks commented on a change in pull request #200: Add SnapshotProducer that can rewrite manifests

danielcweeks commented on a change in pull request #200: Add SnapshotProducer that can rewrite manifests
URL: https://github.com/apache/incubator-iceberg/pull/200#discussion_r290924942
 
 

 ##########
 File path: core/src/main/java/org/apache/iceberg/ReplaceManifests.java
 ##########
 @@ -0,0 +1,221 @@
+/*
+ * 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.Lists;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+
+import static org.apache.iceberg.TableProperties.MANIFEST_TARGET_SIZE_BYTES;
+import static org.apache.iceberg.TableProperties.MANIFEST_TARGET_SIZE_BYTES_DEFAULT;
+
+
+public class ReplaceManifests extends SnapshotProducer<RewriteManifests> implements RewriteManifests {
+  private final TableOperations ops;
+  private final PartitionSpec spec;
+  private final long manifestTargetSizeBytes;
+  private final SnapshotSummary.Builder summaryBuilder = SnapshotSummary.builder();
+  private final List<ManifestFile> newManifests = Collections.synchronizedList(Lists.newArrayList());
+  private final Map<Object, WriterWrapper> writers = Collections.synchronizedMap(new HashMap<>());
+  private final AtomicInteger manifestCount = new AtomicInteger(0);
+
+  private Function<DataFile, Object> clusterByFunc;
+  private Function<ManifestFile, Boolean> filterFunc;
+
+  ReplaceManifests(TableOperations ops) {
+    super(ops);
+    this.ops = ops;
+    this.spec = ops.current().spec();
+    this.manifestTargetSizeBytes =
+      ops.current().propertyAsLong(MANIFEST_TARGET_SIZE_BYTES, MANIFEST_TARGET_SIZE_BYTES_DEFAULT);
+  }
+
+  @Override
+  protected String operation() {
+    return DataOperations.REPLACE;
+  }
+
+  @Override
+  public RewriteManifests set(String property, String value) {
+    summaryBuilder.set(property, value);
+    return this;
+  }
+
+  @Override
+  protected Map<String, String> summary() {
+    return summaryBuilder.build();
+  }
+
+  @Override
+  public ReplaceManifests clusterBy(Function<DataFile, Object> func) {
+    this.clusterByFunc = func;
+    return this;
+  }
+
+  @Override
+  public ReplaceManifests filter(Function<ManifestFile, Boolean> func) {
+    this.filterFunc = func;
+    return this;
+  }
+
+  @Override
+  public List<ManifestFile> apply(TableMetadata base) {
+    Preconditions.checkNotNull(clusterByFunc, "clusterBy function cannot be null");
+
+    List<ManifestFile> existingManifests = Collections.synchronizedList(new ArrayList<>());
+
+    try {
+      Tasks.foreach(base.currentSnapshot().manifests())
+          .executeWith(ThreadPools.getWorkerPool())
+          .run(manifest -> {
+            if (filterFunc != null && !filterFunc.apply(manifest)) {
+              existingManifests.add(manifest);
+            } else {
+              long entryNum = manifest.addedFilesCount() + manifest.existingFilesCount() + manifest.deletedFilesCount();
+              long avgEntryLen = manifest.length() / entryNum;
+
+              try (ManifestReader reader =
+                     ManifestReader.read(ops.io().newInputFile(manifest.path()))) {
+                FilteredManifest filteredManifest = reader.select(Lists.newArrayList("*"));
+                filteredManifest.iterator().forEachRemaining(
+                    file -> appendFile(file, avgEntryLen, clusterByFunc.apply(file))
+                );
+
+              } catch (IOException x) {
+                throw new RuntimeIOException(x);
+              }
+            }
+          });
+    } finally {
+      Tasks.foreach(writers.values()).executeWith(ThreadPools.getWorkerPool()).run(writer -> writer.close());
+    }
+
+    newManifests.addAll(
+        writers.values()
+          .stream()
+          .map(WriterWrapper::toManifestFile)
+          .collect(Collectors.toList())
+    );
+
+    // put new manifests at the beginning
+    List<ManifestFile> apply = new ArrayList<>();
+    apply.addAll(newManifests);
+    apply.addAll(existingManifests);
+
+    return apply;
+  }
+
+  private void appendFile(DataFile file, long avgEntryLen, Object key) {
+    Preconditions.checkNotNull(file, "Data file cannot be null");
+    Preconditions.checkNotNull(key, "Key cannot be null");
+
+    synchronized (summaryBuilder) {
+      summaryBuilder.addedFile(spec, file);
 
 Review comment:
   @bryanck Looks like this is potentially pulling all the partitions into memory during this process.  While this may work for smaller cases, I doubt this will scale given the current summary builder.  We could not write these at all since this is a replace operation and the partition summaries won't be used for any current use case.  I assume we would fallback elsewhere if the summaries don't exist.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org