Copilot commented on code in PR #10106: URL: https://github.com/apache/gravitino/pull/10106#discussion_r2888935178
########## maintenance/jobs/src/main/java/org/apache/gravitino/maintenance/jobs/iceberg/IcebergUpdateStatsAndMetricsJob.java: ########## @@ -0,0 +1,607 @@ +/* + * 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.gravitino.maintenance.jobs.iceberg; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import org.apache.gravitino.NameIdentifier; +import org.apache.gravitino.job.JobTemplateProvider; +import org.apache.gravitino.job.SparkJobTemplate; +import org.apache.gravitino.maintenance.jobs.BuiltInJob; +import org.apache.gravitino.maintenance.optimizer.api.common.MetricPoint; +import org.apache.gravitino.maintenance.optimizer.api.common.PartitionEntry; +import org.apache.gravitino.maintenance.optimizer.api.common.PartitionPath; +import org.apache.gravitino.maintenance.optimizer.api.common.StatisticEntry; +import org.apache.gravitino.maintenance.optimizer.api.updater.MetricsUpdater; +import org.apache.gravitino.maintenance.optimizer.api.updater.StatisticsUpdater; +import org.apache.gravitino.maintenance.optimizer.common.OptimizerEnv; +import org.apache.gravitino.maintenance.optimizer.common.PartitionEntryImpl; +import org.apache.gravitino.maintenance.optimizer.common.StatisticEntryImpl; +import org.apache.gravitino.maintenance.optimizer.common.conf.OptimizerConfig; +import org.apache.gravitino.maintenance.optimizer.common.util.IcebergSparkConfigUtils; +import org.apache.gravitino.maintenance.optimizer.common.util.ProviderUtils; +import org.apache.gravitino.stats.StatisticValues; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Built-in job for computing Iceberg table file statistics and persisting them to Gravitino. */ +public class IcebergUpdateStatsAndMetricsJob implements BuiltInJob { + + private static final Logger LOG = LoggerFactory.getLogger(IcebergUpdateStatsAndMetricsJob.class); + + private static final String NAME = + JobTemplateProvider.BUILTIN_NAME_PREFIX + "iceberg-update-stats"; + private static final String VERSION = "v1"; + private static final String DEFAULT_STATISTICS_UPDATER = "gravitino-statistics-updater"; + private static final String DEFAULT_METRICS_UPDATER = "gravitino-metrics-updater"; + private static final long DEFAULT_TARGET_FILE_SIZE_BYTES = 128L * 1024 * 1024; + private static final long SMALL_FILE_THRESHOLD_BYTES = 32L * 1024 * 1024; + private static final String DEFAULT_UPDATE_MODE = UpdateMode.ALL.modeName; + private static final String CUSTOM_STAT_PREFIX = "custom-"; + + @Override + public SparkJobTemplate jobTemplate() { + return SparkJobTemplate.builder() + .withName(NAME) + .withComment( + "Built-in Iceberg update stats job template for computing datafile MSE and file metrics") + .withExecutable(resolveExecutable(IcebergUpdateStatsAndMetricsJob.class)) + .withClassName(IcebergUpdateStatsAndMetricsJob.class.getName()) + .withArguments(buildArguments()) + .withConfigs(buildSparkConfigs()) + .withCustomFields( + Collections.singletonMap(JobTemplateProvider.PROPERTY_VERSION_KEY, VERSION)) + .build(); + } + + /** Main entry point. */ + public static void main(String[] args) { + Map<String, String> argMap = parseArguments(args); + String catalogName = argMap.get("catalog"); + String tableIdentifier = argMap.get("table"); + UpdateMode updateMode = parseUpdateMode(argMap.get("update-mode")); + + if (catalogName == null || tableIdentifier == null) { + System.err.println("Error: --catalog and --table are required arguments"); + printUsage(); + System.exit(1); + } + + Map<String, String> updaterOptions = parseJsonOptions(argMap.get("updater-options")); + String sparkConfJson = argMap.get("spark-conf"); + + SparkSession.Builder sparkBuilder = + SparkSession.builder().appName("Gravitino Built-in Iceberg Update Stats"); + + if (sparkConfJson != null && !sparkConfJson.isEmpty()) { + Map<String, String> customConfigs = parseCustomSparkConfigs(sparkConfJson); + for (Map.Entry<String, String> entry : customConfigs.entrySet()) { + sparkBuilder.config(entry.getKey(), entry.getValue()); + } + } + + SparkSession spark = sparkBuilder.getOrCreate(); + StatisticsUpdater statisticsUpdater = null; + MetricsUpdater metricsUpdater = null; + try { + Map<String, String> optimizerProperties = buildOptimizerProperties(updaterOptions); + if (updateMode.updateStats) { + String statisticsUpdaterName = + updaterOptions.getOrDefault("statistics_updater", DEFAULT_STATISTICS_UPDATER).trim(); + statisticsUpdater = + createStatisticsUpdater( + statisticsUpdaterName, requireGravitinoConfig(optimizerProperties)); + } + if (updateMode.updateMetrics) { + String metricsUpdaterName = + updaterOptions.getOrDefault("metrics_updater", DEFAULT_METRICS_UPDATER).trim(); + metricsUpdater = createMetricsUpdater(metricsUpdaterName, optimizerProperties); + } + + updateStatistics( + spark, statisticsUpdater, metricsUpdater, updateMode, catalogName, tableIdentifier); + } catch (Exception e) { + LOG.error("Failed to update Iceberg statistics/metrics", e); + System.exit(1); + } finally { + if (statisticsUpdater != null) { + try { + statisticsUpdater.close(); + } catch (Exception e) { + LOG.warn("Failed to close statistics updater", e); + } + } + if (metricsUpdater != null) { + try { + metricsUpdater.close(); + } catch (Exception e) { + LOG.warn("Failed to close metrics updater", e); + } + } + spark.stop(); + } + } + + static void updateStatistics( + SparkSession spark, + StatisticsUpdater statisticsUpdater, + String catalogName, + String tableIdentifier) { + updateStatistics( + spark, statisticsUpdater, null, UpdateMode.STATS, catalogName, tableIdentifier); + } + + static void updateStatistics( + SparkSession spark, + StatisticsUpdater statisticsUpdater, + MetricsUpdater metricsUpdater, + String catalogName, + String tableIdentifier) { + updateStatistics( + spark, statisticsUpdater, metricsUpdater, UpdateMode.ALL, catalogName, tableIdentifier); + } + + static void updateStatistics( + SparkSession spark, + StatisticsUpdater statisticsUpdater, + MetricsUpdater metricsUpdater, + UpdateMode updateMode, + String catalogName, + String tableIdentifier) { + Objects.requireNonNull(updateMode, "updateMode must not be null"); + + if (updateMode.updateStats && statisticsUpdater == null) { + throw new IllegalArgumentException( + "Statistics updater must be configured when update_mode is stats or all"); + } + + if (updateMode.updateMetrics && metricsUpdater == null) { + throw new IllegalArgumentException( + "Metrics updater must be configured when update_mode is metrics or all"); + } + + NameIdentifier gravitinoTableIdentifier = + toGravitinoTableIdentifier(catalogName, tableIdentifier); + long metricTimestamp = System.currentTimeMillis() / 1000L; + boolean partitioned = isPartitionedTable(spark, catalogName, tableIdentifier); + if (partitioned) { + String sql = buildPartitionStatsSql(catalogName, tableIdentifier); + Row[] rows = (Row[]) spark.sql(sql).collect(); + Map<PartitionPath, List<StatisticEntry<?>>> partitionStatistics = new LinkedHashMap<>(); + List<MetricPoint> tableAndPartitionMetrics = new ArrayList<>(); + for (Row row : rows) { + PartitionPath partitionPath = toPartitionPath(row.getAs("partition")); + List<StatisticEntry<?>> statistics = toStatistics(row); + if (updateMode.updateStats) { + partitionStatistics.put(partitionPath, statistics); + } + if (updateMode.updateMetrics) { + tableAndPartitionMetrics.addAll( + toPartitionMetricPoints( + gravitinoTableIdentifier, partitionPath, statistics, metricTimestamp)); + } + } + + if (updateMode.updateStats) { + statisticsUpdater.updatePartitionStatistics(gravitinoTableIdentifier, partitionStatistics); + } + + if (updateMode.updateMetrics && !tableAndPartitionMetrics.isEmpty()) { + metricsUpdater.updateTableAndPartitionMetrics(tableAndPartitionMetrics); + } + + LOG.info( + "Updated partition data in mode {} for {} partitions on {}", + updateMode.modeName, + rows.length, + gravitinoTableIdentifier); + } else { + String sql = buildTableStatsSql(catalogName, tableIdentifier); + Row[] rows = (Row[]) spark.sql(sql).collect(); + List<StatisticEntry<?>> tableStatistics = + rows.length == 0 ? Collections.emptyList() : toStatistics(rows[0]); + + if (updateMode.updateStats) { + statisticsUpdater.updateTableStatistics(gravitinoTableIdentifier, tableStatistics); + } + + if (updateMode.updateMetrics && !tableStatistics.isEmpty()) { + metricsUpdater.updateTableAndPartitionMetrics( + toTableMetricPoints(gravitinoTableIdentifier, tableStatistics, metricTimestamp)); + } + + LOG.info( + "Updated table data in mode {} with {} metrics on {}", + updateMode.modeName, + tableStatistics.size(), + gravitinoTableIdentifier); + } + } + + static String buildTableStatsSql(String catalogName, String tableIdentifier) { + String filesTable = buildFilesTableIdentifier(catalogName, tableIdentifier); + return "SELECT " + + "COUNT(*) AS file_count, " + + "SUM(CASE WHEN content = 0 THEN 1 ELSE 0 END) AS data_files, " + + "SUM(CASE WHEN content = 1 THEN 1 ELSE 0 END) AS position_delete_files, " + + "SUM(CASE WHEN content = 2 THEN 1 ELSE 0 END) AS equality_delete_files, " + + "SUM(CASE WHEN file_size_in_bytes < " + + SMALL_FILE_THRESHOLD_BYTES + + " THEN 1 ELSE 0 END) AS small_files, " + + "AVG(POWER(" + + DEFAULT_TARGET_FILE_SIZE_BYTES + + " - LEAST(" + + DEFAULT_TARGET_FILE_SIZE_BYTES + + ", file_size_in_bytes), 2)) AS datafile_mse, " + + "AVG(file_size_in_bytes) AS avg_size, " + + "SUM(file_size_in_bytes) AS total_size " + + "FROM " + + filesTable; + } + + static String buildPartitionStatsSql(String catalogName, String tableIdentifier) { + String filesTable = buildFilesTableIdentifier(catalogName, tableIdentifier); + return "SELECT " + + "partition, " + + "COUNT(*) AS file_count, " + + "SUM(CASE WHEN content = 0 THEN 1 ELSE 0 END) AS data_files, " + + "SUM(CASE WHEN content = 1 THEN 1 ELSE 0 END) AS position_delete_files, " + + "SUM(CASE WHEN content = 2 THEN 1 ELSE 0 END) AS equality_delete_files, " + + "SUM(CASE WHEN file_size_in_bytes < " + + SMALL_FILE_THRESHOLD_BYTES + + " THEN 1 ELSE 0 END) AS small_files, " + + "AVG(POWER(" + + DEFAULT_TARGET_FILE_SIZE_BYTES + + " - LEAST(" + + DEFAULT_TARGET_FILE_SIZE_BYTES + + ", file_size_in_bytes), 2)) AS datafile_mse, " + + "AVG(file_size_in_bytes) AS avg_size, " + + "SUM(file_size_in_bytes) AS total_size " + + "FROM " + + filesTable + + " GROUP BY partition"; + } + + static boolean isPartitionedTable( + SparkSession spark, String catalogName, String tableIdentifier) { + StructType filesSchema = + spark.table(buildFilesTableIdentifier(catalogName, tableIdentifier)).schema(); + if (!Arrays.asList(filesSchema.fieldNames()).contains("partition")) { + return false; + } + StructField partitionField = filesSchema.apply("partition"); + if (!(partitionField.dataType() instanceof StructType)) { + return false; + } + return ((StructType) partitionField.dataType()).fields().length > 0; + } + + static List<StatisticEntry<?>> toStatistics(Row row) { + List<StatisticEntry<?>> statistics = new ArrayList<>(); + statistics.add( + new StatisticEntryImpl<>( + CUSTOM_STAT_PREFIX + "file_count", + StatisticValues.longValue(toLongValue(row, "file_count")))); + statistics.add( + new StatisticEntryImpl<>( + CUSTOM_STAT_PREFIX + "data_files", + StatisticValues.longValue(toLongValue(row, "data_files")))); + statistics.add( + new StatisticEntryImpl<>( + CUSTOM_STAT_PREFIX + "position_delete_files", + StatisticValues.longValue(toLongValue(row, "position_delete_files")))); + statistics.add( + new StatisticEntryImpl<>( + CUSTOM_STAT_PREFIX + "equality_delete_files", + StatisticValues.longValue(toLongValue(row, "equality_delete_files")))); + statistics.add( + new StatisticEntryImpl<>( + CUSTOM_STAT_PREFIX + "small_files", + StatisticValues.longValue(toLongValue(row, "small_files")))); + statistics.add( + new StatisticEntryImpl<>( + CUSTOM_STAT_PREFIX + "datafile_mse", + StatisticValues.doubleValue(toDoubleValue(row, "datafile_mse")))); + statistics.add( + new StatisticEntryImpl<>( + CUSTOM_STAT_PREFIX + "avg_size", + StatisticValues.doubleValue(toDoubleValue(row, "avg_size")))); + statistics.add( + new StatisticEntryImpl<>( + CUSTOM_STAT_PREFIX + "total_size", + StatisticValues.longValue(toLongValue(row, "total_size")))); + return statistics; + } + + static PartitionPath toPartitionPath(Row partitionRow) { + StructType partitionSchema = partitionRow.schema(); + List<PartitionEntry> entries = new ArrayList<>(partitionSchema.fields().length); + for (int i = 0; i < partitionSchema.fields().length; i++) { + String name = partitionSchema.fields()[i].name(); + Object value = partitionRow.get(i); + entries.add(new PartitionEntryImpl(name, String.valueOf(value))); + } + return PartitionPath.of(entries); + } + + static Map<String, String> parseArguments(String[] args) { + Map<String, String> argMap = new HashMap<>(); + for (int i = 0; i < args.length; i++) { + if (args[i].startsWith("--")) { + String key = args[i].substring(2); + if (i + 1 < args.length && !args[i + 1].startsWith("--")) { + String value = args[i + 1]; + if (value != null && !value.trim().isEmpty()) { + argMap.put(key, value); + } + i++; + } + } + } + return argMap; + } + + static Map<String, String> parseCustomSparkConfigs(String sparkConfJson) { + return parseJsonOptions(sparkConfJson); + } + + static Map<String, String> parseJsonOptions(String json) { + if (json == null || json.isEmpty()) { + return new HashMap<>(); + } + try { + ObjectMapper mapper = new ObjectMapper(); + Map<String, Object> parsedMap = + mapper.readValue(json, new TypeReference<Map<String, Object>>() {}); + Map<String, String> configs = new HashMap<>(); + for (Map.Entry<String, Object> entry : parsedMap.entrySet()) { + Object value = entry.getValue(); + if (value instanceof Map || value instanceof List) { + throw new IllegalArgumentException( + String.format( + Locale.ROOT, + "JSON options must be a flat key-value map, but key '%s' has non-scalar value", + entry.getKey())); + } + configs.put(entry.getKey(), value == null ? "" : value.toString()); + } + return configs; + } catch (Exception e) { + throw new IllegalArgumentException( + "Failed to parse JSON options: " + json + ". Error: " + e.getMessage(), e); + } + } + + static UpdateMode parseUpdateMode(String value) { + if (value == null || value.trim().isEmpty()) { + return UpdateMode.from(DEFAULT_UPDATE_MODE); + } + return UpdateMode.from(value); + } + + static Map<String, String> buildOptimizerProperties(Map<String, String> updaterOptions) { + Map<String, String> optimizerProperties = new HashMap<>(updaterOptions); + + Optional<String> gravitinoUri = + firstNonEmpty( + updaterOptions.get("gravitino_uri"), + updaterOptions.get("gravitino-uri"), + updaterOptions.get(OptimizerConfig.GRAVITINO_URI)); + Optional<String> metalake = + firstNonEmpty( + updaterOptions.get("metalake"), updaterOptions.get(OptimizerConfig.GRAVITINO_METALAKE)); + + gravitinoUri.ifPresent(uri -> optimizerProperties.put(OptimizerConfig.GRAVITINO_URI, uri)); + metalake.ifPresent(value -> optimizerProperties.put(OptimizerConfig.GRAVITINO_METALAKE, value)); + return optimizerProperties; + } + + static Map<String, String> requireGravitinoConfig(Map<String, String> optimizerProperties) { + String gravitinoUri = optimizerProperties.get(OptimizerConfig.GRAVITINO_URI); + String metalake = optimizerProperties.get(OptimizerConfig.GRAVITINO_METALAKE); + + if (gravitinoUri == null || gravitinoUri.trim().isEmpty()) { + throw new IllegalArgumentException( + "updater_options must contain 'gravitino_uri' when update_mode is stats or all"); + } + + if (metalake == null || metalake.trim().isEmpty()) { + throw new IllegalArgumentException( + "updater_options must contain 'metalake' when update_mode is stats or all"); + } + + return optimizerProperties; + } + + private static StatisticsUpdater createStatisticsUpdater( + String updaterName, Map<String, String> optimizerProperties) { + StatisticsUpdater statisticsUpdater = + ProviderUtils.createStatisticsUpdaterInstance(updaterName); + statisticsUpdater.initialize(new OptimizerEnv(new OptimizerConfig(optimizerProperties))); + return statisticsUpdater; + } + + private static MetricsUpdater createMetricsUpdater( + String updaterName, Map<String, String> optimizerProperties) { + MetricsUpdater metricsUpdater = ProviderUtils.createMetricsUpdaterInstance(updaterName); + metricsUpdater.initialize(new OptimizerEnv(new OptimizerConfig(optimizerProperties))); + return metricsUpdater; + } + + private static List<MetricPoint> toTableMetricPoints( + NameIdentifier tableIdentifier, + List<StatisticEntry<?>> tableStatistics, + long metricTimestamp) { + List<MetricPoint> requests = new ArrayList<>(tableStatistics.size()); + for (StatisticEntry<?> statistic : tableStatistics) { + requests.add( + MetricPoint.forTable( + tableIdentifier, statistic.name(), statistic.value(), metricTimestamp)); + } + return requests; + } + + private static List<MetricPoint> toPartitionMetricPoints( + NameIdentifier tableIdentifier, + PartitionPath partitionPath, + List<StatisticEntry<?>> statistics, + long metricTimestamp) { + List<MetricPoint> requests = new ArrayList<>(statistics.size()); + for (StatisticEntry<?> statistic : statistics) { + requests.add( + MetricPoint.forPartition( + tableIdentifier, + partitionPath, + statistic.name(), + statistic.value(), + metricTimestamp)); + } + return requests; + } + + private static NameIdentifier toGravitinoTableIdentifier( + String catalogName, String tableIdentifier) { + String[] levels = tableIdentifier.split("\\."); + if (levels.length != 2) { + throw new IllegalArgumentException( + "--table must use schema.table format, but got: " + tableIdentifier); + } + return NameIdentifier.of(catalogName, levels[0], levels[1]); + } + + private static String buildFilesTableIdentifier(String catalogName, String tableIdentifier) { + String[] levels = tableIdentifier.split("\\."); + if (levels.length != 2) { + throw new IllegalArgumentException( + "--table must use schema.table format, but got: " + tableIdentifier); + } + return escapeSqlIdentifier(catalogName) + + "." + + escapeSqlIdentifier(levels[0]) + + "." + + escapeSqlIdentifier(levels[1]) + + ".files"; + } + + private static String escapeSqlIdentifier(String identifier) { + return identifier.replace("`", "``"); Review Comment: `buildFilesTableIdentifier()` concatenates user-provided `--catalog`/`--table` values directly into a SQL string. `escapeSqlIdentifier()` currently only replaces backticks and does not quote identifiers, so it doesn’t reliably prevent malformed identifiers (and can allow Spark SQL injection depending on parsing). Please validate identifier components (e.g., against an allowed pattern) and/or wrap each identifier part in backticks when building the SQL identifier. ```suggestion String escaped = identifier.replace("`", "``"); return "`" + escaped + "`"; ``` ########## maintenance/optimizer/src/main/java/org/apache/gravitino/maintenance/optimizer/command/SubmitUpdateStatsJobCommand.java: ########## @@ -0,0 +1,248 @@ +/* + * 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.gravitino.maintenance.optimizer.command; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.TreeMap; +import org.apache.commons.lang3.StringUtils; +import org.apache.gravitino.client.GravitinoClient; +import org.apache.gravitino.job.JobHandle; +import org.apache.gravitino.maintenance.optimizer.common.conf.OptimizerConfig; +import org.apache.gravitino.maintenance.optimizer.common.util.GravitinoClientUtils; +import org.apache.gravitino.maintenance.optimizer.common.util.IcebergSparkConfigUtils; + +/** + * Handles CLI command {@code submit-update-stats-job} for submitting built-in Iceberg update stats + * jobs directly from optimizer CLI. + */ +public class SubmitUpdateStatsJobCommand implements OptimizerCommandExecutor { + + private static final String JOB_TEMPLATE_NAME = "builtin-iceberg-update-stats"; + private static final String DEFAULT_UPDATE_MODE = "stats"; + private static final String OPTION_UPDATER_OPTIONS = "updater-options"; + private static final String OPTION_SPARK_CONF = "spark-conf"; + + private static final ObjectMapper MAPPER = new ObjectMapper(); + + @Override + public void execute(OptimizerCommandContext context) throws Exception { + Map<String, String> submitterConfigs = context.optimizerEnv().config().jobSubmitterConfigs(); + + List<TableTarget> tableTargets = + parseTableTargets( + context.identifiers(), + context.optimizerEnv().config().get(OptimizerConfig.GRAVITINO_DEFAULT_CATALOG_CONFIG)); + + String updateMode = + parseUpdateMode( + resolveScalarOption(context.updateMode(), submitterConfigs.get("update_mode"))); + + String updaterOptionsJson = + resolveJsonOption(context.updaterOptions(), submitterConfigs.get("updater_options")); + String sparkConfJson = + resolveJsonOption(context.sparkConf(), submitterConfigs.get("spark_conf")); + + Map<String, String> updaterOptions = + IcebergSparkConfigUtils.parseFlatJsonMap(updaterOptionsJson, OPTION_UPDATER_OPTIONS); + Map<String, String> sparkConfigs = + IcebergSparkConfigUtils.parseFlatJsonMap(sparkConfJson, OPTION_SPARK_CONF); + + validateUpdaterOptions(updateMode, updaterOptions); + validateSparkConfigs(tableTargets, sparkConfigs); + + if (context.dryRun()) { + for (TableTarget tableTarget : tableTargets) { + Map<String, String> jobConfig = + buildJobConfig(tableTarget, updateMode, updaterOptions, sparkConfigs); + context + .output() + .printf( + "DRY-RUN: identifier=%s jobTemplate=%s jobConfig=%s%n", + tableTarget.fullIdentifier, JOB_TEMPLATE_NAME, jobConfig); + } Review Comment: The CLI output prints the full `jobConfig` map for both DRY-RUN and SUBMIT paths. Since `jobConfig` includes `updater_options`/`spark_conf`, this can leak sensitive values (for example JDBC passwords) into terminal history and CI logs. Please redact sensitive keys (e.g., *password*, *secret*, *token*) or avoid printing raw JSON values (print only the identifier/template and maybe the config keys). ########## maintenance/jobs/src/main/java/org/apache/gravitino/maintenance/jobs/iceberg/IcebergUpdateStatsAndMetricsJob.java: ########## @@ -0,0 +1,607 @@ +/* + * 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.gravitino.maintenance.jobs.iceberg; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import org.apache.gravitino.NameIdentifier; +import org.apache.gravitino.job.JobTemplateProvider; +import org.apache.gravitino.job.SparkJobTemplate; +import org.apache.gravitino.maintenance.jobs.BuiltInJob; +import org.apache.gravitino.maintenance.optimizer.api.common.MetricPoint; +import org.apache.gravitino.maintenance.optimizer.api.common.PartitionEntry; +import org.apache.gravitino.maintenance.optimizer.api.common.PartitionPath; +import org.apache.gravitino.maintenance.optimizer.api.common.StatisticEntry; +import org.apache.gravitino.maintenance.optimizer.api.updater.MetricsUpdater; +import org.apache.gravitino.maintenance.optimizer.api.updater.StatisticsUpdater; +import org.apache.gravitino.maintenance.optimizer.common.OptimizerEnv; +import org.apache.gravitino.maintenance.optimizer.common.PartitionEntryImpl; +import org.apache.gravitino.maintenance.optimizer.common.StatisticEntryImpl; +import org.apache.gravitino.maintenance.optimizer.common.conf.OptimizerConfig; +import org.apache.gravitino.maintenance.optimizer.common.util.IcebergSparkConfigUtils; +import org.apache.gravitino.maintenance.optimizer.common.util.ProviderUtils; +import org.apache.gravitino.stats.StatisticValues; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Built-in job for computing Iceberg table file statistics and persisting them to Gravitino. */ +public class IcebergUpdateStatsAndMetricsJob implements BuiltInJob { + + private static final Logger LOG = LoggerFactory.getLogger(IcebergUpdateStatsAndMetricsJob.class); + + private static final String NAME = + JobTemplateProvider.BUILTIN_NAME_PREFIX + "iceberg-update-stats"; + private static final String VERSION = "v1"; + private static final String DEFAULT_STATISTICS_UPDATER = "gravitino-statistics-updater"; + private static final String DEFAULT_METRICS_UPDATER = "gravitino-metrics-updater"; + private static final long DEFAULT_TARGET_FILE_SIZE_BYTES = 128L * 1024 * 1024; + private static final long SMALL_FILE_THRESHOLD_BYTES = 32L * 1024 * 1024; + private static final String DEFAULT_UPDATE_MODE = UpdateMode.ALL.modeName; + private static final String CUSTOM_STAT_PREFIX = "custom-"; + + @Override + public SparkJobTemplate jobTemplate() { + return SparkJobTemplate.builder() + .withName(NAME) + .withComment( + "Built-in Iceberg update stats job template for computing datafile MSE and file metrics") + .withExecutable(resolveExecutable(IcebergUpdateStatsAndMetricsJob.class)) + .withClassName(IcebergUpdateStatsAndMetricsJob.class.getName()) + .withArguments(buildArguments()) + .withConfigs(buildSparkConfigs()) + .withCustomFields( + Collections.singletonMap(JobTemplateProvider.PROPERTY_VERSION_KEY, VERSION)) + .build(); + } + + /** Main entry point. */ + public static void main(String[] args) { + Map<String, String> argMap = parseArguments(args); + String catalogName = argMap.get("catalog"); + String tableIdentifier = argMap.get("table"); + UpdateMode updateMode = parseUpdateMode(argMap.get("update-mode")); + + if (catalogName == null || tableIdentifier == null) { + System.err.println("Error: --catalog and --table are required arguments"); + printUsage(); + System.exit(1); + } + + Map<String, String> updaterOptions = parseJsonOptions(argMap.get("updater-options")); + String sparkConfJson = argMap.get("spark-conf"); + + SparkSession.Builder sparkBuilder = + SparkSession.builder().appName("Gravitino Built-in Iceberg Update Stats"); + + if (sparkConfJson != null && !sparkConfJson.isEmpty()) { + Map<String, String> customConfigs = parseCustomSparkConfigs(sparkConfJson); + for (Map.Entry<String, String> entry : customConfigs.entrySet()) { + sparkBuilder.config(entry.getKey(), entry.getValue()); + } + } + + SparkSession spark = sparkBuilder.getOrCreate(); + StatisticsUpdater statisticsUpdater = null; + MetricsUpdater metricsUpdater = null; + try { + Map<String, String> optimizerProperties = buildOptimizerProperties(updaterOptions); + if (updateMode.updateStats) { + String statisticsUpdaterName = + updaterOptions.getOrDefault("statistics_updater", DEFAULT_STATISTICS_UPDATER).trim(); + statisticsUpdater = + createStatisticsUpdater( + statisticsUpdaterName, requireGravitinoConfig(optimizerProperties)); + } + if (updateMode.updateMetrics) { + String metricsUpdaterName = + updaterOptions.getOrDefault("metrics_updater", DEFAULT_METRICS_UPDATER).trim(); + metricsUpdater = createMetricsUpdater(metricsUpdaterName, optimizerProperties); + } + + updateStatistics( + spark, statisticsUpdater, metricsUpdater, updateMode, catalogName, tableIdentifier); + } catch (Exception e) { + LOG.error("Failed to update Iceberg statistics/metrics", e); + System.exit(1); + } finally { + if (statisticsUpdater != null) { + try { + statisticsUpdater.close(); + } catch (Exception e) { + LOG.warn("Failed to close statistics updater", e); + } + } + if (metricsUpdater != null) { + try { + metricsUpdater.close(); + } catch (Exception e) { + LOG.warn("Failed to close metrics updater", e); + } + } + spark.stop(); + } + } + + static void updateStatistics( + SparkSession spark, + StatisticsUpdater statisticsUpdater, + String catalogName, + String tableIdentifier) { + updateStatistics( + spark, statisticsUpdater, null, UpdateMode.STATS, catalogName, tableIdentifier); + } + + static void updateStatistics( + SparkSession spark, + StatisticsUpdater statisticsUpdater, + MetricsUpdater metricsUpdater, + String catalogName, + String tableIdentifier) { + updateStatistics( + spark, statisticsUpdater, metricsUpdater, UpdateMode.ALL, catalogName, tableIdentifier); + } + + static void updateStatistics( + SparkSession spark, + StatisticsUpdater statisticsUpdater, + MetricsUpdater metricsUpdater, + UpdateMode updateMode, + String catalogName, + String tableIdentifier) { + Objects.requireNonNull(updateMode, "updateMode must not be null"); + + if (updateMode.updateStats && statisticsUpdater == null) { + throw new IllegalArgumentException( + "Statistics updater must be configured when update_mode is stats or all"); + } + + if (updateMode.updateMetrics && metricsUpdater == null) { + throw new IllegalArgumentException( + "Metrics updater must be configured when update_mode is metrics or all"); + } + + NameIdentifier gravitinoTableIdentifier = + toGravitinoTableIdentifier(catalogName, tableIdentifier); + long metricTimestamp = System.currentTimeMillis() / 1000L; + boolean partitioned = isPartitionedTable(spark, catalogName, tableIdentifier); + if (partitioned) { + String sql = buildPartitionStatsSql(catalogName, tableIdentifier); + Row[] rows = (Row[]) spark.sql(sql).collect(); + Map<PartitionPath, List<StatisticEntry<?>>> partitionStatistics = new LinkedHashMap<>(); + List<MetricPoint> tableAndPartitionMetrics = new ArrayList<>(); + for (Row row : rows) { + PartitionPath partitionPath = toPartitionPath(row.getAs("partition")); + List<StatisticEntry<?>> statistics = toStatistics(row); + if (updateMode.updateStats) { + partitionStatistics.put(partitionPath, statistics); + } + if (updateMode.updateMetrics) { + tableAndPartitionMetrics.addAll( + toPartitionMetricPoints( + gravitinoTableIdentifier, partitionPath, statistics, metricTimestamp)); + } + } + + if (updateMode.updateStats) { + statisticsUpdater.updatePartitionStatistics(gravitinoTableIdentifier, partitionStatistics); + } + + if (updateMode.updateMetrics && !tableAndPartitionMetrics.isEmpty()) { + metricsUpdater.updateTableAndPartitionMetrics(tableAndPartitionMetrics); + } + + LOG.info( + "Updated partition data in mode {} for {} partitions on {}", + updateMode.modeName, + rows.length, + gravitinoTableIdentifier); + } else { + String sql = buildTableStatsSql(catalogName, tableIdentifier); + Row[] rows = (Row[]) spark.sql(sql).collect(); + List<StatisticEntry<?>> tableStatistics = + rows.length == 0 ? Collections.emptyList() : toStatistics(rows[0]); + + if (updateMode.updateStats) { + statisticsUpdater.updateTableStatistics(gravitinoTableIdentifier, tableStatistics); + } + + if (updateMode.updateMetrics && !tableStatistics.isEmpty()) { + metricsUpdater.updateTableAndPartitionMetrics( + toTableMetricPoints(gravitinoTableIdentifier, tableStatistics, metricTimestamp)); + } + + LOG.info( + "Updated table data in mode {} with {} metrics on {}", + updateMode.modeName, + tableStatistics.size(), + gravitinoTableIdentifier); + } + } + + static String buildTableStatsSql(String catalogName, String tableIdentifier) { + String filesTable = buildFilesTableIdentifier(catalogName, tableIdentifier); + return "SELECT " + + "COUNT(*) AS file_count, " + + "SUM(CASE WHEN content = 0 THEN 1 ELSE 0 END) AS data_files, " + + "SUM(CASE WHEN content = 1 THEN 1 ELSE 0 END) AS position_delete_files, " + + "SUM(CASE WHEN content = 2 THEN 1 ELSE 0 END) AS equality_delete_files, " + + "SUM(CASE WHEN file_size_in_bytes < " + + SMALL_FILE_THRESHOLD_BYTES + + " THEN 1 ELSE 0 END) AS small_files, " + + "AVG(POWER(" + + DEFAULT_TARGET_FILE_SIZE_BYTES + + " - LEAST(" + + DEFAULT_TARGET_FILE_SIZE_BYTES + + ", file_size_in_bytes), 2)) AS datafile_mse, " + + "AVG(file_size_in_bytes) AS avg_size, " + + "SUM(file_size_in_bytes) AS total_size " + + "FROM " + + filesTable; + } + + static String buildPartitionStatsSql(String catalogName, String tableIdentifier) { + String filesTable = buildFilesTableIdentifier(catalogName, tableIdentifier); + return "SELECT " + + "partition, " + + "COUNT(*) AS file_count, " + + "SUM(CASE WHEN content = 0 THEN 1 ELSE 0 END) AS data_files, " + + "SUM(CASE WHEN content = 1 THEN 1 ELSE 0 END) AS position_delete_files, " + + "SUM(CASE WHEN content = 2 THEN 1 ELSE 0 END) AS equality_delete_files, " + + "SUM(CASE WHEN file_size_in_bytes < " + + SMALL_FILE_THRESHOLD_BYTES + + " THEN 1 ELSE 0 END) AS small_files, " + + "AVG(POWER(" + + DEFAULT_TARGET_FILE_SIZE_BYTES + + " - LEAST(" + + DEFAULT_TARGET_FILE_SIZE_BYTES + + ", file_size_in_bytes), 2)) AS datafile_mse, " + + "AVG(file_size_in_bytes) AS avg_size, " + + "SUM(file_size_in_bytes) AS total_size " + + "FROM " + + filesTable + + " GROUP BY partition"; + } + + static boolean isPartitionedTable( + SparkSession spark, String catalogName, String tableIdentifier) { + StructType filesSchema = + spark.table(buildFilesTableIdentifier(catalogName, tableIdentifier)).schema(); + if (!Arrays.asList(filesSchema.fieldNames()).contains("partition")) { + return false; + } + StructField partitionField = filesSchema.apply("partition"); + if (!(partitionField.dataType() instanceof StructType)) { + return false; + } + return ((StructType) partitionField.dataType()).fields().length > 0; + } + + static List<StatisticEntry<?>> toStatistics(Row row) { + List<StatisticEntry<?>> statistics = new ArrayList<>(); + statistics.add( + new StatisticEntryImpl<>( + CUSTOM_STAT_PREFIX + "file_count", + StatisticValues.longValue(toLongValue(row, "file_count")))); + statistics.add( + new StatisticEntryImpl<>( + CUSTOM_STAT_PREFIX + "data_files", + StatisticValues.longValue(toLongValue(row, "data_files")))); + statistics.add( + new StatisticEntryImpl<>( + CUSTOM_STAT_PREFIX + "position_delete_files", + StatisticValues.longValue(toLongValue(row, "position_delete_files")))); + statistics.add( + new StatisticEntryImpl<>( + CUSTOM_STAT_PREFIX + "equality_delete_files", + StatisticValues.longValue(toLongValue(row, "equality_delete_files")))); + statistics.add( + new StatisticEntryImpl<>( + CUSTOM_STAT_PREFIX + "small_files", + StatisticValues.longValue(toLongValue(row, "small_files")))); + statistics.add( + new StatisticEntryImpl<>( + CUSTOM_STAT_PREFIX + "datafile_mse", + StatisticValues.doubleValue(toDoubleValue(row, "datafile_mse")))); + statistics.add( + new StatisticEntryImpl<>( + CUSTOM_STAT_PREFIX + "avg_size", + StatisticValues.doubleValue(toDoubleValue(row, "avg_size")))); + statistics.add( + new StatisticEntryImpl<>( + CUSTOM_STAT_PREFIX + "total_size", + StatisticValues.longValue(toLongValue(row, "total_size")))); + return statistics; + } + + static PartitionPath toPartitionPath(Row partitionRow) { + StructType partitionSchema = partitionRow.schema(); + List<PartitionEntry> entries = new ArrayList<>(partitionSchema.fields().length); + for (int i = 0; i < partitionSchema.fields().length; i++) { + String name = partitionSchema.fields()[i].name(); + Object value = partitionRow.get(i); Review Comment: `toPartitionPath()` uses `String.valueOf(value)` for partition values; if a partition value is actually null, this becomes the literal string "null" and will be treated as a valid partition value. Please handle nulls explicitly (either reject them with a clear error or map them to the project’s expected null/empty representation) so partition paths don’t silently become incorrect. ```suggestion Object value = partitionRow.get(i); if (value == null) { throw new IllegalArgumentException( String.format("Partition value for column '%s' is null", name)); } ``` -- 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]
