mxm commented on code in PR #12979:
URL: https://github.com/apache/iceberg/pull/12979#discussion_r2077855951


##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java:
##########
@@ -81,6 +81,9 @@ private FlinkWriteOptions() {}
   public static final ConfigOption<Integer> WRITE_PARALLELISM =
       ConfigOptions.key("write-parallelism").intType().noDefaultValue();
 
+  public static final ConfigOption<Boolean> COMPACT_ENABLE =
+      ConfigOptions.key("compact-enabled").booleanType().defaultValue(false);

Review Comment:
   Maybe this is clearer to the user?
   
   ```suggestion
         
ConfigOptions.key("compation-enabled").booleanType().defaultValue(false);
   ```



##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/RewriteDataFilesConfig.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.flink.maintenance.api;
+
+import java.util.Map;
+import org.apache.iceberg.actions.BinPackRewriteFilePlanner;
+import org.apache.iceberg.actions.SizeBasedFileRewritePlanner;
+import org.apache.iceberg.util.PropertyUtil;
+
+public class RewriteDataFilesConfig {
+  private static final String CONFIG_PREFIX = "flink-maintenance.rewrite.";
+
+  private final Map<String, String> properties;
+
+  public RewriteDataFilesConfig(Map<String, String> properties) {
+    this.properties = properties;
+  }
+
+  public static final String PARTIAL_PROGRESS_ENABLE = CONFIG_PREFIX + 
"partial-progress-enabled";
+
+  public static final String PARTIAL_PROGRESS_MAX_COMMITS =
+      CONFIG_PREFIX + "partial-progress-max-commits";
+
+  public static final String MAX_REWRITE_BYTES = CONFIG_PREFIX + "max-bytes";
+
+  public static final String TARGET_FILE_SIZE_BYTES =
+      CONFIG_PREFIX + SizeBasedFileRewritePlanner.TARGET_FILE_SIZE_BYTES;
+
+  public static final String MIN_FILE_SIZE_BYTES =
+      CONFIG_PREFIX + SizeBasedFileRewritePlanner.MIN_FILE_SIZE_BYTES;
+
+  public static final String MAX_FILE_SIZE_BYTES =
+      CONFIG_PREFIX + SizeBasedFileRewritePlanner.MAX_FILE_SIZE_BYTES;
+
+  public static final String MAX_FILE_GROUP_SIZE_BYTES =
+      CONFIG_PREFIX + SizeBasedFileRewritePlanner.MAX_FILE_GROUP_SIZE_BYTES;
+
+  public static final String MIN_INPUT_FILES =
+      CONFIG_PREFIX + SizeBasedFileRewritePlanner.MIN_INPUT_FILES;
+
+  public static final String DELETE_FILE_THRESHOLD =
+      CONFIG_PREFIX + BinPackRewriteFilePlanner.DELETE_FILE_THRESHOLD;
+
+  public static final String REWRITE_ALL = CONFIG_PREFIX + 
SizeBasedFileRewritePlanner.REWRITE_ALL;
+
+  public static final String SCHEDULE_ON_COMMIT_COUNT = CONFIG_PREFIX + 
"schedule-on-commit-count";
+
+  public static final String SCHEDULE_ON_DATA_FILE_COUNT =
+      CONFIG_PREFIX + "schedule-on-data-file-count";
+
+  public static final String SCHEDULE_ON_DATA_FILE_SIZE =
+      CONFIG_PREFIX + "schedule-on-data-file-size";
+
+  public static final String SCHEDULE_ON_INTERVAL_SECOND =
+      CONFIG_PREFIX + "schedule-on-interval-second";
+
+  public Integer getScheduleOnCommitCount() {
+    return PropertyUtil.propertyAsNullableInt(properties, 
SCHEDULE_ON_COMMIT_COUNT);
+  }
+
+  public Integer getScheduleOnDataFileCount() {
+    return PropertyUtil.propertyAsNullableInt(properties, 
SCHEDULE_ON_DATA_FILE_COUNT);
+  }

Review Comment:
   How do we validate that all these options work as intended?



##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/RewriteDataFilesConfig.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.flink.maintenance.api;
+
+import java.util.Map;
+import org.apache.iceberg.actions.BinPackRewriteFilePlanner;
+import org.apache.iceberg.actions.SizeBasedFileRewritePlanner;
+import org.apache.iceberg.util.PropertyUtil;
+
+public class RewriteDataFilesConfig {
+  private static final String CONFIG_PREFIX = "flink-maintenance.rewrite.";
+
+  private final Map<String, String> properties;
+
+  public RewriteDataFilesConfig(Map<String, String> properties) {
+    this.properties = properties;
+  }
+
+  public static final String PARTIAL_PROGRESS_ENABLE = CONFIG_PREFIX + 
"partial-progress-enabled";
+
+  public static final String PARTIAL_PROGRESS_MAX_COMMITS =
+      CONFIG_PREFIX + "partial-progress-max-commits";
+
+  public static final String MAX_REWRITE_BYTES = CONFIG_PREFIX + "max-bytes";
+
+  public static final String TARGET_FILE_SIZE_BYTES =
+      CONFIG_PREFIX + SizeBasedFileRewritePlanner.TARGET_FILE_SIZE_BYTES;
+
+  public static final String MIN_FILE_SIZE_BYTES =
+      CONFIG_PREFIX + SizeBasedFileRewritePlanner.MIN_FILE_SIZE_BYTES;
+
+  public static final String MAX_FILE_SIZE_BYTES =
+      CONFIG_PREFIX + SizeBasedFileRewritePlanner.MAX_FILE_SIZE_BYTES;
+
+  public static final String MAX_FILE_GROUP_SIZE_BYTES =
+      CONFIG_PREFIX + SizeBasedFileRewritePlanner.MAX_FILE_GROUP_SIZE_BYTES;
+
+  public static final String MIN_INPUT_FILES =
+      CONFIG_PREFIX + SizeBasedFileRewritePlanner.MIN_INPUT_FILES;
+
+  public static final String DELETE_FILE_THRESHOLD =
+      CONFIG_PREFIX + BinPackRewriteFilePlanner.DELETE_FILE_THRESHOLD;
+
+  public static final String REWRITE_ALL = CONFIG_PREFIX + 
SizeBasedFileRewritePlanner.REWRITE_ALL;
+
+  public static final String SCHEDULE_ON_COMMIT_COUNT = CONFIG_PREFIX + 
"schedule-on-commit-count";
+
+  public static final String SCHEDULE_ON_DATA_FILE_COUNT =
+      CONFIG_PREFIX + "schedule-on-data-file-count";
+
+  public static final String SCHEDULE_ON_DATA_FILE_SIZE =
+      CONFIG_PREFIX + "schedule-on-data-file-size";
+
+  public static final String SCHEDULE_ON_INTERVAL_SECOND =
+      CONFIG_PREFIX + "schedule-on-interval-second";
+
+  public Integer getScheduleOnCommitCount() {
+    return PropertyUtil.propertyAsNullableInt(properties, 
SCHEDULE_ON_COMMIT_COUNT);
+  }
+
+  public Integer getScheduleOnDataFileCount() {
+    return PropertyUtil.propertyAsNullableInt(properties, 
SCHEDULE_ON_DATA_FILE_COUNT);
+  }

Review Comment:
   We could add a parsing test that validates all options in this class are 
read correctly.



##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/RewriteDataFiles.java:
##########
@@ -170,6 +172,43 @@ public Builder maxFileGroupSizeBytes(long 
maxFileGroupSizeBytes) {
       return this;
     }
 
+    /**
+     * Configures the properties for the rewriter.
+     *
+     * @param properties properties for the rewriter
+     */
+    public Builder properties(Map<String, String> properties) {
+      RewriteDataFilesConfig rewriteDataFilesConfig = new 
RewriteDataFilesConfig(properties);
+      Optional.ofNullable(rewriteDataFilesConfig.getPartialProgressEnable())
+          .ifPresent(this::partialProgressEnabled);
+      
Optional.ofNullable(rewriteDataFilesConfig.getPartialProgressMaxCommits())
+          .ifPresent(this::partialProgressMaxCommits);
+      Optional.ofNullable(rewriteDataFilesConfig.getMaxRewriteBytes())
+          .ifPresent(this::maxRewriteBytes);
+      Optional.ofNullable(rewriteDataFilesConfig.getTargetFileSizeBytes())
+          .ifPresent(this::targetFileSizeBytes);
+      Optional.ofNullable(rewriteDataFilesConfig.getMinFileSizeBytes())
+          .ifPresent(this::minFileSizeBytes);
+      Optional.ofNullable(rewriteDataFilesConfig.getMaxFileSizeBytes())
+          .ifPresent(this::maxFileSizeBytes);
+      Optional.ofNullable(rewriteDataFilesConfig.getMaxFileGroupSizeBytes())
+          .ifPresent(this::maxFileGroupSizeBytes);
+      
Optional.ofNullable(rewriteDataFilesConfig.getMinInputFiles()).ifPresent(this::minInputFiles);
+      Optional.ofNullable(rewriteDataFilesConfig.getDeleteFileThreshold())
+          .ifPresent(this::deleteFileThreshold);
+      
Optional.ofNullable(rewriteDataFilesConfig.getRewriteAll()).ifPresent(this::rewriteAll);
+
+      Optional.ofNullable(rewriteDataFilesConfig.getScheduleOnCommitCount())
+          .ifPresent(this::scheduleOnCommitCount);
+      Optional.ofNullable(rewriteDataFilesConfig.getScheduleOnDataFileCount())
+          .ifPresent(this::scheduleOnDataFileCount);
+      Optional.ofNullable(rewriteDataFilesConfig.getScheduleOnDataFileSize())
+          .ifPresent(this::scheduleOnDataFileSize);
+      Optional.ofNullable(rewriteDataFilesConfig.getScheduleOnIntervalSecond())
+          .ifPresent(intervalSecond -> 
this.scheduleOnInterval(Duration.ofSeconds(intervalSecond)));

Review Comment:
   It would be nice to unit test that the mapping here is correct.



##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/LockFactoryCreator.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.flink.maintenance.api;
+
+import java.util.Locale;
+import java.util.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class LockFactoryCreator {

Review Comment:
   ```suggestion
   public class JdbcLockFactoryCreator {
   ```



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

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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

Reply via email to