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/20 22:55:57 UTC

[GitHub] [iceberg] RussellSpitzer opened a new pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

RussellSpitzer opened a new pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501


   In this commit we declare the interfaces required for implementing a new
   DataCompactionStrategy and Action for Compacting DataFiles.
   
   As discussed in this Design Doc
   
   https://docs.google.com/document/d/1aXo1VzuXxSuqcTzMLSQdnivMVgtLExgDWUFMvWeXRxc/edit?ts=600b0432#heading=h.3laq1izdmipi


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


[GitHub] [iceberg] kbendick commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r627081408



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Rewrite Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.
  */
-public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+public interface RewriteDataFiles extends Action<RewriteDataFiles, RewriteDataFiles.Result> {
+
   /**
-   * Pass a row filter to filter {@link DataFile}s to be rewritten.
-   * <p>
-   * Note that all files that may contain data matching the filter may be rewritten.
-   * <p>
-   * If not set, all files will be rewritten.
-   *
-   * @param expr a row filter to filter out data files
-   * @return this for method chaining
+   * Enable committing groups of files (see max-file-group-size) prior to the entire compaction completing.
+   * This will produce additional commits but allow for progress even if some groups fail to commit. This setting
+   * will not change the correctness of the rewrite operation. The default is false, which produces a single commit
+   * when the entire job has completed.
    */
-  RewriteDataFiles filter(Expression expr);
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
 
   /**
-   * Enables or disables case sensitive expression binding.
-   * <p>
-   * If not set, defaults to false.
-   *
-   * @param caseSensitive caseSensitive
-   * @return this for method chaining
+   * The maximum amount of Iceberg commits that compaction is allowed to produce if partial progress is enabled.
    */
-  RewriteDataFiles caseSensitive(boolean caseSensitive);
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
 
   /**
-   * Pass a PartitionSpec id to specify which PartitionSpec should be used in DataFile rewrite
-   * <p>
-   * If not set, defaults to the table's default spec ID.
-   *
-   * @param specId PartitionSpec id to rewrite
-   * @return this for method chaining
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size into groups. These sub-units of compaction are referred to as file groups. The largest amount of data that
+   * should be compacted in a single group is controlled by MAX_FILE_GROUP_SIZE_BYTES. When grouping files, the
+   * underlying compaction strategy will use this value as to limit the files which will be included in a single file
+   * group. A group will be processed by a single framework "action". For example, in Spark this means that each group
+   * would be rewritten in its own Spark action. A group will never contain files for multiple output partitions.
    */
-  RewriteDataFiles outputSpecId(int specId);
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";
+  long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
 
   /**
-   * Specify the target data file size in bytes.
-   * <p>
-   * If not set, defaults to the table's target file size.
-   *
-   * @param targetSizeInBytes size in bytes of rewrite data file
-   * @return this for method chaining
+   * The max number of file groups to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the group is determined by the compaction strategy. Each file group will be rewritten
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_FILE_GROUP_ACTIONS = "max-concurrent-file-group-actions";
+  int MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT = 1;
+
+  /**
+   * The output file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.
+   */
+  String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes";
+
+  /**
+   * The partition spec to use when writing the output data from this operation. By default uses the
+   * current table partition spec.
    */
-  RewriteDataFiles targetSizeInBytes(long targetSizeInBytes);
+  String PARTITION_SPEC_ID = "partition-spec-id";
+
+  enum RewriteStrategyName {
+    BINPACK,
+    SORT
+  }
 
   /**
-   * Specify the number of "bins" considered when trying to pack the next file split into a task. Increasing this
-   * usually makes tasks a bit more even by considering more ways to pack file regions into a single task with extra
-   * planning cost.
-   * <p>
-   * This configuration can reorder the incoming file regions, to preserve order for lower/upper bounds in file
-   * metadata, user can use a lookback of 1.
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
    *
-   * @param splitLookback number of "bins" considered when trying to pack the next file split into a task.
+   * @param strategyName name of the strategy
    * @return this for method chaining
    */
-  RewriteDataFiles splitLookback(int splitLookback);
+  RewriteDataFiles strategy(RewriteStrategyName strategyName);
 
   /**
-   * Specify the cost of opening a file that will be taken into account during packing files into
-   * bins. If the size of the file is smaller than the cost of opening, then this value will be used
-   * instead of the actual file size.
-   * <p>
-   * If not set, defaults to the table's open file cost.
+   * 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. For example this would be used for providing a
+   * restriction to only run compaction on a specific partition.
    *
-   * @param splitOpenFileCost minimum file size to count to pack into one "bin".
-   * @return this for method chaining
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
    */
-  RewriteDataFiles splitOpenFileCost(long splitOpenFileCost);
+  RewriteDataFiles filter(Expression expression);
 
   /**
-   * The action result that contains a summary of the execution.
+   * A pairing of file group information to the result of the rewriting that file group. If the results are null then
+   * that particular file group failed. We should only have failed groups if partial progress is enabled otherwise we
+   * will report a total failure for the job.
    */
   interface Result {
+    Map<FileGroupInfo, FileGroupRewriteResult> resultMap();
+  }

Review comment:
       Possibly I missed this in the discussion, but should we return something other than null for the failure? I can imagine that we'd want to possibly return metadata about the failure, possibly trimmed stack traces, or something else specific to an individual compaction strategy.
   
   I prefer error responses over null for errors as null is a lot easier to misinterpret or simply miss entirely and then NPE.

##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Rewrite Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.
  */
-public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+public interface RewriteDataFiles extends Action<RewriteDataFiles, RewriteDataFiles.Result> {
+
   /**
-   * Pass a row filter to filter {@link DataFile}s to be rewritten.
-   * <p>
-   * Note that all files that may contain data matching the filter may be rewritten.
-   * <p>
-   * If not set, all files will be rewritten.
-   *
-   * @param expr a row filter to filter out data files
-   * @return this for method chaining
+   * Enable committing groups of files (see max-file-group-size) prior to the entire compaction completing.
+   * This will produce additional commits but allow for progress even if some groups fail to commit. This setting
+   * will not change the correctness of the rewrite operation. The default is false, which produces a single commit
+   * when the entire job has completed.
    */
-  RewriteDataFiles filter(Expression expr);
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
 
   /**
-   * Enables or disables case sensitive expression binding.
-   * <p>
-   * If not set, defaults to false.
-   *
-   * @param caseSensitive caseSensitive
-   * @return this for method chaining
+   * The maximum amount of Iceberg commits that compaction is allowed to produce if partial progress is enabled.
    */
-  RewriteDataFiles caseSensitive(boolean caseSensitive);
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
 
   /**
-   * Pass a PartitionSpec id to specify which PartitionSpec should be used in DataFile rewrite
-   * <p>
-   * If not set, defaults to the table's default spec ID.
-   *
-   * @param specId PartitionSpec id to rewrite
-   * @return this for method chaining
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size into groups. These sub-units of compaction are referred to as file groups. The largest amount of data that
+   * should be compacted in a single group is controlled by MAX_FILE_GROUP_SIZE_BYTES. When grouping files, the
+   * underlying compaction strategy will use this value as to limit the files which will be included in a single file
+   * group. A group will be processed by a single framework "action". For example, in Spark this means that each group
+   * would be rewritten in its own Spark action. A group will never contain files for multiple output partitions.
    */
-  RewriteDataFiles outputSpecId(int specId);
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";
+  long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
 
   /**
-   * Specify the target data file size in bytes.
-   * <p>
-   * If not set, defaults to the table's target file size.
-   *
-   * @param targetSizeInBytes size in bytes of rewrite data file
-   * @return this for method chaining
+   * The max number of file groups to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the group is determined by the compaction strategy. Each file group will be rewritten
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_FILE_GROUP_ACTIONS = "max-concurrent-file-group-actions";
+  int MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT = 1;
+
+  /**
+   * The output file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.
+   */
+  String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes";
+
+  /**
+   * The partition spec to use when writing the output data from this operation. By default uses the
+   * current table partition spec.
    */
-  RewriteDataFiles targetSizeInBytes(long targetSizeInBytes);
+  String PARTITION_SPEC_ID = "partition-spec-id";
+
+  enum RewriteStrategyName {
+    BINPACK,
+    SORT
+  }
 
   /**
-   * Specify the number of "bins" considered when trying to pack the next file split into a task. Increasing this
-   * usually makes tasks a bit more even by considering more ways to pack file regions into a single task with extra
-   * planning cost.
-   * <p>
-   * This configuration can reorder the incoming file regions, to preserve order for lower/upper bounds in file
-   * metadata, user can use a lookback of 1.
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
    *
-   * @param splitLookback number of "bins" considered when trying to pack the next file split into a task.
+   * @param strategyName name of the strategy

Review comment:
       Maybe "name of the compaction strategy" so that the param definition isn't _exactly_ the param name?
   
   Though I recognize this is partially to appease the linter so might be too nit-picky on my part. 🙂 

##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Rewrite Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.
  */
-public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+public interface RewriteDataFiles extends Action<RewriteDataFiles, RewriteDataFiles.Result> {
+
   /**
-   * Pass a row filter to filter {@link DataFile}s to be rewritten.
-   * <p>
-   * Note that all files that may contain data matching the filter may be rewritten.
-   * <p>
-   * If not set, all files will be rewritten.
-   *
-   * @param expr a row filter to filter out data files
-   * @return this for method chaining
+   * Enable committing groups of files (see max-file-group-size) prior to the entire compaction completing.
+   * This will produce additional commits but allow for progress even if some groups fail to commit. This setting
+   * will not change the correctness of the rewrite operation. The default is false, which produces a single commit
+   * when the entire job has completed.
    */
-  RewriteDataFiles filter(Expression expr);
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
 
   /**
-   * Enables or disables case sensitive expression binding.
-   * <p>
-   * If not set, defaults to false.
-   *
-   * @param caseSensitive caseSensitive
-   * @return this for method chaining
+   * The maximum amount of Iceberg commits that compaction is allowed to produce if partial progress is enabled.
    */
-  RewriteDataFiles caseSensitive(boolean caseSensitive);
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
 
   /**
-   * Pass a PartitionSpec id to specify which PartitionSpec should be used in DataFile rewrite
-   * <p>
-   * If not set, defaults to the table's default spec ID.
-   *
-   * @param specId PartitionSpec id to rewrite
-   * @return this for method chaining
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size into groups. These sub-units of compaction are referred to as file groups. The largest amount of data that
+   * should be compacted in a single group is controlled by MAX_FILE_GROUP_SIZE_BYTES. When grouping files, the
+   * underlying compaction strategy will use this value as to limit the files which will be included in a single file
+   * group. A group will be processed by a single framework "action". For example, in Spark this means that each group
+   * would be rewritten in its own Spark action. A group will never contain files for multiple output partitions.
    */
-  RewriteDataFiles outputSpecId(int specId);
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";
+  long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
 
   /**
-   * Specify the target data file size in bytes.
-   * <p>
-   * If not set, defaults to the table's target file size.
-   *
-   * @param targetSizeInBytes size in bytes of rewrite data file
-   * @return this for method chaining
+   * The max number of file groups to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the group is determined by the compaction strategy. Each file group will be rewritten
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_FILE_GROUP_ACTIONS = "max-concurrent-file-group-actions";
+  int MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT = 1;
+
+  /**
+   * The output file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.
+   */
+  String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes";
+
+  /**
+   * The partition spec to use when writing the output data from this operation. By default uses the
+   * current table partition spec.
    */
-  RewriteDataFiles targetSizeInBytes(long targetSizeInBytes);
+  String PARTITION_SPEC_ID = "partition-spec-id";

Review comment:
       +1 on keeping `partition` in the term. 

##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Rewrite Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.
  */
-public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+public interface RewriteDataFiles extends Action<RewriteDataFiles, RewriteDataFiles.Result> {
+
   /**
-   * Pass a row filter to filter {@link DataFile}s to be rewritten.
-   * <p>
-   * Note that all files that may contain data matching the filter may be rewritten.
-   * <p>
-   * If not set, all files will be rewritten.
-   *
-   * @param expr a row filter to filter out data files
-   * @return this for method chaining
+   * Enable committing groups of files (see max-file-group-size) prior to the entire compaction completing.
+   * This will produce additional commits but allow for progress even if some groups fail to commit. This setting
+   * will not change the correctness of the rewrite operation. The default is false, which produces a single commit
+   * when the entire job has completed.
    */
-  RewriteDataFiles filter(Expression expr);
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
 
   /**
-   * Enables or disables case sensitive expression binding.
-   * <p>
-   * If not set, defaults to false.
-   *
-   * @param caseSensitive caseSensitive
-   * @return this for method chaining
+   * The maximum amount of Iceberg commits that compaction is allowed to produce if partial progress is enabled.
    */
-  RewriteDataFiles caseSensitive(boolean caseSensitive);
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
 
   /**
-   * Pass a PartitionSpec id to specify which PartitionSpec should be used in DataFile rewrite
-   * <p>
-   * If not set, defaults to the table's default spec ID.
-   *
-   * @param specId PartitionSpec id to rewrite
-   * @return this for method chaining
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size into groups. These sub-units of compaction are referred to as file groups. The largest amount of data that
+   * should be compacted in a single group is controlled by MAX_FILE_GROUP_SIZE_BYTES. When grouping files, the
+   * underlying compaction strategy will use this value as to limit the files which will be included in a single file
+   * group. A group will be processed by a single framework "action". For example, in Spark this means that each group
+   * would be rewritten in its own Spark action. A group will never contain files for multiple output partitions.
    */
-  RewriteDataFiles outputSpecId(int specId);
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";
+  long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
 
   /**
-   * Specify the target data file size in bytes.
-   * <p>
-   * If not set, defaults to the table's target file size.
-   *
-   * @param targetSizeInBytes size in bytes of rewrite data file
-   * @return this for method chaining
+   * The max number of file groups to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the group is determined by the compaction strategy. Each file group will be rewritten
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_FILE_GROUP_ACTIONS = "max-concurrent-file-group-actions";
+  int MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT = 1;
+
+  /**
+   * The output file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.
+   */
+  String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes";
+
+  /**
+   * The partition spec to use when writing the output data from this operation. By default uses the
+   * current table partition spec.
    */
-  RewriteDataFiles targetSizeInBytes(long targetSizeInBytes);
+  String PARTITION_SPEC_ID = "partition-spec-id";
+
+  enum RewriteStrategyName {
+    BINPACK,
+    SORT
+  }
 
   /**
-   * Specify the number of "bins" considered when trying to pack the next file split into a task. Increasing this
-   * usually makes tasks a bit more even by considering more ways to pack file regions into a single task with extra
-   * planning cost.
-   * <p>
-   * This configuration can reorder the incoming file regions, to preserve order for lower/upper bounds in file
-   * metadata, user can use a lookback of 1.
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
    *
-   * @param splitLookback number of "bins" considered when trying to pack the next file split into a task.
+   * @param strategyName name of the strategy
    * @return this for method chaining
    */
-  RewriteDataFiles splitLookback(int splitLookback);
+  RewriteDataFiles strategy(RewriteStrategyName strategyName);
 
   /**
-   * Specify the cost of opening a file that will be taken into account during packing files into
-   * bins. If the size of the file is smaller than the cost of opening, then this value will be used
-   * instead of the actual file size.
-   * <p>
-   * If not set, defaults to the table's open file cost.
+   * 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. For example this would be used for providing a
+   * restriction to only run compaction on a specific partition.
    *
-   * @param splitOpenFileCost minimum file size to count to pack into one "bin".
-   * @return this for method chaining
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
    */
-  RewriteDataFiles splitOpenFileCost(long splitOpenFileCost);
+  RewriteDataFiles filter(Expression expression);
 
   /**
-   * The action result that contains a summary of the execution.
+   * A pairing of file group information to the result of the rewriting that file group. If the results are null then

Review comment:
       Nit: Pairing makes it sound like there's only one entry in the map to me. Maybe just "a map"?

##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Rewrite Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.
  */
-public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+public interface RewriteDataFiles extends Action<RewriteDataFiles, RewriteDataFiles.Result> {
+
   /**
-   * Pass a row filter to filter {@link DataFile}s to be rewritten.
-   * <p>
-   * Note that all files that may contain data matching the filter may be rewritten.
-   * <p>
-   * If not set, all files will be rewritten.
-   *
-   * @param expr a row filter to filter out data files
-   * @return this for method chaining
+   * Enable committing groups of files (see max-file-group-size) prior to the entire compaction completing.
+   * This will produce additional commits but allow for progress even if some groups fail to commit. This setting
+   * will not change the correctness of the rewrite operation. The default is false, which produces a single commit
+   * when the entire job has completed.
    */
-  RewriteDataFiles filter(Expression expr);
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
 
   /**
-   * Enables or disables case sensitive expression binding.
-   * <p>
-   * If not set, defaults to false.
-   *
-   * @param caseSensitive caseSensitive
-   * @return this for method chaining
+   * The maximum amount of Iceberg commits that compaction is allowed to produce if partial progress is enabled.
    */
-  RewriteDataFiles caseSensitive(boolean caseSensitive);
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
 
   /**
-   * Pass a PartitionSpec id to specify which PartitionSpec should be used in DataFile rewrite
-   * <p>
-   * If not set, defaults to the table's default spec ID.
-   *
-   * @param specId PartitionSpec id to rewrite
-   * @return this for method chaining
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size into groups. These sub-units of compaction are referred to as file groups. The largest amount of data that
+   * should be compacted in a single group is controlled by MAX_FILE_GROUP_SIZE_BYTES. When grouping files, the
+   * underlying compaction strategy will use this value as to limit the files which will be included in a single file
+   * group. A group will be processed by a single framework "action". For example, in Spark this means that each group
+   * would be rewritten in its own Spark action. A group will never contain files for multiple output partitions.
    */
-  RewriteDataFiles outputSpecId(int specId);
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";
+  long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
 
   /**
-   * Specify the target data file size in bytes.
-   * <p>
-   * If not set, defaults to the table's target file size.
-   *
-   * @param targetSizeInBytes size in bytes of rewrite data file
-   * @return this for method chaining
+   * The max number of file groups to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the group is determined by the compaction strategy. Each file group will be rewritten
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_FILE_GROUP_ACTIONS = "max-concurrent-file-group-actions";
+  int MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT = 1;
+
+  /**
+   * The output file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.

Review comment:
       +1. Aligns with many other similar configs and is more explicit.

##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Rewrite Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.
  */
-public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+public interface RewriteDataFiles extends Action<RewriteDataFiles, RewriteDataFiles.Result> {
+
   /**
-   * Pass a row filter to filter {@link DataFile}s to be rewritten.
-   * <p>
-   * Note that all files that may contain data matching the filter may be rewritten.
-   * <p>
-   * If not set, all files will be rewritten.
-   *
-   * @param expr a row filter to filter out data files
-   * @return this for method chaining
+   * Enable committing groups of files (see max-file-group-size) prior to the entire compaction completing.
+   * This will produce additional commits but allow for progress even if some groups fail to commit. This setting
+   * will not change the correctness of the rewrite operation. The default is false, which produces a single commit
+   * when the entire job has completed.
    */
-  RewriteDataFiles filter(Expression expr);
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
 
   /**
-   * Enables or disables case sensitive expression binding.
-   * <p>
-   * If not set, defaults to false.
-   *
-   * @param caseSensitive caseSensitive
-   * @return this for method chaining
+   * The maximum amount of Iceberg commits that compaction is allowed to produce if partial progress is enabled.
    */
-  RewriteDataFiles caseSensitive(boolean caseSensitive);
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
 
   /**
-   * Pass a PartitionSpec id to specify which PartitionSpec should be used in DataFile rewrite
-   * <p>
-   * If not set, defaults to the table's default spec ID.
-   *
-   * @param specId PartitionSpec id to rewrite
-   * @return this for method chaining
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size into groups. These sub-units of compaction are referred to as file groups. The largest amount of data that
+   * should be compacted in a single group is controlled by MAX_FILE_GROUP_SIZE_BYTES. When grouping files, the
+   * underlying compaction strategy will use this value as to limit the files which will be included in a single file
+   * group. A group will be processed by a single framework "action". For example, in Spark this means that each group
+   * would be rewritten in its own Spark action. A group will never contain files for multiple output partitions.
    */
-  RewriteDataFiles outputSpecId(int specId);
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";
+  long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
 
   /**
-   * Specify the target data file size in bytes.
-   * <p>
-   * If not set, defaults to the table's target file size.
-   *
-   * @param targetSizeInBytes size in bytes of rewrite data file
-   * @return this for method chaining
+   * The max number of file groups to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the group is determined by the compaction strategy. Each file group will be rewritten
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_FILE_GROUP_ACTIONS = "max-concurrent-file-group-actions";
+  int MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT = 1;
+
+  /**
+   * The output file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.
+   */
+  String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes";
+
+  /**
+   * The partition spec to use when writing the output data from this operation. By default uses the
+   * current table partition spec.
    */
-  RewriteDataFiles targetSizeInBytes(long targetSizeInBytes);
+  String PARTITION_SPEC_ID = "partition-spec-id";
+
+  enum RewriteStrategyName {
+    BINPACK,
+    SORT
+  }
 
   /**
-   * Specify the number of "bins" considered when trying to pack the next file split into a task. Increasing this
-   * usually makes tasks a bit more even by considering more ways to pack file regions into a single task with extra
-   * planning cost.
-   * <p>
-   * This configuration can reorder the incoming file regions, to preserve order for lower/upper bounds in file
-   * metadata, user can use a lookback of 1.
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
    *
-   * @param splitLookback number of "bins" considered when trying to pack the next file split into a task.
+   * @param strategyName name of the strategy
    * @return this for method chaining
    */
-  RewriteDataFiles splitLookback(int splitLookback);
+  RewriteDataFiles strategy(RewriteStrategyName strategyName);
 
   /**
-   * Specify the cost of opening a file that will be taken into account during packing files into
-   * bins. If the size of the file is smaller than the cost of opening, then this value will be used
-   * instead of the actual file size.
-   * <p>
-   * If not set, defaults to the table's open file cost.
+   * 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. For example this would be used for providing a
+   * restriction to only run compaction on a specific partition.
    *
-   * @param splitOpenFileCost minimum file size to count to pack into one "bin".
-   * @return this for method chaining
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
    */
-  RewriteDataFiles splitOpenFileCost(long splitOpenFileCost);
+  RewriteDataFiles filter(Expression expression);
 
   /**
-   * The action result that contains a summary of the execution.
+   * A pairing of file group information to the result of the rewriting that file group. If the results are null then
+   * that particular file group failed. We should only have failed groups if partial progress is enabled otherwise we
+   * will report a total failure for the job.
    */
   interface Result {
+    Map<FileGroupInfo, FileGroupRewriteResult> resultMap();
+  }
+
+  interface FileGroupRewriteResult {
+    int addedDataFilesCount();
+
+    int rewrittenDataFilesCount();
+  }
+
+  /**
+   * A description of a file group, when it was processed, and within which partition. For use
+   * tracking rewrite operations and for returning results.
+   */
+  interface FileGroupInfo {
+
+    /**
+     * returns which file group this is out of the total set of file groups for this compaction
+     */
+    int globalIndex();

Review comment:
       Correct me if I'm wrong, but I think that since this is `FileGroupInfo` and not the Result, the indexes are likely included so the engine assign and track work etc as opposed to purely for tracking on output. 

##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Rewrite Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.
  */
-public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+public interface RewriteDataFiles extends Action<RewriteDataFiles, RewriteDataFiles.Result> {
+
   /**
-   * Pass a row filter to filter {@link DataFile}s to be rewritten.
-   * <p>
-   * Note that all files that may contain data matching the filter may be rewritten.
-   * <p>
-   * If not set, all files will be rewritten.
-   *
-   * @param expr a row filter to filter out data files
-   * @return this for method chaining
+   * Enable committing groups of files (see max-file-group-size) prior to the entire compaction completing.
+   * This will produce additional commits but allow for progress even if some groups fail to commit. This setting
+   * will not change the correctness of the rewrite operation. The default is false, which produces a single commit
+   * when the entire job has completed.
    */
-  RewriteDataFiles filter(Expression expr);
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
 
   /**
-   * Enables or disables case sensitive expression binding.
-   * <p>
-   * If not set, defaults to false.
-   *
-   * @param caseSensitive caseSensitive
-   * @return this for method chaining
+   * The maximum amount of Iceberg commits that compaction is allowed to produce if partial progress is enabled.
    */
-  RewriteDataFiles caseSensitive(boolean caseSensitive);
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
 
   /**
-   * Pass a PartitionSpec id to specify which PartitionSpec should be used in DataFile rewrite
-   * <p>
-   * If not set, defaults to the table's default spec ID.
-   *
-   * @param specId PartitionSpec id to rewrite
-   * @return this for method chaining
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size into groups. These sub-units of compaction are referred to as file groups. The largest amount of data that
+   * should be compacted in a single group is controlled by MAX_FILE_GROUP_SIZE_BYTES. When grouping files, the
+   * underlying compaction strategy will use this value as to limit the files which will be included in a single file
+   * group. A group will be processed by a single framework "action". For example, in Spark this means that each group
+   * would be rewritten in its own Spark action. A group will never contain files for multiple output partitions.
    */
-  RewriteDataFiles outputSpecId(int specId);
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";
+  long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
 
   /**
-   * Specify the target data file size in bytes.
-   * <p>
-   * If not set, defaults to the table's target file size.
-   *
-   * @param targetSizeInBytes size in bytes of rewrite data file
-   * @return this for method chaining
+   * The max number of file groups to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the group is determined by the compaction strategy. Each file group will be rewritten
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_FILE_GROUP_ACTIONS = "max-concurrent-file-group-actions";
+  int MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT = 1;
+
+  /**
+   * The output file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.
+   */
+  String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes";
+
+  /**
+   * The partition spec to use when writing the output data from this operation. By default uses the
+   * current table partition spec.
    */
-  RewriteDataFiles targetSizeInBytes(long targetSizeInBytes);
+  String PARTITION_SPEC_ID = "partition-spec-id";
+
+  enum RewriteStrategyName {
+    BINPACK,
+    SORT
+  }
 
   /**
-   * Specify the number of "bins" considered when trying to pack the next file split into a task. Increasing this
-   * usually makes tasks a bit more even by considering more ways to pack file regions into a single task with extra
-   * planning cost.
-   * <p>
-   * This configuration can reorder the incoming file regions, to preserve order for lower/upper bounds in file
-   * metadata, user can use a lookback of 1.
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.

Review comment:
       Is the `currently only support options BINPACK and SORT options` comment necessary?. To me, what's supported is a natural extension of the enum. And then if we declare an enum that's not implemented, we can specify that. Feels like less to maintain / less likely to have the comments drift from the reality, but I'd like to hear what others have to say (and it might be a non-issue).

##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Rewrite Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.
  */
-public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+public interface RewriteDataFiles extends Action<RewriteDataFiles, RewriteDataFiles.Result> {
+
   /**
-   * Pass a row filter to filter {@link DataFile}s to be rewritten.
-   * <p>
-   * Note that all files that may contain data matching the filter may be rewritten.
-   * <p>
-   * If not set, all files will be rewritten.
-   *
-   * @param expr a row filter to filter out data files
-   * @return this for method chaining
+   * Enable committing groups of files (see max-file-group-size) prior to the entire compaction completing.
+   * This will produce additional commits but allow for progress even if some groups fail to commit. This setting
+   * will not change the correctness of the rewrite operation. The default is false, which produces a single commit
+   * when the entire job has completed.
    */
-  RewriteDataFiles filter(Expression expr);
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
 
   /**
-   * Enables or disables case sensitive expression binding.
-   * <p>
-   * If not set, defaults to false.
-   *
-   * @param caseSensitive caseSensitive
-   * @return this for method chaining
+   * The maximum amount of Iceberg commits that compaction is allowed to produce if partial progress is enabled.
    */
-  RewriteDataFiles caseSensitive(boolean caseSensitive);
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
 
   /**
-   * Pass a PartitionSpec id to specify which PartitionSpec should be used in DataFile rewrite
-   * <p>
-   * If not set, defaults to the table's default spec ID.
-   *
-   * @param specId PartitionSpec id to rewrite
-   * @return this for method chaining
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size into groups. These sub-units of compaction are referred to as file groups. The largest amount of data that
+   * should be compacted in a single group is controlled by MAX_FILE_GROUP_SIZE_BYTES. When grouping files, the
+   * underlying compaction strategy will use this value as to limit the files which will be included in a single file
+   * group. A group will be processed by a single framework "action". For example, in Spark this means that each group
+   * would be rewritten in its own Spark action. A group will never contain files for multiple output partitions.
    */
-  RewriteDataFiles outputSpecId(int specId);
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";
+  long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
 
   /**
-   * Specify the target data file size in bytes.
-   * <p>
-   * If not set, defaults to the table's target file size.
-   *
-   * @param targetSizeInBytes size in bytes of rewrite data file
-   * @return this for method chaining
+   * The max number of file groups to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the group is determined by the compaction strategy. Each file group will be rewritten
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_FILE_GROUP_ACTIONS = "max-concurrent-file-group-actions";
+  int MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT = 1;
+
+  /**
+   * The output file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.
+   */
+  String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes";
+
+  /**
+   * The partition spec to use when writing the output data from this operation. By default uses the
+   * current table partition spec.
    */
-  RewriteDataFiles targetSizeInBytes(long targetSizeInBytes);
+  String PARTITION_SPEC_ID = "partition-spec-id";
+
+  enum RewriteStrategyName {
+    BINPACK,
+    SORT
+  }
 
   /**
-   * Specify the number of "bins" considered when trying to pack the next file split into a task. Increasing this
-   * usually makes tasks a bit more even by considering more ways to pack file regions into a single task with extra
-   * planning cost.
-   * <p>
-   * This configuration can reorder the incoming file regions, to preserve order for lower/upper bounds in file
-   * metadata, user can use a lookback of 1.
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
    *
-   * @param splitLookback number of "bins" considered when trying to pack the next file split into a task.
+   * @param strategyName name of the strategy
    * @return this for method chaining
    */
-  RewriteDataFiles splitLookback(int splitLookback);
+  RewriteDataFiles strategy(RewriteStrategyName strategyName);
 
   /**
-   * Specify the cost of opening a file that will be taken into account during packing files into
-   * bins. If the size of the file is smaller than the cost of opening, then this value will be used
-   * instead of the actual file size.
-   * <p>
-   * If not set, defaults to the table's open file cost.
+   * 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. For example this would be used for providing a
+   * restriction to only run compaction on a specific partition.
    *
-   * @param splitOpenFileCost minimum file size to count to pack into one "bin".
-   * @return this for method chaining
+   * @param expression only entries that pass this filter will be compacted

Review comment:
       Nit: Possibly start the param definition with the noun or focus on the expression rather than the result? Like `Filter expression used to select entries for compaction` or something? Reads a little funny to me but I might not have had enough coffee today. 🙂 




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r627077406



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Rewrite Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.
  */
-public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+public interface RewriteDataFiles extends Action<RewriteDataFiles, RewriteDataFiles.Result> {
+
   /**
-   * Pass a row filter to filter {@link DataFile}s to be rewritten.
-   * <p>
-   * Note that all files that may contain data matching the filter may be rewritten.
-   * <p>
-   * If not set, all files will be rewritten.
-   *
-   * @param expr a row filter to filter out data files
-   * @return this for method chaining
+   * Enable committing groups of files (see max-file-group-size) prior to the entire compaction completing.
+   * This will produce additional commits but allow for progress even if some groups fail to commit. This setting
+   * will not change the correctness of the rewrite operation. The default is false, which produces a single commit
+   * when the entire job has completed.
    */
-  RewriteDataFiles filter(Expression expr);
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
 
   /**
-   * Enables or disables case sensitive expression binding.
-   * <p>
-   * If not set, defaults to false.
-   *
-   * @param caseSensitive caseSensitive
-   * @return this for method chaining
+   * The maximum amount of Iceberg commits that compaction is allowed to produce if partial progress is enabled.
    */
-  RewriteDataFiles caseSensitive(boolean caseSensitive);
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
 
   /**
-   * Pass a PartitionSpec id to specify which PartitionSpec should be used in DataFile rewrite
-   * <p>
-   * If not set, defaults to the table's default spec ID.
-   *
-   * @param specId PartitionSpec id to rewrite
-   * @return this for method chaining
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size into groups. These sub-units of compaction are referred to as file groups. The largest amount of data that
+   * should be compacted in a single group is controlled by MAX_FILE_GROUP_SIZE_BYTES. When grouping files, the
+   * underlying compaction strategy will use this value as to limit the files which will be included in a single file
+   * group. A group will be processed by a single framework "action". For example, in Spark this means that each group
+   * would be rewritten in its own Spark action. A group will never contain files for multiple output partitions.
    */
-  RewriteDataFiles outputSpecId(int specId);
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";
+  long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
 
   /**
-   * Specify the target data file size in bytes.
-   * <p>
-   * If not set, defaults to the table's target file size.
-   *
-   * @param targetSizeInBytes size in bytes of rewrite data file
-   * @return this for method chaining
+   * The max number of file groups to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the group is determined by the compaction strategy. Each file group will be rewritten
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_FILE_GROUP_ACTIONS = "max-concurrent-file-group-actions";
+  int MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT = 1;
+
+  /**
+   * The output file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.
+   */
+  String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes";
+
+  /**
+   * The partition spec to use when writing the output data from this operation. By default uses the
+   * current table partition spec.
    */
-  RewriteDataFiles targetSizeInBytes(long targetSizeInBytes);
+  String PARTITION_SPEC_ID = "partition-spec-id";
+
+  enum RewriteStrategyName {

Review comment:
       strategy it is. This does make the next method call "RewriteDataFiles strategy(Strategy strategy)"




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617842650



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;

Review comment:
       I feel like this will just be a real trap, kind of like our default expire snapshots 'older-than' value which trips up every first user who starts by just seeing if it does anything but it doesn't. I feel like we are always answering questions about that.
   
   I don't know if there is a best in most cases value here, but I still think 1 is the least surprising default.




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r627062730



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Rewrite Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.
  */
-public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+public interface RewriteDataFiles extends Action<RewriteDataFiles, RewriteDataFiles.Result> {
+
   /**
-   * Pass a row filter to filter {@link DataFile}s to be rewritten.
-   * <p>
-   * Note that all files that may contain data matching the filter may be rewritten.
-   * <p>
-   * If not set, all files will be rewritten.
-   *
-   * @param expr a row filter to filter out data files
-   * @return this for method chaining
+   * Enable committing groups of files (see max-file-group-size) prior to the entire compaction completing.
+   * This will produce additional commits but allow for progress even if some groups fail to commit. This setting
+   * will not change the correctness of the rewrite operation. The default is false, which produces a single commit
+   * when the entire job has completed.
    */
-  RewriteDataFiles filter(Expression expr);
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
 
   /**
-   * Enables or disables case sensitive expression binding.
-   * <p>
-   * If not set, defaults to false.
-   *
-   * @param caseSensitive caseSensitive
-   * @return this for method chaining
+   * The maximum amount of Iceberg commits that compaction is allowed to produce if partial progress is enabled.
    */
-  RewriteDataFiles caseSensitive(boolean caseSensitive);
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
 
   /**
-   * Pass a PartitionSpec id to specify which PartitionSpec should be used in DataFile rewrite
-   * <p>
-   * If not set, defaults to the table's default spec ID.
-   *
-   * @param specId PartitionSpec id to rewrite
-   * @return this for method chaining
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size into groups. These sub-units of compaction are referred to as file groups. The largest amount of data that
+   * should be compacted in a single group is controlled by MAX_FILE_GROUP_SIZE_BYTES. When grouping files, the
+   * underlying compaction strategy will use this value as to limit the files which will be included in a single file
+   * group. A group will be processed by a single framework "action". For example, in Spark this means that each group
+   * would be rewritten in its own Spark action. A group will never contain files for multiple output partitions.
    */
-  RewriteDataFiles outputSpecId(int specId);
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";
+  long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
 
   /**
-   * Specify the target data file size in bytes.
-   * <p>
-   * If not set, defaults to the table's target file size.
-   *
-   * @param targetSizeInBytes size in bytes of rewrite data file
-   * @return this for method chaining
+   * The max number of file groups to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the group is determined by the compaction strategy. Each file group will be rewritten
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_FILE_GROUP_ACTIONS = "max-concurrent-file-group-actions";
+  int MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT = 1;
+
+  /**
+   * The output file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.
+   */
+  String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes";
+
+  /**
+   * The partition spec to use when writing the output data from this operation. By default uses the
+   * current table partition spec.
    */
-  RewriteDataFiles targetSizeInBytes(long targetSizeInBytes);
+  String PARTITION_SPEC_ID = "partition-spec-id";
+
+  enum RewriteStrategyName {
+    BINPACK,
+    SORT

Review comment:
       Let's not include sort for now. We will start with bin-packing for now.




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r627029787



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Rewrite Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.
  */
-public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+public interface RewriteDataFiles extends Action<RewriteDataFiles, RewriteDataFiles.Result> {

Review comment:
       Shouldn't it extend `SnapshotUpdate` as this action produces snapshots and may associate key/value pairs?




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r619464875



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.expressions.Expression;
+
+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();

Review comment:
       I think discarding calls with invalid options will help. If I set an option, I expect it to have an effect.




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617753663



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;
+
+  /**
+   * The parallelism level to use when processing chunks generated by the Compaction Strategy. The structure and
+   * contents of the chunk 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 this for chaining
+   */
+  CompactDataFiles parallelism(int par);
+
+  /**
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
+   *
+   * @param strategyName name of the strategy
+   * @return this for chaining
+   */
+  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. For example this would be used for providing a
+   * restriction to only run compaction on a specific partition.
+   *
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
+   */
+  CompactDataFiles filter(Expression expression);
+
+  /**
+   * A pairing of Chunk information to the result of that chunk's results. If the results are null then that particular
+   * chunk failed. We should only have failed chunks if partial progress is enabled.
+   */
+  class Result {
+    private Map<CompactionChunkInfo, CompactionChunkResult> resultMap;
+
+
+    public Result(Map<CompactionChunkInfo, CompactionChunkResult> resultMap) {
+      this.resultMap = resultMap;
+    }
+
+    public Map<CompactionChunkInfo, CompactionChunkResult> resultMap() {
+      return resultMap;
+    }
+  }
+
+  interface CompactionChunkResult {
+
+    int numFilesAdded();
+
+    int numFilesRemoved();
+  }
+
+  interface CompactionChunkInfo {
+
+    /**
+     * returns which chunk this is out of the total set of chunks for this compaction
+     */
+    int chunkIndex();

Review comment:
       This one is global, the next one is within the partition




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617953616



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;
+
+  /**
+   * The parallelism level to use when processing chunks generated by the Compaction Strategy. The structure and
+   * contents of the chunk 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 this for chaining
+   */
+  CompactDataFiles parallelism(int par);

Review comment:
       @rdblue, any ideas on naming?




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


[GitHub] [iceberg] aokolnychyi commented on pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#issuecomment-839360481


   Thanks for reviewing, @stevenzwu @rdblue @kbendick @yyanyy @chenjunjiedada @jackye1995!


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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r626030854



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;
+
+  /**
+   * The parallelism level to use when processing chunks generated by the Compaction Strategy. The structure and
+   * contents of the chunk 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 this for chaining
+   */
+  CompactDataFiles parallelism(int par);
+
+  /**
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
+   *
+   * @param strategyName name of the strategy
+   * @return this for chaining
+   */
+  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. For example this would be used for providing a
+   * restriction to only run compaction on a specific partition.
+   *
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
+   */
+  CompactDataFiles filter(Expression expression);
+
+  /**
+   * A pairing of Chunk information to the result of that chunk's results. If the results are null then that particular
+   * chunk failed. We should only have failed chunks if partial progress is enabled.
+   */
+  class Result {
+    private Map<CompactionChunkInfo, CompactionChunkResult> resultMap;
+
+
+    public Result(Map<CompactionChunkInfo, CompactionChunkResult> resultMap) {
+      this.resultMap = resultMap;
+    }
+
+    public Map<CompactionChunkInfo, CompactionChunkResult> resultMap() {
+      return resultMap;
+    }
+  }
+
+  interface CompactionChunkResult {
+
+    int numFilesAdded();
+
+    int numFilesRemoved();
+  }
+
+  interface CompactionChunkInfo {
+
+    /**
+     * returns which chunk this is out of the total set of chunks for this compaction
+     */
+    int chunkIndex();
+
+    /**
+     * returns which chunk this is out of the set of chunks for this partition
+     */
+    int partitionIndex();

Review comment:
       I'd like to be logging like
   
   Log (File group 1 of Total amount, File group 1 of X for this partition)
   
   Then the procedure results look like
   
   | FileGroup | Partition | Partition Index | Files Added | Files Rewritten |
   |-----------|---------|----------------|-------------|---------------|
   | 1                | [1996]   |   1                       | 5                  |.           10          |
   
   Or something like that




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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r626221355



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+
+interface DataCompactionStrategy extends Serializable {

Review comment:
       Do we use both `rewrite` and `compaction`? 




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


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r623563761



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+
+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();
+
+  DataCompactionStrategy withOptions(Map<String, String> options);
+
+  /**
+   * 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.
+   *
+   * @param dataFiles iterator of live datafiles in a given partition
+   * @return iterator containing only files to be rewritten
+   */
+  Iterable<FileScanTask> selectFilesToCompact(Iterable<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
+   */
+  Iterable<List<FileScanTask>> groupFilesIntoChunks(Iterable<FileScanTask> dataFiles);

Review comment:
       Why not use `Set`?




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r626249099



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Compaction Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.
  */
-public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+public interface RewriteDataFiles extends Action<RewriteDataFiles, RewriteDataFiles.Result> {
+
   /**
-   * Pass a row filter to filter {@link DataFile}s to be rewritten.
-   * <p>
-   * Note that all files that may contain data matching the filter may be rewritten.
-   * <p>
-   * If not set, all files will be rewritten.
-   *
-   * @param expr a row filter to filter out data files
-   * @return this for method chaining
+   * Enable committing groups of files (see max-file-group-size) prior to the entire compaction completing.
+   * This will produce additional commits but allow for progress even if some groups fail to commit. This setting
+   * will not change the correctness of the rewrite operation. The default is false, which produces a single commit
+   * when the entire job has completed.
    */
-  RewriteDataFiles filter(Expression expr);
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
 
   /**
-   * Enables or disables case sensitive expression binding.
-   * <p>
-   * If not set, defaults to false.
-   *
-   * @param caseSensitive caseSensitive
-   * @return this for method chaining
+   * The maximum amount of Iceberg commits that compaction is allowed to produce if partial progress is enabled.
    */
-  RewriteDataFiles caseSensitive(boolean caseSensitive);
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
 
   /**
-   * Pass a PartitionSpec id to specify which PartitionSpec should be used in DataFile rewrite
-   * <p>
-   * If not set, defaults to the table's default spec ID.
-   *
-   * @param specId PartitionSpec id to rewrite
-   * @return this for method chaining
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size into groups. These sub-units of compaction are referred to as file groups. The largest amount of data that
+   * should be compacted in a single group is controlled by MAX_FILE_GROUP_SIZE_BYTES. When grouping files, the
+   * underlying compaction strategy will use this value as to limit the files which will be included in a single file
+   * group. A group will be processed by a single framework "action". For example, in Spark this means that each group
+   * would be rewritten in its own Spark action. A group will never contain files for multiple output partitions.
    */
-  RewriteDataFiles outputSpecId(int specId);
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";
+  long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
 
   /**
-   * Specify the target data file size in bytes.
-   * <p>
-   * If not set, defaults to the table's target file size.
-   *
-   * @param targetSizeInBytes size in bytes of rewrite data file
-   * @return this for method chaining
+   * The max number of file groups to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the group is determined by the compaction strategy. Each file group will be rewritten
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_FILE_GROUP_ACTIONS = "max-concurrent-file-group-actions";
+  int MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT = 1;
+
+  /**
+   * The output file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.
+   */
+  String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes";
+
+  /**
+   * The partition spec to use when writing the output data from this operation. By default uses the
+   * current table partition spec.
    */
-  RewriteDataFiles targetSizeInBytes(long targetSizeInBytes);
+  String PARTITION_SPEC_ID = "partition-spec-id";
+
+  enum CompactionStrategyName {

Review comment:
       Sure we have been doing a lot of renaming




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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r626719643



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+
+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();
+
+  DataCompactionStrategy withOptions(Map<String, String> options);
+
+  /**
+   * Selects files which this strategy believes are valid targets to be rewritten.
+   *
+   * @param dataFiles iterator of live datafiles in a given partition
+   * @return iterator containing only files to be rewritten
+   */
+  Iterable<FileScanTask> selectFilesToCompact(Iterable<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
+   */
+  Iterable<Set<FileScanTask>> groupFilesIntoChunks(Iterable<FileScanTask> dataFiles);

Review comment:
       ok. I just thought Iceberg already had an interface `CombinedScanTask` to capture of a group of files. Reuse it might make make sense in the `api` module.




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617688545



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";

Review comment:
       Iceberg uses `-` instead of `_` in table props and read/write options. Shall we follow that here too?

##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;
+
+  /**
+   * The parallelism level to use when processing chunks generated by the Compaction Strategy. The structure and
+   * contents of the chunk 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 this for chaining
+   */
+  CompactDataFiles parallelism(int par);
+
+  /**
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
+   *
+   * @param strategyName name of the strategy
+   * @return this for chaining
+   */
+  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. For example this would be used for providing a
+   * restriction to only run compaction on a specific partition.
+   *
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
+   */
+  CompactDataFiles filter(Expression expression);
+
+  /**
+   * A pairing of Chunk information to the result of that chunk's results. If the results are null then that particular
+   * chunk failed. We should only have failed chunks if partial progress is enabled.
+   */
+  class Result {

Review comment:
       I think we either have to switch to classes in all actions or continue to use interfaces.
   
   Maybe, having result implementations in `core` is an overkill and we can move them to actions directly. I just took a look, I think that will be alright. Probably, we won't need engine-specific results.
   
   Thoughts, @RussellSpitzer @rdblue @openinx?

##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {

Review comment:
       Do we want to offer this under a new name? I thought this would swap the `RewriteDataFiles` implementation in the new actions API. Right now, `RewriteDataFiles` from the new API is not implemented so we don't have to update any other place, can just move the new logic there.
   
   Plus, I tend to think `rewrite` fits slightly better as we not necessarily compact. We can split files or recluster them, for example. 

##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";

Review comment:
       These may be specific to bin-packing. Shall we wait a bit until we know how the sort-based compaction is going to look like? We can always add them here.

##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes

Review comment:
       nit: `...SIZE_BYTES_DEFAULT`?

##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.

Review comment:
       Shall we mention the default?

##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;

Review comment:
       I'd pick the default value based on what would work best in most cases as we are trying to minimize the input from the user. I know we set this as 1 due to a concern that a rewrite may be noop otherwise. However, is there any realistic use case where users may want to always compact 2 tiny files? If the user would need to always change this value, I'd pick another default value.

##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;
+
+  /**
+   * The parallelism level to use when processing chunks generated by the Compaction Strategy. The structure and
+   * contents of the chunk 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 this for chaining
+   */
+  CompactDataFiles parallelism(int par);
+
+  /**
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
+   *
+   * @param strategyName name of the strategy
+   * @return this for chaining
+   */
+  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. For example this would be used for providing a
+   * restriction to only run compaction on a specific partition.
+   *
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
+   */
+  CompactDataFiles filter(Expression expression);
+
+  /**
+   * A pairing of Chunk information to the result of that chunk's results. If the results are null then that particular
+   * chunk failed. We should only have failed chunks if partial progress is enabled.
+   */
+  class Result {
+    private Map<CompactionChunkInfo, CompactionChunkResult> resultMap;
+
+
+    public Result(Map<CompactionChunkInfo, CompactionChunkResult> resultMap) {
+      this.resultMap = resultMap;
+    }
+
+    public Map<CompactionChunkInfo, CompactionChunkResult> resultMap() {
+      return resultMap;
+    }
+  }
+
+  interface CompactionChunkResult {
+
+    int numFilesAdded();
+
+    int numFilesRemoved();
+  }
+
+  interface CompactionChunkInfo {
+
+    /**
+     * returns which chunk this is out of the total set of chunks for this compaction
+     */
+    int chunkIndex();

Review comment:
       Is it global or per partition?

##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";

Review comment:
       `TARGET_FILE_SIZE_BYTES`?

##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;
+
+  /**
+   * The parallelism level to use when processing chunks generated by the Compaction Strategy. The structure and
+   * contents of the chunk 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 this for chaining
+   */
+  CompactDataFiles parallelism(int par);
+
+  /**
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
+   *
+   * @param strategyName name of the strategy
+   * @return this for chaining
+   */
+  CompactDataFiles compactionStrategy(String strategyName);

Review comment:
       `compactionStrategy` -> `strategy`?

##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;
+
+  /**
+   * The parallelism level to use when processing chunks generated by the Compaction Strategy. The structure and
+   * contents of the chunk 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 this for chaining
+   */
+  CompactDataFiles parallelism(int par);
+
+  /**
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
+   *
+   * @param strategyName name of the strategy
+   * @return this for chaining
+   */
+  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. For example this would be used for providing a
+   * restriction to only run compaction on a specific partition.
+   *
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
+   */
+  CompactDataFiles filter(Expression expression);
+
+  /**
+   * A pairing of Chunk information to the result of that chunk's results. If the results are null then that particular
+   * chunk failed. We should only have failed chunks if partial progress is enabled.
+   */
+  class Result {
+    private Map<CompactionChunkInfo, CompactionChunkResult> resultMap;
+
+
+    public Result(Map<CompactionChunkInfo, CompactionChunkResult> resultMap) {
+      this.resultMap = resultMap;
+    }
+
+    public Map<CompactionChunkInfo, CompactionChunkResult> resultMap() {

Review comment:
       `chunkResults` or something?

##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;
+
+  /**
+   * The parallelism level to use when processing chunks generated by the Compaction Strategy. The structure and
+   * contents of the chunk 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 this for chaining
+   */
+  CompactDataFiles parallelism(int par);
+
+  /**
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
+   *
+   * @param strategyName name of the strategy
+   * @return this for chaining
+   */
+  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. For example this would be used for providing a
+   * restriction to only run compaction on a specific partition.
+   *
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
+   */
+  CompactDataFiles filter(Expression expression);
+
+  /**
+   * A pairing of Chunk information to the result of that chunk's results. If the results are null then that particular
+   * chunk failed. We should only have failed chunks if partial progress is enabled.
+   */
+  class Result {
+    private Map<CompactionChunkInfo, CompactionChunkResult> resultMap;
+
+
+    public Result(Map<CompactionChunkInfo, CompactionChunkResult> resultMap) {
+      this.resultMap = resultMap;
+    }
+
+    public Map<CompactionChunkInfo, CompactionChunkResult> resultMap() {
+      return resultMap;
+    }
+  }
+
+  interface CompactionChunkResult {
+
+    int numFilesAdded();
+
+    int numFilesRemoved();
+  }
+
+  interface CompactionChunkInfo {
+
+    /**
+     * returns which chunk this is out of the total set of chunks for this compaction
+     */
+    int chunkIndex();
+
+    /**
+     * returns which chunk this is out of the set of chunks for this partition
+     */
+    int partitionIndex();
+
+    /**
+     * returns which partition this chunk contains files from
+     */
+    String partition();

Review comment:
       Should it be `StructLike` instead? Or how will we derive the string?

##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;
+
+  /**
+   * The parallelism level to use when processing chunks generated by the Compaction Strategy. The structure and
+   * contents of the chunk 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 this for chaining
+   */
+  CompactDataFiles parallelism(int par);
+
+  /**
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
+   *
+   * @param strategyName name of the strategy
+   * @return this for chaining
+   */
+  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. For example this would be used for providing a
+   * restriction to only run compaction on a specific partition.
+   *
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
+   */
+  CompactDataFiles filter(Expression expression);
+
+  /**
+   * A pairing of Chunk information to the result of that chunk's results. If the results are null then that particular
+   * chunk failed. We should only have failed chunks if partial progress is enabled.
+   */
+  class Result {
+    private Map<CompactionChunkInfo, CompactionChunkResult> resultMap;
+
+

Review comment:
       nit: extra line

##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;
+
+  /**
+   * The parallelism level to use when processing chunks generated by the Compaction Strategy. The structure and
+   * contents of the chunk 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 this for chaining
+   */
+  CompactDataFiles parallelism(int par);
+
+  /**
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
+   *
+   * @param strategyName name of the strategy
+   * @return this for chaining
+   */
+  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. For example this would be used for providing a
+   * restriction to only run compaction on a specific partition.
+   *
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
+   */
+  CompactDataFiles filter(Expression expression);
+
+  /**
+   * A pairing of Chunk information to the result of that chunk's results. If the results are null then that particular
+   * chunk failed. We should only have failed chunks if partial progress is enabled.
+   */
+  class Result {
+    private Map<CompactionChunkInfo, CompactionChunkResult> resultMap;
+
+
+    public Result(Map<CompactionChunkInfo, CompactionChunkResult> resultMap) {
+      this.resultMap = resultMap;
+    }
+
+    public Map<CompactionChunkInfo, CompactionChunkResult> resultMap() {
+      return resultMap;
+    }
+  }
+
+  interface CompactionChunkResult {
+
+    int numFilesAdded();
+
+    int numFilesRemoved();

Review comment:
       I think we use `xxxCount` frequently. Shall it be `rewrittenDataFilesCount`?

##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.expressions.Expression;
+
+public interface DataCompactionStrategy extends Serializable {

Review comment:
       Do we want to make it public in the first iteration? We could make it non-public in `core` to start with.

##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/BinPack.java
##########
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+public interface BinPack {

Review comment:
       Would it make sense to add this one once we have a bit more clarity on the implementation? Right now, it is only used in the default compaction strategy constant. We can probably add it later and focus on the common API for now.
   
   Just asking.

##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;
+
+  /**
+   * The parallelism level to use when processing chunks generated by the Compaction Strategy. The structure and
+   * contents of the chunk 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 this for chaining
+   */
+  CompactDataFiles parallelism(int par);
+
+  /**
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
+   *
+   * @param strategyName name of the strategy
+   * @return this for chaining
+   */
+  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. For example this would be used for providing a
+   * restriction to only run compaction on a specific partition.
+   *
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
+   */
+  CompactDataFiles filter(Expression expression);
+
+  /**
+   * A pairing of Chunk information to the result of that chunk's results. If the results are null then that particular
+   * chunk failed. We should only have failed chunks if partial progress is enabled.
+   */
+  class Result {
+    private Map<CompactionChunkInfo, CompactionChunkResult> resultMap;
+
+
+    public Result(Map<CompactionChunkInfo, CompactionChunkResult> resultMap) {
+      this.resultMap = resultMap;
+    }
+
+    public Map<CompactionChunkInfo, CompactionChunkResult> resultMap() {
+      return resultMap;
+    }
+  }
+
+  interface CompactionChunkResult {
+
+    int numFilesAdded();

Review comment:
       `addedDataFilesCount`?

##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.expressions.Expression;
+
+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();
+
+  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. This would be used for pushing down filter expressions to the manifest entry scanning phase.
+   *
+   * @return an Iceberg expression to use when discovering file scan tasks
+   */
+  Expression preFilter();
+
+  /**
+   * 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);

Review comment:
       Do we think working with `Iterator` would be easier than with `Iterable`? Just out of curiosity.

##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.expressions.Expression;
+
+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();
+
+  DataCompactionStrategy withOptions(Map<String, String> options);

Review comment:
       Do we need a method in the action to pass the options? I assume the user won't be instantiating the strategy for now.

##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.expressions.Expression;
+
+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();
+
+  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. This would be used for pushing down filter expressions to the manifest entry scanning phase.
+   *
+   * @return an Iceberg expression to use when discovering file scan tasks
+   */
+  Expression preFilter();
+
+  /**
+   * 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>> groupFilesIntoChunks(Iterator<FileScanTask> dataFiles);
+
+  /**
+   * Method which will rewrite files based on this particular DataCompactionStrategy'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 written files
+   */
+  List<DataFile> rewriteFiles(Table table, List<FileScanTask> filesToRewrite);

Review comment:
       Question: `RewriteFiles` needs a set. Shall we return a set here to avoid any conversions later?

##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.expressions.Expression;
+
+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();
+
+  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. This would be used for pushing down filter expressions to the manifest entry scanning phase.
+   *
+   * @return an Iceberg expression to use when discovering file scan tasks
+   */
+  Expression preFilter();

Review comment:
       Can this be only a data filter?




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r626031963



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,119 @@
 

Review comment:
       This file now contains the new RewriteDataFiles API The old File is in RewriteDataFilesV1




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r626027392



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {

Review comment:
       Ok if no one else has comments i'll have this be RewriteDataFiles and we will rename the old one to v1




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


[GitHub] [iceberg] aokolnychyi merged pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi merged pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501


   


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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r618499227



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.expressions.Expression;
+
+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();
+
+  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. This would be used for pushing down filter expressions to the manifest entry scanning phase.
+   *
+   * @return an Iceberg expression to use when discovering file scan tasks
+   */
+  Expression preFilter();

Review comment:
       metadataPushdownFilter ?




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


[GitHub] [iceberg] rdblue commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617775845



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.expressions.Expression;
+
+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();
+
+  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. This would be used for pushing down filter expressions to the manifest entry scanning phase.
+   *
+   * @return an Iceberg expression to use when discovering file scan tasks
+   */
+  Expression preFilter();
+
+  /**
+   * 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);

Review comment:
       I generally prefer `Iterable`, but is it a good idea to work on a collection here? Why not `canSkip(FileScanTask)` or something?




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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r625294894



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,123 @@
+/*
+ * 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 org.apache.iceberg.StructLike;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior see max-chunk-size to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when the entire job has completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  // TODO be set once we have an ENUM in core
+  // String COMPACTION_STRATEGY_DEFAULT;
+
+  /**
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size. These sub-units of compaction are referred to as chunks. The largest amount of data that should be
+   * compacted in a single chunk is controlled by MAX_CHUNK_SIZE_BYTES. When grouping files, the underlying
+   * compaction strategy will use this value to but an upper bound on the number of files included in a single
+   * chunk. A chunk will be processed by a single framework "job". For example, in Spark this means that each chunk
+   * would be processed in it's own Spark action. A chunk will never contain files for multiple output partitions.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max-chunk-size-bytes";
+  long MAX_CHUNK_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.
+   */
+  String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes";
+
+  /**
+   * The max number of chunks to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the chunk is determined by the compaction strategy. When running each job chunk will be run
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_CHUNKS = "max-concurrent-chunks";

Review comment:
       `MAX_ACTION_SIZE_BYTES` looks clear to me. +1




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r627087689



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Rewrite Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.
  */
-public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+public interface RewriteDataFiles extends Action<RewriteDataFiles, RewriteDataFiles.Result> {
+
   /**
-   * Pass a row filter to filter {@link DataFile}s to be rewritten.
-   * <p>
-   * Note that all files that may contain data matching the filter may be rewritten.
-   * <p>
-   * If not set, all files will be rewritten.
-   *
-   * @param expr a row filter to filter out data files
-   * @return this for method chaining
+   * Enable committing groups of files (see max-file-group-size) prior to the entire compaction completing.
+   * This will produce additional commits but allow for progress even if some groups fail to commit. This setting
+   * will not change the correctness of the rewrite operation. The default is false, which produces a single commit
+   * when the entire job has completed.
    */
-  RewriteDataFiles filter(Expression expr);
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
 
   /**
-   * Enables or disables case sensitive expression binding.
-   * <p>
-   * If not set, defaults to false.
-   *
-   * @param caseSensitive caseSensitive
-   * @return this for method chaining
+   * The maximum amount of Iceberg commits that compaction is allowed to produce if partial progress is enabled.
    */
-  RewriteDataFiles caseSensitive(boolean caseSensitive);
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
 
   /**
-   * Pass a PartitionSpec id to specify which PartitionSpec should be used in DataFile rewrite
-   * <p>
-   * If not set, defaults to the table's default spec ID.
-   *
-   * @param specId PartitionSpec id to rewrite
-   * @return this for method chaining
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size into groups. These sub-units of compaction are referred to as file groups. The largest amount of data that
+   * should be compacted in a single group is controlled by MAX_FILE_GROUP_SIZE_BYTES. When grouping files, the
+   * underlying compaction strategy will use this value as to limit the files which will be included in a single file
+   * group. A group will be processed by a single framework "action". For example, in Spark this means that each group
+   * would be rewritten in its own Spark action. A group will never contain files for multiple output partitions.
    */
-  RewriteDataFiles outputSpecId(int specId);
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";
+  long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
 
   /**
-   * Specify the target data file size in bytes.
-   * <p>
-   * If not set, defaults to the table's target file size.
-   *
-   * @param targetSizeInBytes size in bytes of rewrite data file
-   * @return this for method chaining
+   * The max number of file groups to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the group is determined by the compaction strategy. Each file group will be rewritten
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_FILE_GROUP_ACTIONS = "max-concurrent-file-group-actions";
+  int MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT = 1;
+
+  /**
+   * The output file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.
+   */
+  String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes";
+
+  /**
+   * The partition spec to use when writing the output data from this operation. By default uses the
+   * current table partition spec.
    */
-  RewriteDataFiles targetSizeInBytes(long targetSizeInBytes);
+  String PARTITION_SPEC_ID = "partition-spec-id";
+
+  enum RewriteStrategyName {
+    BINPACK,
+    SORT
+  }
 
   /**
-   * Specify the number of "bins" considered when trying to pack the next file split into a task. Increasing this
-   * usually makes tasks a bit more even by considering more ways to pack file regions into a single task with extra
-   * planning cost.
-   * <p>
-   * This configuration can reorder the incoming file regions, to preserve order for lower/upper bounds in file
-   * metadata, user can use a lookback of 1.
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.

Review comment:
       Sure i'll remove it




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r627037674



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Rewrite Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.
  */
-public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+public interface RewriteDataFiles extends Action<RewriteDataFiles, RewriteDataFiles.Result> {
+
   /**
-   * Pass a row filter to filter {@link DataFile}s to be rewritten.
-   * <p>
-   * Note that all files that may contain data matching the filter may be rewritten.
-   * <p>
-   * If not set, all files will be rewritten.
-   *
-   * @param expr a row filter to filter out data files
-   * @return this for method chaining
+   * Enable committing groups of files (see max-file-group-size) prior to the entire compaction completing.
+   * This will produce additional commits but allow for progress even if some groups fail to commit. This setting
+   * will not change the correctness of the rewrite operation. The default is false, which produces a single commit
+   * when the entire job has completed.
    */
-  RewriteDataFiles filter(Expression expr);
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
 
   /**
-   * Enables or disables case sensitive expression binding.
-   * <p>
-   * If not set, defaults to false.
-   *
-   * @param caseSensitive caseSensitive
-   * @return this for method chaining
+   * The maximum amount of Iceberg commits that compaction is allowed to produce if partial progress is enabled.
    */
-  RewriteDataFiles caseSensitive(boolean caseSensitive);
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
 
   /**
-   * Pass a PartitionSpec id to specify which PartitionSpec should be used in DataFile rewrite
-   * <p>
-   * If not set, defaults to the table's default spec ID.
-   *
-   * @param specId PartitionSpec id to rewrite
-   * @return this for method chaining
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size into groups. These sub-units of compaction are referred to as file groups. The largest amount of data that
+   * should be compacted in a single group is controlled by MAX_FILE_GROUP_SIZE_BYTES. When grouping files, the
+   * underlying compaction strategy will use this value as to limit the files which will be included in a single file
+   * group. A group will be processed by a single framework "action". For example, in Spark this means that each group
+   * would be rewritten in its own Spark action. A group will never contain files for multiple output partitions.
    */
-  RewriteDataFiles outputSpecId(int specId);
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";
+  long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
 
   /**
-   * Specify the target data file size in bytes.
-   * <p>
-   * If not set, defaults to the table's target file size.
-   *
-   * @param targetSizeInBytes size in bytes of rewrite data file
-   * @return this for method chaining
+   * The max number of file groups to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the group is determined by the compaction strategy. Each file group will be rewritten
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_FILE_GROUP_ACTIONS = "max-concurrent-file-group-actions";

Review comment:
       I am ok calling actions. I'd also suggest something like `max-concurrent-file-group-rewrites`. I think that's is slightly more accurate but I'd want to hear what other folks think. 




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r618498247



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;
+
+  /**
+   * The parallelism level to use when processing chunks generated by the Compaction Strategy. The structure and
+   * contents of the chunk 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 this for chaining
+   */
+  CompactDataFiles parallelism(int par);
+
+  /**
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
+   *
+   * @param strategyName name of the strategy
+   * @return this for chaining
+   */
+  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. For example this would be used for providing a
+   * restriction to only run compaction on a specific partition.
+   *
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
+   */
+  CompactDataFiles filter(Expression expression);
+
+  /**
+   * A pairing of Chunk information to the result of that chunk's results. If the results are null then that particular
+   * chunk failed. We should only have failed chunks if partial progress is enabled.
+   */
+  class Result {
+    private Map<CompactionChunkInfo, CompactionChunkResult> resultMap;
+
+
+    public Result(Map<CompactionChunkInfo, CompactionChunkResult> resultMap) {
+      this.resultMap = resultMap;
+    }
+
+    public Map<CompactionChunkInfo, CompactionChunkResult> resultMap() {
+      return resultMap;
+    }
+  }
+
+  interface CompactionChunkResult {
+
+    int numFilesAdded();
+
+    int numFilesRemoved();
+  }
+
+  interface CompactionChunkInfo {
+
+    /**
+     * returns which chunk this is out of the total set of chunks for this compaction
+     */
+    int chunkIndex();

Review comment:
       I think so, if I tell you that the currently running job is Partition "fooBar" do you know how many partitions are left?
   Basically I think there is "overall job progress and partition job progress" I think both are valuable




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r618531447



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.expressions.Expression;
+
+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();

Review comment:
       I have a personal dislike of noop options, that is you can pass an option which cannot do anything. I feel like leads to a lot of errors with typos or using an option which is only valid in a separate configuration. In this case we may have a bunch of parameters for Sort that don't apply to BinPack, we don't want someone to think they are doing something if the options are actually useless.
   
   So mainly
   1. Avoid typos doing nothing
   2. Avoid options which are only valid for another compaction strategy




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r627080538



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Rewrite Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.
  */
-public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+public interface RewriteDataFiles extends Action<RewriteDataFiles, RewriteDataFiles.Result> {
+
   /**
-   * Pass a row filter to filter {@link DataFile}s to be rewritten.
-   * <p>
-   * Note that all files that may contain data matching the filter may be rewritten.
-   * <p>
-   * If not set, all files will be rewritten.
-   *
-   * @param expr a row filter to filter out data files
-   * @return this for method chaining
+   * Enable committing groups of files (see max-file-group-size) prior to the entire compaction completing.
+   * This will produce additional commits but allow for progress even if some groups fail to commit. This setting
+   * will not change the correctness of the rewrite operation. The default is false, which produces a single commit
+   * when the entire job has completed.
    */
-  RewriteDataFiles filter(Expression expr);
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
 
   /**
-   * Enables or disables case sensitive expression binding.
-   * <p>
-   * If not set, defaults to false.
-   *
-   * @param caseSensitive caseSensitive
-   * @return this for method chaining
+   * The maximum amount of Iceberg commits that compaction is allowed to produce if partial progress is enabled.
    */
-  RewriteDataFiles caseSensitive(boolean caseSensitive);
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
 
   /**
-   * Pass a PartitionSpec id to specify which PartitionSpec should be used in DataFile rewrite
-   * <p>
-   * If not set, defaults to the table's default spec ID.
-   *
-   * @param specId PartitionSpec id to rewrite
-   * @return this for method chaining
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size into groups. These sub-units of compaction are referred to as file groups. The largest amount of data that
+   * should be compacted in a single group is controlled by MAX_FILE_GROUP_SIZE_BYTES. When grouping files, the
+   * underlying compaction strategy will use this value as to limit the files which will be included in a single file
+   * group. A group will be processed by a single framework "action". For example, in Spark this means that each group
+   * would be rewritten in its own Spark action. A group will never contain files for multiple output partitions.
    */
-  RewriteDataFiles outputSpecId(int specId);
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";
+  long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
 
   /**
-   * Specify the target data file size in bytes.
-   * <p>
-   * If not set, defaults to the table's target file size.
-   *
-   * @param targetSizeInBytes size in bytes of rewrite data file
-   * @return this for method chaining
+   * The max number of file groups to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the group is determined by the compaction strategy. Each file group will be rewritten
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_FILE_GROUP_ACTIONS = "max-concurrent-file-group-actions";
+  int MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT = 1;
+
+  /**
+   * The output file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.
+   */
+  String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes";
+
+  /**
+   * The partition spec to use when writing the output data from this operation. By default uses the
+   * current table partition spec.
    */
-  RewriteDataFiles targetSizeInBytes(long targetSizeInBytes);
+  String PARTITION_SPEC_ID = "partition-spec-id";
+
+  enum RewriteStrategyName {
+    BINPACK,
+    SORT
+  }
 
   /**
-   * Specify the number of "bins" considered when trying to pack the next file split into a task. Increasing this
-   * usually makes tasks a bit more even by considering more ways to pack file regions into a single task with extra
-   * planning cost.
-   * <p>
-   * This configuration can reorder the incoming file regions, to preserve order for lower/upper bounds in file
-   * metadata, user can use a lookback of 1.
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
    *
-   * @param splitLookback number of "bins" considered when trying to pack the next file split into a task.
+   * @param strategyName name of the strategy
    * @return this for method chaining
    */
-  RewriteDataFiles splitLookback(int splitLookback);
+  RewriteDataFiles strategy(RewriteStrategyName strategyName);
 
   /**
-   * Specify the cost of opening a file that will be taken into account during packing files into
-   * bins. If the size of the file is smaller than the cost of opening, then this value will be used
-   * instead of the actual file size.
-   * <p>
-   * If not set, defaults to the table's open file cost.
+   * 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. For example this would be used for providing a
+   * restriction to only run compaction on a specific partition.
    *
-   * @param splitOpenFileCost minimum file size to count to pack into one "bin".
-   * @return this for method chaining
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
    */
-  RewriteDataFiles splitOpenFileCost(long splitOpenFileCost);
+  RewriteDataFiles filter(Expression expression);
 
   /**
-   * The action result that contains a summary of the execution.
+   * A pairing of file group information to the result of the rewriting that file group. If the results are null then
+   * that particular file group failed. We should only have failed groups if partial progress is enabled otherwise we
+   * will report a total failure for the job.
    */
   interface Result {
+    Map<FileGroupInfo, FileGroupRewriteResult> resultMap();
+  }
+
+  interface FileGroupRewriteResult {
+    int addedDataFilesCount();
+
+    int rewrittenDataFilesCount();
+  }
+
+  /**
+   * A description of a file group, when it was processed, and within which partition. For use
+   * tracking rewrite operations and for returning results.
+   */
+  interface FileGroupInfo {
+
+    /**
+     * returns which file group this is out of the total set of file groups for this compaction
+     */
+    int globalIndex();

Review comment:
       I'd rather keep the full results accessible here and do any summarizing when we are actually preparing command output for SQL.




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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r626222682



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+
+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();
+
+  DataCompactionStrategy withOptions(Map<String, String> options);
+
+  /**
+   * Selects files which this strategy believes are valid targets to be rewritten.
+   *
+   * @param dataFiles iterator of live datafiles in a given partition
+   * @return iterator containing only files to be rewritten
+   */
+  Iterable<FileScanTask> selectFilesToCompact(Iterable<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
+   */
+  Iterable<Set<FileScanTask>> groupFilesIntoChunks(Iterable<FileScanTask> dataFiles);

Review comment:
       Can we use `Set` here? it doesn't seem that `BaseFileScanTask` implements any `hashCode` or `equals` method.




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r627038186



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Rewrite Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.
  */
-public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+public interface RewriteDataFiles extends Action<RewriteDataFiles, RewriteDataFiles.Result> {
+
   /**
-   * Pass a row filter to filter {@link DataFile}s to be rewritten.
-   * <p>
-   * Note that all files that may contain data matching the filter may be rewritten.
-   * <p>
-   * If not set, all files will be rewritten.
-   *
-   * @param expr a row filter to filter out data files
-   * @return this for method chaining
+   * Enable committing groups of files (see max-file-group-size) prior to the entire compaction completing.
+   * This will produce additional commits but allow for progress even if some groups fail to commit. This setting
+   * will not change the correctness of the rewrite operation. The default is false, which produces a single commit
+   * when the entire job has completed.
    */
-  RewriteDataFiles filter(Expression expr);
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
 
   /**
-   * Enables or disables case sensitive expression binding.
-   * <p>
-   * If not set, defaults to false.
-   *
-   * @param caseSensitive caseSensitive
-   * @return this for method chaining
+   * The maximum amount of Iceberg commits that compaction is allowed to produce if partial progress is enabled.
    */
-  RewriteDataFiles caseSensitive(boolean caseSensitive);
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
 
   /**
-   * Pass a PartitionSpec id to specify which PartitionSpec should be used in DataFile rewrite
-   * <p>
-   * If not set, defaults to the table's default spec ID.
-   *
-   * @param specId PartitionSpec id to rewrite
-   * @return this for method chaining
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size into groups. These sub-units of compaction are referred to as file groups. The largest amount of data that
+   * should be compacted in a single group is controlled by MAX_FILE_GROUP_SIZE_BYTES. When grouping files, the
+   * underlying compaction strategy will use this value as to limit the files which will be included in a single file
+   * group. A group will be processed by a single framework "action". For example, in Spark this means that each group
+   * would be rewritten in its own Spark action. A group will never contain files for multiple output partitions.
    */
-  RewriteDataFiles outputSpecId(int specId);
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";
+  long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
 
   /**
-   * Specify the target data file size in bytes.
-   * <p>
-   * If not set, defaults to the table's target file size.
-   *
-   * @param targetSizeInBytes size in bytes of rewrite data file
-   * @return this for method chaining
+   * The max number of file groups to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the group is determined by the compaction strategy. Each file group will be rewritten
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_FILE_GROUP_ACTIONS = "max-concurrent-file-group-actions";
+  int MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT = 1;
+
+  /**
+   * The output file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.
+   */
+  String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes";
+
+  /**
+   * The partition spec to use when writing the output data from this operation. By default uses the
+   * current table partition spec.
    */
-  RewriteDataFiles targetSizeInBytes(long targetSizeInBytes);
+  String PARTITION_SPEC_ID = "partition-spec-id";

Review comment:
       Shall it be `output-spec-id` or `output-partition-spec-id` to indicate it is used for the result 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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617752608



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/BinPack.java
##########
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+public interface BinPack {

Review comment:
       Yeah sorry this was just a placeholder, I think an Enum would probably be fine. We'll have to determine the actual "BinPack" at the action level since we'll need a different class depending on the executing framework.




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


[GitHub] [iceberg] rdblue commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617741434



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;
+
+  /**
+   * The parallelism level to use when processing chunks generated by the Compaction Strategy. The structure and
+   * contents of the chunk 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 this for chaining
+   */
+  CompactDataFiles parallelism(int par);

Review comment:
       I would prefer to make this more specific, since parallelism could refer to a lot of things including the number of executors or tasks used.




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


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r618000815



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";

Review comment:
       Maybe we need to clarify that this will produce one commit for one chunk/partition. And In the future, we may offer a mechanism to commit multiple chunks in one commit? Right?




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


[GitHub] [iceberg] rdblue commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617767854



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.expressions.Expression;
+
+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();
+
+  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. This would be used for pushing down filter expressions to the manifest entry scanning phase.
+   *
+   * @return an Iceberg expression to use when discovering file scan tasks
+   */
+  Expression preFilter();

Review comment:
       We can add partition filters later if we want.




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r623412679



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.expressions.Expression;
+
+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();
+
+  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. This would be used for pushing down filter expressions to the manifest entry scanning phase.
+   *
+   * @return an Iceberg expression to use when discovering file scan tasks
+   */
+  Expression preFilter();

Review comment:
       sgtm




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r626247948



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+
+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();
+
+  DataCompactionStrategy withOptions(Map<String, String> options);
+
+  /**
+   * Selects files which this strategy believes are valid targets to be rewritten.
+   *
+   * @param dataFiles iterator of live datafiles in a given partition
+   * @return iterator containing only files to be rewritten
+   */
+  Iterable<FileScanTask> selectFilesToCompact(Iterable<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
+   */
+  Iterable<Set<FileScanTask>> groupFilesIntoChunks(Iterable<FileScanTask> dataFiles);

Review comment:
       This was originally List but was changed to set to accommodate a previous reviewer's request. I think you're correct and we should switch it back to list. I'll also do a rename to planFileGroups




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r619467784



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.expressions.Expression;
+
+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();
+
+  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. This would be used for pushing down filter expressions to the manifest entry scanning phase.
+   *
+   * @return an Iceberg expression to use when discovering file scan tasks
+   */
+  Expression preFilter();
+
+  /**
+   * 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>> groupFilesIntoChunks(Iterator<FileScanTask> dataFiles);
+
+  /**
+   * Method which will rewrite files based on this particular DataCompactionStrategy'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 written files
+   */
+  List<DataFile> rewriteFiles(Table table, List<FileScanTask> filesToRewrite);

Review comment:
       I am ok with that.




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


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r618025490



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.expressions.Expression;
+
+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();
+
+  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. This would be used for pushing down filter expressions to the manifest entry scanning phase.
+   *
+   * @return an Iceberg expression to use when discovering file scan tasks
+   */
+  Expression preFilter();
+
+  /**
+   * 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>> groupFilesIntoChunks(Iterator<FileScanTask> dataFiles);

Review comment:
       This could also be iterable to iterable, right? Each chunk should contain a set of files without duplication, so we can use `Set`.




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r627079472



##########
File path: api/src/main/java/org/apache/iceberg/actions/rewrite/RewriteStrategy.java
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.rewrite;
+
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+
+interface RewriteStrategy 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();
+
+  RewriteStrategy withOptions(Map<String, String> options);
+
+  /**
+   * Selects files which this strategy believes are valid targets to be rewritten.
+   *
+   * @param dataFiles iterable of FileScanTasks for files in a given partition
+   * @return iterable containing only FileScanTasks to be rewritten
+   */
+  Iterable<FileScanTask> selectFilesToCompact(Iterable<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 iterable of FileScanTasks to be rewritten
+   * @return iterable of lists of FileScanTasks which will be processed together
+   */
+  Iterable<List<FileScanTask>> planFileGroups(Iterable<FileScanTask> dataFiles);
+
+  /**
+   * Method which will rewrite files based on this particular RewriteStrategy's algorithm.
+   * This will most likely be Action framework specific (Spark/Presto/Flink ....).
+   *
+   * @param table          table being modified
+   * @param filesToRewrite a group of files to be rewritten together
+   * @return a list of newly written files
+   */
+  List<DataFile> rewriteFiles(Table table, List<FileScanTask> filesToRewrite);

Review comment:
       Well before we couldn't serialize tables :) Now I guess we can so we can instead have the Strategy Implementation have table as a field.




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


[GitHub] [iceberg] yyanyy commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
yyanyy commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r623443530



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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 org.apache.iceberg.StructLike;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when the entire job has completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  // TODO be set once we have an ENUM in core
+  // String COMPACTION_STRATEGY_DEFAULT;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max-chunk-size-bytes";

Review comment:
       Do we want to define what a "chunk" is here? Sounds like a collection of data files to be processed together (under one spark job?), but it's not super obvious to me by reading this class. If my assumption is correct, then what happens if a single file's file size is above this threshold? I'd imagine we will treat it as a chunk (instead of splitting it into multiple chunks) and I'd hope we can describe it more here. 

##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;
+
+  /**
+   * The parallelism level to use when processing chunks generated by the Compaction Strategy. The structure and
+   * contents of the chunk 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 this for chaining
+   */
+  CompactDataFiles parallelism(int par);
+
+  /**
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
+   *
+   * @param strategyName name of the strategy
+   * @return this for chaining
+   */
+  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. For example this would be used for providing a
+   * restriction to only run compaction on a specific partition.
+   *
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
+   */
+  CompactDataFiles filter(Expression expression);
+
+  /**
+   * A pairing of Chunk information to the result of that chunk's results. If the results are null then that particular
+   * chunk failed. We should only have failed chunks if partial progress is enabled.
+   */
+  class Result {
+    private Map<CompactionChunkInfo, CompactionChunkResult> resultMap;
+
+
+    public Result(Map<CompactionChunkInfo, CompactionChunkResult> resultMap) {
+      this.resultMap = resultMap;
+    }
+
+    public Map<CompactionChunkInfo, CompactionChunkResult> resultMap() {
+      return resultMap;
+    }
+  }
+
+  interface CompactionChunkResult {
+
+    int numFilesAdded();
+
+    int numFilesRemoved();
+  }
+
+  interface CompactionChunkInfo {
+
+    /**
+     * returns which chunk this is out of the total set of chunks for this compaction
+     */
+    int chunkIndex();

Review comment:
       if we do want this, do we want to rename it to `globalIndex`/`globalChunkIndex` then? Also unless we print the total number of chunks we want to process within the same compaction operation somewhere, I think just having the index of individual finished chunks we still don't know how many remaining ones are left? How do we plan to convey how many chunks we will process in total, is the plan to check size of `DataCompactionStrategy.groupFilesIntoChunks()`? 

##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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 org.apache.iceberg.StructLike;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when the entire job has completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  // TODO be set once we have an ENUM in core
+  // String COMPACTION_STRATEGY_DEFAULT;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max-chunk-size-bytes";
+  long MAX_CHUNK_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.
+   */
+  String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes";
+
+  /**
+   * The max number of chunks to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the chunk is determined by the compaction strategy. When running each job chunk will be run
+   * be run independently and asynchronously.

Review comment:
       nit: duplicated "be run"




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r623401236



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;
+
+  /**
+   * The parallelism level to use when processing chunks generated by the Compaction Strategy. The structure and
+   * contents of the chunk 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 this for chaining
+   */
+  CompactDataFiles parallelism(int par);
+
+  /**
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
+   *
+   * @param strategyName name of the strategy
+   * @return this for chaining
+   */
+  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. For example this would be used for providing a
+   * restriction to only run compaction on a specific partition.
+   *
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
+   */
+  CompactDataFiles filter(Expression expression);
+
+  /**
+   * A pairing of Chunk information to the result of that chunk's results. If the results are null then that particular
+   * chunk failed. We should only have failed chunks if partial progress is enabled.
+   */
+  class Result {
+    private Map<CompactionChunkInfo, CompactionChunkResult> resultMap;
+
+
+    public Result(Map<CompactionChunkInfo, CompactionChunkResult> resultMap) {
+      this.resultMap = resultMap;
+    }
+
+    public Map<CompactionChunkInfo, CompactionChunkResult> resultMap() {
+      return resultMap;
+    }
+  }
+
+  interface CompactionChunkResult {
+
+    int numFilesAdded();
+
+    int numFilesRemoved();
+  }
+
+  interface CompactionChunkInfo {
+
+    /**
+     * returns which chunk this is out of the total set of chunks for this compaction
+     */
+    int chunkIndex();

Review comment:
       I think this would make it much easier to track things down in the logs, but that's just me. Basically I was thinking this class is a description for a Chunk which is about to be compacted all wrapped together.




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


[GitHub] [iceberg] yyanyy commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
yyanyy commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r625269726



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,123 @@
+/*
+ * 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 org.apache.iceberg.StructLike;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior see max-chunk-size to the entire compaction completing. This will produce additional commits

Review comment:
       Is this "see max-chunk-size" added unintentionally? 




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r619450841



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";

Review comment:
       To me, the max number of commits is fine. I'd interpret this as the number of commits I as a user allow the compaction to do. I agree with Russell the error message must be as detailed as possible. I believe the current proposal should already cover case 2 mentioned by @jackye1995 if we have a high enough default value. 




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r626252875



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+
+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();
+
+  DataCompactionStrategy withOptions(Map<String, String> options);
+
+  /**
+   * Selects files which this strategy believes are valid targets to be rewritten.
+   *
+   * @param dataFiles iterator of live datafiles in a given partition
+   * @return iterator containing only files to be rewritten
+   */
+  Iterable<FileScanTask> selectFilesToCompact(Iterable<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
+   */
+  Iterable<Set<FileScanTask>> groupFilesIntoChunks(Iterable<FileScanTask> dataFiles);

Review comment:
       It's better the keep it as FileScanTasks if you see the work we've been doing on 
   FileScanTaskSetManager  https://github.com/apache/iceberg/pull/2472
   and SparkRewriteManger https://github.com/apache/iceberg/pull/2500
   




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617950961



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/BinPack.java
##########
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+public interface BinPack {

Review comment:
       I'd support an idea of an enum or something in `core` and not exposing it for now.




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617960670



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.expressions.Expression;
+
+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();
+
+  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. This would be used for pushing down filter expressions to the manifest entry scanning phase.
+   *
+   * @return an Iceberg expression to use when discovering file scan tasks
+   */
+  Expression preFilter();
+
+  /**
+   * 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>> groupFilesIntoChunks(Iterator<FileScanTask> dataFiles);
+
+  /**
+   * Method which will rewrite files based on this particular DataCompactionStrategy'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 written files
+   */
+  List<DataFile> rewriteFiles(Table table, List<FileScanTask> filesToRewrite);

Review comment:
       @RussellSpitzer, is the plan to have BaseXXX strategies in core and then SparkXXX and FlinkXXX strategies that would implement `rewriteFiles` differently?
   
   cc @rdblue @openinx 




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r627027835



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,119 @@
 

Review comment:
       Why do we need `RewriteDataFilesV1` in the new API? I'd say we should only expose the new logic. Let's keep the current implementation under the old interface and old entry point.
   
   It should be safe to simply replace the content of `RewriteDataFiles` as there is no implementation and we delayed the discussion how the API should look like up until now.




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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r626223369



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+
+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();
+
+  DataCompactionStrategy withOptions(Map<String, String> options);
+
+  /**
+   * Selects files which this strategy believes are valid targets to be rewritten.
+   *
+   * @param dataFiles iterator of live datafiles in a given partition
+   * @return iterator containing only files to be rewritten
+   */
+  Iterable<FileScanTask> selectFilesToCompact(Iterable<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
+   */
+  Iterable<Set<FileScanTask>> groupFilesIntoChunks(Iterable<FileScanTask> dataFiles);

Review comment:
       since Iceberg's `TableScan` returns `CloseableIterable`, should we use it here too (instead of the regular `Iterable`).  if we shouldn't use `Set`, maybe we just use `CombinedScanTask`. so basically returns `CloseableIterable<CombinedScanTask>`?




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r624061406



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+
+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();
+
+  DataCompactionStrategy withOptions(Map<String, String> options);
+
+  /**
+   * Removes all file references which this plan will not rewrite or change. Unlike the preFilter, this method can

Review comment:
       Switched this to be a positive phrasing, removed the line about "prefilter" entirely since we no longer are going to include that in this verison.




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


[GitHub] [iceberg] rdblue commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617742473



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;
+
+  /**
+   * The parallelism level to use when processing chunks generated by the Compaction Strategy. The structure and
+   * contents of the chunk 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 this for chaining
+   */
+  CompactDataFiles parallelism(int par);
+
+  /**
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
+   *
+   * @param strategyName name of the strategy
+   * @return this for chaining
+   */
+  CompactDataFiles compactionStrategy(String strategyName);
+
+  /**
+   * A user provided filter for determining which files will be considered by the Compaction strategy. This will be used

Review comment:
       This capitalizes "compaction strategy" differently than in other methods' Javadoc. In general, I'd prefer not to capitalize nouns just to call them out. I think "compaction strategy" makes sense without the capitalization.




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r624029937



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;
+
+  /**
+   * The parallelism level to use when processing chunks generated by the Compaction Strategy. The structure and
+   * contents of the chunk 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 this for chaining

Review comment:
       Removed, no longer method, now an option




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r627073523



##########
File path: api/src/main/java/org/apache/iceberg/actions/rewrite/RewriteStrategy.java
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.rewrite;
+
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+
+interface RewriteStrategy 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();
+
+  RewriteStrategy withOptions(Map<String, String> options);
+
+  /**
+   * Selects files which this strategy believes are valid targets to be rewritten.
+   *
+   * @param dataFiles iterable of FileScanTasks for files in a given partition
+   * @return iterable containing only FileScanTasks to be rewritten
+   */
+  Iterable<FileScanTask> selectFilesToCompact(Iterable<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 iterable of FileScanTasks to be rewritten
+   * @return iterable of lists of FileScanTasks which will be processed together
+   */
+  Iterable<List<FileScanTask>> planFileGroups(Iterable<FileScanTask> dataFiles);
+
+  /**
+   * Method which will rewrite files based on this particular RewriteStrategy's algorithm.
+   * This will most likely be Action framework specific (Spark/Presto/Flink ....).
+   *
+   * @param table          table being modified
+   * @param filesToRewrite a group of files to be rewritten together
+   * @return a list of newly written files
+   */
+  List<DataFile> rewriteFiles(Table table, List<FileScanTask> filesToRewrite);

Review comment:
       Why do we pass `Table` here?




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r627069963



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Rewrite Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.
  */
-public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+public interface RewriteDataFiles extends Action<RewriteDataFiles, RewriteDataFiles.Result> {

Review comment:
       Sure I didn't know about that class




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


[GitHub] [iceberg] rdblue commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617739983



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;
+
+  /**
+   * The parallelism level to use when processing chunks generated by the Compaction Strategy. The structure and
+   * contents of the chunk 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 this for chaining

Review comment:
       Nit: this for method chaining.




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r618531997



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.expressions.Expression;
+
+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();
+
+  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. This would be used for pushing down filter expressions to the manifest entry scanning phase.
+   *
+   * @return an Iceberg expression to use when discovering file scan tasks
+   */
+  Expression preFilter();
+
+  /**
+   * 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>> groupFilesIntoChunks(Iterator<FileScanTask> dataFiles);

Review comment:
       Yeah I think the consensus is leaning towards Iterable over iterator




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r624034373



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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 org.apache.iceberg.StructLike;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when the entire job has completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  // TODO be set once we have an ENUM in core
+  // String COMPACTION_STRATEGY_DEFAULT;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max-chunk-size-bytes";

Review comment:
       Yes it's a collection of files that is processed in a single Spark action for spark. I'm not familiar with how other systems terminology but basically we are talking about an independent set of files that has been selected out of our total set of files to be rewritten at the same time. I'll try to elaborate in the java doc.




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617837560



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/BinPack.java
##########
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+public interface BinPack {

Review comment:
       I think my initial intention was to put Bin-pack specific options in here, but I think we can just keep them in the base class.




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617756665



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.expressions.Expression;
+
+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();
+
+  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. This would be used for pushing down filter expressions to the manifest entry scanning phase.
+   *
+   * @return an Iceberg expression to use when discovering file scan tasks
+   */
+  Expression preFilter();
+
+  /**
+   * 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);

Review comment:
       I don't have a strong feeling on this, I initially wanted to stress the single pass nature to try to allow frameworks to pass lazy iterators (like spark) but i'm beginning to think all of our actual implementations for more complicated things will need to materialize the whole set of tasks anyway ...




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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r624568161



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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 org.apache.iceberg.StructLike;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when the entire job has completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  // TODO be set once we have an ENUM in core
+  // String COMPACTION_STRATEGY_DEFAULT;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max-chunk-size-bytes";

Review comment:
       what about `MAX_FILE_GROUP_SIZE_BYTES"? To me, chunk has the implication of a split/slice of a file.




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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r626222750



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+
+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();
+
+  DataCompactionStrategy withOptions(Map<String, String> options);
+
+  /**
+   * Selects files which this strategy believes are valid targets to be rewritten.
+   *
+   * @param dataFiles iterator of live datafiles in a given partition
+   * @return iterator containing only files to be rewritten
+   */
+  Iterable<FileScanTask> selectFilesToCompact(Iterable<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
+   */
+  Iterable<Set<FileScanTask>> groupFilesIntoChunks(Iterable<FileScanTask> dataFiles);
+
+  /**
+   * Method which will rewrite files based on this particular DataCompactionStrategy's algorithm.
+   * This will most likely be Action framework specific (Spark/Presto/Flink ....).
+   *
+   * @param table          table being modified
+   * @param filesToRewrite a group of files to be rewritten together
+   * @return a list of newly written files
+   */
+  List<DataFile> rewriteFiles(Table table, Set<FileScanTask> filesToRewrite);

Review comment:
       similar question for the use of `Set`




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


[GitHub] [iceberg] RussellSpitzer commented on pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#issuecomment-830265714


   One thing i'm thinking about with all these indexes is that we can present results like
   
   | GlobalIndex | Partition | PartitionIndex | Files Added | Files Rewritten | 


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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r627032084



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Rewrite Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.
  */
-public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+public interface RewriteDataFiles extends Action<RewriteDataFiles, RewriteDataFiles.Result> {
+
   /**
-   * Pass a row filter to filter {@link DataFile}s to be rewritten.
-   * <p>
-   * Note that all files that may contain data matching the filter may be rewritten.
-   * <p>
-   * If not set, all files will be rewritten.
-   *
-   * @param expr a row filter to filter out data files
-   * @return this for method chaining
+   * Enable committing groups of files (see max-file-group-size) prior to the entire compaction completing.
+   * This will produce additional commits but allow for progress even if some groups fail to commit. This setting
+   * will not change the correctness of the rewrite operation. The default is false, which produces a single commit

Review comment:
       I'd add a bit more details to emphasize the data correctness is not affected.
   Like `... of the rewrite operation as file groups can be compacted independently` or something. 




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r627073911



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Rewrite Strategy. Generally used for

Review comment:
       Corrected, RewriteStrategy is the class name though so I switched to a link. Only issue is as long as RewriteStrategy is package private the link just shows up as RewriteStrategy. This is actually an issue with us making that interface package private since an implementation in spark would have to be in a different package than the Spark action implementations which are all in `o.a.iceberg.spark.actions` and not `o.a.iceberg.actions` ... may have to think about this later




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


[GitHub] [iceberg] rdblue commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617760725



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.expressions.Expression;
+
+public interface DataCompactionStrategy extends Serializable {

Review comment:
       +1 for not exposing what we don't need to yet.




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r627062634



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Rewrite Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.
  */
-public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+public interface RewriteDataFiles extends Action<RewriteDataFiles, RewriteDataFiles.Result> {
+
   /**
-   * Pass a row filter to filter {@link DataFile}s to be rewritten.
-   * <p>
-   * Note that all files that may contain data matching the filter may be rewritten.
-   * <p>
-   * If not set, all files will be rewritten.
-   *
-   * @param expr a row filter to filter out data files
-   * @return this for method chaining
+   * Enable committing groups of files (see max-file-group-size) prior to the entire compaction completing.
+   * This will produce additional commits but allow for progress even if some groups fail to commit. This setting
+   * will not change the correctness of the rewrite operation. The default is false, which produces a single commit
+   * when the entire job has completed.
    */
-  RewriteDataFiles filter(Expression expr);
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
 
   /**
-   * Enables or disables case sensitive expression binding.
-   * <p>
-   * If not set, defaults to false.
-   *
-   * @param caseSensitive caseSensitive
-   * @return this for method chaining
+   * The maximum amount of Iceberg commits that compaction is allowed to produce if partial progress is enabled.
    */
-  RewriteDataFiles caseSensitive(boolean caseSensitive);
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
 
   /**
-   * Pass a PartitionSpec id to specify which PartitionSpec should be used in DataFile rewrite
-   * <p>
-   * If not set, defaults to the table's default spec ID.
-   *
-   * @param specId PartitionSpec id to rewrite
-   * @return this for method chaining
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size into groups. These sub-units of compaction are referred to as file groups. The largest amount of data that
+   * should be compacted in a single group is controlled by MAX_FILE_GROUP_SIZE_BYTES. When grouping files, the
+   * underlying compaction strategy will use this value as to limit the files which will be included in a single file
+   * group. A group will be processed by a single framework "action". For example, in Spark this means that each group
+   * would be rewritten in its own Spark action. A group will never contain files for multiple output partitions.
    */
-  RewriteDataFiles outputSpecId(int specId);
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";
+  long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
 
   /**
-   * Specify the target data file size in bytes.
-   * <p>
-   * If not set, defaults to the table's target file size.
-   *
-   * @param targetSizeInBytes size in bytes of rewrite data file
-   * @return this for method chaining
+   * The max number of file groups to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the group is determined by the compaction strategy. Each file group will be rewritten
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_FILE_GROUP_ACTIONS = "max-concurrent-file-group-actions";
+  int MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT = 1;
+
+  /**
+   * The output file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.
+   */
+  String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes";
+
+  /**
+   * The partition spec to use when writing the output data from this operation. By default uses the
+   * current table partition spec.
    */
-  RewriteDataFiles targetSizeInBytes(long targetSizeInBytes);
+  String PARTITION_SPEC_ID = "partition-spec-id";
+
+  enum RewriteStrategyName {

Review comment:
       Well, I am not sure here. It is a bit weird to have the strategy name enum and the strategy interface separate from each other. I don't have an alternative, though. Maybe, it is actually fine.
   
   I'd consider calling it simply `Strategy` or `StrategyName` as it is nested inside `RewriteDataFiles`.




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r627072548



##########
File path: api/src/main/java/org/apache/iceberg/actions/rewrite/RewriteStrategy.java
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.rewrite;
+
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+
+interface RewriteStrategy 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();
+
+  RewriteStrategy withOptions(Map<String, String> options);

Review comment:
       nit: I think we decided not to use `with` prefix in the V2 APIs.




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


[GitHub] [iceberg] kbendick commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r627085739



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFilesV1.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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 org.apache.iceberg.DataFile;
+import org.apache.iceberg.expressions.Expression;
+
+/**
+ * An action that rewrites data files.
+ */
+public interface RewriteDataFilesV1 extends SnapshotUpdate<RewriteDataFilesV1, RewriteDataFilesV1.Result> {
+  /**
+   * Pass a row filter to filter {@link DataFile}s to be rewritten.
+   * <p>
+   * Note that all files that may contain data matching the filter may be rewritten.
+   * <p>
+   * If not set, all files will be rewritten.

Review comment:
       Seems I don't have permissions to close this. Something weird has happened to my GitHub account in this repo. On another PR I was tagged in, I'm not able to comment at all (even though it's still open). Feel free to close this!




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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r624622663



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,123 @@
+/*
+ * 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 org.apache.iceberg.StructLike;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior see max-chunk-size to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when the entire job has completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  // TODO be set once we have an ENUM in core
+  // String COMPACTION_STRATEGY_DEFAULT;
+
+  /**
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size. These sub-units of compaction are referred to as chunks. The largest amount of data that should be
+   * compacted in a single chunk is controlled by MAX_CHUNK_SIZE_BYTES. When grouping files, the underlying
+   * compaction strategy will use this value to but an upper bound on the number of files included in a single
+   * chunk. A chunk will be processed by a single framework "job". For example, in Spark this means that each chunk
+   * would be processed in it's own Spark action. A chunk will never contain files for multiple output partitions.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max-chunk-size-bytes";
+  long MAX_CHUNK_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.
+   */
+  String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes";
+
+  /**
+   * The max number of chunks to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the chunk is determined by the compaction strategy. When running each job chunk will be run
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_CHUNKS = "max-concurrent-chunks";

Review comment:
       what about renaming `Chunk` as `Assignment`? Action for an assignment. In another comment, I also suggested 'FileGroup` for `Chunk`.
   
   also for the prop name, maybe `max-concurrent-actions` is more accurate?
   




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617755137



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;
+
+  /**
+   * The parallelism level to use when processing chunks generated by the Compaction Strategy. The structure and
+   * contents of the chunk 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 this for chaining
+   */
+  CompactDataFiles parallelism(int par);
+
+  /**
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
+   *
+   * @param strategyName name of the strategy
+   * @return this for chaining
+   */
+  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. For example this would be used for providing a
+   * restriction to only run compaction on a specific partition.
+   *
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
+   */
+  CompactDataFiles filter(Expression expression);
+
+  /**
+   * A pairing of Chunk information to the result of that chunk's results. If the results are null then that particular
+   * chunk failed. We should only have failed chunks if partial progress is enabled.
+   */
+  class Result {

Review comment:
       Interface is fine, I was actually thinking that when I stripped out all the private members that were present in the WIP I just forgot to change this here




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


[GitHub] [iceberg] rdblue commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617746647



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/BinPack.java
##########
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+public interface BinPack {

Review comment:
       Why not an enum for the strategies?




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r623390758



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";

Review comment:
       Sounds good removing them for now
   




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617957465



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;
+
+  /**
+   * The parallelism level to use when processing chunks generated by the Compaction Strategy. The structure and
+   * contents of the chunk 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 this for chaining
+   */
+  CompactDataFiles parallelism(int par);
+
+  /**
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
+   *
+   * @param strategyName name of the strategy
+   * @return this for chaining
+   */
+  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. For example this would be used for providing a
+   * restriction to only run compaction on a specific partition.
+   *
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
+   */
+  CompactDataFiles filter(Expression expression);
+
+  /**
+   * A pairing of Chunk information to the result of that chunk's results. If the results are null then that particular
+   * chunk failed. We should only have failed chunks if partial progress is enabled.
+   */
+  class Result {
+    private Map<CompactionChunkInfo, CompactionChunkResult> resultMap;
+
+
+    public Result(Map<CompactionChunkInfo, CompactionChunkResult> resultMap) {
+      this.resultMap = resultMap;
+    }
+
+    public Map<CompactionChunkInfo, CompactionChunkResult> resultMap() {
+      return resultMap;
+    }
+  }
+
+  interface CompactionChunkResult {
+
+    int numFilesAdded();
+
+    int numFilesRemoved();
+  }
+
+  interface CompactionChunkInfo {
+
+    /**
+     * returns which chunk this is out of the total set of chunks for this compaction
+     */
+    int chunkIndex();

Review comment:
       Do we need it at the global level? Wouldn't just the partition and chunk within that partition be enough? Like partition A, part 1, partition A, part 2.




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


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r618013445



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.expressions.Expression;
+
+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();

Review comment:
       Why we need the allowed list? What kind of options that users may pass to runtime and compaction strategy does not use?




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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r626220808



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Compaction Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.
  */
-public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+public interface RewriteDataFiles extends Action<RewriteDataFiles, RewriteDataFiles.Result> {
+
   /**
-   * Pass a row filter to filter {@link DataFile}s to be rewritten.
-   * <p>
-   * Note that all files that may contain data matching the filter may be rewritten.
-   * <p>
-   * If not set, all files will be rewritten.
-   *
-   * @param expr a row filter to filter out data files
-   * @return this for method chaining
+   * Enable committing groups of files (see max-file-group-size) prior to the entire compaction completing.
+   * This will produce additional commits but allow for progress even if some groups fail to commit. This setting
+   * will not change the correctness of the rewrite operation. The default is false, which produces a single commit
+   * when the entire job has completed.
    */
-  RewriteDataFiles filter(Expression expr);
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
 
   /**
-   * Enables or disables case sensitive expression binding.
-   * <p>
-   * If not set, defaults to false.
-   *
-   * @param caseSensitive caseSensitive
-   * @return this for method chaining
+   * The maximum amount of Iceberg commits that compaction is allowed to produce if partial progress is enabled.
    */
-  RewriteDataFiles caseSensitive(boolean caseSensitive);
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
 
   /**
-   * Pass a PartitionSpec id to specify which PartitionSpec should be used in DataFile rewrite
-   * <p>
-   * If not set, defaults to the table's default spec ID.
-   *
-   * @param specId PartitionSpec id to rewrite
-   * @return this for method chaining
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size into groups. These sub-units of compaction are referred to as file groups. The largest amount of data that
+   * should be compacted in a single group is controlled by MAX_FILE_GROUP_SIZE_BYTES. When grouping files, the
+   * underlying compaction strategy will use this value as to limit the files which will be included in a single file
+   * group. A group will be processed by a single framework "action". For example, in Spark this means that each group
+   * would be rewritten in its own Spark action. A group will never contain files for multiple output partitions.
    */
-  RewriteDataFiles outputSpecId(int specId);
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";
+  long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
 
   /**
-   * Specify the target data file size in bytes.
-   * <p>
-   * If not set, defaults to the table's target file size.
-   *
-   * @param targetSizeInBytes size in bytes of rewrite data file
-   * @return this for method chaining
+   * The max number of file groups to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the group is determined by the compaction strategy. Each file group will be rewritten
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_FILE_GROUP_ACTIONS = "max-concurrent-file-group-actions";
+  int MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT = 1;
+
+  /**
+   * The output file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.
+   */
+  String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes";
+
+  /**
+   * The partition spec to use when writing the output data from this operation. By default uses the
+   * current table partition spec.
    */
-  RewriteDataFiles targetSizeInBytes(long targetSizeInBytes);
+  String PARTITION_SPEC_ID = "partition-spec-id";
+
+  enum CompactionStrategyName {
+    BinPack,
+    Sort
+  }
 
   /**
-   * Specify the number of "bins" considered when trying to pack the next file split into a task. Increasing this
-   * usually makes tasks a bit more even by considering more ways to pack file regions into a single task with extra
-   * planning cost.
-   * <p>
-   * This configuration can reorder the incoming file regions, to preserve order for lower/upper bounds in file
-   * metadata, user can use a lookback of 1.
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
    *
-   * @param splitLookback number of "bins" considered when trying to pack the next file split into a task.
+   * @param strategyName name of the strategy
    * @return this for method chaining
    */
-  RewriteDataFiles splitLookback(int splitLookback);
+  RewriteDataFiles strategy(CompactionStrategyName strategyName);
 
   /**
-   * Specify the cost of opening a file that will be taken into account during packing files into
-   * bins. If the size of the file is smaller than the cost of opening, then this value will be used
-   * instead of the actual file size.
-   * <p>
-   * If not set, defaults to the table's open file cost.
+   * 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. For example this would be used for providing a
+   * restriction to only run compaction on a specific partition.
    *
-   * @param splitOpenFileCost minimum file size to count to pack into one "bin".
-   * @return this for method chaining
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
    */
-  RewriteDataFiles splitOpenFileCost(long splitOpenFileCost);
+  RewriteDataFiles filter(Expression expression);
 
   /**
-   * The action result that contains a summary of the execution.
+   * A pairing of file group information to the result of the rewriting that file group. If the results are null then
+   * that particular chunk failed. We should only have failed groups if partial progress is enabled otherwise we will
+   * report a total failure for the job.
    */
   interface Result {
+    Map<FileGroupInfo, FileGroupResult> resultMap();
+  }
+
+  interface FileGroupResult {

Review comment:
       is `RewriteResult` more accurate?




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r627029312



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Rewrite Strategy. Generally used for

Review comment:
       nit: `datafiles` -> `data files`, `Rewrite Strategy` -> `rewrite strategy`




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r624061625



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+
+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();
+
+  DataCompactionStrategy withOptions(Map<String, String> options);
+
+  /**
+   * 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.
+   *
+   * @param dataFiles iterator of live datafiles in a given partition
+   * @return iterator containing only files to be rewritten
+   */
+  Iterable<FileScanTask> selectFilesToCompact(Iterable<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
+   */
+  Iterable<List<FileScanTask>> groupFilesIntoChunks(Iterable<FileScanTask> dataFiles);

Review comment:
       Good idea




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r626249976



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+
+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();
+
+  DataCompactionStrategy withOptions(Map<String, String> options);
+
+  /**
+   * Selects files which this strategy believes are valid targets to be rewritten.
+   *
+   * @param dataFiles iterator of live datafiles in a given partition
+   * @return iterator containing only files to be rewritten
+   */
+  Iterable<FileScanTask> selectFilesToCompact(Iterable<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
+   */
+  Iterable<Set<FileScanTask>> groupFilesIntoChunks(Iterable<FileScanTask> dataFiles);

Review comment:
       I don't think we should use CloseableIterable, I think it adds a bit of complexity to the code which we won't really benefit from.




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


[GitHub] [iceberg] kbendick commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r627084846



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFilesV1.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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 org.apache.iceberg.DataFile;
+import org.apache.iceberg.expressions.Expression;
+
+/**
+ * An action that rewrites data files.
+ */
+public interface RewriteDataFilesV1 extends SnapshotUpdate<RewriteDataFilesV1, RewriteDataFilesV1.Result> {
+  /**
+   * Pass a row filter to filter {@link DataFile}s to be rewritten.
+   * <p>
+   * Note that all files that may contain data matching the filter may be rewritten.
+   * <p>
+   * If not set, all files will be rewritten.

Review comment:
       Oh. I see that this is just a copy of the original one. Then not necessarily in scope to update the doc here.

##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFilesV1.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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 org.apache.iceberg.DataFile;
+import org.apache.iceberg.expressions.Expression;
+
+/**
+ * An action that rewrites data files.
+ */
+public interface RewriteDataFilesV1 extends SnapshotUpdate<RewriteDataFilesV1, RewriteDataFilesV1.Result> {
+  /**
+   * Pass a row filter to filter {@link DataFile}s to be rewritten.
+   * <p>
+   * Note that all files that may contain data matching the filter may be rewritten.
+   * <p>
+   * If not set, all files will be rewritten.

Review comment:
       Am I reading this correctly, that we rewrite all files whenever there's no filter? I thought that we only considered all files for rewrite, but left some alone under different criteria.
   
   Could totally be nit-picking, but it makes it sound potentially much more expensive than it is if the docs say that every file is rewritten if some are only opened and then left as is. Might not be an important distinction though.




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r623394748



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;
+
+  /**
+   * The parallelism level to use when processing chunks generated by the Compaction Strategy. The structure and
+   * contents of the chunk 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 this for chaining
+   */
+  CompactDataFiles parallelism(int par);

Review comment:
       Swapped to an option, rather than method. max-concurrent-chunks is fine to me




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r627063475



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Rewrite Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.
  */
-public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+public interface RewriteDataFiles extends Action<RewriteDataFiles, RewriteDataFiles.Result> {
+
   /**
-   * Pass a row filter to filter {@link DataFile}s to be rewritten.
-   * <p>
-   * Note that all files that may contain data matching the filter may be rewritten.
-   * <p>
-   * If not set, all files will be rewritten.
-   *
-   * @param expr a row filter to filter out data files
-   * @return this for method chaining
+   * Enable committing groups of files (see max-file-group-size) prior to the entire compaction completing.
+   * This will produce additional commits but allow for progress even if some groups fail to commit. This setting
+   * will not change the correctness of the rewrite operation. The default is false, which produces a single commit
+   * when the entire job has completed.
    */
-  RewriteDataFiles filter(Expression expr);
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
 
   /**
-   * Enables or disables case sensitive expression binding.
-   * <p>
-   * If not set, defaults to false.
-   *
-   * @param caseSensitive caseSensitive
-   * @return this for method chaining
+   * The maximum amount of Iceberg commits that compaction is allowed to produce if partial progress is enabled.
    */
-  RewriteDataFiles caseSensitive(boolean caseSensitive);
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
 
   /**
-   * Pass a PartitionSpec id to specify which PartitionSpec should be used in DataFile rewrite
-   * <p>
-   * If not set, defaults to the table's default spec ID.
-   *
-   * @param specId PartitionSpec id to rewrite
-   * @return this for method chaining
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size into groups. These sub-units of compaction are referred to as file groups. The largest amount of data that
+   * should be compacted in a single group is controlled by MAX_FILE_GROUP_SIZE_BYTES. When grouping files, the
+   * underlying compaction strategy will use this value as to limit the files which will be included in a single file
+   * group. A group will be processed by a single framework "action". For example, in Spark this means that each group
+   * would be rewritten in its own Spark action. A group will never contain files for multiple output partitions.
    */
-  RewriteDataFiles outputSpecId(int specId);
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";
+  long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
 
   /**
-   * Specify the target data file size in bytes.
-   * <p>
-   * If not set, defaults to the table's target file size.
-   *
-   * @param targetSizeInBytes size in bytes of rewrite data file
-   * @return this for method chaining
+   * The max number of file groups to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the group is determined by the compaction strategy. Each file group will be rewritten
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_FILE_GROUP_ACTIONS = "max-concurrent-file-group-actions";
+  int MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT = 1;
+
+  /**
+   * The output file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.
+   */
+  String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes";
+
+  /**
+   * The partition spec to use when writing the output data from this operation. By default uses the
+   * current table partition spec.
    */
-  RewriteDataFiles targetSizeInBytes(long targetSizeInBytes);
+  String PARTITION_SPEC_ID = "partition-spec-id";
+
+  enum RewriteStrategyName {
+    BINPACK,
+    SORT
+  }
 
   /**
-   * Specify the number of "bins" considered when trying to pack the next file split into a task. Increasing this
-   * usually makes tasks a bit more even by considering more ways to pack file regions into a single task with extra
-   * planning cost.
-   * <p>
-   * This configuration can reorder the incoming file regions, to preserve order for lower/upper bounds in file
-   * metadata, user can use a lookback of 1.
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
    *
-   * @param splitLookback number of "bins" considered when trying to pack the next file split into a task.
+   * @param strategyName name of the strategy
    * @return this for method chaining
    */
-  RewriteDataFiles splitLookback(int splitLookback);
+  RewriteDataFiles strategy(RewriteStrategyName strategyName);
 
   /**
-   * Specify the cost of opening a file that will be taken into account during packing files into
-   * bins. If the size of the file is smaller than the cost of opening, then this value will be used
-   * instead of the actual file size.
-   * <p>
-   * If not set, defaults to the table's open file cost.
+   * 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. For example this would be used for providing a
+   * restriction to only run compaction on a specific partition.
    *
-   * @param splitOpenFileCost minimum file size to count to pack into one "bin".
-   * @return this for method chaining
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
    */
-  RewriteDataFiles splitOpenFileCost(long splitOpenFileCost);
+  RewriteDataFiles filter(Expression expression);
 
   /**
-   * The action result that contains a summary of the execution.
+   * A pairing of file group information to the result of the rewriting that file group. If the results are null then
+   * that particular file group failed. We should only have failed groups if partial progress is enabled otherwise we
+   * will report a total failure for the job.
    */
   interface Result {
+    Map<FileGroupInfo, FileGroupRewriteResult> resultMap();
+  }
+
+  interface FileGroupRewriteResult {

Review comment:
       nit: Javadoc?




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r627034911



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Rewrite Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.
  */
-public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+public interface RewriteDataFiles extends Action<RewriteDataFiles, RewriteDataFiles.Result> {
+
   /**
-   * Pass a row filter to filter {@link DataFile}s to be rewritten.
-   * <p>
-   * Note that all files that may contain data matching the filter may be rewritten.
-   * <p>
-   * If not set, all files will be rewritten.
-   *
-   * @param expr a row filter to filter out data files
-   * @return this for method chaining
+   * Enable committing groups of files (see max-file-group-size) prior to the entire compaction completing.
+   * This will produce additional commits but allow for progress even if some groups fail to commit. This setting
+   * will not change the correctness of the rewrite operation. The default is false, which produces a single commit
+   * when the entire job has completed.
    */
-  RewriteDataFiles filter(Expression expr);
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
 
   /**
-   * Enables or disables case sensitive expression binding.
-   * <p>
-   * If not set, defaults to false.
-   *
-   * @param caseSensitive caseSensitive
-   * @return this for method chaining
+   * The maximum amount of Iceberg commits that compaction is allowed to produce if partial progress is enabled.
    */
-  RewriteDataFiles caseSensitive(boolean caseSensitive);
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
 
   /**
-   * Pass a PartitionSpec id to specify which PartitionSpec should be used in DataFile rewrite
-   * <p>
-   * If not set, defaults to the table's default spec ID.
-   *
-   * @param specId PartitionSpec id to rewrite
-   * @return this for method chaining
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size into groups. These sub-units of compaction are referred to as file groups. The largest amount of data that
+   * should be compacted in a single group is controlled by MAX_FILE_GROUP_SIZE_BYTES. When grouping files, the
+   * underlying compaction strategy will use this value as to limit the files which will be included in a single file
+   * group. A group will be processed by a single framework "action". For example, in Spark this means that each group
+   * would be rewritten in its own Spark action. A group will never contain files for multiple output partitions.

Review comment:
       Shall we also mention the files are split into groups for scalability?
   




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r627086236



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Rewrite Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.
  */
-public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+public interface RewriteDataFiles extends Action<RewriteDataFiles, RewriteDataFiles.Result> {
+
   /**
-   * Pass a row filter to filter {@link DataFile}s to be rewritten.
-   * <p>
-   * Note that all files that may contain data matching the filter may be rewritten.
-   * <p>
-   * If not set, all files will be rewritten.
-   *
-   * @param expr a row filter to filter out data files
-   * @return this for method chaining
+   * Enable committing groups of files (see max-file-group-size) prior to the entire compaction completing.
+   * This will produce additional commits but allow for progress even if some groups fail to commit. This setting
+   * will not change the correctness of the rewrite operation. The default is false, which produces a single commit
+   * when the entire job has completed.
    */
-  RewriteDataFiles filter(Expression expr);
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
 
   /**
-   * Enables or disables case sensitive expression binding.
-   * <p>
-   * If not set, defaults to false.
-   *
-   * @param caseSensitive caseSensitive
-   * @return this for method chaining
+   * The maximum amount of Iceberg commits that compaction is allowed to produce if partial progress is enabled.
    */
-  RewriteDataFiles caseSensitive(boolean caseSensitive);
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
 
   /**
-   * Pass a PartitionSpec id to specify which PartitionSpec should be used in DataFile rewrite
-   * <p>
-   * If not set, defaults to the table's default spec ID.
-   *
-   * @param specId PartitionSpec id to rewrite
-   * @return this for method chaining
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size into groups. These sub-units of compaction are referred to as file groups. The largest amount of data that
+   * should be compacted in a single group is controlled by MAX_FILE_GROUP_SIZE_BYTES. When grouping files, the
+   * underlying compaction strategy will use this value as to limit the files which will be included in a single file
+   * group. A group will be processed by a single framework "action". For example, in Spark this means that each group
+   * would be rewritten in its own Spark action. A group will never contain files for multiple output partitions.
    */
-  RewriteDataFiles outputSpecId(int specId);
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";
+  long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
 
   /**
-   * Specify the target data file size in bytes.
-   * <p>
-   * If not set, defaults to the table's target file size.
-   *
-   * @param targetSizeInBytes size in bytes of rewrite data file
-   * @return this for method chaining
+   * The max number of file groups to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the group is determined by the compaction strategy. Each file group will be rewritten
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_FILE_GROUP_ACTIONS = "max-concurrent-file-group-actions";
+  int MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT = 1;
+
+  /**
+   * The output file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.
+   */
+  String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes";
+
+  /**
+   * The partition spec to use when writing the output data from this operation. By default uses the
+   * current table partition spec.
    */
-  RewriteDataFiles targetSizeInBytes(long targetSizeInBytes);
+  String PARTITION_SPEC_ID = "partition-spec-id";
+
+  enum RewriteStrategyName {
+    BINPACK,
+    SORT
+  }
 
   /**
-   * Specify the number of "bins" considered when trying to pack the next file split into a task. Increasing this
-   * usually makes tasks a bit more even by considering more ways to pack file regions into a single task with extra
-   * planning cost.
-   * <p>
-   * This configuration can reorder the incoming file regions, to preserve order for lower/upper bounds in file
-   * metadata, user can use a lookback of 1.
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
    *
-   * @param splitLookback number of "bins" considered when trying to pack the next file split into a task.
+   * @param strategyName name of the strategy
    * @return this for method chaining
    */
-  RewriteDataFiles splitLookback(int splitLookback);
+  RewriteDataFiles strategy(RewriteStrategyName strategyName);
 
   /**
-   * Specify the cost of opening a file that will be taken into account during packing files into
-   * bins. If the size of the file is smaller than the cost of opening, then this value will be used
-   * instead of the actual file size.
-   * <p>
-   * If not set, defaults to the table's open file cost.
+   * 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. For example this would be used for providing a
+   * restriction to only run compaction on a specific partition.
    *
-   * @param splitOpenFileCost minimum file size to count to pack into one "bin".
-   * @return this for method chaining
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
    */
-  RewriteDataFiles splitOpenFileCost(long splitOpenFileCost);
+  RewriteDataFiles filter(Expression expression);
 
   /**
-   * The action result that contains a summary of the execution.
+   * A pairing of file group information to the result of the rewriting that file group. If the results are null then
+   * that particular file group failed. We should only have failed groups if partial progress is enabled otherwise we
+   * will report a total failure for the job.
    */
   interface Result {
+    Map<FileGroupInfo, FileGroupRewriteResult> resultMap();
+  }

Review comment:
       We only get null in the results during partial progress, the error messages would still be logged, but not part of the result set, at least that was my idea here. Then a summary can be created from the result information we make here if that's what we want to do eventually.




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r627069070



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Rewrite Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.
  */
-public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+public interface RewriteDataFiles extends Action<RewriteDataFiles, RewriteDataFiles.Result> {
+
   /**
-   * Pass a row filter to filter {@link DataFile}s to be rewritten.
-   * <p>
-   * Note that all files that may contain data matching the filter may be rewritten.
-   * <p>
-   * If not set, all files will be rewritten.
-   *
-   * @param expr a row filter to filter out data files
-   * @return this for method chaining
+   * Enable committing groups of files (see max-file-group-size) prior to the entire compaction completing.
+   * This will produce additional commits but allow for progress even if some groups fail to commit. This setting
+   * will not change the correctness of the rewrite operation. The default is false, which produces a single commit
+   * when the entire job has completed.
    */
-  RewriteDataFiles filter(Expression expr);
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
 
   /**
-   * Enables or disables case sensitive expression binding.
-   * <p>
-   * If not set, defaults to false.
-   *
-   * @param caseSensitive caseSensitive
-   * @return this for method chaining
+   * The maximum amount of Iceberg commits that compaction is allowed to produce if partial progress is enabled.
    */
-  RewriteDataFiles caseSensitive(boolean caseSensitive);
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
 
   /**
-   * Pass a PartitionSpec id to specify which PartitionSpec should be used in DataFile rewrite
-   * <p>
-   * If not set, defaults to the table's default spec ID.
-   *
-   * @param specId PartitionSpec id to rewrite
-   * @return this for method chaining
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size into groups. These sub-units of compaction are referred to as file groups. The largest amount of data that
+   * should be compacted in a single group is controlled by MAX_FILE_GROUP_SIZE_BYTES. When grouping files, the
+   * underlying compaction strategy will use this value as to limit the files which will be included in a single file
+   * group. A group will be processed by a single framework "action". For example, in Spark this means that each group
+   * would be rewritten in its own Spark action. A group will never contain files for multiple output partitions.
    */
-  RewriteDataFiles outputSpecId(int specId);
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";
+  long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
 
   /**
-   * Specify the target data file size in bytes.
-   * <p>
-   * If not set, defaults to the table's target file size.
-   *
-   * @param targetSizeInBytes size in bytes of rewrite data file
-   * @return this for method chaining
+   * The max number of file groups to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the group is determined by the compaction strategy. Each file group will be rewritten
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_FILE_GROUP_ACTIONS = "max-concurrent-file-group-actions";
+  int MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT = 1;
+
+  /**
+   * The output file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.
+   */
+  String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes";
+
+  /**
+   * The partition spec to use when writing the output data from this operation. By default uses the
+   * current table partition spec.
    */
-  RewriteDataFiles targetSizeInBytes(long targetSizeInBytes);
+  String PARTITION_SPEC_ID = "partition-spec-id";
+
+  enum RewriteStrategyName {
+    BINPACK,
+    SORT

Review comment:
       It' doesn't really matter since the implementation in BaseActions can always just say "not available", we'll have to do this for flink for example until we have a sorted writer there




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r618011274



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";

Review comment:
       By default it will do as many chunks per commit as required to hit our PARTIAL_PROGRESS_MAX_COMMITS parameter below. I've already prototyped a mechanism for doing this that I'll submit once we get this whole api settled.




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617747892



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;
+
+  /**
+   * The parallelism level to use when processing chunks generated by the Compaction Strategy. The structure and
+   * contents of the chunk 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 this for chaining
+   */
+  CompactDataFiles parallelism(int par);
+
+  /**
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
+   *
+   * @param strategyName name of the strategy
+   * @return this for chaining
+   */
+  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. For example this would be used for providing a
+   * restriction to only run compaction on a specific partition.
+   *
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
+   */
+  CompactDataFiles filter(Expression expression);
+
+  /**
+   * A pairing of Chunk information to the result of that chunk's results. If the results are null then that particular
+   * chunk failed. We should only have failed chunks if partial progress is enabled.
+   */
+  class Result {
+    private Map<CompactionChunkInfo, CompactionChunkResult> resultMap;
+
+
+    public Result(Map<CompactionChunkInfo, CompactionChunkResult> resultMap) {
+      this.resultMap = resultMap;
+    }
+
+    public Map<CompactionChunkInfo, CompactionChunkResult> resultMap() {
+      return resultMap;
+    }
+  }
+
+  interface CompactionChunkResult {
+
+    int numFilesAdded();
+
+    int numFilesRemoved();
+  }
+
+  interface CompactionChunkInfo {
+
+    /**
+     * returns which chunk this is out of the total set of chunks for this compaction
+     */
+    int chunkIndex();
+
+    /**
+     * returns which chunk this is out of the set of chunks for this partition
+     */
+    int partitionIndex();

Review comment:
       Mostly for Logging, it and providing information in the Spark UI. For example you would want to know how far into a compaction job you are and where in the partition you are. We've had this issue where a Compaction is running but it is difficult for an observer to tell how close it is to finishing.




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r625271812



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,123 @@
+/*
+ * 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 org.apache.iceberg.StructLike;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior see max-chunk-size to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when the entire job has completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  // TODO be set once we have an ENUM in core
+  // String COMPACTION_STRATEGY_DEFAULT;
+
+  /**
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size. These sub-units of compaction are referred to as chunks. The largest amount of data that should be
+   * compacted in a single chunk is controlled by MAX_CHUNK_SIZE_BYTES. When grouping files, the underlying
+   * compaction strategy will use this value to but an upper bound on the number of files included in a single
+   * chunk. A chunk will be processed by a single framework "job". For example, in Spark this means that each chunk
+   * would be processed in it's own Spark action. A chunk will never contain files for multiple output partitions.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max-chunk-size-bytes";
+  long MAX_CHUNK_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.
+   */
+  String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes";
+
+  /**
+   * The max number of chunks to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the chunk is determined by the compaction strategy. When running each job chunk will be run
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_CHUNKS = "max-concurrent-chunks";

Review comment:
       I think that's ok, as long as we think "Action" is a global enough descriptor to apply to other frameworks that spark. I really feel like all the parameters should use the same noun so if it's CONCURRENT_ACTIONS we should have  MAX_ACTION_SIZE_BYTES?
   
   I'm willing to go with anything we have consensus on




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


[GitHub] [iceberg] rdblue commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617744072



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;
+
+  /**
+   * The parallelism level to use when processing chunks generated by the Compaction Strategy. The structure and
+   * contents of the chunk 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 this for chaining
+   */
+  CompactDataFiles parallelism(int par);
+
+  /**
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
+   *
+   * @param strategyName name of the strategy
+   * @return this for chaining
+   */
+  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. For example this would be used for providing a
+   * restriction to only run compaction on a specific partition.
+   *
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
+   */
+  CompactDataFiles filter(Expression expression);
+
+  /**
+   * A pairing of Chunk information to the result of that chunk's results. If the results are null then that particular
+   * chunk failed. We should only have failed chunks if partial progress is enabled.
+   */
+  class Result {

Review comment:
       I agree we should probably be consistent. I don't like impl classes in general without hiding them inline and having a factory method, like `Result.create`, to instantiate them.




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r618523681



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {

Review comment:
       I was wondering if we really need the output spec. Shouldn't we always be targeting the latest spec? and if someone wants to change spec and then optimize, then they should do it as two operations?




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


[GitHub] [iceberg] rdblue commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617738665



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";

Review comment:
       I don't think that it is obvious what "chunk" is without the Javadoc, so I'm not sure I would make this public. I'll try to think about alternatives.




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r619467425



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.expressions.Expression;
+
+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();
+
+  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. This would be used for pushing down filter expressions to the manifest entry scanning phase.
+   *
+   * @return an Iceberg expression to use when discovering file scan tasks
+   */
+  Expression preFilter();

Review comment:
       Do we need this in the first iteration? I think the proposed strategies won't use it. I'd add this once needed.




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


[GitHub] [iceberg] aokolnychyi commented on pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#issuecomment-824214891


   I had only minor questions/comments. Let's hear from other folks before updating. 


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


[GitHub] [iceberg] rdblue commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617746324



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;
+
+  /**
+   * The parallelism level to use when processing chunks generated by the Compaction Strategy. The structure and
+   * contents of the chunk 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 this for chaining
+   */
+  CompactDataFiles parallelism(int par);
+
+  /**
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
+   *
+   * @param strategyName name of the strategy
+   * @return this for chaining
+   */
+  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. For example this would be used for providing a
+   * restriction to only run compaction on a specific partition.
+   *
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
+   */
+  CompactDataFiles filter(Expression expression);
+
+  /**
+   * A pairing of Chunk information to the result of that chunk's results. If the results are null then that particular
+   * chunk failed. We should only have failed chunks if partial progress is enabled.
+   */
+  class Result {
+    private Map<CompactionChunkInfo, CompactionChunkResult> resultMap;
+
+
+    public Result(Map<CompactionChunkInfo, CompactionChunkResult> resultMap) {
+      this.resultMap = resultMap;
+    }
+
+    public Map<CompactionChunkInfo, CompactionChunkResult> resultMap() {
+      return resultMap;
+    }
+  }
+
+  interface CompactionChunkResult {
+
+    int numFilesAdded();
+
+    int numFilesRemoved();
+  }
+
+  interface CompactionChunkInfo {
+
+    /**
+     * returns which chunk this is out of the total set of chunks for this compaction
+     */
+    int chunkIndex();
+
+    /**
+     * returns which chunk this is out of the set of chunks for this partition
+     */
+    int partitionIndex();

Review comment:
       What are these indexes useful for? Does the caller know about the ahead of time?
   
   I'd prefer to keep these results simple if we don't need these.




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r627073290



##########
File path: api/src/main/java/org/apache/iceberg/actions/rewrite/RewriteStrategy.java
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.rewrite;
+
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+
+interface RewriteStrategy 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();

Review comment:
       I support the idea of validating options and throwing an exception but we may do that in the strategy when the action passes options to us. I don't mind an explicit method that exposes supported options too.
   
   I'm fine either way, 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:
users@infra.apache.org



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


[GitHub] [iceberg] rdblue commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617745589



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;
+
+  /**
+   * The parallelism level to use when processing chunks generated by the Compaction Strategy. The structure and
+   * contents of the chunk 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 this for chaining
+   */
+  CompactDataFiles parallelism(int par);
+
+  /**
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
+   *
+   * @param strategyName name of the strategy
+   * @return this for chaining
+   */
+  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. For example this would be used for providing a
+   * restriction to only run compaction on a specific partition.
+   *
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
+   */
+  CompactDataFiles filter(Expression expression);
+
+  /**
+   * A pairing of Chunk information to the result of that chunk's results. If the results are null then that particular
+   * chunk failed. We should only have failed chunks if partial progress is enabled.
+   */
+  class Result {
+    private Map<CompactionChunkInfo, CompactionChunkResult> resultMap;
+
+
+    public Result(Map<CompactionChunkInfo, CompactionChunkResult> resultMap) {
+      this.resultMap = resultMap;
+    }
+
+    public Map<CompactionChunkInfo, CompactionChunkResult> resultMap() {
+      return resultMap;
+    }
+  }
+
+  interface CompactionChunkResult {
+
+    int numFilesAdded();
+
+    int numFilesRemoved();
+  }
+
+  interface CompactionChunkInfo {
+
+    /**
+     * returns which chunk this is out of the total set of chunks for this compaction
+     */
+    int chunkIndex();
+
+    /**
+     * returns which chunk this is out of the set of chunks for this partition
+     */
+    int partitionIndex();
+
+    /**
+     * returns which partition this chunk contains files from
+     */
+    String partition();

Review comment:
       There's a method to do this in `PartitionSpec`, but I agree that we shouldn't convert to String too early. If we want to return this data to the caller then I think it makes sense to return `StructLike`.




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r627070469



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Rewrite Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.
  */
-public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+public interface RewriteDataFiles extends Action<RewriteDataFiles, RewriteDataFiles.Result> {
+
   /**
-   * Pass a row filter to filter {@link DataFile}s to be rewritten.
-   * <p>
-   * Note that all files that may contain data matching the filter may be rewritten.
-   * <p>
-   * If not set, all files will be rewritten.
-   *
-   * @param expr a row filter to filter out data files
-   * @return this for method chaining
+   * Enable committing groups of files (see max-file-group-size) prior to the entire compaction completing.
+   * This will produce additional commits but allow for progress even if some groups fail to commit. This setting
+   * will not change the correctness of the rewrite operation. The default is false, which produces a single commit
+   * when the entire job has completed.
    */
-  RewriteDataFiles filter(Expression expr);
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
 
   /**
-   * Enables or disables case sensitive expression binding.
-   * <p>
-   * If not set, defaults to false.
-   *
-   * @param caseSensitive caseSensitive
-   * @return this for method chaining
+   * The maximum amount of Iceberg commits that compaction is allowed to produce if partial progress is enabled.
    */
-  RewriteDataFiles caseSensitive(boolean caseSensitive);
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
 
   /**
-   * Pass a PartitionSpec id to specify which PartitionSpec should be used in DataFile rewrite
-   * <p>
-   * If not set, defaults to the table's default spec ID.
-   *
-   * @param specId PartitionSpec id to rewrite
-   * @return this for method chaining
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size into groups. These sub-units of compaction are referred to as file groups. The largest amount of data that
+   * should be compacted in a single group is controlled by MAX_FILE_GROUP_SIZE_BYTES. When grouping files, the
+   * underlying compaction strategy will use this value as to limit the files which will be included in a single file
+   * group. A group will be processed by a single framework "action". For example, in Spark this means that each group
+   * would be rewritten in its own Spark action. A group will never contain files for multiple output partitions.
    */
-  RewriteDataFiles outputSpecId(int specId);
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";
+  long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
 
   /**
-   * Specify the target data file size in bytes.
-   * <p>
-   * If not set, defaults to the table's target file size.
-   *
-   * @param targetSizeInBytes size in bytes of rewrite data file
-   * @return this for method chaining
+   * The max number of file groups to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the group is determined by the compaction strategy. Each file group will be rewritten
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_FILE_GROUP_ACTIONS = "max-concurrent-file-group-actions";
+  int MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT = 1;
+
+  /**
+   * The output file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.
+   */
+  String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes";
+
+  /**
+   * The partition spec to use when writing the output data from this operation. By default uses the
+   * current table partition spec.
    */
-  RewriteDataFiles targetSizeInBytes(long targetSizeInBytes);
+  String PARTITION_SPEC_ID = "partition-spec-id";

Review comment:
       I'll go with the second, I feel like we have way to many places where we use "spec-id" and don't specify that this has to do with partitioning and it's confusing to me




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


[GitHub] [iceberg] kbendick commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r627084846



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFilesV1.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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 org.apache.iceberg.DataFile;
+import org.apache.iceberg.expressions.Expression;
+
+/**
+ * An action that rewrites data files.
+ */
+public interface RewriteDataFilesV1 extends SnapshotUpdate<RewriteDataFilesV1, RewriteDataFilesV1.Result> {
+  /**
+   * Pass a row filter to filter {@link DataFile}s to be rewritten.
+   * <p>
+   * Note that all files that may contain data matching the filter may be rewritten.
+   * <p>
+   * If not set, all files will be rewritten.

Review comment:
       Oh. I see that this is just a copy of the original one. Then not necessarily in scope to update the doc here.
   
   I'm going to close this since this is just a copy of the original.




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


[GitHub] [iceberg] rdblue commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617771422



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.expressions.Expression;
+
+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();
+
+  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. This would be used for pushing down filter expressions to the manifest entry scanning phase.
+   *
+   * @return an Iceberg expression to use when discovering file scan tasks
+   */
+  Expression preFilter();
+
+  /**
+   * 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

Review comment:
       This method should be `Serializable`? I think this is a class requirement.




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r627073428



##########
File path: api/src/main/java/org/apache/iceberg/actions/rewrite/RewriteStrategy.java
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.rewrite;
+
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+
+interface RewriteStrategy 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();
+
+  RewriteStrategy withOptions(Map<String, String> options);
+
+  /**
+   * Selects files which this strategy believes are valid targets to be rewritten.
+   *
+   * @param dataFiles iterable of FileScanTasks for files in a given partition
+   * @return iterable containing only FileScanTasks to be rewritten
+   */
+  Iterable<FileScanTask> selectFilesToCompact(Iterable<FileScanTask> dataFiles);

Review comment:
       nit: `toCompact` -> `toRewrite`




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617805996



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.expressions.Expression;
+
+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();
+
+  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. This would be used for pushing down filter expressions to the manifest entry scanning phase.
+   *
+   * @return an Iceberg expression to use when discovering file scan tasks
+   */
+  Expression preFilter();
+
+  /**
+   * 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);

Review comment:
       With Sort for example we need to see a file in context with others to know if we are going to keep it. So either we have access to the whole set of files here, or the implementer needs to keep static state about previous file's seen.




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r629672049



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,129 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting data files according to a rewrite strategy.
+ * Generally used for optimizing the sizing and layout of data files within a table.
  */
 public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+
   /**
-   * Pass a row filter to filter {@link DataFile}s to be rewritten.
-   * <p>
-   * Note that all files that may contain data matching the filter may be rewritten.
+   * Enable committing groups of files (see max-file-group-size) prior to the entire rewrite completing.

Review comment:
       nit: `max-file-group-size` -> `max-file-group-size-bytes`

##########
File path: api/src/main/java/org/apache/iceberg/actions/rewrite/RewriteStrategy.java
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.rewrite;
+
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+
+interface RewriteStrategy 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();

Review comment:
       Makes sense.

##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Rewrite Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.
  */
-public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+public interface RewriteDataFiles extends Action<RewriteDataFiles, RewriteDataFiles.Result> {
+
   /**
-   * Pass a row filter to filter {@link DataFile}s to be rewritten.
-   * <p>
-   * Note that all files that may contain data matching the filter may be rewritten.
-   * <p>
-   * If not set, all files will be rewritten.
-   *
-   * @param expr a row filter to filter out data files
-   * @return this for method chaining
+   * Enable committing groups of files (see max-file-group-size) prior to the entire compaction completing.
+   * This will produce additional commits but allow for progress even if some groups fail to commit. This setting
+   * will not change the correctness of the rewrite operation. The default is false, which produces a single commit
+   * when the entire job has completed.
    */
-  RewriteDataFiles filter(Expression expr);
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
 
   /**
-   * Enables or disables case sensitive expression binding.
-   * <p>
-   * If not set, defaults to false.
-   *
-   * @param caseSensitive caseSensitive
-   * @return this for method chaining
+   * The maximum amount of Iceberg commits that compaction is allowed to produce if partial progress is enabled.
    */
-  RewriteDataFiles caseSensitive(boolean caseSensitive);
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
 
   /**
-   * Pass a PartitionSpec id to specify which PartitionSpec should be used in DataFile rewrite
-   * <p>
-   * If not set, defaults to the table's default spec ID.
-   *
-   * @param specId PartitionSpec id to rewrite
-   * @return this for method chaining
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size into groups. These sub-units of compaction are referred to as file groups. The largest amount of data that
+   * should be compacted in a single group is controlled by MAX_FILE_GROUP_SIZE_BYTES. When grouping files, the
+   * underlying compaction strategy will use this value as to limit the files which will be included in a single file
+   * group. A group will be processed by a single framework "action". For example, in Spark this means that each group
+   * would be rewritten in its own Spark action. A group will never contain files for multiple output partitions.
    */
-  RewriteDataFiles outputSpecId(int specId);
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";
+  long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
 
   /**
-   * Specify the target data file size in bytes.
-   * <p>
-   * If not set, defaults to the table's target file size.
-   *
-   * @param targetSizeInBytes size in bytes of rewrite data file
-   * @return this for method chaining
+   * The max number of file groups to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the group is determined by the compaction strategy. Each file group will be rewritten
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_FILE_GROUP_ACTIONS = "max-concurrent-file-group-actions";
+  int MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT = 1;
+
+  /**
+   * The output file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.
+   */
+  String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes";
+
+  /**
+   * The partition spec to use when writing the output data from this operation. By default uses the
+   * current table partition spec.
    */
-  RewriteDataFiles targetSizeInBytes(long targetSizeInBytes);
+  String PARTITION_SPEC_ID = "partition-spec-id";
+
+  enum RewriteStrategyName {
+    BINPACK,
+    SORT

Review comment:
       The thing is I am not sure it is going to be called SORT. Technically, it could be called CLUSTER or something as we may include sorting in the the bin-pack strategy if the table has a sort key defined.




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617951966



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";

Review comment:
       Any naming alternatives would helpful. We have already discarded job, unit and a few others.




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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r624622663



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,123 @@
+/*
+ * 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 org.apache.iceberg.StructLike;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior see max-chunk-size to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when the entire job has completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  // TODO be set once we have an ENUM in core
+  // String COMPACTION_STRATEGY_DEFAULT;
+
+  /**
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size. These sub-units of compaction are referred to as chunks. The largest amount of data that should be
+   * compacted in a single chunk is controlled by MAX_CHUNK_SIZE_BYTES. When grouping files, the underlying
+   * compaction strategy will use this value to but an upper bound on the number of files included in a single
+   * chunk. A chunk will be processed by a single framework "job". For example, in Spark this means that each chunk
+   * would be processed in it's own Spark action. A chunk will never contain files for multiple output partitions.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max-chunk-size-bytes";
+  long MAX_CHUNK_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.
+   */
+  String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes";
+
+  /**
+   * The max number of chunks to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the chunk is determined by the compaction strategy. When running each job chunk will be run
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_CHUNKS = "max-concurrent-chunks";

Review comment:
       what about name `Chunk` as `Assignment`? Action for an assignment.
   




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r619461223



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;
+
+  /**
+   * The parallelism level to use when processing chunks generated by the Compaction Strategy. The structure and
+   * contents of the chunk 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 this for chaining
+   */
+  CompactDataFiles parallelism(int par);

Review comment:
       One option is to include whatever we decide to call our "chunks". Something like `maxNumConcurrentChunks`. I don't really like that particular name but if anyone has a good alternative, this may work.
   
   Another idea I currently have is to be as specific as possible. This will actually control the thread pool size that will submit chunks. So something like `submitThreadPoolSize` can be an option.




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


[GitHub] [iceberg] RussellSpitzer commented on pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#issuecomment-830264629


   @yyanyy The Chunk info was going to a conatiner class used for indentifying a chunk within the baseaction class, I just choose to define it here as well so that we can include it in the result set rather than making two different container classes.
   
   So an implementation will have something like
   ```
   Create Groups
   Sort Groups 
   Tasks.run(groups) 
   {
      Create Info Class
      Log details based on info
      Perform Rewrite
      Add Info,Result to resultMap
   }
   ```
   At least this is my thought processes


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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617753257



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {

Review comment:
       I think we should go for Optimize, I feel like Rewrite is a bit difficult because a successful run of this command can end up not changing anything. Which I think makes more sense for Optimize which if it does nothing, implies it is already optimized.




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r618529121



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.expressions.Expression;
+
+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();
+
+  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. This would be used for pushing down filter expressions to the manifest entry scanning phase.
+   *
+   * @return an Iceberg expression to use when discovering file scan tasks
+   */
+  Expression preFilter();
+
+  /**
+   * 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>> groupFilesIntoChunks(Iterator<FileScanTask> dataFiles);
+
+  /**
+   * Method which will rewrite files based on this particular DataCompactionStrategy'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 written files
+   */
+  List<DataFile> rewriteFiles(Table table, List<FileScanTask> filesToRewrite);

Review comment:
       That's my current thought, Then when you choose a strategy the action would know how to map that enum value to the correct one for it's particular implementation.




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r627035224



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Rewrite Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.
  */
-public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+public interface RewriteDataFiles extends Action<RewriteDataFiles, RewriteDataFiles.Result> {
+
   /**
-   * Pass a row filter to filter {@link DataFile}s to be rewritten.
-   * <p>
-   * Note that all files that may contain data matching the filter may be rewritten.
-   * <p>
-   * If not set, all files will be rewritten.
-   *
-   * @param expr a row filter to filter out data files
-   * @return this for method chaining
+   * Enable committing groups of files (see max-file-group-size) prior to the entire compaction completing.
+   * This will produce additional commits but allow for progress even if some groups fail to commit. This setting
+   * will not change the correctness of the rewrite operation. The default is false, which produces a single commit
+   * when the entire job has completed.
    */
-  RewriteDataFiles filter(Expression expr);
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
 
   /**
-   * Enables or disables case sensitive expression binding.
-   * <p>
-   * If not set, defaults to false.
-   *
-   * @param caseSensitive caseSensitive
-   * @return this for method chaining
+   * The maximum amount of Iceberg commits that compaction is allowed to produce if partial progress is enabled.
    */
-  RewriteDataFiles caseSensitive(boolean caseSensitive);
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
 
   /**
-   * Pass a PartitionSpec id to specify which PartitionSpec should be used in DataFile rewrite
-   * <p>
-   * If not set, defaults to the table's default spec ID.
-   *
-   * @param specId PartitionSpec id to rewrite
-   * @return this for method chaining
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size into groups. These sub-units of compaction are referred to as file groups. The largest amount of data that
+   * should be compacted in a single group is controlled by MAX_FILE_GROUP_SIZE_BYTES. When grouping files, the
+   * underlying compaction strategy will use this value as to limit the files which will be included in a single file
+   * group. A group will be processed by a single framework "action". For example, in Spark this means that each group
+   * would be rewritten in its own Spark action. A group will never contain files for multiple output partitions.
    */
-  RewriteDataFiles outputSpecId(int specId);
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";

Review comment:
       +1 on calling it file group




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617960670



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.expressions.Expression;
+
+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();
+
+  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. This would be used for pushing down filter expressions to the manifest entry scanning phase.
+   *
+   * @return an Iceberg expression to use when discovering file scan tasks
+   */
+  Expression preFilter();
+
+  /**
+   * 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>> groupFilesIntoChunks(Iterator<FileScanTask> dataFiles);
+
+  /**
+   * Method which will rewrite files based on this particular DataCompactionStrategy'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 written files
+   */
+  List<DataFile> rewriteFiles(Table table, List<FileScanTask> filesToRewrite);

Review comment:
       @RussellSpitzer, is the plan to have BaseXXX strategies in core and then SparkXXX and FlinkXXX strategies that would implement `rewriteFiles` differently?




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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r626252215



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+
+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();
+
+  DataCompactionStrategy withOptions(Map<String, String> options);
+
+  /**
+   * Selects files which this strategy believes are valid targets to be rewritten.
+   *
+   * @param dataFiles iterator of live datafiles in a given partition
+   * @return iterator containing only files to be rewritten
+   */
+  Iterable<FileScanTask> selectFilesToCompact(Iterable<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
+   */
+  Iterable<Set<FileScanTask>> groupFilesIntoChunks(Iterable<FileScanTask> dataFiles);

Review comment:
       agree with you on Iterable. maybe just use `CombinedScanTask` interface and return `Iterable<CombinedScanTask>`. Each CombinedScanTask contains a group of files for rewrite action?




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617957808



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;
+
+  /**
+   * The parallelism level to use when processing chunks generated by the Compaction Strategy. The structure and
+   * contents of the chunk 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 this for chaining
+   */
+  CompactDataFiles parallelism(int par);
+
+  /**
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
+   *
+   * @param strategyName name of the strategy
+   * @return this for chaining
+   */
+  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. For example this would be used for providing a
+   * restriction to only run compaction on a specific partition.
+   *
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
+   */
+  CompactDataFiles filter(Expression expression);
+
+  /**
+   * A pairing of Chunk information to the result of that chunk's results. If the results are null then that particular
+   * chunk failed. We should only have failed chunks if partial progress is enabled.
+   */
+  class Result {
+    private Map<CompactionChunkInfo, CompactionChunkResult> resultMap;
+
+
+    public Result(Map<CompactionChunkInfo, CompactionChunkResult> resultMap) {
+      this.resultMap = resultMap;
+    }
+
+    public Map<CompactionChunkInfo, CompactionChunkResult> resultMap() {
+      return resultMap;
+    }
+  }
+
+  interface CompactionChunkResult {
+
+    int numFilesAdded();
+
+    int numFilesRemoved();
+  }
+
+  interface CompactionChunkInfo {
+
+    /**
+     * returns which chunk this is out of the total set of chunks for this compaction
+     */
+    int chunkIndex();
+
+    /**
+     * returns which chunk this is out of the set of chunks for this partition
+     */
+    int partitionIndex();

Review comment:
       I think it would be helpful to show the partition we compact and the chunk index within that partition so that users have an idea of what we currently optimize.




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r623410308



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.expressions.Expression;
+
+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();
+
+  DataCompactionStrategy withOptions(Map<String, String> options);

Review comment:
       Yeah, the Action will be passing through actions that are passed to it




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


[GitHub] [iceberg] jackye1995 commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r620821355



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";

Review comment:
       I see, that sounds good to me then.




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r619463957



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;
+
+  /**
+   * The parallelism level to use when processing chunks generated by the Compaction Strategy. The structure and
+   * contents of the chunk 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 this for chaining
+   */
+  CompactDataFiles parallelism(int par);
+
+  /**
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
+   *
+   * @param strategyName name of the strategy
+   * @return this for chaining
+   */
+  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. For example this would be used for providing a
+   * restriction to only run compaction on a specific partition.
+   *
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
+   */
+  CompactDataFiles filter(Expression expression);
+
+  /**
+   * A pairing of Chunk information to the result of that chunk's results. If the results are null then that particular
+   * chunk failed. We should only have failed chunks if partial progress is enabled.
+   */
+  class Result {
+    private Map<CompactionChunkInfo, CompactionChunkResult> resultMap;
+
+
+    public Result(Map<CompactionChunkInfo, CompactionChunkResult> resultMap) {
+      this.resultMap = resultMap;
+    }
+
+    public Map<CompactionChunkInfo, CompactionChunkResult> resultMap() {
+      return resultMap;
+    }
+  }
+
+  interface CompactionChunkResult {
+
+    int numFilesAdded();
+
+    int numFilesRemoved();
+  }
+
+  interface CompactionChunkInfo {
+
+    /**
+     * returns which chunk this is out of the total set of chunks for this compaction
+     */
+    int chunkIndex();

Review comment:
       I definitely agree knowing how many partition we have already compacted and how many left is going to be useful in Spark UI. I am not sure about the results, though. I feel like the overall progress is something we can track in the action and the result may include a chunk id which will be specific to a partition. I.e., it will start from 1 to num_chunks in each partition.




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


[GitHub] [iceberg] RussellSpitzer commented on pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#issuecomment-829609056


   Went through all the comments I could. Made changes were asked, if anyone any additional feedback please let me know. I'd like to post a PR with the Spark Action and Bin-pack implementation asap.


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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r624567994



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,123 @@
+/*
+ * 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 org.apache.iceberg.StructLike;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior see max-chunk-size to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when the entire job has completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  // TODO be set once we have an ENUM in core
+  // String COMPACTION_STRATEGY_DEFAULT;
+
+  /**
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size. These sub-units of compaction are referred to as chunks. The largest amount of data that should be
+   * compacted in a single chunk is controlled by MAX_CHUNK_SIZE_BYTES. When grouping files, the underlying
+   * compaction strategy will use this value to but an upper bound on the number of files included in a single

Review comment:
       typo? "but an upper bound" -> "put an upper bound"?




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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r624606278



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,123 @@
+/*
+ * 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 org.apache.iceberg.StructLike;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior see max-chunk-size to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when the entire job has completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  // TODO be set once we have an ENUM in core
+  // String COMPACTION_STRATEGY_DEFAULT;
+
+  /**
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size. These sub-units of compaction are referred to as chunks. The largest amount of data that should be
+   * compacted in a single chunk is controlled by MAX_CHUNK_SIZE_BYTES. When grouping files, the underlying
+   * compaction strategy will use this value to but an upper bound on the number of files included in a single
+   * chunk. A chunk will be processed by a single framework "job". For example, in Spark this means that each chunk
+   * would be processed in it's own Spark action. A chunk will never contain files for multiple output partitions.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max-chunk-size-bytes";
+  long MAX_CHUNK_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.
+   */
+  String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes";
+
+  /**
+   * The max number of chunks to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the chunk is determined by the compaction strategy. When running each job chunk will be run
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_CHUNKS = "max-concurrent-chunks";
+  int MAX_CONCURRENT_CHUNKS_DEFAULT = 1;
+
+  /**
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
+   *
+   * @param strategyName name of the strategy
+   * @return this for method chaining
+   */
+  CompactDataFiles strategy(String strategyName);

Review comment:
       Should we define a `enum Strategy`?




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


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r623562836



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+
+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();
+
+  DataCompactionStrategy withOptions(Map<String, String> options);
+
+  /**
+   * Removes all file references which this plan will not rewrite or change. Unlike the preFilter, this method can

Review comment:
       Unlike the preFilter -> Unlike the {@link filter}




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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r626221881



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+
+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();
+
+  DataCompactionStrategy withOptions(Map<String, String> options);
+
+  /**
+   * Selects files which this strategy believes are valid targets to be rewritten.
+   *
+   * @param dataFiles iterator of live datafiles in a given partition
+   * @return iterator containing only files to be rewritten
+   */
+  Iterable<FileScanTask> selectFilesToCompact(Iterable<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
+   */
+  Iterable<Set<FileScanTask>> groupFilesIntoChunks(Iterable<FileScanTask> dataFiles);

Review comment:
       just `groupFiles`? or `planFileGroups`?




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r627082661



##########
File path: api/src/main/java/org/apache/iceberg/actions/rewrite/RewriteStrategy.java
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.rewrite;
+
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+
+interface RewriteStrategy 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();

Review comment:
       I think this sort of thing is helpful for future "help" commands since we can then have each strategy displayed with it's valid options or an error message can correctly report the options which were allowed and not allowed without a implementation specific error message.




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


[GitHub] [iceberg] rdblue commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617742473



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;
+
+  /**
+   * The parallelism level to use when processing chunks generated by the Compaction Strategy. The structure and
+   * contents of the chunk 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 this for chaining
+   */
+  CompactDataFiles parallelism(int par);
+
+  /**
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
+   *
+   * @param strategyName name of the strategy
+   * @return this for chaining
+   */
+  CompactDataFiles compactionStrategy(String strategyName);
+
+  /**
+   * A user provided filter for determining which files will be considered by the Compaction strategy. This will be used

Review comment:
       Nit: This capitalizes "compaction strategy" differently than in other methods' Javadoc. In general, I'd prefer not to capitalize nouns just to call them out. I think "compaction strategy" makes sense without the capitalization.




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617949833



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {

Review comment:
       I am flexible here. I think `RewriteDataFiles` was fine as it was generic enough to indicate we are rewriting data files without changing data in them. In addition, users are already familiar with the existing name. Not a strong opinion. If we are to change the name, now is the time to do this.
   
   As this action can be used to rewrite data with another output spec, using `OptimizeDataFiles` may not fit very well. 
   
   




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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r626221736



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Compaction Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.
  */
-public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+public interface RewriteDataFiles extends Action<RewriteDataFiles, RewriteDataFiles.Result> {
+
   /**
-   * Pass a row filter to filter {@link DataFile}s to be rewritten.
-   * <p>
-   * Note that all files that may contain data matching the filter may be rewritten.
-   * <p>
-   * If not set, all files will be rewritten.
-   *
-   * @param expr a row filter to filter out data files
-   * @return this for method chaining
+   * Enable committing groups of files (see max-file-group-size) prior to the entire compaction completing.
+   * This will produce additional commits but allow for progress even if some groups fail to commit. This setting
+   * will not change the correctness of the rewrite operation. The default is false, which produces a single commit
+   * when the entire job has completed.
    */
-  RewriteDataFiles filter(Expression expr);
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
 
   /**
-   * Enables or disables case sensitive expression binding.
-   * <p>
-   * If not set, defaults to false.
-   *
-   * @param caseSensitive caseSensitive
-   * @return this for method chaining
+   * The maximum amount of Iceberg commits that compaction is allowed to produce if partial progress is enabled.
    */
-  RewriteDataFiles caseSensitive(boolean caseSensitive);
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
 
   /**
-   * Pass a PartitionSpec id to specify which PartitionSpec should be used in DataFile rewrite
-   * <p>
-   * If not set, defaults to the table's default spec ID.
-   *
-   * @param specId PartitionSpec id to rewrite
-   * @return this for method chaining
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size into groups. These sub-units of compaction are referred to as file groups. The largest amount of data that
+   * should be compacted in a single group is controlled by MAX_FILE_GROUP_SIZE_BYTES. When grouping files, the
+   * underlying compaction strategy will use this value as to limit the files which will be included in a single file
+   * group. A group will be processed by a single framework "action". For example, in Spark this means that each group
+   * would be rewritten in its own Spark action. A group will never contain files for multiple output partitions.
    */
-  RewriteDataFiles outputSpecId(int specId);
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";
+  long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
 
   /**
-   * Specify the target data file size in bytes.
-   * <p>
-   * If not set, defaults to the table's target file size.
-   *
-   * @param targetSizeInBytes size in bytes of rewrite data file
-   * @return this for method chaining
+   * The max number of file groups to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the group is determined by the compaction strategy. Each file group will be rewritten
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_FILE_GROUP_ACTIONS = "max-concurrent-file-group-actions";
+  int MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT = 1;
+
+  /**
+   * The output file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.
+   */
+  String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes";
+
+  /**
+   * The partition spec to use when writing the output data from this operation. By default uses the
+   * current table partition spec.
    */
-  RewriteDataFiles targetSizeInBytes(long targetSizeInBytes);
+  String PARTITION_SPEC_ID = "partition-spec-id";
+
+  enum CompactionStrategyName {
+    BinPack,
+    Sort
+  }
 
   /**
-   * Specify the number of "bins" considered when trying to pack the next file split into a task. Increasing this
-   * usually makes tasks a bit more even by considering more ways to pack file regions into a single task with extra
-   * planning cost.
-   * <p>
-   * This configuration can reorder the incoming file regions, to preserve order for lower/upper bounds in file
-   * metadata, user can use a lookback of 1.
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
    *
-   * @param splitLookback number of "bins" considered when trying to pack the next file split into a task.
+   * @param strategyName name of the strategy
    * @return this for method chaining
    */
-  RewriteDataFiles splitLookback(int splitLookback);
+  RewriteDataFiles strategy(CompactionStrategyName strategyName);
 
   /**
-   * Specify the cost of opening a file that will be taken into account during packing files into
-   * bins. If the size of the file is smaller than the cost of opening, then this value will be used
-   * instead of the actual file size.
-   * <p>
-   * If not set, defaults to the table's open file cost.
+   * 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. For example this would be used for providing a
+   * restriction to only run compaction on a specific partition.
    *
-   * @param splitOpenFileCost minimum file size to count to pack into one "bin".
-   * @return this for method chaining
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
    */
-  RewriteDataFiles splitOpenFileCost(long splitOpenFileCost);
+  RewriteDataFiles filter(Expression expression);
 
   /**
-   * The action result that contains a summary of the execution.
+   * A pairing of file group information to the result of the rewriting that file group. If the results are null then
+   * that particular chunk failed. We should only have failed groups if partial progress is enabled otherwise we will
+   * report a total failure for the job.
    */
   interface Result {
+    Map<FileGroupInfo, FileGroupResult> resultMap();
+  }
+
+  interface FileGroupResult {
+    int addedDataFilesCount();
+
+    int rewrittenDataFilesCount();
+  }
+
+  /**
+   * A description of a file group, when it was processed, and within which partition. For use
+   * tracking rewrite operations and for returning results.
+   */
+  interface FileGroupInfo {
+
+    /**
+     * returns which chunk this is out of the total set of chunks for this compaction

Review comment:
       chunk -> file group?




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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r626220629



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Compaction Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.
  */
-public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+public interface RewriteDataFiles extends Action<RewriteDataFiles, RewriteDataFiles.Result> {
+
   /**
-   * Pass a row filter to filter {@link DataFile}s to be rewritten.
-   * <p>
-   * Note that all files that may contain data matching the filter may be rewritten.
-   * <p>
-   * If not set, all files will be rewritten.
-   *
-   * @param expr a row filter to filter out data files
-   * @return this for method chaining
+   * Enable committing groups of files (see max-file-group-size) prior to the entire compaction completing.
+   * This will produce additional commits but allow for progress even if some groups fail to commit. This setting
+   * will not change the correctness of the rewrite operation. The default is false, which produces a single commit
+   * when the entire job has completed.
    */
-  RewriteDataFiles filter(Expression expr);
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
 
   /**
-   * Enables or disables case sensitive expression binding.
-   * <p>
-   * If not set, defaults to false.
-   *
-   * @param caseSensitive caseSensitive
-   * @return this for method chaining
+   * The maximum amount of Iceberg commits that compaction is allowed to produce if partial progress is enabled.
    */
-  RewriteDataFiles caseSensitive(boolean caseSensitive);
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
 
   /**
-   * Pass a PartitionSpec id to specify which PartitionSpec should be used in DataFile rewrite
-   * <p>
-   * If not set, defaults to the table's default spec ID.
-   *
-   * @param specId PartitionSpec id to rewrite
-   * @return this for method chaining
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size into groups. These sub-units of compaction are referred to as file groups. The largest amount of data that
+   * should be compacted in a single group is controlled by MAX_FILE_GROUP_SIZE_BYTES. When grouping files, the
+   * underlying compaction strategy will use this value as to limit the files which will be included in a single file
+   * group. A group will be processed by a single framework "action". For example, in Spark this means that each group
+   * would be rewritten in its own Spark action. A group will never contain files for multiple output partitions.
    */
-  RewriteDataFiles outputSpecId(int specId);
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";
+  long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
 
   /**
-   * Specify the target data file size in bytes.
-   * <p>
-   * If not set, defaults to the table's target file size.
-   *
-   * @param targetSizeInBytes size in bytes of rewrite data file
-   * @return this for method chaining
+   * The max number of file groups to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the group is determined by the compaction strategy. Each file group will be rewritten
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_FILE_GROUP_ACTIONS = "max-concurrent-file-group-actions";
+  int MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT = 1;
+
+  /**
+   * The output file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.
+   */
+  String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes";
+
+  /**
+   * The partition spec to use when writing the output data from this operation. By default uses the
+   * current table partition spec.
    */
-  RewriteDataFiles targetSizeInBytes(long targetSizeInBytes);
+  String PARTITION_SPEC_ID = "partition-spec-id";
+
+  enum CompactionStrategyName {

Review comment:
       since the class has been renamed to `RewriteDataFiles`, should the enum be renamed as `RewriteStrategy`? `Name` seems a little redundant.




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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r624568409



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,123 @@
+/*
+ * 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 org.apache.iceberg.StructLike;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior see max-chunk-size to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when the entire job has completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  // TODO be set once we have an ENUM in core
+  // String COMPACTION_STRATEGY_DEFAULT;
+
+  /**
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size. These sub-units of compaction are referred to as chunks. The largest amount of data that should be
+   * compacted in a single chunk is controlled by MAX_CHUNK_SIZE_BYTES. When grouping files, the underlying
+   * compaction strategy will use this value to but an upper bound on the number of files included in a single
+   * chunk. A chunk will be processed by a single framework "job". For example, in Spark this means that each chunk
+   * would be processed in it's own Spark action. A chunk will never contain files for multiple output partitions.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max-chunk-size-bytes";
+  long MAX_CHUNK_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.
+   */
+  String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes";
+
+  /**
+   * The max number of chunks to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the chunk is determined by the compaction strategy. When running each job chunk will be run
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_CHUNKS = "max-concurrent-chunks";

Review comment:
       is each chunk processed by a single job? or one job can process multiple chunks and hence the `max-concurrent-chunks`? wondering if this should be called `max-concurrent-actions`.




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r627076062



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Rewrite Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.
  */
-public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+public interface RewriteDataFiles extends Action<RewriteDataFiles, RewriteDataFiles.Result> {
+
   /**
-   * Pass a row filter to filter {@link DataFile}s to be rewritten.
-   * <p>
-   * Note that all files that may contain data matching the filter may be rewritten.
-   * <p>
-   * If not set, all files will be rewritten.
-   *
-   * @param expr a row filter to filter out data files
-   * @return this for method chaining
+   * Enable committing groups of files (see max-file-group-size) prior to the entire compaction completing.
+   * This will produce additional commits but allow for progress even if some groups fail to commit. This setting
+   * will not change the correctness of the rewrite operation. The default is false, which produces a single commit
+   * when the entire job has completed.
    */
-  RewriteDataFiles filter(Expression expr);
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
 
   /**
-   * Enables or disables case sensitive expression binding.
-   * <p>
-   * If not set, defaults to false.
-   *
-   * @param caseSensitive caseSensitive
-   * @return this for method chaining
+   * The maximum amount of Iceberg commits that compaction is allowed to produce if partial progress is enabled.
    */
-  RewriteDataFiles caseSensitive(boolean caseSensitive);
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
 
   /**
-   * Pass a PartitionSpec id to specify which PartitionSpec should be used in DataFile rewrite
-   * <p>
-   * If not set, defaults to the table's default spec ID.
-   *
-   * @param specId PartitionSpec id to rewrite
-   * @return this for method chaining
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size into groups. These sub-units of compaction are referred to as file groups. The largest amount of data that
+   * should be compacted in a single group is controlled by MAX_FILE_GROUP_SIZE_BYTES. When grouping files, the
+   * underlying compaction strategy will use this value as to limit the files which will be included in a single file
+   * group. A group will be processed by a single framework "action". For example, in Spark this means that each group
+   * would be rewritten in its own Spark action. A group will never contain files for multiple output partitions.
    */
-  RewriteDataFiles outputSpecId(int specId);
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";
+  long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
 
   /**
-   * Specify the target data file size in bytes.
-   * <p>
-   * If not set, defaults to the table's target file size.
-   *
-   * @param targetSizeInBytes size in bytes of rewrite data file
-   * @return this for method chaining
+   * The max number of file groups to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the group is determined by the compaction strategy. Each file group will be rewritten
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_FILE_GROUP_ACTIONS = "max-concurrent-file-group-actions";

Review comment:
       sounds good to me




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r627029444



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Rewrite Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.

Review comment:
       nit: `datafiles` -> `data 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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r625270391



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,123 @@
+/*
+ * 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 org.apache.iceberg.StructLike;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior see max-chunk-size to the entire compaction completing. This will produce additional commits

Review comment:
       That should be parenthetical, I forgot the parens :(




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r619451104



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;

Review comment:
       Let's consider adding this once we have an enum in core.




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


[GitHub] [iceberg] jackye1995 commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r618033799



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";

Review comment:
       I would like to revisit a bit of the decision about using this `max_commits`. The reason we decided to go with this was to not produce 1 commit per partition and results in millions of commits in some cases. But now I think about it, using max commits has the following 2 concerns:
   1. the more partitions there are, the more partitions are compacted in a single commit, so the more likely it is for each commit to fail, which kind of defeats our purpose to make partial progress for tables with many partitions.
   2. For tables with only a few partitions (e.g. region partitioned), it might be fine to have 1 commit per partition, which means I need to customize a different number input for each table I compact, which makes this config inflexible.
   
   So what about using something like a percentage? for example, `PARTIAL_PROGRESS_NEW_COMMIT_THRESHOLD`, which defines the percentage of partitions/chunks to be included in a single commit, so that for issue (2), user can set just generic numbers such as 100% for small tables, 10% for big tables, etc.
   
   To tackle (1), another option we can add is a lower bound, something like `PARTIAL_PROGRESS_MAX_CHUNKS_PER_COMMIT`, so that we guarantee each commit can still succeed with reasonable likelihood. When working together with other options, this value will override the MAX_COMMIT or percentage, so that when set, if one commit might include too many things, this config can act as a safeguard.




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r618528084



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.expressions.Expression;
+
+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();
+
+  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. This would be used for pushing down filter expressions to the manifest entry scanning phase.
+   *
+   * @return an Iceberg expression to use when discovering file scan tasks
+   */
+  Expression preFilter();
+
+  /**
+   * 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

Review comment:
       Yep let me remove that note. The whole class should be serializable but this function does not need to be.




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r624617766



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,123 @@
+/*
+ * 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 org.apache.iceberg.StructLike;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior see max-chunk-size to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when the entire job has completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  // TODO be set once we have an ENUM in core
+  // String COMPACTION_STRATEGY_DEFAULT;
+
+  /**
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size. These sub-units of compaction are referred to as chunks. The largest amount of data that should be
+   * compacted in a single chunk is controlled by MAX_CHUNK_SIZE_BYTES. When grouping files, the underlying
+   * compaction strategy will use this value to but an upper bound on the number of files included in a single
+   * chunk. A chunk will be processed by a single framework "job". For example, in Spark this means that each chunk
+   * would be processed in it's own Spark action. A chunk will never contain files for multiple output partitions.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max-chunk-size-bytes";
+  long MAX_CHUNK_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.
+   */
+  String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes";
+
+  /**
+   * The max number of chunks to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the chunk is determined by the compaction strategy. When running each job chunk will be run
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_CHUNKS = "max-concurrent-chunks";

Review comment:
       This is where the terminology is difficult. In the "Spark" a single chunk could actually require multiple "Spark Jobs" (Although one may be more likely for our current plans) but would be triggered but a single "Spark Action" and would only be a part of a single Iceberg "Compaction Action commit"
   
   The "Chunk" is the independent isolated unit of work that can be processed in a single "Action" of the underlying framework. So in sparks case, a single write/collect per chunk. These actions would happen concurrently if this parameter MAX_CONCURRENT_CHUNKS is not 1




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617956728



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;
+
+  /**
+   * The parallelism level to use when processing chunks generated by the Compaction Strategy. The structure and
+   * contents of the chunk 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 this for chaining
+   */
+  CompactDataFiles parallelism(int par);
+
+  /**
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
+   *
+   * @param strategyName name of the strategy
+   * @return this for chaining
+   */
+  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. For example this would be used for providing a
+   * restriction to only run compaction on a specific partition.
+   *
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
+   */
+  CompactDataFiles filter(Expression expression);
+
+  /**
+   * A pairing of Chunk information to the result of that chunk's results. If the results are null then that particular
+   * chunk failed. We should only have failed chunks if partial progress is enabled.
+   */
+  class Result {

Review comment:
       Since we cannot have private classes in interfaces, I think we should continue to follow the approach we have in other actions now (base implementations in core).




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r619445197



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {

Review comment:
       I also thought a bit more about the naming. I think I'd still go with the old `RewriteDataFiles` name.
   
   Definitely not a strong opinion, though.




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r627072083



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Rewrite Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.
  */
-public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+public interface RewriteDataFiles extends Action<RewriteDataFiles, RewriteDataFiles.Result> {
+
   /**
-   * Pass a row filter to filter {@link DataFile}s to be rewritten.
-   * <p>
-   * Note that all files that may contain data matching the filter may be rewritten.
-   * <p>
-   * If not set, all files will be rewritten.
-   *
-   * @param expr a row filter to filter out data files
-   * @return this for method chaining
+   * Enable committing groups of files (see max-file-group-size) prior to the entire compaction completing.
+   * This will produce additional commits but allow for progress even if some groups fail to commit. This setting
+   * will not change the correctness of the rewrite operation. The default is false, which produces a single commit
+   * when the entire job has completed.
    */
-  RewriteDataFiles filter(Expression expr);
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
 
   /**
-   * Enables or disables case sensitive expression binding.
-   * <p>
-   * If not set, defaults to false.
-   *
-   * @param caseSensitive caseSensitive
-   * @return this for method chaining
+   * The maximum amount of Iceberg commits that compaction is allowed to produce if partial progress is enabled.
    */
-  RewriteDataFiles caseSensitive(boolean caseSensitive);
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
 
   /**
-   * Pass a PartitionSpec id to specify which PartitionSpec should be used in DataFile rewrite
-   * <p>
-   * If not set, defaults to the table's default spec ID.
-   *
-   * @param specId PartitionSpec id to rewrite
-   * @return this for method chaining
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size into groups. These sub-units of compaction are referred to as file groups. The largest amount of data that
+   * should be compacted in a single group is controlled by MAX_FILE_GROUP_SIZE_BYTES. When grouping files, the
+   * underlying compaction strategy will use this value as to limit the files which will be included in a single file
+   * group. A group will be processed by a single framework "action". For example, in Spark this means that each group
+   * would be rewritten in its own Spark action. A group will never contain files for multiple output partitions.
    */
-  RewriteDataFiles outputSpecId(int specId);
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";
+  long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
 
   /**
-   * Specify the target data file size in bytes.
-   * <p>
-   * If not set, defaults to the table's target file size.
-   *
-   * @param targetSizeInBytes size in bytes of rewrite data file
-   * @return this for method chaining
+   * The max number of file groups to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the group is determined by the compaction strategy. Each file group will be rewritten
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_FILE_GROUP_ACTIONS = "max-concurrent-file-group-actions";
+  int MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT = 1;
+
+  /**
+   * The output file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.
+   */
+  String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes";
+
+  /**
+   * The partition spec to use when writing the output data from this operation. By default uses the
+   * current table partition spec.
    */
-  RewriteDataFiles targetSizeInBytes(long targetSizeInBytes);
+  String PARTITION_SPEC_ID = "partition-spec-id";
+
+  enum RewriteStrategyName {
+    BINPACK,
+    SORT
+  }
 
   /**
-   * Specify the number of "bins" considered when trying to pack the next file split into a task. Increasing this
-   * usually makes tasks a bit more even by considering more ways to pack file regions into a single task with extra
-   * planning cost.
-   * <p>
-   * This configuration can reorder the incoming file regions, to preserve order for lower/upper bounds in file
-   * metadata, user can use a lookback of 1.
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
    *
-   * @param splitLookback number of "bins" considered when trying to pack the next file split into a task.
+   * @param strategyName name of the strategy
    * @return this for method chaining
    */
-  RewriteDataFiles splitLookback(int splitLookback);
+  RewriteDataFiles strategy(RewriteStrategyName strategyName);
 
   /**
-   * Specify the cost of opening a file that will be taken into account during packing files into
-   * bins. If the size of the file is smaller than the cost of opening, then this value will be used
-   * instead of the actual file size.
-   * <p>
-   * If not set, defaults to the table's open file cost.
+   * 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. For example this would be used for providing a
+   * restriction to only run compaction on a specific partition.
    *
-   * @param splitOpenFileCost minimum file size to count to pack into one "bin".
-   * @return this for method chaining
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
    */
-  RewriteDataFiles splitOpenFileCost(long splitOpenFileCost);
+  RewriteDataFiles filter(Expression expression);
 
   /**
-   * The action result that contains a summary of the execution.
+   * A pairing of file group information to the result of the rewriting that file group. If the results are null then
+   * that particular file group failed. We should only have failed groups if partial progress is enabled otherwise we
+   * will report a total failure for the job.
    */
   interface Result {
+    Map<FileGroupInfo, FileGroupRewriteResult> resultMap();
+  }
+
+  interface FileGroupRewriteResult {
+    int addedDataFilesCount();
+
+    int rewrittenDataFilesCount();
+  }
+
+  /**
+   * A description of a file group, when it was processed, and within which partition. For use
+   * tracking rewrite operations and for returning results.
+   */
+  interface FileGroupInfo {
+
+    /**
+     * returns which file group this is out of the total set of file groups for this compaction
+     */
+    int globalIndex();

Review comment:
       I think both global as well as per partition counts are useful only if we know how many groups we had overall and how many groups we had per partition. That's something we should show in Spark UI for each job description as well as in logs.
   
   I am not sure how useful it is in the result, though. For example, I can see the action result as something like this
   
   ```
   StructLike -> num rewritten file groups, num failed file groups, num rewritten files, num added files
   ```
   
   While the job is running, I am interested in how many groups I compacted and how many left. In the action result (i.e. output of my OPTIMIZE command), I'd say a partition summary is more important.




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r626251383



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+
+interface DataCompactionStrategy extends Serializable {

Review comment:
       we can switch everything to rewrite now, I'll swap the package as well




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


[GitHub] [iceberg] RussellSpitzer commented on pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#issuecomment-823656457


   I added a bunch of folks but please anyone who is interested comment. Especially those planning writing their own platform specific implementations or new compaction strategies in the future


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


[GitHub] [iceberg] jackye1995 commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r620822077



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.expressions.Expression;
+
+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();
+
+  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. This would be used for pushing down filter expressions to the manifest entry scanning phase.
+   *
+   * @return an Iceberg expression to use when discovering file scan tasks
+   */
+  Expression preFilter();
+
+  /**
+   * 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>> groupFilesIntoChunks(Iterator<FileScanTask> dataFiles);

Review comment:
       +1, `Iterable` is always preferred as it covers both `Iterator` and `Spliterator`




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


[GitHub] [iceberg] kbendick commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r627082461



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Rewrite Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.
  */
-public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+public interface RewriteDataFiles extends Action<RewriteDataFiles, RewriteDataFiles.Result> {
+
   /**
-   * Pass a row filter to filter {@link DataFile}s to be rewritten.
-   * <p>
-   * Note that all files that may contain data matching the filter may be rewritten.
-   * <p>
-   * If not set, all files will be rewritten.
-   *
-   * @param expr a row filter to filter out data files
-   * @return this for method chaining
+   * Enable committing groups of files (see max-file-group-size) prior to the entire compaction completing.
+   * This will produce additional commits but allow for progress even if some groups fail to commit. This setting
+   * will not change the correctness of the rewrite operation. The default is false, which produces a single commit
+   * when the entire job has completed.
    */
-  RewriteDataFiles filter(Expression expr);
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
 
   /**
-   * Enables or disables case sensitive expression binding.
-   * <p>
-   * If not set, defaults to false.
-   *
-   * @param caseSensitive caseSensitive
-   * @return this for method chaining
+   * The maximum amount of Iceberg commits that compaction is allowed to produce if partial progress is enabled.
    */
-  RewriteDataFiles caseSensitive(boolean caseSensitive);
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
 
   /**
-   * Pass a PartitionSpec id to specify which PartitionSpec should be used in DataFile rewrite
-   * <p>
-   * If not set, defaults to the table's default spec ID.
-   *
-   * @param specId PartitionSpec id to rewrite
-   * @return this for method chaining
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size into groups. These sub-units of compaction are referred to as file groups. The largest amount of data that
+   * should be compacted in a single group is controlled by MAX_FILE_GROUP_SIZE_BYTES. When grouping files, the
+   * underlying compaction strategy will use this value as to limit the files which will be included in a single file
+   * group. A group will be processed by a single framework "action". For example, in Spark this means that each group
+   * would be rewritten in its own Spark action. A group will never contain files for multiple output partitions.
    */
-  RewriteDataFiles outputSpecId(int specId);
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";
+  long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
 
   /**
-   * Specify the target data file size in bytes.
-   * <p>
-   * If not set, defaults to the table's target file size.
-   *
-   * @param targetSizeInBytes size in bytes of rewrite data file
-   * @return this for method chaining
+   * The max number of file groups to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the group is determined by the compaction strategy. Each file group will be rewritten
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_FILE_GROUP_ACTIONS = "max-concurrent-file-group-actions";
+  int MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT = 1;
+
+  /**
+   * The output file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.
+   */
+  String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes";
+
+  /**
+   * The partition spec to use when writing the output data from this operation. By default uses the
+   * current table partition spec.
    */
-  RewriteDataFiles targetSizeInBytes(long targetSizeInBytes);
+  String PARTITION_SPEC_ID = "partition-spec-id";
+
+  enum RewriteStrategyName {
+    BINPACK,
+    SORT
+  }
 
   /**
-   * Specify the number of "bins" considered when trying to pack the next file split into a task. Increasing this
-   * usually makes tasks a bit more even by considering more ways to pack file regions into a single task with extra
-   * planning cost.
-   * <p>
-   * This configuration can reorder the incoming file regions, to preserve order for lower/upper bounds in file
-   * metadata, user can use a lookback of 1.
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
    *
-   * @param splitLookback number of "bins" considered when trying to pack the next file split into a task.
+   * @param strategyName name of the strategy
    * @return this for method chaining
    */
-  RewriteDataFiles splitLookback(int splitLookback);
+  RewriteDataFiles strategy(RewriteStrategyName strategyName);
 
   /**
-   * Specify the cost of opening a file that will be taken into account during packing files into
-   * bins. If the size of the file is smaller than the cost of opening, then this value will be used
-   * instead of the actual file size.
-   * <p>
-   * If not set, defaults to the table's open file cost.
+   * 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. For example this would be used for providing a
+   * restriction to only run compaction on a specific partition.
    *
-   * @param splitOpenFileCost minimum file size to count to pack into one "bin".
-   * @return this for method chaining
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
    */
-  RewriteDataFiles splitOpenFileCost(long splitOpenFileCost);
+  RewriteDataFiles filter(Expression expression);
 
   /**
-   * The action result that contains a summary of the execution.
+   * A pairing of file group information to the result of the rewriting that file group. If the results are null then
+   * that particular file group failed. We should only have failed groups if partial progress is enabled otherwise we
+   * will report a total failure for the job.
    */
   interface Result {
+    Map<FileGroupInfo, FileGroupRewriteResult> resultMap();
+  }
+
+  interface FileGroupRewriteResult {
+    int addedDataFilesCount();
+
+    int rewrittenDataFilesCount();
+  }
+
+  /**
+   * A description of a file group, when it was processed, and within which partition. For use
+   * tracking rewrite operations and for returning results.
+   */
+  interface FileGroupInfo {
+
+    /**
+     * returns which file group this is out of the total set of file groups for this compaction
+     */
+    int globalIndex();

Review comment:
       Correct me if I'm wrong, but I think that since this is `FileGroupInfo` and not the Result, the indexes are likely included so the engine assign and track work etc as opposed to purely for tracking on output. 
   
   EDIT: I wrote this comment before @RussellSpitzer's response was included, so it might not be relevant now.




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r618498247



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;
+
+  /**
+   * The parallelism level to use when processing chunks generated by the Compaction Strategy. The structure and
+   * contents of the chunk 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 this for chaining
+   */
+  CompactDataFiles parallelism(int par);
+
+  /**
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
+   *
+   * @param strategyName name of the strategy
+   * @return this for chaining
+   */
+  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. For example this would be used for providing a
+   * restriction to only run compaction on a specific partition.
+   *
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
+   */
+  CompactDataFiles filter(Expression expression);
+
+  /**
+   * A pairing of Chunk information to the result of that chunk's results. If the results are null then that particular
+   * chunk failed. We should only have failed chunks if partial progress is enabled.
+   */
+  class Result {
+    private Map<CompactionChunkInfo, CompactionChunkResult> resultMap;
+
+
+    public Result(Map<CompactionChunkInfo, CompactionChunkResult> resultMap) {
+      this.resultMap = resultMap;
+    }
+
+    public Map<CompactionChunkInfo, CompactionChunkResult> resultMap() {
+      return resultMap;
+    }
+  }
+
+  interface CompactionChunkResult {
+
+    int numFilesAdded();
+
+    int numFilesRemoved();
+  }
+
+  interface CompactionChunkInfo {
+
+    /**
+     * returns which chunk this is out of the total set of chunks for this compaction
+     */
+    int chunkIndex();

Review comment:
       I think so, if I tell you that the currently running job is Partition "fooBar" do you know how many partitions are left?




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


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r618019058



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.expressions.Expression;
+
+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();
+
+  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. This would be used for pushing down filter expressions to the manifest entry scanning phase.
+   *
+   * @return an Iceberg expression to use when discovering file scan tasks
+   */
+  Expression preFilter();

Review comment:
       Can this be more specific? metadataFilter? planFilter? 




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


[GitHub] [iceberg] aokolnychyi commented on pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#issuecomment-824430614


   cc @rymurr @chenjunjiedada @flyrain @karuppayya @jackye1995 @kbendick 


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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r618496315



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";

Review comment:
       I did like having a single parameter that described the output, If we could avoid having multiple parameters I'd definitely prefer that. 
   
   I feel like the current parameter can cover the two cases specified above, for example you just increase the value if nothing is successfully committing. I thought this is pretty clean because in an error message we can always just say "increase the value of this parameter".
   
   The scale is basically
   
   1 - (Do a single Commit regardless of the number of chunks
   LongMaxValue = Do a commit per chunk




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617958128



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;
+
+  /**
+   * The parallelism level to use when processing chunks generated by the Compaction Strategy. The structure and
+   * contents of the chunk 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 this for chaining
+   */
+  CompactDataFiles parallelism(int par);
+
+  /**
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
+   *
+   * @param strategyName name of the strategy
+   * @return this for chaining
+   */
+  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. For example this would be used for providing a
+   * restriction to only run compaction on a specific partition.
+   *
+   * @param expression only entries that pass this filter will be compacted
+   * @return this for chaining
+   */
+  CompactDataFiles filter(Expression expression);
+
+  /**
+   * A pairing of Chunk information to the result of that chunk's results. If the results are null then that particular
+   * chunk failed. We should only have failed chunks if partial progress is enabled.
+   */
+  class Result {
+    private Map<CompactionChunkInfo, CompactionChunkResult> resultMap;
+
+
+    public Result(Map<CompactionChunkInfo, CompactionChunkResult> resultMap) {
+      this.resultMap = resultMap;
+    }
+
+    public Map<CompactionChunkInfo, CompactionChunkResult> resultMap() {
+      return resultMap;
+    }
+  }
+
+  interface CompactionChunkResult {
+
+    int numFilesAdded();
+
+    int numFilesRemoved();
+  }
+
+  interface CompactionChunkInfo {
+
+    /**
+     * returns which chunk this is out of the total set of chunks for this compaction
+     */
+    int chunkIndex();
+
+    /**
+     * returns which chunk this is out of the set of chunks for this partition
+     */
+    int partitionIndex();
+
+    /**
+     * returns which partition this chunk contains files from
+     */
+    String partition();

Review comment:
       Yeah, exactly. I'd image we will convert it to string while showing in Spark UI, for example.




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r627037847



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Rewrite Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.
  */
-public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+public interface RewriteDataFiles extends Action<RewriteDataFiles, RewriteDataFiles.Result> {
+
   /**
-   * Pass a row filter to filter {@link DataFile}s to be rewritten.
-   * <p>
-   * Note that all files that may contain data matching the filter may be rewritten.
-   * <p>
-   * If not set, all files will be rewritten.
-   *
-   * @param expr a row filter to filter out data files
-   * @return this for method chaining
+   * Enable committing groups of files (see max-file-group-size) prior to the entire compaction completing.
+   * This will produce additional commits but allow for progress even if some groups fail to commit. This setting
+   * will not change the correctness of the rewrite operation. The default is false, which produces a single commit
+   * when the entire job has completed.
    */
-  RewriteDataFiles filter(Expression expr);
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
 
   /**
-   * Enables or disables case sensitive expression binding.
-   * <p>
-   * If not set, defaults to false.
-   *
-   * @param caseSensitive caseSensitive
-   * @return this for method chaining
+   * The maximum amount of Iceberg commits that compaction is allowed to produce if partial progress is enabled.
    */
-  RewriteDataFiles caseSensitive(boolean caseSensitive);
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
 
   /**
-   * Pass a PartitionSpec id to specify which PartitionSpec should be used in DataFile rewrite
-   * <p>
-   * If not set, defaults to the table's default spec ID.
-   *
-   * @param specId PartitionSpec id to rewrite
-   * @return this for method chaining
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size into groups. These sub-units of compaction are referred to as file groups. The largest amount of data that
+   * should be compacted in a single group is controlled by MAX_FILE_GROUP_SIZE_BYTES. When grouping files, the
+   * underlying compaction strategy will use this value as to limit the files which will be included in a single file
+   * group. A group will be processed by a single framework "action". For example, in Spark this means that each group
+   * would be rewritten in its own Spark action. A group will never contain files for multiple output partitions.
    */
-  RewriteDataFiles outputSpecId(int specId);
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";
+  long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
 
   /**
-   * Specify the target data file size in bytes.
-   * <p>
-   * If not set, defaults to the table's target file size.
-   *
-   * @param targetSizeInBytes size in bytes of rewrite data file
-   * @return this for method chaining
+   * The max number of file groups to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the group is determined by the compaction strategy. Each file group will be rewritten
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_FILE_GROUP_ACTIONS = "max-concurrent-file-group-actions";
+  int MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT = 1;
+
+  /**
+   * The output file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.

Review comment:
       nit: `write.target-size` -> `write.target-file-size-bytes`?




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617956243



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when all chunks have completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial_progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial_progress.max_commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  String COMPACTION_STRATEGY_DEFAULT = BinPack.NAME;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max_chunk_size_bytes";
+  long MAX_CHUNK_SIZE_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files.
+   */
+  String TARGET_FILE_SIZE = "target_file_size";
+
+  /**
+   * A threshold for preventing compaction of partitions whose output will not more than MIN_COMPACTION_OUTPUT_FILES
+   * files.
+   */
+  String MIN_COMPACTION_OUTPUT_FILES = "min_output_files";
+  int MIN_COMPACTION_OUTPUT_FILES_DEFAULT = 1;
+
+  /**
+   * A threshold for preventing compaction of partitions which do not have more than MIN_COMPACTION_INPUT_FILES to
+   * compact.
+   */
+  String MIN_COMPACTION_INPUT_FILES = "min_input_files";
+  int MIN_COMPACTION_INPUT_FILES_DEFAULT = 1;

Review comment:
       I don't feel strongly here. I am fine with 1. 




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617960020



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.expressions.Expression;
+
+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();
+
+  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. This would be used for pushing down filter expressions to the manifest entry scanning phase.
+   *
+   * @return an Iceberg expression to use when discovering file scan tasks
+   */
+  Expression preFilter();
+
+  /**
+   * 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);

Review comment:
       I think we mostly use `Iterable` in core and Guava transforms that are lazy. It would be nice to express this as a filter but the proposed strategies will need to know a broader picture so I think it should be fine to operate on iterables or iterator here. 




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r624617835



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,123 @@
+/*
+ * 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 org.apache.iceberg.StructLike;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior see max-chunk-size to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when the entire job has completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  // TODO be set once we have an ENUM in core
+  // String COMPACTION_STRATEGY_DEFAULT;
+
+  /**
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size. These sub-units of compaction are referred to as chunks. The largest amount of data that should be
+   * compacted in a single chunk is controlled by MAX_CHUNK_SIZE_BYTES. When grouping files, the underlying
+   * compaction strategy will use this value to but an upper bound on the number of files included in a single
+   * chunk. A chunk will be processed by a single framework "job". For example, in Spark this means that each chunk
+   * would be processed in it's own Spark action. A chunk will never contain files for multiple output partitions.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max-chunk-size-bytes";
+  long MAX_CHUNK_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.
+   */
+  String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes";
+
+  /**
+   * The max number of chunks to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the chunk is determined by the compaction strategy. When running each job chunk will be run
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_CHUNKS = "max-concurrent-chunks";
+  int MAX_CONCURRENT_CHUNKS_DEFAULT = 1;
+
+  /**
+   * The name of the compaction strategy to be used when compacting data files. Currently we only support BINPACK and
+   * SORT as options.
+   *
+   * @param strategyName name of the strategy
+   * @return this for method chaining
+   */
+  CompactDataFiles strategy(String strategyName);

Review comment:
       This is the plan




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r623412148



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.expressions.Expression;
+
+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();
+
+  DataCompactionStrategy withOptions(Map<String, String> options);

Review comment:
       Ie no separate method for passing options specifically for the Strategy, any options for the action that don't apply to the action go to the strategy.




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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r625995051



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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 org.apache.iceberg.StructLike;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {
+
+  /**
+   * Enable committing groups of chunks prior to the entire compaction completing. This will produce additional commits
+   * but allow for progress even if some chunks fail to commit. The default is false, which produces a single commit
+   * when the entire job has completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of commits that compaction is allowed to produce if partial progress is enabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  // TODO be set once we have an ENUM in core
+  // String COMPACTION_STRATEGY_DEFAULT;
+
+  /**
+   * The largest amount of data that should be compacted in a single chunk by the underlying framework. This bounds the
+   * amount of data that would be used in a single shuffle for example.
+   */
+  String MAX_CHUNK_SIZE_BYTES = "max-chunk-size-bytes";

Review comment:
       Switched to FILE_GROUP everywhere




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


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r618021599



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/DataCompactionStrategy.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.expressions.Expression;
+
+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();
+
+  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. This would be used for pushing down filter expressions to the manifest entry scanning phase.
+   *
+   * @return an Iceberg expression to use when discovering file scan tasks
+   */
+  Expression preFilter();
+
+  /**
+   * 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);

Review comment:
       This is used to select candidates from a set of files, how about calling it with select or filter prefix?




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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r626221579



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java
##########
@@ -19,92 +19,120 @@
 
 package org.apache.iceberg.actions;
 
-import org.apache.iceberg.DataFile;
+import java.util.Map;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
 
 /**
- * An action that rewrites data files.
+ * An action for rewriting datafiles according to a Compaction Strategy. Generally used for
+ * optimizing the sizing and layout of datafiles within a table.
  */
-public interface RewriteDataFiles extends SnapshotUpdate<RewriteDataFiles, RewriteDataFiles.Result> {
+public interface RewriteDataFiles extends Action<RewriteDataFiles, RewriteDataFiles.Result> {
+
   /**
-   * Pass a row filter to filter {@link DataFile}s to be rewritten.
-   * <p>
-   * Note that all files that may contain data matching the filter may be rewritten.
-   * <p>
-   * If not set, all files will be rewritten.
-   *
-   * @param expr a row filter to filter out data files
-   * @return this for method chaining
+   * Enable committing groups of files (see max-file-group-size) prior to the entire compaction completing.
+   * This will produce additional commits but allow for progress even if some groups fail to commit. This setting
+   * will not change the correctness of the rewrite operation. The default is false, which produces a single commit
+   * when the entire job has completed.
    */
-  RewriteDataFiles filter(Expression expr);
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
 
   /**
-   * Enables or disables case sensitive expression binding.
-   * <p>
-   * If not set, defaults to false.
-   *
-   * @param caseSensitive caseSensitive
-   * @return this for method chaining
+   * The maximum amount of Iceberg commits that compaction is allowed to produce if partial progress is enabled.
    */
-  RewriteDataFiles caseSensitive(boolean caseSensitive);
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
 
   /**
-   * Pass a PartitionSpec id to specify which PartitionSpec should be used in DataFile rewrite
-   * <p>
-   * If not set, defaults to the table's default spec ID.
-   *
-   * @param specId PartitionSpec id to rewrite
-   * @return this for method chaining
+   * The entire compaction operation is broken down into pieces based on partitioning and within partitions based
+   * on size into groups. These sub-units of compaction are referred to as file groups. The largest amount of data that
+   * should be compacted in a single group is controlled by MAX_FILE_GROUP_SIZE_BYTES. When grouping files, the
+   * underlying compaction strategy will use this value as to limit the files which will be included in a single file
+   * group. A group will be processed by a single framework "action". For example, in Spark this means that each group
+   * would be rewritten in its own Spark action. A group will never contain files for multiple output partitions.
    */
-  RewriteDataFiles outputSpecId(int specId);
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";
+  long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
 
   /**
-   * Specify the target data file size in bytes.
-   * <p>
-   * If not set, defaults to the table's target file size.
-   *
-   * @param targetSizeInBytes size in bytes of rewrite data file
-   * @return this for method chaining
+   * The max number of file groups to be simultaneously rewritten by the compaction strategy. The structure and
+   * contents of the group is determined by the compaction strategy. Each file group will be rewritten
+   * independently and asynchronously.
+   **/
+  String MAX_CONCURRENT_FILE_GROUP_ACTIONS = "max-concurrent-file-group-actions";
+  int MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT = 1;
+
+  /**
+   * The output file size that this compaction strategy will attempt to generate when rewriting files. By default this
+   * will use the write.target-size value in the table properties of the table being updated.
+   */
+  String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes";
+
+  /**
+   * The partition spec to use when writing the output data from this operation. By default uses the
+   * current table partition spec.
    */
-  RewriteDataFiles targetSizeInBytes(long targetSizeInBytes);
+  String PARTITION_SPEC_ID = "partition-spec-id";
+
+  enum CompactionStrategyName {

Review comment:
       hmm. seems that we might need the `Name` to distinguish with the `DataCompactionStrategy` interface later.




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


[GitHub] [iceberg] rdblue commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617747190



##########
File path: api/src/main/java/org/apache/iceberg/actions/compaction/BinPack.java
##########
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+public interface BinPack {

Review comment:
       This should probably also be private. There's no need to expose this, I think.




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r617950009



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {

Review comment:
       That reminds me that we need an option for the output spec.




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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2501: API: API For CompactDataFiles and DataCompactionStrategy

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2501:
URL: https://github.com/apache/iceberg/pull/2501#discussion_r619443696



##########
File path: api/src/main/java/org/apache/iceberg/actions/CompactDataFiles.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.iceberg.actions.compaction.BinPack;
+import org.apache.iceberg.expressions.Expression;
+
+public interface CompactDataFiles extends Action<CompactDataFiles, CompactDataFiles.Result> {

Review comment:
       I don't think it will be common but it may be a valid use case to have multiple specs in a table. For example, the volume of data for old partitions may be much smaller so we may actually want to keep the old spec for some parts of the data.




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