difin commented on code in PR #5529:
URL: https://github.com/apache/hive/pull/5529#discussion_r1918722582


##########
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/compaction/evaluator/IcebergCompactionEvaluator.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.mr.hive.compaction.evaluator;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Collections;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.CompactionType;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.mr.hive.compaction.IcebergCompactionUtil;
+import 
org.apache.iceberg.mr.hive.compaction.evaluator.amoro.CommonPartitionEvaluator;
+import 
org.apache.iceberg.mr.hive.compaction.evaluator.amoro.IcebergTableFileScanHelper;
+import org.apache.iceberg.mr.hive.compaction.evaluator.amoro.OptimizingConfig;
+import 
org.apache.iceberg.mr.hive.compaction.evaluator.amoro.TableConfiguration;
+import 
org.apache.iceberg.mr.hive.compaction.evaluator.amoro.TableFileScanHelper;
+import org.apache.iceberg.mr.hive.compaction.evaluator.amoro.TableFormat;
+import org.apache.iceberg.mr.hive.compaction.evaluator.amoro.TableRuntime;
+import org.apache.iceberg.mr.hive.compaction.evaluator.amoro.TableRuntimeMeta;
+import org.apache.iceberg.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class IcebergCompactionEvaluator {
+
+  private static final long lastOptimizeTime = 0;
+  private static final int triggerInterval = 0;
+
+  private IcebergCompactionEvaluator() {
+
+  }
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(IcebergCompactionEvaluator.class);
+
+  public static boolean isEligibleForCompaction(Table icebergTable, String 
partitionPath,
+      CompactionType compactionType, HiveConf conf) {
+
+    if (icebergTable.currentSnapshot() == null) {
+      LOG.info("Table {}{} doesn't require compaction because it is empty", 
icebergTable,
+          partitionPath == null ? "" : " partition " + partitionPath);
+      return false;
+    }
+
+    CommonPartitionEvaluator partitionEvaluator = 
createCommonPartitionEvaluator(icebergTable, partitionPath, conf);
+
+    if (partitionEvaluator == null) {
+      return false;
+    }
+
+    switch (compactionType) {
+      case MINOR:
+        return partitionEvaluator.isMinorNecessary();
+      case MAJOR:
+        return partitionEvaluator.isFullNecessary() || 
partitionEvaluator.isMajorNecessary();
+      default:
+        return false;
+    }
+  }
+
+  private static TableRuntime createTableRuntime(Table icebergTable, HiveConf 
conf) {
+    long fileSizeInBytesThreshold = HiveConf.getSizeVar(conf,
+        HiveConf.ConfVars.HIVE_ICEBERG_COMPACTION_FILE_SIZE_THRESHOLD);
+
+    OptimizingConfig optimizingConfig = 
OptimizingConfig.parse(Collections.emptyMap());
+    optimizingConfig.setTargetSize(fileSizeInBytesThreshold);
+    optimizingConfig.setFullTriggerInterval(triggerInterval);

Review Comment:
   Amoro compaction evaluator has logic that depends on trigger intervals. It 
can refuse compaction requests based on them.
   Setting trigger interval because defaults ones have the following values:
   ```
   public static final int SELF_OPTIMIZING_MINOR_TRIGGER_INTERVAL_DEFAULT = 
3600000; // 1 h
   public static final int SELF_OPTIMIZING_FULL_TRIGGER_INTERVAL_DEFAULT = -1; 
// not trigger
   ```
   
   Regarding overriding target size config, I think we need the ability to 
override the default value in order to use FILE_SIZE_THRESHOLD clause in a 
compaction command.



-- 
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: gitbox-unsubscr...@hive.apache.org

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


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

Reply via email to