turboFei commented on code in PR #3776:
URL: https://github.com/apache/amoro/pull/3776#discussion_r2570058547


##########
amoro-format-iceberg/src/main/java/org/apache/amoro/table/TableProperties.java:
##########
@@ -133,6 +133,18 @@ private TableProperties() {}
       "self-optimizing.min-plan-interval";
   public static final long SELF_OPTIMIZING_MIN_PLAN_INTERVAL_DEFAULT = 60000;
 
+  /** metric-based evaluation related properties */
+  public static final String SELF_OPTIMIZING_EVALUATION_FALLBACK_INTERVAL =
+      "self-optimizing.evaluation.fallback-interval"; // fallback evaluation 
interval in
+  // milliseconds
+
+  public static final int SELF_OPTIMIZING_EVALUATION_FALLBACK_INTERVAL_DEFAULT 
=
+      -1; // event-based evaluation not in effect
+
+  public static final String 
SELF_OPTIMIZING_EVALUATION_FILE_SIZE_MSE_TOLERANCE =
+      "self-optimizing.evaluation.file-size.mse-tolerance";

Review Comment:
   maybe add some comments for the MSE abbreviation 



##########
amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/TableRuntimeRefreshExecutor.java:
##########
@@ -62,6 +63,16 @@ private void tryEvaluatingPendingInput(DefaultTableRuntime 
tableRuntime, MixedTa
     // only evaluate pending input when optimizing is enabled and in idle state
     if (tableRuntime.getTableConfiguration().getOptimizingConfig().isEnabled()
         && tableRuntime.getOptimizingStatus().equals(OptimizingStatus.IDLE)) {
+
+      if 
(tableRuntime.getTableConfiguration().getOptimizingConfig().isEventBasedTriggerEnabled()
+          && !MetricBasedEvaluationEvent.isEvaluatingNecessary(
+              tableRuntime.getOptimizingConfig(), table, 
tableRuntime.getLastPlanTime())) {

Review Comment:
   what is the difference between 
   
   `tableRuntime.getTableConfiguration().getOptimizingConfig()` and 
`tableRuntime.getOptimizingConfig()`, it looks confuse. 



##########
amoro-format-iceberg/src/main/java/org/apache/amoro/optimizing/plan/CommonPartitionEvaluator.java:
##########
@@ -208,6 +217,11 @@ private boolean addTargetSizeReachedFile(DataFile 
dataFile, List<ContentFile<?>>
     return false;
   }
 
+  private void updateFileSizeSquaredErrorSum(DataFile dataFile) {
+    long diffSize = minTargetSize - Math.min(dataFile.fileSizeInBytes(), 
minTargetSize);

Review Comment:
   maybe add comments that only accumulates squared error for files smaller 
than minTargetSize.



##########
amoro-format-iceberg/src/main/java/org/apache/amoro/optimizing/evaluation/MetricBasedEvaluationEvent.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.amoro.optimizing.evaluation;
+
+import org.apache.amoro.TableFormat;
+import org.apache.amoro.config.OptimizingConfig;
+import 
org.apache.amoro.shade.guava32.com.google.common.annotations.VisibleForTesting;
+import org.apache.amoro.table.MixedTable;
+import org.apache.amoro.utils.MemorySize;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.util.PropertyUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+
+public class MetricBasedEvaluationEvent {
+  private static final Logger logger = 
LoggerFactory.getLogger(MetricBasedEvaluationEvent.class);
+
+  public static boolean isReachFallbackInterval(OptimizingConfig config, long 
lastPlanTime) {
+    long fallbackInterval = config.getEvaluationFallbackInterval();
+    return fallbackInterval >= 0 && System.currentTimeMillis() - lastPlanTime 
> fallbackInterval;

Review Comment:
   nit:
   ```
   >= fallbackInterval
   ```



##########
amoro-format-iceberg/src/main/java/org/apache/amoro/optimizing/evaluation/MetricBasedEvaluationEvent.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.amoro.optimizing.evaluation;
+
+import org.apache.amoro.TableFormat;
+import org.apache.amoro.config.OptimizingConfig;
+import 
org.apache.amoro.shade.guava32.com.google.common.annotations.VisibleForTesting;
+import org.apache.amoro.table.MixedTable;
+import org.apache.amoro.utils.MemorySize;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.util.PropertyUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+
+public class MetricBasedEvaluationEvent {
+  private static final Logger logger = 
LoggerFactory.getLogger(MetricBasedEvaluationEvent.class);
+
+  public static boolean isReachFallbackInterval(OptimizingConfig config, long 
lastPlanTime) {
+    long fallbackInterval = config.getEvaluationFallbackInterval();
+    return fallbackInterval >= 0 && System.currentTimeMillis() - lastPlanTime 
> fallbackInterval;
+  }
+
+  public static boolean isEvaluatingNecessary(
+      OptimizingConfig config, MixedTable table, long lastPlanTime) {
+    if (table.format() != TableFormat.ICEBERG && table.format() != 
TableFormat.MIXED_ICEBERG) {
+      logger.debug(
+          "MetricBasedRefreshEvent only support ICEBERG/MIXED_ICEBERG tables. 
Always return true for other table formats.");
+      return true;
+    }
+    // Step 1: Perform periodic scheduling according to the fallback interval 
to avoid false
+    // positives or
+    // missed triggers based on metadata metric-driven evaluation
+    if (isReachFallbackInterval(config, lastPlanTime)) {
+      logger.info("Maximum interval for evaluating table {} has reached.", 
table.id());
+      return true;
+    }
+
+    // Step 2: Empty table should skip evaluating pending input
+    BasicTableStats basicStats = new BasicTableStats(table);
+    if (basicStats.dataFileCnt == 0) {
+      logger.info("Table {} contains no data files, skip evaluating pending 
input.", table.id());
+      return false;
+    }
+
+    // Step 3: If the condition `delete file=0 && avg file size > target size 
* ratio` is satisfied,
+    // then evaluating the pending input is considered unnecessary and will be 
skipped.
+    long minTargetSize = (long) (config.getTargetSize() * 
config.getMinTargetSizeRatio());
+    double avgFileSize =
+        basicStats.dataFileCnt > 0 ? (double) basicStats.fileSize / 
basicStats.dataFileCnt : 0;
+
+    if (basicStats.deleteFileCnt == 0 && avgFileSize > minTargetSize) {
+      logger.info(
+          "Table {} contains only appended data and no deleted files (average 
file size: {}), skip evaluating pending input.",
+          table.id(),
+          avgFileSize);
+      return false;
+    }
+
+    return true;
+  }
+
+  public static boolean isPartitionPendingNecessary(
+      OptimizingConfig config, long partitionSquaredErrorSum, long 
partitionFileCount) {
+    long mseTolerance = config.getEvaluationMseTolerance();
+    return partitionFileCount > 1
+        && (mseTolerance == 0
+            || (double) partitionSquaredErrorSum / partitionFileCount
+                >= mseTolerance * mseTolerance);
+  }
+
+  static class BasicTableStats {
+    int deleteFileCnt = 0;
+    int dataFileCnt = 0;
+    long fileSize = 0;
+
+    @VisibleForTesting
+    BasicTableStats() {}
+
+    BasicTableStats(MixedTable table) {
+      if (table.isUnkeyedTable()) {
+        acceptSnapshotIfPresent(table.asUnkeyedTable().currentSnapshot());
+      } else {
+        
acceptSnapshotIfPresent(table.asKeyedTable().baseTable().currentSnapshot());
+        
acceptSnapshotIfPresent(table.asKeyedTable().changeTable().currentSnapshot());
+      }
+    }
+
+    private void acceptSnapshotIfPresent(Snapshot snapshot) {
+      if (snapshot != null) {
+        accept(snapshot.summary());
+      }
+    }
+
+    void accept(Map<String, String> summary) {
+      deleteFileCnt +=
+          PropertyUtil.propertyAsInt(summary, 
SnapshotSummary.TOTAL_DELETE_FILES_PROP, 0);
+      dataFileCnt += PropertyUtil.propertyAsInt(summary, 
SnapshotSummary.TOTAL_DATA_FILES_PROP, 0);
+      fileSize +=
+          MemorySize.parse(
+                  PropertyUtil.propertyAsString(summary, 
SnapshotSummary.TOTAL_FILE_SIZE_PROP, "0"))

Review Comment:
   PropertyUtil.propertyAsLong should be fine for TOTAL_FILE_SIZE_PROP



##########
amoro-format-iceberg/src/main/java/org/apache/amoro/optimizing/evaluation/MetricBasedEvaluationEvent.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.amoro.optimizing.evaluation;
+
+import org.apache.amoro.TableFormat;
+import org.apache.amoro.config.OptimizingConfig;
+import 
org.apache.amoro.shade.guava32.com.google.common.annotations.VisibleForTesting;
+import org.apache.amoro.table.MixedTable;
+import org.apache.amoro.utils.MemorySize;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.util.PropertyUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+
+public class MetricBasedEvaluationEvent {
+  private static final Logger logger = 
LoggerFactory.getLogger(MetricBasedEvaluationEvent.class);
+
+  public static boolean isReachFallbackInterval(OptimizingConfig config, long 
lastPlanTime) {
+    long fallbackInterval = config.getEvaluationFallbackInterval();
+    return fallbackInterval >= 0 && System.currentTimeMillis() - lastPlanTime 
> fallbackInterval;
+  }
+
+  public static boolean isEvaluatingNecessary(
+      OptimizingConfig config, MixedTable table, long lastPlanTime) {
+    if (table.format() != TableFormat.ICEBERG && table.format() != 
TableFormat.MIXED_ICEBERG) {
+      logger.debug(
+          "MetricBasedRefreshEvent only support ICEBERG/MIXED_ICEBERG tables. 
Always return true for other table formats.");
+      return true;
+    }
+    // Step 1: Perform periodic scheduling according to the fallback interval 
to avoid false
+    // positives or
+    // missed triggers based on metadata metric-driven evaluation
+    if (isReachFallbackInterval(config, lastPlanTime)) {
+      logger.info("Maximum interval for evaluating table {} has reached.", 
table.id());
+      return true;
+    }
+
+    // Step 2: Empty table should skip evaluating pending input
+    BasicTableStats basicStats = new BasicTableStats(table);
+    if (basicStats.dataFileCnt == 0) {
+      logger.info("Table {} contains no data files, skip evaluating pending 
input.", table.id());
+      return false;
+    }
+
+    // Step 3: If the condition `delete file=0 && avg file size > target size 
* ratio` is satisfied,
+    // then evaluating the pending input is considered unnecessary and will be 
skipped.
+    long minTargetSize = (long) (config.getTargetSize() * 
config.getMinTargetSizeRatio());
+    double avgFileSize =
+        basicStats.dataFileCnt > 0 ? (double) basicStats.fileSize / 
basicStats.dataFileCnt : 0;
+
+    if (basicStats.deleteFileCnt == 0 && avgFileSize > minTargetSize) {
+      logger.info(
+          "Table {} contains only appended data and no deleted files (average 
file size: {}), skip evaluating pending input.",
+          table.id(),
+          avgFileSize);
+      return false;
+    }
+
+    return true;
+  }
+
+  public static boolean isPartitionPendingNecessary(
+      OptimizingConfig config, long partitionSquaredErrorSum, long 
partitionFileCount) {
+    long mseTolerance = config.getEvaluationMseTolerance();
+    return partitionFileCount > 1
+        && (mseTolerance == 0
+            || (double) partitionSquaredErrorSum / partitionFileCount
+                >= mseTolerance * mseTolerance);

Review Comment:
   `>= (double) mseTolerance * mseTolerance`



##########
amoro-ams/src/main/java/org/apache/amoro/server/scheduler/inline/TableRuntimeRefreshExecutor.java:
##########
@@ -62,6 +63,16 @@ private void tryEvaluatingPendingInput(DefaultTableRuntime 
tableRuntime, MixedTa
     // only evaluate pending input when optimizing is enabled and in idle state
     if (tableRuntime.getTableConfiguration().getOptimizingConfig().isEnabled()
         && tableRuntime.getOptimizingStatus().equals(OptimizingStatus.IDLE)) {
+
+      if 
(tableRuntime.getTableConfiguration().getOptimizingConfig().isEventBasedTriggerEnabled()
+          && !MetricBasedEvaluationEvent.isEvaluatingNecessary(
+              tableRuntime.getOptimizingConfig(), table, 
tableRuntime.getLastPlanTime())) {

Review Comment:
   Can we define variable likes `OptimizingConfig config = 
tableRuntime.getOptimizingConfig()` and reuse it



##########
amoro-format-iceberg/src/main/java/org/apache/amoro/optimizing/plan/CommonPartitionEvaluator.java:
##########
@@ -208,6 +217,11 @@ private boolean addTargetSizeReachedFile(DataFile 
dataFile, List<ContentFile<?>>
     return false;
   }
 
+  private void updateFileSizeSquaredErrorSum(DataFile dataFile) {
+    long diffSize = minTargetSize - Math.min(dataFile.fileSizeInBytes(), 
minTargetSize);
+    fileSizeSquaredErrorSum += diffSize * diffSize;

Review Comment:
   Is there a possibility of overflow here?



##########
amoro-format-iceberg/src/main/java/org/apache/amoro/table/TableProperties.java:
##########
@@ -133,6 +133,18 @@ private TableProperties() {}
       "self-optimizing.min-plan-interval";
   public static final long SELF_OPTIMIZING_MIN_PLAN_INTERVAL_DEFAULT = 60000;
 
+  /** metric-based evaluation related properties */
+  public static final String SELF_OPTIMIZING_EVALUATION_FALLBACK_INTERVAL =
+      "self-optimizing.evaluation.fallback-interval"; // fallback evaluation 
interval in
+  // milliseconds
+
+  public static final int SELF_OPTIMIZING_EVALUATION_FALLBACK_INTERVAL_DEFAULT 
=
+      -1; // event-based evaluation not in effect
+
+  public static final String 
SELF_OPTIMIZING_EVALUATION_FILE_SIZE_MSE_TOLERANCE =
+      "self-optimizing.evaluation.file-size.mse-tolerance";
+  public static final long 
SELF_OPTIMIZING_EVALUATION_FILE_SIZE_MSE_TOLERANCE_DEFAULT = 0;

Review Comment:
   I saw that, now 
   
   if `SELF_OPTIMIZING_EVALUATION_FALLBACK_INTERVAL_DEFAULT > 0`, 
`isEventBasedTriggerEnabled` is true.
   
   Should event-based evaluation be considered "enabled" if fallbackInterval ≥ 
0 but evaluationMseTolerance == 0.
   
   If not, can we give a default value for `mse-tolerance` or give some 
suggestions to determine the mse tolerance?
   



##########
amoro-format-iceberg/src/main/java/org/apache/amoro/optimizing/evaluation/MetricBasedEvaluationEvent.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.amoro.optimizing.evaluation;
+
+import org.apache.amoro.TableFormat;
+import org.apache.amoro.config.OptimizingConfig;
+import 
org.apache.amoro.shade.guava32.com.google.common.annotations.VisibleForTesting;
+import org.apache.amoro.table.MixedTable;
+import org.apache.amoro.utils.MemorySize;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.util.PropertyUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+
+public class MetricBasedEvaluationEvent {
+  private static final Logger logger = 
LoggerFactory.getLogger(MetricBasedEvaluationEvent.class);
+
+  public static boolean isReachFallbackInterval(OptimizingConfig config, long 
lastPlanTime) {
+    long fallbackInterval = config.getEvaluationFallbackInterval();
+    return fallbackInterval >= 0 && System.currentTimeMillis() - lastPlanTime 
> fallbackInterval;
+  }
+
+  public static boolean isEvaluatingNecessary(
+      OptimizingConfig config, MixedTable table, long lastPlanTime) {
+    if (table.format() != TableFormat.ICEBERG && table.format() != 
TableFormat.MIXED_ICEBERG) {
+      logger.debug(
+          "MetricBasedRefreshEvent only support ICEBERG/MIXED_ICEBERG tables. 
Always return true for other table formats.");

Review Comment:
   is it by design?
   
   `MetricBasedRefreshEvent` or `MetricBasedEvaluationEvent`



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to