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 2021/04/08 06:53:16 UTC

[GitHub] [iceberg] openinx commented on a change in pull request #2379: Compaction Strategies

openinx commented on a change in pull request #2379:
URL: https://github.com/apache/iceberg/pull/2379#discussion_r609346859



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,134 @@
+/*
+ * 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 java.util.Map;
+import java.util.Objects;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.actions.compaction.DataCompactionStrategy;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * The parallelism level to use when processing jobs generated by the Compaction Strategy. The structure
+   * and contents of the jobs is determined by the Compaction Strategy and options passed to it. When running
+   * each job will be run independently and asynchronously.
+   *
+   * @param par number of concurrent jobs
+   * @return
+   */
+  CompactDataFiles parallelism(int par);
+
+  /**
+   * Pass an already constructed DataCompactionStrategy
+   *
+   * @param strategy the algorithm to be used during compaction
+   * @return
+   */
+  CompactDataFiles compactionStrategy(DataCompactionStrategy strategy);
+
+  // Todo Do we want to allow generic class loading here? I think yes this will probably be framework specific
+  CompactDataFiles compactionStrategy(String strategyName);
+
+  /**
+   * A user provided filter for determining which files will be considered by the
+   * Compaction strategy. This will be used in addition to whatever rules the Compaction strategy
+   * generates.
+   *
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
+   */
+  CompactDataFiles filter(Expression expression);
+
+  class Result {
+    private Map<CompactionJobInfo, CompactionResult> resultMap;
+
+
+    public Result(
+        Map<CompactionJobInfo, CompactionResult> resultMap) {
+      this.resultMap = resultMap;
+    }
+
+    public Map<CompactionJobInfo, CompactionResult> resultMap() {
+      return resultMap;
+    }
+  }
+
+  class CompactionResult {
+    private int numFilesRemoved;
+    private int numFilesAdded;
+
+    public CompactionResult(int numFilesRemoved, int numFilesAdded) {
+      this.numFilesRemoved = numFilesRemoved;
+      this.numFilesAdded = numFilesAdded;
+    }
+
+    public int numFilesAdded() {
+      return numFilesAdded;
+    }
+
+    public int numFilesRemoved() {
+      return numFilesRemoved;
+    }
+  }
+
+  class CompactionJobInfo {
+    private final int jobIndex;
+    private final int partitionIndex;
+    private final StructLike partition;
+
+    public CompactionJobInfo(int jobIndex, int partitionIndex, StructLike partition) {
+      this.jobIndex = jobIndex;
+      this.partitionIndex = partitionIndex;
+      this.partition = partition;
+    }
+
+    public int jobIndex() {
+      return jobIndex;
+    }
+
+    public int partitionIndex() {
+      return partitionIndex;
+    }
+
+    public StructLike partition() {
+      return partition;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      CompactionJobInfo that = (CompactionJobInfo) o;
+      return jobIndex == that.jobIndex && partitionIndex == that.partitionIndex &&
+          Objects.equals(partition, that.partition);

Review comment:
       The correct way to compare two different `StructLike` is using the comparator that is built from `Comparators#forType` because almost all of the `StructLike` implementation does not implement the `equals` & `hashCode` method.  One example is: https://github.com/apache/iceberg/blob/90225d6c9413016d611e2ce5eff37db1bc1b4fc5/core/src/main/java/org/apache/iceberg/util/StructLikeWrapper.java#L76 

##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/compaction/SparkBinningCompactionStrategy.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.compaction;
+
+import java.util.List;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.spark.source.SourceUtil;
+import org.apache.spark.sql.SparkSession;
+
+public class SparkBinningCompactionStrategy extends BinningCompactionStrategy {

Review comment:
       Here , we've introduced a `SparkBinningCompactionStrategy` ,   that sounds like is customized for the spark query engine.  I think the compaction strategy is common to various query engines, it just use its own strategy to plan the sub-tasks for each job,  the real job execution is binded to query engine.  Maybe we will need to move `rewriteFiles` out of the strategy intefaces.

##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,134 @@
+/*
+ * 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 java.util.Map;
+import java.util.Objects;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.actions.compaction.DataCompactionStrategy;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * The parallelism level to use when processing jobs generated by the Compaction Strategy. The structure
+   * and contents of the jobs is determined by the Compaction Strategy and options passed to it. When running
+   * each job will be run independently and asynchronously.
+   *
+   * @param par number of concurrent jobs
+   * @return
+   */
+  CompactDataFiles parallelism(int par);
+
+  /**
+   * Pass an already constructed DataCompactionStrategy
+   *
+   * @param strategy the algorithm to be used during compaction
+   * @return
+   */
+  CompactDataFiles compactionStrategy(DataCompactionStrategy strategy);
+
+  // Todo Do we want to allow generic class loading here? I think yes this will probably be framework specific
+  CompactDataFiles compactionStrategy(String strategyName);
+
+  /**
+   * A user provided filter for determining which files will be considered by the
+   * Compaction strategy. This will be used in addition to whatever rules the Compaction strategy
+   * generates.
+   *
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
+   */
+  CompactDataFiles filter(Expression expression);
+
+  class Result {
+    private Map<CompactionJobInfo, CompactionResult> resultMap;
+
+
+    public Result(
+        Map<CompactionJobInfo, CompactionResult> resultMap) {
+      this.resultMap = resultMap;
+    }
+
+    public Map<CompactionJobInfo, CompactionResult> resultMap() {
+      return resultMap;
+    }
+  }
+
+  class CompactionResult {
+    private int numFilesRemoved;
+    private int numFilesAdded;
+
+    public CompactionResult(int numFilesRemoved, int numFilesAdded) {
+      this.numFilesRemoved = numFilesRemoved;
+      this.numFilesAdded = numFilesAdded;
+    }
+
+    public int numFilesAdded() {
+      return numFilesAdded;
+    }
+
+    public int numFilesRemoved() {
+      return numFilesRemoved;
+    }
+  }
+
+  class CompactionJobInfo {
+    private final int jobIndex;
+    private final int partitionIndex;
+    private final StructLike partition;
+
+    public CompactionJobInfo(int jobIndex, int partitionIndex, StructLike partition) {
+      this.jobIndex = jobIndex;
+      this.partitionIndex = partitionIndex;
+      this.partition = partition;
+    }
+
+    public int jobIndex() {
+      return jobIndex;
+    }
+
+    public int partitionIndex() {
+      return partitionIndex;
+    }
+
+    public StructLike partition() {
+      return partition;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      CompactionJobInfo that = (CompactionJobInfo) o;
+      return jobIndex == that.jobIndex && partitionIndex == that.partitionIndex &&
+          Objects.equals(partition, that.partition);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(jobIndex, partitionIndex, partition);

Review comment:
       The `partition` instance will need `JavaHash.forType()` to generate the hash code.

##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.compaction;
+
+import java.io.Serializable;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+
+public interface DataCompactionStrategy extends Serializable {
+
+  /**
+   * Returns the name of this compaction strategy
+   */
+  String name();
+
+  /**
+   * Returns a set of options which this compaction strategy can use. This is an allowed-list and any options not
+   * specified here will be rejected at runtime.
+   */
+  Set<String> validOptions();
+
+  default void validateOptions(Map<String, String> options) {
+    Sets.SetView<String> invalidOptions = Sets.difference(options.keySet(), validOptions());
+    if (!invalidOptions.isEmpty()) {
+      String invalidColumnString = invalidOptions.stream().collect(Collectors.joining(",", "[", "]"));
+      String validColumnString = validOptions().stream().collect(Collectors.joining(",", "[", "]"));
+
+      throw new IllegalArgumentException(String.format(
+          "Cannot use strategy %s with unknown options %s. This strategy accepts %s",
+          name(), invalidColumnString, validColumnString));
+    }
+  }
+
+  DataCompactionStrategy withOptions(Map<String, String> options);
+
+  /**
+   * Before the compaction strategy rules are applied, the underlying action has the ability to use this expression
+   * to filter the FileScanTasks which will be created when planning file reads that will later be run through
+   * this compaction strategy.
+   *
+   * @return an Iceberg expression to use when discovering file scan tasks
+   */
+  default Expression preFilter() {
+    return Expressions.alwaysTrue();
+  }
+
+  /**
+   * Removes all file references which this plan will not rewrite or change. Unlike the preFilter, this method can
+   * execute arbitrary code and isn't restricted to just Iceberg Expressions. This should be serializable so that
+   * Actions which run remotely can utilize the method.
+   *
+   * @param dataFiles iterator of live datafiles in a given partition
+   * @return iterator containing only files to be rewritten
+   */
+  Iterator<FileScanTask> filesToCompact(Iterator<FileScanTask> dataFiles);
+
+  /**
+   * Groups file scans into lists which will be processed in a single executable
+   * unit. Each group will end up being committed as an independent set of
+   * changes. This creates the jobs which will eventually be run as by the underlying Action.
+   *
+   * @param dataFiles iterator of files to be rewritten
+   * @return iterator of sets of files to be processed together
+   */
+  Iterator<List<FileScanTask>> groupsFilesIntoJobs(Iterator<FileScanTask> dataFiles);
+
+  /**
+   * Method which will actually rewrite and commit changes to a group of files
+   * based on the particular CompactionStrategy's Algorithm. This will most likely be
+   * Action framework specific.
+   *
+   * @param table table being modified
+   * @param filesToRewrite a group of files to be rewritten together
+   * @return a list of newly committed files
+   */
+  List<DataFile> rewriteFiles(Table table, List<FileScanTask> filesToRewrite);

Review comment:
       I think we need to consider the format v2's compaction strategy , at least in the API design level. Otherwise we will deprecate this API soon once we introduce compaction implementation for v2 in https://github.com/apache/iceberg/pull/2303/files




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



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