diff --git a/persistence/relational-jdbc/build.gradle.kts b/persistence/relational-jdbc/build.gradle.kts index c3e4253727..3de2526dd3 100644 --- a/persistence/relational-jdbc/build.gradle.kts +++ b/persistence/relational-jdbc/build.gradle.kts @@ -29,10 +29,16 @@ dependencies { compileOnly(platform(libs.jackson.bom)) compileOnly("com.fasterxml.jackson.core:jackson-annotations") + compileOnly("com.fasterxml.jackson.core:jackson-databind") compileOnly(libs.jakarta.annotation.api) compileOnly(libs.jakarta.enterprise.cdi.api) compileOnly(libs.jakarta.inject.api) + // Iceberg API for metrics report conversion + compileOnly(platform(libs.iceberg.bom)) + compileOnly("org.apache.iceberg:iceberg-api") + compileOnly("org.apache.iceberg:iceberg-core") + implementation(libs.smallrye.common.annotation) // @Identifier implementation(libs.postgresql) diff --git a/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/JdbcBasePersistenceImpl.java b/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/JdbcBasePersistenceImpl.java index 9401df2dd0..35e7d9c30e 100644 --- a/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/JdbcBasePersistenceImpl.java +++ b/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/JdbcBasePersistenceImpl.java @@ -69,11 +69,15 @@ import org.apache.polaris.core.storage.PolarisStorageIntegrationProvider; import org.apache.polaris.core.storage.StorageLocation; import org.apache.polaris.persistence.relational.jdbc.models.EntityNameLookupRecordConverter; +import org.apache.polaris.persistence.relational.jdbc.models.ModelCommitMetricsReport; +import org.apache.polaris.persistence.relational.jdbc.models.ModelCommitMetricsReportConverter; import org.apache.polaris.persistence.relational.jdbc.models.ModelEntity; import org.apache.polaris.persistence.relational.jdbc.models.ModelEvent; import org.apache.polaris.persistence.relational.jdbc.models.ModelGrantRecord; import org.apache.polaris.persistence.relational.jdbc.models.ModelPolicyMappingRecord; import org.apache.polaris.persistence.relational.jdbc.models.ModelPrincipalAuthenticationData; +import org.apache.polaris.persistence.relational.jdbc.models.ModelScanMetricsReport; +import org.apache.polaris.persistence.relational.jdbc.models.ModelScanMetricsReportConverter; import org.apache.polaris.persistence.relational.jdbc.models.SchemaVersion; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -315,6 +319,264 @@ public void writeEvents(@Nonnull List events) { } } + /** + * Writes a scan metrics report to the database as a first-class entity. + * + * @param report the scan metrics report to persist + */ + public void writeScanMetricsReport(@Nonnull ModelScanMetricsReport report) { + try { + PreparedQuery pq = + QueryGenerator.generateInsertQueryWithoutRealmId( + ModelScanMetricsReport.ALL_COLUMNS, + ModelScanMetricsReport.TABLE_NAME, + report.toMap(datasourceOperations.getDatabaseType()).values().stream().toList()); + int updated = datasourceOperations.executeUpdate(pq); + if (updated == 0) { + throw new SQLException("Scan metrics report was not inserted."); + } + } catch (SQLException e) { + throw new RuntimeException( + String.format("Failed to write scan metrics report due to %s", e.getMessage()), e); + } + } + + /** + * Writes a commit metrics report to the database as a first-class entity. + * + * @param report the commit metrics report to persist + */ + public void writeCommitMetricsReport(@Nonnull ModelCommitMetricsReport report) { + try { + PreparedQuery pq = + QueryGenerator.generateInsertQueryWithoutRealmId( + ModelCommitMetricsReport.ALL_COLUMNS, + ModelCommitMetricsReport.TABLE_NAME, + report.toMap(datasourceOperations.getDatabaseType()).values().stream().toList()); + int updated = datasourceOperations.executeUpdate(pq); + if (updated == 0) { + throw new SQLException("Commit metrics report was not inserted."); + } + } catch (SQLException e) { + throw new RuntimeException( + String.format("Failed to write commit metrics report due to %s", e.getMessage()), e); + } + } + + /** + * Retrieves scan metrics reports for a specific table within a time range. + * + * @param catalogName the catalog name + * @param namespace the namespace + * @param tableName the table name + * @param startTimeMs start of time range (inclusive), or null for no lower bound + * @param endTimeMs end of time range (exclusive), or null for no upper bound + * @param limit maximum number of results to return + * @return list of scan metrics reports matching the criteria + */ + @Nonnull + public List queryScanMetricsReports( + @Nonnull String catalogName, + @Nonnull String namespace, + @Nonnull String tableName, + @Nullable Long startTimeMs, + @Nullable Long endTimeMs, + int limit) { + try { + StringBuilder whereClause = new StringBuilder(); + whereClause.append("realm_id = ? AND catalog_name = ? AND namespace = ? AND table_name = ?"); + List values = new ArrayList<>(List.of(realmId, catalogName, namespace, tableName)); + + if (startTimeMs != null) { + whereClause.append(" AND timestamp_ms >= ?"); + values.add(startTimeMs); + } + if (endTimeMs != null) { + whereClause.append(" AND timestamp_ms < ?"); + values.add(endTimeMs); + } + + String sql = + "SELECT * FROM " + + QueryGenerator.getFullyQualifiedTableName(ModelScanMetricsReport.TABLE_NAME) + + " WHERE " + + whereClause + + " ORDER BY timestamp_ms DESC LIMIT " + + limit; + + PreparedQuery query = new PreparedQuery(sql, values); + var results = + datasourceOperations.executeSelect(query, new ModelScanMetricsReportConverter()); + return results == null ? Collections.emptyList() : results; + } catch (SQLException e) { + throw new RuntimeException( + String.format("Failed to query scan metrics reports due to %s", e.getMessage()), e); + } + } + + /** + * Retrieves commit metrics reports for a specific table within a time range. + * + * @param catalogName the catalog name + * @param namespace the namespace + * @param tableName the table name + * @param startTimeMs start of time range (inclusive), or null for no lower bound + * @param endTimeMs end of time range (exclusive), or null for no upper bound + * @param limit maximum number of results to return + * @return list of commit metrics reports matching the criteria + */ + @Nonnull + public List queryCommitMetricsReports( + @Nonnull String catalogName, + @Nonnull String namespace, + @Nonnull String tableName, + @Nullable Long startTimeMs, + @Nullable Long endTimeMs, + int limit) { + try { + List values = new ArrayList<>(List.of(realmId, catalogName, namespace, tableName)); + + StringBuilder whereClause = new StringBuilder(); + whereClause.append("realm_id = ? AND catalog_name = ? AND namespace = ? AND table_name = ?"); + + if (startTimeMs != null) { + whereClause.append(" AND timestamp_ms >= ?"); + values.add(startTimeMs); + } + if (endTimeMs != null) { + whereClause.append(" AND timestamp_ms < ?"); + values.add(endTimeMs); + } + + String sql = + "SELECT * FROM " + + QueryGenerator.getFullyQualifiedTableName(ModelCommitMetricsReport.TABLE_NAME) + + " WHERE " + + whereClause + + " ORDER BY timestamp_ms DESC LIMIT " + + limit; + + PreparedQuery query = new PreparedQuery(sql, values); + var results = + datasourceOperations.executeSelect(query, new ModelCommitMetricsReportConverter()); + return results == null ? Collections.emptyList() : results; + } catch (SQLException e) { + throw new RuntimeException( + String.format("Failed to query commit metrics reports due to %s", e.getMessage()), e); + } + } + + /** + * Retrieves scan metrics reports by OpenTelemetry trace ID. + * + * @param traceId the OpenTelemetry trace ID + * @return list of scan metrics reports with the given trace ID + */ + @Nonnull + public List queryScanMetricsReportsByTraceId(@Nonnull String traceId) { + try { + String sql = + "SELECT * FROM " + + QueryGenerator.getFullyQualifiedTableName(ModelScanMetricsReport.TABLE_NAME) + + " WHERE realm_id = ? AND otel_trace_id = ? ORDER BY timestamp_ms DESC"; + + PreparedQuery query = new PreparedQuery(sql, List.of(realmId, traceId)); + var results = + datasourceOperations.executeSelect(query, new ModelScanMetricsReportConverter()); + return results == null ? Collections.emptyList() : results; + } catch (SQLException e) { + throw new RuntimeException( + String.format( + "Failed to query scan metrics reports by trace ID due to %s", e.getMessage()), + e); + } + } + + /** + * Retrieves commit metrics reports by OpenTelemetry trace ID. + * + * @param traceId the OpenTelemetry trace ID + * @return list of commit metrics reports with the given trace ID + */ + @Nonnull + public List queryCommitMetricsReportsByTraceId( + @Nonnull String traceId) { + try { + String sql = + "SELECT * FROM " + + QueryGenerator.getFullyQualifiedTableName(ModelCommitMetricsReport.TABLE_NAME) + + " WHERE realm_id = ? AND otel_trace_id = ? ORDER BY timestamp_ms DESC"; + + PreparedQuery query = new PreparedQuery(sql, List.of(realmId, traceId)); + var results = + datasourceOperations.executeSelect(query, new ModelCommitMetricsReportConverter()); + return results == null ? Collections.emptyList() : results; + } catch (SQLException e) { + throw new RuntimeException( + String.format( + "Failed to query commit metrics reports by trace ID due to %s", e.getMessage()), + e); + } + } + + /** + * Deletes scan metrics reports older than the specified timestamp. + * + * @param olderThanMs timestamp in milliseconds; reports with timestamp_ms less than this will be + * deleted + * @return the number of reports deleted + */ + public int deleteScanMetricsReportsOlderThan(long olderThanMs) { + try { + String sql = + "DELETE FROM " + + QueryGenerator.getFullyQualifiedTableName(ModelScanMetricsReport.TABLE_NAME) + + " WHERE realm_id = ? AND timestamp_ms < ?"; + + PreparedQuery query = new PreparedQuery(sql, List.of(realmId, olderThanMs)); + return datasourceOperations.executeUpdate(query); + } catch (SQLException e) { + throw new RuntimeException( + String.format("Failed to delete old scan metrics reports due to %s", e.getMessage()), e); + } + } + + /** + * Deletes commit metrics reports older than the specified timestamp. + * + * @param olderThanMs timestamp in milliseconds; reports with timestamp_ms less than this will be + * deleted + * @return the number of reports deleted + */ + public int deleteCommitMetricsReportsOlderThan(long olderThanMs) { + try { + String sql = + "DELETE FROM " + + QueryGenerator.getFullyQualifiedTableName(ModelCommitMetricsReport.TABLE_NAME) + + " WHERE realm_id = ? AND timestamp_ms < ?"; + + PreparedQuery query = new PreparedQuery(sql, List.of(realmId, olderThanMs)); + return datasourceOperations.executeUpdate(query); + } catch (SQLException e) { + throw new RuntimeException( + String.format("Failed to delete old commit metrics reports due to %s", e.getMessage()), + e); + } + } + + /** + * Deletes all metrics reports (both scan and commit) older than the specified timestamp. + * + * @param olderThanMs timestamp in milliseconds; reports with timestamp_ms less than this will be + * deleted + * @return the total number of reports deleted (scan + commit) + */ + public int deleteAllMetricsReportsOlderThan(long olderThanMs) { + int scanDeleted = deleteScanMetricsReportsOlderThan(olderThanMs); + int commitDeleted = deleteCommitMetricsReportsOlderThan(olderThanMs); + return scanDeleted + commitDeleted; + } + @Override public void deleteEntity(@Nonnull PolarisCallContext callCtx, @Nonnull PolarisBaseEntity entity) { ModelEntity modelEntity = ModelEntity.fromEntity(entity, schemaVersion); diff --git a/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/QueryGenerator.java b/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/QueryGenerator.java index 485956ed85..423e965bd9 100644 --- a/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/QueryGenerator.java +++ b/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/QueryGenerator.java @@ -169,6 +169,30 @@ public static PreparedQuery generateInsertQuery( return new PreparedQuery(sql, finalValues); } + /** + * Generates an INSERT query for a given table without appending realm_id. Use this when the + * columns already include realm_id. + * + * @param allColumns Columns to insert values into (should already include realm_id if needed). + * @param tableName Target table name. + * @param values Values for each column (must match order of columns). + * @return INSERT query with value bindings. + */ + public static PreparedQuery generateInsertQueryWithoutRealmId( + @Nonnull List allColumns, @Nonnull String tableName, List values) { + String columns = String.join(", ", allColumns); + String placeholders = allColumns.stream().map(c -> "?").collect(Collectors.joining(", ")); + String sql = + "INSERT INTO " + + getFullyQualifiedTableName(tableName) + + " (" + + columns + + ") VALUES (" + + placeholders + + ")"; + return new PreparedQuery(sql, values); + } + /** * Builds an UPDATE query. * @@ -317,7 +341,7 @@ public static PreparedQuery generateOverlapQuery( return new PreparedQuery(query.sql(), where.parameters()); } - private static String getFullyQualifiedTableName(String tableName) { + static String getFullyQualifiedTableName(String tableName) { // TODO: make schema name configurable. return "POLARIS_SCHEMA." + tableName; } diff --git a/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/models/MetricsReportConverter.java b/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/models/MetricsReportConverter.java new file mode 100644 index 0000000000..c9660c2e84 --- /dev/null +++ b/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/models/MetricsReportConverter.java @@ -0,0 +1,275 @@ +/* + * 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.polaris.persistence.relational.jdbc.models; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import jakarta.annotation.Nullable; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.iceberg.metrics.CommitMetricsResult; +import org.apache.iceberg.metrics.CommitReport; +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanMetricsResult; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.iceberg.metrics.TimerResult; + +/** + * Converter utility class for transforming Iceberg metrics reports into persistence model classes. + */ +public final class MetricsReportConverter { + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private MetricsReportConverter() { + // Utility class + } + + /** + * Converts an Iceberg ScanReport to a ModelScanMetricsReport. + * + * @param scanReport the Iceberg scan report + * @param realmId the realm ID for multi-tenancy + * @param catalogId the catalog ID + * @param catalogName the catalog name + * @param namespace the namespace (dot-separated) + * @param principalName the principal who initiated the scan (optional) + * @param requestId the request ID (optional) + * @param otelTraceId OpenTelemetry trace ID (optional) + * @param otelSpanId OpenTelemetry span ID (optional) + * @return the converted ModelScanMetricsReport + */ + public static ModelScanMetricsReport fromScanReport( + ScanReport scanReport, + String realmId, + String catalogId, + String catalogName, + String namespace, + @Nullable String principalName, + @Nullable String requestId, + @Nullable String otelTraceId, + @Nullable String otelSpanId) { + + String reportId = UUID.randomUUID().toString(); + long timestampMs = System.currentTimeMillis(); + + ScanMetricsResult metrics = scanReport.scanMetrics(); + + ImmutableModelScanMetricsReport.Builder builder = + ImmutableModelScanMetricsReport.builder() + .reportId(reportId) + .realmId(realmId) + .catalogId(catalogId) + .catalogName(catalogName) + .namespace(namespace) + .tableName(scanReport.tableName()) + .timestampMs(timestampMs) + .principalName(principalName) + .requestId(requestId) + .otelTraceId(otelTraceId) + .otelSpanId(otelSpanId) + .snapshotId(scanReport.snapshotId()) + .schemaId(scanReport.schemaId()) + .filterExpression(scanReport.filter() != null ? scanReport.filter().toString() : null) + .projectedFieldIds(formatIntegerList(scanReport.projectedFieldIds())) + .projectedFieldNames(formatStringList(scanReport.projectedFieldNames())); + + // Extract metrics values + if (metrics != null) { + builder + .resultDataFiles(getCounterValue(metrics.resultDataFiles())) + .resultDeleteFiles(getCounterValue(metrics.resultDeleteFiles())) + .totalFileSizeBytes(getCounterValue(metrics.totalFileSizeInBytes())) + .totalDataManifests(getCounterValue(metrics.totalDataManifests())) + .totalDeleteManifests(getCounterValue(metrics.totalDeleteManifests())) + .scannedDataManifests(getCounterValue(metrics.scannedDataManifests())) + .scannedDeleteManifests(getCounterValue(metrics.scannedDeleteManifests())) + .skippedDataManifests(getCounterValue(metrics.skippedDataManifests())) + .skippedDeleteManifests(getCounterValue(metrics.skippedDeleteManifests())) + .skippedDataFiles(getCounterValue(metrics.skippedDataFiles())) + .skippedDeleteFiles(getCounterValue(metrics.skippedDeleteFiles())) + .totalPlanningDurationMs(getTimerValueMs(metrics.totalPlanningDuration())) + .equalityDeleteFiles(getCounterValue(metrics.equalityDeleteFiles())) + .positionalDeleteFiles(getCounterValue(metrics.positionalDeleteFiles())) + .indexedDeleteFiles(getCounterValue(metrics.indexedDeleteFiles())) + .totalDeleteFileSizeBytes(getCounterValue(metrics.totalDeleteFileSizeInBytes())); + } else { + builder + .resultDataFiles(0L) + .resultDeleteFiles(0L) + .totalFileSizeBytes(0L) + .totalDataManifests(0L) + .totalDeleteManifests(0L) + .scannedDataManifests(0L) + .scannedDeleteManifests(0L) + .skippedDataManifests(0L) + .skippedDeleteManifests(0L) + .skippedDataFiles(0L) + .skippedDeleteFiles(0L) + .totalPlanningDurationMs(0L) + .equalityDeleteFiles(0L) + .positionalDeleteFiles(0L) + .indexedDeleteFiles(0L) + .totalDeleteFileSizeBytes(0L); + } + + // Store additional metadata as JSON + Map metadata = scanReport.metadata(); + if (metadata != null && !metadata.isEmpty()) { + builder.metadata(toJson(metadata)); + } + + return builder.build(); + } + + /** + * Converts an Iceberg CommitReport to a ModelCommitMetricsReport. + * + * @param commitReport the Iceberg commit report + * @param realmId the realm ID for multi-tenancy + * @param catalogId the catalog ID + * @param catalogName the catalog name + * @param namespace the namespace (dot-separated) + * @param principalName the principal who initiated the commit (optional) + * @param requestId the request ID (optional) + * @param otelTraceId OpenTelemetry trace ID (optional) + * @param otelSpanId OpenTelemetry span ID (optional) + * @return the converted ModelCommitMetricsReport + */ + public static ModelCommitMetricsReport fromCommitReport( + CommitReport commitReport, + String realmId, + String catalogId, + String catalogName, + String namespace, + @Nullable String principalName, + @Nullable String requestId, + @Nullable String otelTraceId, + @Nullable String otelSpanId) { + + String reportId = UUID.randomUUID().toString(); + long timestampMs = System.currentTimeMillis(); + + CommitMetricsResult metrics = commitReport.commitMetrics(); + + ImmutableModelCommitMetricsReport.Builder builder = + ImmutableModelCommitMetricsReport.builder() + .reportId(reportId) + .realmId(realmId) + .catalogId(catalogId) + .catalogName(catalogName) + .namespace(namespace) + .tableName(commitReport.tableName()) + .timestampMs(timestampMs) + .principalName(principalName) + .requestId(requestId) + .otelTraceId(otelTraceId) + .otelSpanId(otelSpanId) + .snapshotId(commitReport.snapshotId()) + .sequenceNumber(commitReport.sequenceNumber()) + .operation(commitReport.operation() != null ? commitReport.operation() : "UNKNOWN"); + + // Extract metrics values + if (metrics != null) { + builder + .addedDataFiles(getCounterValue(metrics.addedDataFiles())) + .removedDataFiles(getCounterValue(metrics.removedDataFiles())) + .totalDataFiles(getCounterValue(metrics.totalDataFiles())) + .addedDeleteFiles(getCounterValue(metrics.addedDeleteFiles())) + .removedDeleteFiles(getCounterValue(metrics.removedDeleteFiles())) + .totalDeleteFiles(getCounterValue(metrics.totalDeleteFiles())) + .addedEqualityDeleteFiles(getCounterValue(metrics.addedEqualityDeleteFiles())) + .removedEqualityDeleteFiles(getCounterValue(metrics.removedEqualityDeleteFiles())) + .addedPositionalDeleteFiles(getCounterValue(metrics.addedPositionalDeleteFiles())) + .removedPositionalDeleteFiles(getCounterValue(metrics.removedPositionalDeleteFiles())) + .addedRecords(getCounterValue(metrics.addedRecords())) + .removedRecords(getCounterValue(metrics.removedRecords())) + .totalRecords(getCounterValue(metrics.totalRecords())) + .addedFileSizeBytes(getCounterValue(metrics.addedFilesSizeInBytes())) + .removedFileSizeBytes(getCounterValue(metrics.removedFilesSizeInBytes())) + .totalFileSizeBytes(getCounterValue(metrics.totalFilesSizeInBytes())) + .totalDurationMs(getTimerValueMs(metrics.totalDuration())) + .attempts(getCounterValueInt(metrics.attempts())); + } else { + builder + .addedDataFiles(0L) + .removedDataFiles(0L) + .totalDataFiles(0L) + .addedDeleteFiles(0L) + .removedDeleteFiles(0L) + .totalDeleteFiles(0L) + .addedEqualityDeleteFiles(0L) + .removedEqualityDeleteFiles(0L) + .addedPositionalDeleteFiles(0L) + .removedPositionalDeleteFiles(0L) + .addedRecords(0L) + .removedRecords(0L) + .totalRecords(0L) + .addedFileSizeBytes(0L) + .removedFileSizeBytes(0L) + .totalFileSizeBytes(0L) + .totalDurationMs(0L) + .attempts(1); + } + + // Store additional metadata as JSON + Map metadata = commitReport.metadata(); + if (metadata != null && !metadata.isEmpty()) { + builder.metadata(toJson(metadata)); + } + + return builder.build(); + } + + private static long getCounterValue(@Nullable CounterResult counter) { + return counter != null ? counter.value() : 0L; + } + + private static int getCounterValueInt(@Nullable CounterResult counter) { + return counter != null ? (int) counter.value() : 1; + } + + private static long getTimerValueMs(@Nullable TimerResult timer) { + return timer != null && timer.totalDuration() != null ? timer.totalDuration().toMillis() : 0L; + } + + private static String formatIntegerList(@Nullable List list) { + if (list == null || list.isEmpty()) { + return null; + } + return list.stream().map(String::valueOf).collect(Collectors.joining(",")); + } + + private static String formatStringList(@Nullable List list) { + if (list == null || list.isEmpty()) { + return null; + } + return String.join(",", list); + } + + private static String toJson(Map map) { + try { + return OBJECT_MAPPER.writeValueAsString(map); + } catch (JsonProcessingException e) { + return "{}"; + } + } +} diff --git a/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/models/ModelCommitMetricsReport.java b/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/models/ModelCommitMetricsReport.java new file mode 100644 index 0000000000..a2c7b420c6 --- /dev/null +++ b/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/models/ModelCommitMetricsReport.java @@ -0,0 +1,299 @@ +/* + * 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.polaris.persistence.relational.jdbc.models; + +import jakarta.annotation.Nullable; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import org.apache.polaris.immutables.PolarisImmutable; +import org.apache.polaris.persistence.relational.jdbc.DatabaseType; + +/** Model class for commit_metrics_report table - stores commit metrics as first-class entities. */ +@PolarisImmutable +public interface ModelCommitMetricsReport extends Converter { + String TABLE_NAME = "COMMIT_METRICS_REPORT"; + + // Column names + String REPORT_ID = "report_id"; + String REALM_ID = "realm_id"; + String CATALOG_ID = "catalog_id"; + String CATALOG_NAME = "catalog_name"; + String NAMESPACE = "namespace"; + String TABLE_NAME_COL = "table_name"; + String TIMESTAMP_MS = "timestamp_ms"; + String PRINCIPAL_NAME = "principal_name"; + String REQUEST_ID = "request_id"; + String OTEL_TRACE_ID = "otel_trace_id"; + String OTEL_SPAN_ID = "otel_span_id"; + String REPORT_TRACE_ID = "report_trace_id"; + String SNAPSHOT_ID = "snapshot_id"; + String SEQUENCE_NUMBER = "sequence_number"; + String OPERATION = "operation"; + String ADDED_DATA_FILES = "added_data_files"; + String REMOVED_DATA_FILES = "removed_data_files"; + String TOTAL_DATA_FILES = "total_data_files"; + String ADDED_DELETE_FILES = "added_delete_files"; + String REMOVED_DELETE_FILES = "removed_delete_files"; + String TOTAL_DELETE_FILES = "total_delete_files"; + String ADDED_EQUALITY_DELETE_FILES = "added_equality_delete_files"; + String REMOVED_EQUALITY_DELETE_FILES = "removed_equality_delete_files"; + String ADDED_POSITIONAL_DELETE_FILES = "added_positional_delete_files"; + String REMOVED_POSITIONAL_DELETE_FILES = "removed_positional_delete_files"; + String ADDED_RECORDS = "added_records"; + String REMOVED_RECORDS = "removed_records"; + String TOTAL_RECORDS = "total_records"; + String ADDED_FILE_SIZE_BYTES = "added_file_size_bytes"; + String REMOVED_FILE_SIZE_BYTES = "removed_file_size_bytes"; + String TOTAL_FILE_SIZE_BYTES = "total_file_size_bytes"; + String TOTAL_DURATION_MS = "total_duration_ms"; + String ATTEMPTS = "attempts"; + String METADATA = "metadata"; + + List ALL_COLUMNS = + List.of( + REPORT_ID, + REALM_ID, + CATALOG_ID, + CATALOG_NAME, + NAMESPACE, + TABLE_NAME_COL, + TIMESTAMP_MS, + PRINCIPAL_NAME, + REQUEST_ID, + OTEL_TRACE_ID, + OTEL_SPAN_ID, + REPORT_TRACE_ID, + SNAPSHOT_ID, + SEQUENCE_NUMBER, + OPERATION, + ADDED_DATA_FILES, + REMOVED_DATA_FILES, + TOTAL_DATA_FILES, + ADDED_DELETE_FILES, + REMOVED_DELETE_FILES, + TOTAL_DELETE_FILES, + ADDED_EQUALITY_DELETE_FILES, + REMOVED_EQUALITY_DELETE_FILES, + ADDED_POSITIONAL_DELETE_FILES, + REMOVED_POSITIONAL_DELETE_FILES, + ADDED_RECORDS, + REMOVED_RECORDS, + TOTAL_RECORDS, + ADDED_FILE_SIZE_BYTES, + REMOVED_FILE_SIZE_BYTES, + TOTAL_FILE_SIZE_BYTES, + TOTAL_DURATION_MS, + ATTEMPTS, + METADATA); + + // Getters + String getReportId(); + + String getRealmId(); + + String getCatalogId(); + + String getCatalogName(); + + String getNamespace(); + + String getTableName(); + + long getTimestampMs(); + + @Nullable + String getPrincipalName(); + + @Nullable + String getRequestId(); + + @Nullable + String getOtelTraceId(); + + @Nullable + String getOtelSpanId(); + + @Nullable + String getReportTraceId(); + + long getSnapshotId(); + + @Nullable + Long getSequenceNumber(); + + String getOperation(); + + long getAddedDataFiles(); + + long getRemovedDataFiles(); + + long getTotalDataFiles(); + + long getAddedDeleteFiles(); + + long getRemovedDeleteFiles(); + + long getTotalDeleteFiles(); + + long getAddedEqualityDeleteFiles(); + + long getRemovedEqualityDeleteFiles(); + + long getAddedPositionalDeleteFiles(); + + long getRemovedPositionalDeleteFiles(); + + long getAddedRecords(); + + long getRemovedRecords(); + + long getTotalRecords(); + + long getAddedFileSizeBytes(); + + long getRemovedFileSizeBytes(); + + long getTotalFileSizeBytes(); + + long getTotalDurationMs(); + + int getAttempts(); + + @Nullable + String getMetadata(); + + @Override + default ModelCommitMetricsReport fromResultSet(ResultSet rs) throws SQLException { + return ImmutableModelCommitMetricsReport.builder() + .reportId(rs.getString(REPORT_ID)) + .realmId(rs.getString(REALM_ID)) + .catalogId(rs.getString(CATALOG_ID)) + .catalogName(rs.getString(CATALOG_NAME)) + .namespace(rs.getString(NAMESPACE)) + .tableName(rs.getString(TABLE_NAME_COL)) + .timestampMs(rs.getLong(TIMESTAMP_MS)) + .principalName(rs.getString(PRINCIPAL_NAME)) + .requestId(rs.getString(REQUEST_ID)) + .otelTraceId(rs.getString(OTEL_TRACE_ID)) + .otelSpanId(rs.getString(OTEL_SPAN_ID)) + .reportTraceId(rs.getString(REPORT_TRACE_ID)) + .snapshotId(rs.getLong(SNAPSHOT_ID)) + .sequenceNumber(rs.getObject(SEQUENCE_NUMBER, Long.class)) + .operation(rs.getString(OPERATION)) + .addedDataFiles(rs.getLong(ADDED_DATA_FILES)) + .removedDataFiles(rs.getLong(REMOVED_DATA_FILES)) + .totalDataFiles(rs.getLong(TOTAL_DATA_FILES)) + .addedDeleteFiles(rs.getLong(ADDED_DELETE_FILES)) + .removedDeleteFiles(rs.getLong(REMOVED_DELETE_FILES)) + .totalDeleteFiles(rs.getLong(TOTAL_DELETE_FILES)) + .addedEqualityDeleteFiles(rs.getLong(ADDED_EQUALITY_DELETE_FILES)) + .removedEqualityDeleteFiles(rs.getLong(REMOVED_EQUALITY_DELETE_FILES)) + .addedPositionalDeleteFiles(rs.getLong(ADDED_POSITIONAL_DELETE_FILES)) + .removedPositionalDeleteFiles(rs.getLong(REMOVED_POSITIONAL_DELETE_FILES)) + .addedRecords(rs.getLong(ADDED_RECORDS)) + .removedRecords(rs.getLong(REMOVED_RECORDS)) + .totalRecords(rs.getLong(TOTAL_RECORDS)) + .addedFileSizeBytes(rs.getLong(ADDED_FILE_SIZE_BYTES)) + .removedFileSizeBytes(rs.getLong(REMOVED_FILE_SIZE_BYTES)) + .totalFileSizeBytes(rs.getLong(TOTAL_FILE_SIZE_BYTES)) + .totalDurationMs(rs.getLong(TOTAL_DURATION_MS)) + .attempts(rs.getInt(ATTEMPTS)) + .metadata(rs.getString(METADATA)) + .build(); + } + + @Override + default Map toMap(DatabaseType databaseType) { + Map map = new LinkedHashMap<>(); + map.put(REPORT_ID, getReportId()); + map.put(REALM_ID, getRealmId()); + map.put(CATALOG_ID, getCatalogId()); + map.put(CATALOG_NAME, getCatalogName()); + map.put(NAMESPACE, getNamespace()); + map.put(TABLE_NAME_COL, getTableName()); + map.put(TIMESTAMP_MS, getTimestampMs()); + map.put(PRINCIPAL_NAME, getPrincipalName()); + map.put(REQUEST_ID, getRequestId()); + map.put(OTEL_TRACE_ID, getOtelTraceId()); + map.put(OTEL_SPAN_ID, getOtelSpanId()); + map.put(REPORT_TRACE_ID, getReportTraceId()); + map.put(SNAPSHOT_ID, getSnapshotId()); + map.put(SEQUENCE_NUMBER, getSequenceNumber()); + map.put(OPERATION, getOperation()); + map.put(ADDED_DATA_FILES, getAddedDataFiles()); + map.put(REMOVED_DATA_FILES, getRemovedDataFiles()); + map.put(TOTAL_DATA_FILES, getTotalDataFiles()); + map.put(ADDED_DELETE_FILES, getAddedDeleteFiles()); + map.put(REMOVED_DELETE_FILES, getRemovedDeleteFiles()); + map.put(TOTAL_DELETE_FILES, getTotalDeleteFiles()); + map.put(ADDED_EQUALITY_DELETE_FILES, getAddedEqualityDeleteFiles()); + map.put(REMOVED_EQUALITY_DELETE_FILES, getRemovedEqualityDeleteFiles()); + map.put(ADDED_POSITIONAL_DELETE_FILES, getAddedPositionalDeleteFiles()); + map.put(REMOVED_POSITIONAL_DELETE_FILES, getRemovedPositionalDeleteFiles()); + map.put(ADDED_RECORDS, getAddedRecords()); + map.put(REMOVED_RECORDS, getRemovedRecords()); + map.put(TOTAL_RECORDS, getTotalRecords()); + map.put(ADDED_FILE_SIZE_BYTES, getAddedFileSizeBytes()); + map.put(REMOVED_FILE_SIZE_BYTES, getRemovedFileSizeBytes()); + map.put(TOTAL_FILE_SIZE_BYTES, getTotalFileSizeBytes()); + map.put(TOTAL_DURATION_MS, getTotalDurationMs()); + map.put(ATTEMPTS, getAttempts()); + if (databaseType.equals(DatabaseType.POSTGRES)) { + map.put(METADATA, toJsonbPGobject(getMetadata() != null ? getMetadata() : "{}")); + } else { + map.put(METADATA, getMetadata() != null ? getMetadata() : "{}"); + } + return map; + } + + /** Dummy instance to be used as a Converter when calling fromResultSet(). */ + ModelCommitMetricsReport CONVERTER = + ImmutableModelCommitMetricsReport.builder() + .reportId("") + .realmId("") + .catalogId("") + .catalogName("") + .namespace("") + .tableName("") + .timestampMs(0L) + .snapshotId(0L) + .operation("") + .addedDataFiles(0L) + .removedDataFiles(0L) + .totalDataFiles(0L) + .addedDeleteFiles(0L) + .removedDeleteFiles(0L) + .totalDeleteFiles(0L) + .addedEqualityDeleteFiles(0L) + .removedEqualityDeleteFiles(0L) + .addedPositionalDeleteFiles(0L) + .removedPositionalDeleteFiles(0L) + .addedRecords(0L) + .removedRecords(0L) + .totalRecords(0L) + .addedFileSizeBytes(0L) + .removedFileSizeBytes(0L) + .totalFileSizeBytes(0L) + .totalDurationMs(0L) + .attempts(1) + .build(); +} diff --git a/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/models/ModelCommitMetricsReportConverter.java b/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/models/ModelCommitMetricsReportConverter.java new file mode 100644 index 0000000000..16417e4afe --- /dev/null +++ b/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/models/ModelCommitMetricsReportConverter.java @@ -0,0 +1,79 @@ +/* + * 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.polaris.persistence.relational.jdbc.models; + +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.Map; +import org.apache.polaris.persistence.relational.jdbc.DatabaseType; + +/** + * Converter for reading ModelCommitMetricsReport from database result sets. This class is needed + * because the Immutables-generated class cannot be instantiated without required fields. + */ +public class ModelCommitMetricsReportConverter implements Converter { + + @Override + public ModelCommitMetricsReport fromResultSet(ResultSet rs) throws SQLException { + return ImmutableModelCommitMetricsReport.builder() + .reportId(rs.getString(ModelCommitMetricsReport.REPORT_ID)) + .realmId(rs.getString(ModelCommitMetricsReport.REALM_ID)) + .catalogId(rs.getString(ModelCommitMetricsReport.CATALOG_ID)) + .catalogName(rs.getString(ModelCommitMetricsReport.CATALOG_NAME)) + .namespace(rs.getString(ModelCommitMetricsReport.NAMESPACE)) + .tableName(rs.getString(ModelCommitMetricsReport.TABLE_NAME_COL)) + .timestampMs(rs.getLong(ModelCommitMetricsReport.TIMESTAMP_MS)) + .principalName(rs.getString(ModelCommitMetricsReport.PRINCIPAL_NAME)) + .requestId(rs.getString(ModelCommitMetricsReport.REQUEST_ID)) + .otelTraceId(rs.getString(ModelCommitMetricsReport.OTEL_TRACE_ID)) + .otelSpanId(rs.getString(ModelCommitMetricsReport.OTEL_SPAN_ID)) + .reportTraceId(rs.getString(ModelCommitMetricsReport.REPORT_TRACE_ID)) + .snapshotId(rs.getObject(ModelCommitMetricsReport.SNAPSHOT_ID, Long.class)) + .sequenceNumber(rs.getObject(ModelCommitMetricsReport.SEQUENCE_NUMBER, Long.class)) + .operation(rs.getString(ModelCommitMetricsReport.OPERATION)) + .addedDataFiles(rs.getLong(ModelCommitMetricsReport.ADDED_DATA_FILES)) + .removedDataFiles(rs.getLong(ModelCommitMetricsReport.REMOVED_DATA_FILES)) + .totalDataFiles(rs.getLong(ModelCommitMetricsReport.TOTAL_DATA_FILES)) + .addedDeleteFiles(rs.getLong(ModelCommitMetricsReport.ADDED_DELETE_FILES)) + .removedDeleteFiles(rs.getLong(ModelCommitMetricsReport.REMOVED_DELETE_FILES)) + .totalDeleteFiles(rs.getLong(ModelCommitMetricsReport.TOTAL_DELETE_FILES)) + .addedEqualityDeleteFiles(rs.getLong(ModelCommitMetricsReport.ADDED_EQUALITY_DELETE_FILES)) + .removedEqualityDeleteFiles( + rs.getLong(ModelCommitMetricsReport.REMOVED_EQUALITY_DELETE_FILES)) + .addedPositionalDeleteFiles( + rs.getLong(ModelCommitMetricsReport.ADDED_POSITIONAL_DELETE_FILES)) + .removedPositionalDeleteFiles( + rs.getLong(ModelCommitMetricsReport.REMOVED_POSITIONAL_DELETE_FILES)) + .addedRecords(rs.getLong(ModelCommitMetricsReport.ADDED_RECORDS)) + .removedRecords(rs.getLong(ModelCommitMetricsReport.REMOVED_RECORDS)) + .totalRecords(rs.getLong(ModelCommitMetricsReport.TOTAL_RECORDS)) + .addedFileSizeBytes(rs.getLong(ModelCommitMetricsReport.ADDED_FILE_SIZE_BYTES)) + .removedFileSizeBytes(rs.getLong(ModelCommitMetricsReport.REMOVED_FILE_SIZE_BYTES)) + .totalFileSizeBytes(rs.getLong(ModelCommitMetricsReport.TOTAL_FILE_SIZE_BYTES)) + .totalDurationMs(rs.getObject(ModelCommitMetricsReport.TOTAL_DURATION_MS, Long.class)) + .attempts(rs.getObject(ModelCommitMetricsReport.ATTEMPTS, Integer.class)) + .metadata(rs.getString(ModelCommitMetricsReport.METADATA)) + .build(); + } + + @Override + public Map toMap(DatabaseType databaseType) { + throw new UnsupportedOperationException("Converter is read-only"); + } +} diff --git a/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/models/ModelScanMetricsReport.java b/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/models/ModelScanMetricsReport.java new file mode 100644 index 0000000000..8a44d44390 --- /dev/null +++ b/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/models/ModelScanMetricsReport.java @@ -0,0 +1,299 @@ +/* + * 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.polaris.persistence.relational.jdbc.models; + +import jakarta.annotation.Nullable; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import org.apache.polaris.immutables.PolarisImmutable; +import org.apache.polaris.persistence.relational.jdbc.DatabaseType; + +/** Model class for scan_metrics_report table - stores scan metrics as first-class entities. */ +@PolarisImmutable +public interface ModelScanMetricsReport extends Converter { + String TABLE_NAME = "SCAN_METRICS_REPORT"; + + // Column names + String REPORT_ID = "report_id"; + String REALM_ID = "realm_id"; + String CATALOG_ID = "catalog_id"; + String CATALOG_NAME = "catalog_name"; + String NAMESPACE = "namespace"; + String TABLE_NAME_COL = "table_name"; + String TIMESTAMP_MS = "timestamp_ms"; + String PRINCIPAL_NAME = "principal_name"; + String REQUEST_ID = "request_id"; + String OTEL_TRACE_ID = "otel_trace_id"; + String OTEL_SPAN_ID = "otel_span_id"; + String REPORT_TRACE_ID = "report_trace_id"; + String SNAPSHOT_ID = "snapshot_id"; + String SCHEMA_ID = "schema_id"; + String FILTER_EXPRESSION = "filter_expression"; + String PROJECTED_FIELD_IDS = "projected_field_ids"; + String PROJECTED_FIELD_NAMES = "projected_field_names"; + String RESULT_DATA_FILES = "result_data_files"; + String RESULT_DELETE_FILES = "result_delete_files"; + String TOTAL_FILE_SIZE_BYTES = "total_file_size_bytes"; + String TOTAL_DATA_MANIFESTS = "total_data_manifests"; + String TOTAL_DELETE_MANIFESTS = "total_delete_manifests"; + String SCANNED_DATA_MANIFESTS = "scanned_data_manifests"; + String SCANNED_DELETE_MANIFESTS = "scanned_delete_manifests"; + String SKIPPED_DATA_MANIFESTS = "skipped_data_manifests"; + String SKIPPED_DELETE_MANIFESTS = "skipped_delete_manifests"; + String SKIPPED_DATA_FILES = "skipped_data_files"; + String SKIPPED_DELETE_FILES = "skipped_delete_files"; + String TOTAL_PLANNING_DURATION_MS = "total_planning_duration_ms"; + String EQUALITY_DELETE_FILES = "equality_delete_files"; + String POSITIONAL_DELETE_FILES = "positional_delete_files"; + String INDEXED_DELETE_FILES = "indexed_delete_files"; + String TOTAL_DELETE_FILE_SIZE_BYTES = "total_delete_file_size_bytes"; + String METADATA = "metadata"; + + List ALL_COLUMNS = + List.of( + REPORT_ID, + REALM_ID, + CATALOG_ID, + CATALOG_NAME, + NAMESPACE, + TABLE_NAME_COL, + TIMESTAMP_MS, + PRINCIPAL_NAME, + REQUEST_ID, + OTEL_TRACE_ID, + OTEL_SPAN_ID, + REPORT_TRACE_ID, + SNAPSHOT_ID, + SCHEMA_ID, + FILTER_EXPRESSION, + PROJECTED_FIELD_IDS, + PROJECTED_FIELD_NAMES, + RESULT_DATA_FILES, + RESULT_DELETE_FILES, + TOTAL_FILE_SIZE_BYTES, + TOTAL_DATA_MANIFESTS, + TOTAL_DELETE_MANIFESTS, + SCANNED_DATA_MANIFESTS, + SCANNED_DELETE_MANIFESTS, + SKIPPED_DATA_MANIFESTS, + SKIPPED_DELETE_MANIFESTS, + SKIPPED_DATA_FILES, + SKIPPED_DELETE_FILES, + TOTAL_PLANNING_DURATION_MS, + EQUALITY_DELETE_FILES, + POSITIONAL_DELETE_FILES, + INDEXED_DELETE_FILES, + TOTAL_DELETE_FILE_SIZE_BYTES, + METADATA); + + // Getters + String getReportId(); + + String getRealmId(); + + String getCatalogId(); + + String getCatalogName(); + + String getNamespace(); + + String getTableName(); + + long getTimestampMs(); + + @Nullable + String getPrincipalName(); + + @Nullable + String getRequestId(); + + @Nullable + String getOtelTraceId(); + + @Nullable + String getOtelSpanId(); + + @Nullable + String getReportTraceId(); + + @Nullable + Long getSnapshotId(); + + @Nullable + Integer getSchemaId(); + + @Nullable + String getFilterExpression(); + + @Nullable + String getProjectedFieldIds(); + + @Nullable + String getProjectedFieldNames(); + + long getResultDataFiles(); + + long getResultDeleteFiles(); + + long getTotalFileSizeBytes(); + + long getTotalDataManifests(); + + long getTotalDeleteManifests(); + + long getScannedDataManifests(); + + long getScannedDeleteManifests(); + + long getSkippedDataManifests(); + + long getSkippedDeleteManifests(); + + long getSkippedDataFiles(); + + long getSkippedDeleteFiles(); + + long getTotalPlanningDurationMs(); + + long getEqualityDeleteFiles(); + + long getPositionalDeleteFiles(); + + long getIndexedDeleteFiles(); + + long getTotalDeleteFileSizeBytes(); + + @Nullable + String getMetadata(); + + @Override + default ModelScanMetricsReport fromResultSet(ResultSet rs) throws SQLException { + return ImmutableModelScanMetricsReport.builder() + .reportId(rs.getString(REPORT_ID)) + .realmId(rs.getString(REALM_ID)) + .catalogId(rs.getString(CATALOG_ID)) + .catalogName(rs.getString(CATALOG_NAME)) + .namespace(rs.getString(NAMESPACE)) + .tableName(rs.getString(TABLE_NAME_COL)) + .timestampMs(rs.getLong(TIMESTAMP_MS)) + .principalName(rs.getString(PRINCIPAL_NAME)) + .requestId(rs.getString(REQUEST_ID)) + .otelTraceId(rs.getString(OTEL_TRACE_ID)) + .otelSpanId(rs.getString(OTEL_SPAN_ID)) + .reportTraceId(rs.getString(REPORT_TRACE_ID)) + .snapshotId(rs.getObject(SNAPSHOT_ID, Long.class)) + .schemaId(rs.getObject(SCHEMA_ID, Integer.class)) + .filterExpression(rs.getString(FILTER_EXPRESSION)) + .projectedFieldIds(rs.getString(PROJECTED_FIELD_IDS)) + .projectedFieldNames(rs.getString(PROJECTED_FIELD_NAMES)) + .resultDataFiles(rs.getLong(RESULT_DATA_FILES)) + .resultDeleteFiles(rs.getLong(RESULT_DELETE_FILES)) + .totalFileSizeBytes(rs.getLong(TOTAL_FILE_SIZE_BYTES)) + .totalDataManifests(rs.getLong(TOTAL_DATA_MANIFESTS)) + .totalDeleteManifests(rs.getLong(TOTAL_DELETE_MANIFESTS)) + .scannedDataManifests(rs.getLong(SCANNED_DATA_MANIFESTS)) + .scannedDeleteManifests(rs.getLong(SCANNED_DELETE_MANIFESTS)) + .skippedDataManifests(rs.getLong(SKIPPED_DATA_MANIFESTS)) + .skippedDeleteManifests(rs.getLong(SKIPPED_DELETE_MANIFESTS)) + .skippedDataFiles(rs.getLong(SKIPPED_DATA_FILES)) + .skippedDeleteFiles(rs.getLong(SKIPPED_DELETE_FILES)) + .totalPlanningDurationMs(rs.getLong(TOTAL_PLANNING_DURATION_MS)) + .equalityDeleteFiles(rs.getLong(EQUALITY_DELETE_FILES)) + .positionalDeleteFiles(rs.getLong(POSITIONAL_DELETE_FILES)) + .indexedDeleteFiles(rs.getLong(INDEXED_DELETE_FILES)) + .totalDeleteFileSizeBytes(rs.getLong(TOTAL_DELETE_FILE_SIZE_BYTES)) + .metadata(rs.getString(METADATA)) + .build(); + } + + @Override + default Map toMap(DatabaseType databaseType) { + Map map = new LinkedHashMap<>(); + map.put(REPORT_ID, getReportId()); + map.put(REALM_ID, getRealmId()); + map.put(CATALOG_ID, getCatalogId()); + map.put(CATALOG_NAME, getCatalogName()); + map.put(NAMESPACE, getNamespace()); + map.put(TABLE_NAME_COL, getTableName()); + map.put(TIMESTAMP_MS, getTimestampMs()); + map.put(PRINCIPAL_NAME, getPrincipalName()); + map.put(REQUEST_ID, getRequestId()); + map.put(OTEL_TRACE_ID, getOtelTraceId()); + map.put(OTEL_SPAN_ID, getOtelSpanId()); + map.put(REPORT_TRACE_ID, getReportTraceId()); + map.put(SNAPSHOT_ID, getSnapshotId()); + map.put(SCHEMA_ID, getSchemaId()); + map.put(FILTER_EXPRESSION, getFilterExpression()); + map.put(PROJECTED_FIELD_IDS, getProjectedFieldIds()); + map.put(PROJECTED_FIELD_NAMES, getProjectedFieldNames()); + map.put(RESULT_DATA_FILES, getResultDataFiles()); + map.put(RESULT_DELETE_FILES, getResultDeleteFiles()); + map.put(TOTAL_FILE_SIZE_BYTES, getTotalFileSizeBytes()); + map.put(TOTAL_DATA_MANIFESTS, getTotalDataManifests()); + map.put(TOTAL_DELETE_MANIFESTS, getTotalDeleteManifests()); + map.put(SCANNED_DATA_MANIFESTS, getScannedDataManifests()); + map.put(SCANNED_DELETE_MANIFESTS, getScannedDeleteManifests()); + map.put(SKIPPED_DATA_MANIFESTS, getSkippedDataManifests()); + map.put(SKIPPED_DELETE_MANIFESTS, getSkippedDeleteManifests()); + map.put(SKIPPED_DATA_FILES, getSkippedDataFiles()); + map.put(SKIPPED_DELETE_FILES, getSkippedDeleteFiles()); + map.put(TOTAL_PLANNING_DURATION_MS, getTotalPlanningDurationMs()); + map.put(EQUALITY_DELETE_FILES, getEqualityDeleteFiles()); + map.put(POSITIONAL_DELETE_FILES, getPositionalDeleteFiles()); + map.put(INDEXED_DELETE_FILES, getIndexedDeleteFiles()); + map.put(TOTAL_DELETE_FILE_SIZE_BYTES, getTotalDeleteFileSizeBytes()); + if (databaseType.equals(DatabaseType.POSTGRES)) { + map.put(METADATA, toJsonbPGobject(getMetadata() != null ? getMetadata() : "{}")); + } else { + map.put(METADATA, getMetadata() != null ? getMetadata() : "{}"); + } + return map; + } + + /** Dummy instance to be used as a Converter when calling fromResultSet(). */ + ModelScanMetricsReport CONVERTER = + ImmutableModelScanMetricsReport.builder() + .reportId("") + .realmId("") + .catalogId("") + .catalogName("") + .namespace("") + .tableName("") + .timestampMs(0L) + .resultDataFiles(0L) + .resultDeleteFiles(0L) + .totalFileSizeBytes(0L) + .totalDataManifests(0L) + .totalDeleteManifests(0L) + .scannedDataManifests(0L) + .scannedDeleteManifests(0L) + .skippedDataManifests(0L) + .skippedDeleteManifests(0L) + .skippedDataFiles(0L) + .skippedDeleteFiles(0L) + .totalPlanningDurationMs(0L) + .equalityDeleteFiles(0L) + .positionalDeleteFiles(0L) + .indexedDeleteFiles(0L) + .totalDeleteFileSizeBytes(0L) + .build(); +} diff --git a/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/models/ModelScanMetricsReportConverter.java b/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/models/ModelScanMetricsReportConverter.java new file mode 100644 index 0000000000..e754b3c85d --- /dev/null +++ b/persistence/relational-jdbc/src/main/java/org/apache/polaris/persistence/relational/jdbc/models/ModelScanMetricsReportConverter.java @@ -0,0 +1,76 @@ +/* + * 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.polaris.persistence.relational.jdbc.models; + +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.Map; +import org.apache.polaris.persistence.relational.jdbc.DatabaseType; + +/** + * Converter for reading ModelScanMetricsReport from database result sets. This class is needed + * because the Immutables-generated class cannot be instantiated without required fields. + */ +public class ModelScanMetricsReportConverter implements Converter { + + @Override + public ModelScanMetricsReport fromResultSet(ResultSet rs) throws SQLException { + return ImmutableModelScanMetricsReport.builder() + .reportId(rs.getString(ModelScanMetricsReport.REPORT_ID)) + .realmId(rs.getString(ModelScanMetricsReport.REALM_ID)) + .catalogId(rs.getString(ModelScanMetricsReport.CATALOG_ID)) + .catalogName(rs.getString(ModelScanMetricsReport.CATALOG_NAME)) + .namespace(rs.getString(ModelScanMetricsReport.NAMESPACE)) + .tableName(rs.getString(ModelScanMetricsReport.TABLE_NAME_COL)) + .timestampMs(rs.getLong(ModelScanMetricsReport.TIMESTAMP_MS)) + .principalName(rs.getString(ModelScanMetricsReport.PRINCIPAL_NAME)) + .requestId(rs.getString(ModelScanMetricsReport.REQUEST_ID)) + .otelTraceId(rs.getString(ModelScanMetricsReport.OTEL_TRACE_ID)) + .otelSpanId(rs.getString(ModelScanMetricsReport.OTEL_SPAN_ID)) + .reportTraceId(rs.getString(ModelScanMetricsReport.REPORT_TRACE_ID)) + .snapshotId(rs.getObject(ModelScanMetricsReport.SNAPSHOT_ID, Long.class)) + .schemaId(rs.getObject(ModelScanMetricsReport.SCHEMA_ID, Integer.class)) + .filterExpression(rs.getString(ModelScanMetricsReport.FILTER_EXPRESSION)) + .projectedFieldIds(rs.getString(ModelScanMetricsReport.PROJECTED_FIELD_IDS)) + .projectedFieldNames(rs.getString(ModelScanMetricsReport.PROJECTED_FIELD_NAMES)) + .resultDataFiles(rs.getLong(ModelScanMetricsReport.RESULT_DATA_FILES)) + .resultDeleteFiles(rs.getLong(ModelScanMetricsReport.RESULT_DELETE_FILES)) + .totalFileSizeBytes(rs.getLong(ModelScanMetricsReport.TOTAL_FILE_SIZE_BYTES)) + .totalDataManifests(rs.getLong(ModelScanMetricsReport.TOTAL_DATA_MANIFESTS)) + .totalDeleteManifests(rs.getLong(ModelScanMetricsReport.TOTAL_DELETE_MANIFESTS)) + .scannedDataManifests(rs.getLong(ModelScanMetricsReport.SCANNED_DATA_MANIFESTS)) + .scannedDeleteManifests(rs.getLong(ModelScanMetricsReport.SCANNED_DELETE_MANIFESTS)) + .skippedDataManifests(rs.getLong(ModelScanMetricsReport.SKIPPED_DATA_MANIFESTS)) + .skippedDeleteManifests(rs.getLong(ModelScanMetricsReport.SKIPPED_DELETE_MANIFESTS)) + .skippedDataFiles(rs.getLong(ModelScanMetricsReport.SKIPPED_DATA_FILES)) + .skippedDeleteFiles(rs.getLong(ModelScanMetricsReport.SKIPPED_DELETE_FILES)) + .totalPlanningDurationMs(rs.getLong(ModelScanMetricsReport.TOTAL_PLANNING_DURATION_MS)) + .equalityDeleteFiles(rs.getLong(ModelScanMetricsReport.EQUALITY_DELETE_FILES)) + .positionalDeleteFiles(rs.getLong(ModelScanMetricsReport.POSITIONAL_DELETE_FILES)) + .indexedDeleteFiles(rs.getLong(ModelScanMetricsReport.INDEXED_DELETE_FILES)) + .totalDeleteFileSizeBytes(rs.getLong(ModelScanMetricsReport.TOTAL_DELETE_FILE_SIZE_BYTES)) + .metadata(rs.getString(ModelScanMetricsReport.METADATA)) + .build(); + } + + @Override + public Map toMap(DatabaseType databaseType) { + throw new UnsupportedOperationException("Converter is read-only"); + } +} diff --git a/persistence/relational-jdbc/src/main/resources/h2/schema-v4.sql b/persistence/relational-jdbc/src/main/resources/h2/schema-v4.sql new file mode 100644 index 0000000000..754f8f1d8d --- /dev/null +++ b/persistence/relational-jdbc/src/main/resources/h2/schema-v4.sql @@ -0,0 +1,166 @@ +-- +-- 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. +-- + +-- Changes from v3: +-- * Added `scan_metrics_report` table for scan metrics as first-class entities +-- * Added `commit_metrics_report` table for commit metrics as first-class entities + +CREATE SCHEMA IF NOT EXISTS POLARIS_SCHEMA; +SET SCHEMA POLARIS_SCHEMA; + +CREATE TABLE IF NOT EXISTS version ( + version_key VARCHAR PRIMARY KEY, + version_value INTEGER NOT NULL +); + +MERGE INTO version (version_key, version_value) + KEY (version_key) + VALUES ('version', 4); + +COMMENT ON TABLE version IS 'the version of the JDBC schema in use'; + +-- Scan Metrics Report Entity Table +CREATE TABLE IF NOT EXISTS scan_metrics_report ( + report_id TEXT NOT NULL, + realm_id TEXT NOT NULL, + catalog_id TEXT NOT NULL, + catalog_name TEXT NOT NULL, + namespace TEXT NOT NULL, + table_name TEXT NOT NULL, + + -- Report metadata + timestamp_ms BIGINT NOT NULL, + principal_name TEXT, + request_id TEXT, + + -- Trace correlation + otel_trace_id TEXT, + otel_span_id TEXT, + report_trace_id TEXT, + + -- Scan context + snapshot_id BIGINT, + schema_id INTEGER, + filter_expression TEXT, + projected_field_ids TEXT, + projected_field_names TEXT, + + -- Scan metrics + result_data_files BIGINT DEFAULT 0, + result_delete_files BIGINT DEFAULT 0, + total_file_size_bytes BIGINT DEFAULT 0, + total_data_manifests BIGINT DEFAULT 0, + total_delete_manifests BIGINT DEFAULT 0, + scanned_data_manifests BIGINT DEFAULT 0, + scanned_delete_manifests BIGINT DEFAULT 0, + skipped_data_manifests BIGINT DEFAULT 0, + skipped_delete_manifests BIGINT DEFAULT 0, + skipped_data_files BIGINT DEFAULT 0, + skipped_delete_files BIGINT DEFAULT 0, + total_planning_duration_ms BIGINT DEFAULT 0, + + -- Equality/positional delete metrics + equality_delete_files BIGINT DEFAULT 0, + positional_delete_files BIGINT DEFAULT 0, + indexed_delete_files BIGINT DEFAULT 0, + total_delete_file_size_bytes BIGINT DEFAULT 0, + + -- Additional metadata (for extensibility) + metadata TEXT DEFAULT '{}', + + PRIMARY KEY (report_id) +); + +COMMENT ON TABLE scan_metrics_report IS 'Scan metrics reports as first-class entities'; + +-- Indexes for scan_metrics_report +CREATE INDEX IF NOT EXISTS idx_scan_report_timestamp ON scan_metrics_report(timestamp_ms); +CREATE INDEX IF NOT EXISTS idx_scan_report_table ON scan_metrics_report(catalog_name, namespace, table_name); +CREATE INDEX IF NOT EXISTS idx_scan_report_trace ON scan_metrics_report(otel_trace_id); +CREATE INDEX IF NOT EXISTS idx_scan_report_realm ON scan_metrics_report(realm_id); + +-- Commit Metrics Report Entity Table +CREATE TABLE IF NOT EXISTS commit_metrics_report ( + report_id TEXT NOT NULL, + realm_id TEXT NOT NULL, + catalog_id TEXT NOT NULL, + catalog_name TEXT NOT NULL, + namespace TEXT NOT NULL, + table_name TEXT NOT NULL, + + -- Report metadata + timestamp_ms BIGINT NOT NULL, + principal_name TEXT, + request_id TEXT, + + -- Trace correlation + otel_trace_id TEXT, + otel_span_id TEXT, + report_trace_id TEXT, + + -- Commit context + snapshot_id BIGINT NOT NULL, + sequence_number BIGINT, + operation TEXT NOT NULL, + + -- File metrics + added_data_files BIGINT DEFAULT 0, + removed_data_files BIGINT DEFAULT 0, + total_data_files BIGINT DEFAULT 0, + added_delete_files BIGINT DEFAULT 0, + removed_delete_files BIGINT DEFAULT 0, + total_delete_files BIGINT DEFAULT 0, + + -- Equality delete files + added_equality_delete_files BIGINT DEFAULT 0, + removed_equality_delete_files BIGINT DEFAULT 0, + + -- Positional delete files + added_positional_delete_files BIGINT DEFAULT 0, + removed_positional_delete_files BIGINT DEFAULT 0, + + -- Record metrics + added_records BIGINT DEFAULT 0, + removed_records BIGINT DEFAULT 0, + total_records BIGINT DEFAULT 0, + + -- Size metrics + added_file_size_bytes BIGINT DEFAULT 0, + removed_file_size_bytes BIGINT DEFAULT 0, + total_file_size_bytes BIGINT DEFAULT 0, + + -- Duration and attempts + total_duration_ms BIGINT DEFAULT 0, + attempts INTEGER DEFAULT 1, + + -- Additional metadata (for extensibility) + metadata TEXT DEFAULT '{}', + + PRIMARY KEY (report_id) +); + +COMMENT ON TABLE commit_metrics_report IS 'Commit metrics reports as first-class entities'; + +-- Indexes for commit_metrics_report +CREATE INDEX IF NOT EXISTS idx_commit_report_timestamp ON commit_metrics_report(timestamp_ms); +CREATE INDEX IF NOT EXISTS idx_commit_report_table ON commit_metrics_report(catalog_name, namespace, table_name); +CREATE INDEX IF NOT EXISTS idx_commit_report_trace ON commit_metrics_report(otel_trace_id); +CREATE INDEX IF NOT EXISTS idx_commit_report_operation ON commit_metrics_report(operation); +CREATE INDEX IF NOT EXISTS idx_commit_report_realm ON commit_metrics_report(realm_id); + diff --git a/persistence/relational-jdbc/src/main/resources/postgres/schema-v4.sql b/persistence/relational-jdbc/src/main/resources/postgres/schema-v4.sql new file mode 100644 index 0000000000..d92d5af756 --- /dev/null +++ b/persistence/relational-jdbc/src/main/resources/postgres/schema-v4.sql @@ -0,0 +1,194 @@ +-- +-- 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. +-- + +-- Changes from v3: +-- * Added `scan_metrics_report` table for scan metrics as first-class entities +-- * Added `commit_metrics_report` table for commit metrics as first-class entities + +CREATE SCHEMA IF NOT EXISTS POLARIS_SCHEMA; +SET search_path TO POLARIS_SCHEMA; + +CREATE TABLE IF NOT EXISTS version ( + version_key TEXT PRIMARY KEY, + version_value INTEGER NOT NULL +); +INSERT INTO version (version_key, version_value) +VALUES ('version', 4) +ON CONFLICT (version_key) DO UPDATE +SET version_value = EXCLUDED.version_value; +COMMENT ON TABLE version IS 'the version of the JDBC schema in use'; + +-- Include all tables from v3 +-- (entities, grant_records, principal_authentication_data, policy_mapping_record, events) +-- These are assumed to already exist from v3 migration + +-- Scan Metrics Report Entity Table +CREATE TABLE IF NOT EXISTS scan_metrics_report ( + report_id TEXT NOT NULL, + realm_id TEXT NOT NULL, + catalog_id TEXT NOT NULL, + catalog_name TEXT NOT NULL, + namespace TEXT NOT NULL, + table_name TEXT NOT NULL, + + -- Report metadata + timestamp_ms BIGINT NOT NULL, + principal_name TEXT, + request_id TEXT, + + -- Trace correlation + otel_trace_id TEXT, + otel_span_id TEXT, + report_trace_id TEXT, + + -- Scan context + snapshot_id BIGINT, + schema_id INTEGER, + filter_expression TEXT, + projected_field_ids TEXT, + projected_field_names TEXT, + + -- Scan metrics + result_data_files BIGINT DEFAULT 0, + result_delete_files BIGINT DEFAULT 0, + total_file_size_bytes BIGINT DEFAULT 0, + total_data_manifests BIGINT DEFAULT 0, + total_delete_manifests BIGINT DEFAULT 0, + scanned_data_manifests BIGINT DEFAULT 0, + scanned_delete_manifests BIGINT DEFAULT 0, + skipped_data_manifests BIGINT DEFAULT 0, + skipped_delete_manifests BIGINT DEFAULT 0, + skipped_data_files BIGINT DEFAULT 0, + skipped_delete_files BIGINT DEFAULT 0, + total_planning_duration_ms BIGINT DEFAULT 0, + + -- Equality/positional delete metrics + equality_delete_files BIGINT DEFAULT 0, + positional_delete_files BIGINT DEFAULT 0, + indexed_delete_files BIGINT DEFAULT 0, + total_delete_file_size_bytes BIGINT DEFAULT 0, + + -- Additional metadata (for extensibility) + metadata JSONB DEFAULT '{}'::JSONB, + + PRIMARY KEY (report_id) +); + +COMMENT ON TABLE scan_metrics_report IS 'Scan metrics reports as first-class entities'; +COMMENT ON COLUMN scan_metrics_report.report_id IS 'Unique identifier for the report'; +COMMENT ON COLUMN scan_metrics_report.realm_id IS 'Realm ID for multi-tenancy'; +COMMENT ON COLUMN scan_metrics_report.catalog_id IS 'Catalog ID'; +COMMENT ON COLUMN scan_metrics_report.otel_trace_id IS 'OpenTelemetry trace ID from HTTP headers'; +COMMENT ON COLUMN scan_metrics_report.report_trace_id IS 'Trace ID from report metadata'; + +-- Indexes for scan_metrics_report +CREATE INDEX IF NOT EXISTS idx_scan_report_timestamp + ON scan_metrics_report(timestamp_ms DESC); +CREATE INDEX IF NOT EXISTS idx_scan_report_table + ON scan_metrics_report(catalog_name, namespace, table_name); +CREATE INDEX IF NOT EXISTS idx_scan_report_trace + ON scan_metrics_report(otel_trace_id) WHERE otel_trace_id IS NOT NULL; +CREATE INDEX IF NOT EXISTS idx_scan_report_principal + ON scan_metrics_report(principal_name) WHERE principal_name IS NOT NULL; +CREATE INDEX IF NOT EXISTS idx_scan_report_realm + ON scan_metrics_report(realm_id); + + +-- Commit Metrics Report Entity Table +CREATE TABLE IF NOT EXISTS commit_metrics_report ( + report_id TEXT NOT NULL, + realm_id TEXT NOT NULL, + catalog_id TEXT NOT NULL, + catalog_name TEXT NOT NULL, + namespace TEXT NOT NULL, + table_name TEXT NOT NULL, + + -- Report metadata + timestamp_ms BIGINT NOT NULL, + principal_name TEXT, + request_id TEXT, + + -- Trace correlation + otel_trace_id TEXT, + otel_span_id TEXT, + report_trace_id TEXT, + + -- Commit context + snapshot_id BIGINT NOT NULL, + sequence_number BIGINT, + operation TEXT NOT NULL, + + -- File metrics + added_data_files BIGINT DEFAULT 0, + removed_data_files BIGINT DEFAULT 0, + total_data_files BIGINT DEFAULT 0, + added_delete_files BIGINT DEFAULT 0, + removed_delete_files BIGINT DEFAULT 0, + total_delete_files BIGINT DEFAULT 0, + + -- Equality delete files + added_equality_delete_files BIGINT DEFAULT 0, + removed_equality_delete_files BIGINT DEFAULT 0, + + -- Positional delete files + added_positional_delete_files BIGINT DEFAULT 0, + removed_positional_delete_files BIGINT DEFAULT 0, + + -- Record metrics + added_records BIGINT DEFAULT 0, + removed_records BIGINT DEFAULT 0, + total_records BIGINT DEFAULT 0, + + -- Size metrics + added_file_size_bytes BIGINT DEFAULT 0, + removed_file_size_bytes BIGINT DEFAULT 0, + total_file_size_bytes BIGINT DEFAULT 0, + + -- Duration and attempts + total_duration_ms BIGINT DEFAULT 0, + attempts INTEGER DEFAULT 1, + + -- Additional metadata (for extensibility) + metadata JSONB DEFAULT '{}'::JSONB, + + PRIMARY KEY (report_id) +); + +COMMENT ON TABLE commit_metrics_report IS 'Commit metrics reports as first-class entities'; +COMMENT ON COLUMN commit_metrics_report.report_id IS 'Unique identifier for the report'; +COMMENT ON COLUMN commit_metrics_report.realm_id IS 'Realm ID for multi-tenancy'; +COMMENT ON COLUMN commit_metrics_report.operation IS 'Commit operation type: append, overwrite, delete, replace'; +COMMENT ON COLUMN commit_metrics_report.otel_trace_id IS 'OpenTelemetry trace ID from HTTP headers'; + +-- Indexes for commit_metrics_report +CREATE INDEX IF NOT EXISTS idx_commit_report_timestamp + ON commit_metrics_report(timestamp_ms DESC); +CREATE INDEX IF NOT EXISTS idx_commit_report_table + ON commit_metrics_report(catalog_name, namespace, table_name); +CREATE INDEX IF NOT EXISTS idx_commit_report_trace + ON commit_metrics_report(otel_trace_id) WHERE otel_trace_id IS NOT NULL; +CREATE INDEX IF NOT EXISTS idx_commit_report_principal + ON commit_metrics_report(principal_name) WHERE principal_name IS NOT NULL; +CREATE INDEX IF NOT EXISTS idx_commit_report_operation + ON commit_metrics_report(operation); +CREATE INDEX IF NOT EXISTS idx_commit_report_realm + ON commit_metrics_report(realm_id); +CREATE INDEX IF NOT EXISTS idx_commit_report_snapshot + ON commit_metrics_report(snapshot_id); + diff --git a/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/MetricsReportPersistenceTest.java b/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/MetricsReportPersistenceTest.java new file mode 100644 index 0000000000..252a19b920 --- /dev/null +++ b/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/MetricsReportPersistenceTest.java @@ -0,0 +1,508 @@ +/* + * 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.polaris.persistence.relational.jdbc; + +import static org.apache.polaris.core.persistence.PrincipalSecretsGenerator.RANDOM_SECRETS; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.InputStream; +import java.sql.SQLException; +import java.util.Optional; +import java.util.UUID; +import javax.sql.DataSource; +import org.apache.polaris.core.PolarisDefaultDiagServiceImpl; +import org.apache.polaris.core.PolarisDiagnostics; +import org.apache.polaris.core.context.RealmContext; +import org.apache.polaris.persistence.relational.jdbc.models.ImmutableModelCommitMetricsReport; +import org.apache.polaris.persistence.relational.jdbc.models.ImmutableModelScanMetricsReport; +import org.apache.polaris.persistence.relational.jdbc.models.ModelCommitMetricsReport; +import org.apache.polaris.persistence.relational.jdbc.models.ModelScanMetricsReport; +import org.h2.jdbcx.JdbcConnectionPool; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mockito; + +/** + * Integration tests for metrics report persistence using JdbcBasePersistenceImpl. Tests the full + * flow of writing scan and commit metrics reports to the database. + */ +class MetricsReportPersistenceTest { + + private JdbcBasePersistenceImpl persistence; + private DatasourceOperations datasourceOperations; + + @BeforeEach + void setUp() throws SQLException { + DataSource dataSource = + JdbcConnectionPool.create( + "jdbc:h2:mem:test_metrics_" + UUID.randomUUID() + ";DB_CLOSE_DELAY=-1", "sa", ""); + + datasourceOperations = new DatasourceOperations(dataSource, new TestJdbcConfiguration()); + + // Execute schema v4 which includes metrics tables + ClassLoader classLoader = DatasourceOperations.class.getClassLoader(); + InputStream scriptStream = classLoader.getResourceAsStream("h2/schema-v4.sql"); + datasourceOperations.executeScript(scriptStream); + + PolarisDiagnostics diagServices = new PolarisDefaultDiagServiceImpl(); + RealmContext realmContext = () -> "TEST_REALM"; + + persistence = + new JdbcBasePersistenceImpl( + diagServices, + datasourceOperations, + RANDOM_SECRETS, + Mockito.mock(), + realmContext.getRealmIdentifier(), + 4); + } + + @Test + void testWriteScanMetricsReport() { + ModelScanMetricsReport report = + ImmutableModelScanMetricsReport.builder() + .reportId(UUID.randomUUID().toString()) + .realmId("TEST_REALM") + .catalogId("test-catalog") + .catalogName("test-catalog") + .namespace("db.schema") + .tableName("test_table") + .timestampMs(System.currentTimeMillis()) + .snapshotId(12345L) + .schemaId(1) + .filterExpression("id > 100") + .resultDataFiles(10L) + .resultDeleteFiles(2L) + .totalFileSizeBytes(1024000L) + .totalDataManifests(5L) + .totalDeleteManifests(1L) + .scannedDataManifests(3L) + .scannedDeleteManifests(1L) + .skippedDataManifests(2L) + .skippedDeleteManifests(0L) + .skippedDataFiles(5L) + .skippedDeleteFiles(0L) + .totalPlanningDurationMs(150L) + .equalityDeleteFiles(1L) + .positionalDeleteFiles(1L) + .indexedDeleteFiles(0L) + .totalDeleteFileSizeBytes(10240L) + .principalName("test-user") + .requestId("req-123") + .otelTraceId("trace-abc") + .otelSpanId("span-xyz") + .reportTraceId("report-trace-1") + .build(); + + // Should not throw + persistence.writeScanMetricsReport(report); + } + + @Test + void testWriteCommitMetricsReport() { + ModelCommitMetricsReport report = + ImmutableModelCommitMetricsReport.builder() + .reportId(UUID.randomUUID().toString()) + .realmId("TEST_REALM") + .catalogId("test-catalog") + .catalogName("test-catalog") + .namespace("db.schema") + .tableName("test_table") + .timestampMs(System.currentTimeMillis()) + .snapshotId(12345L) + .sequenceNumber(1L) + .operation("append") + .addedDataFiles(5L) + .removedDataFiles(0L) + .totalDataFiles(100L) + .addedDeleteFiles(0L) + .removedDeleteFiles(0L) + .totalDeleteFiles(2L) + .addedEqualityDeleteFiles(0L) + .removedEqualityDeleteFiles(0L) + .addedPositionalDeleteFiles(0L) + .removedPositionalDeleteFiles(0L) + .addedRecords(1000L) + .removedRecords(0L) + .totalRecords(50000L) + .addedFileSizeBytes(102400L) + .removedFileSizeBytes(0L) + .totalFileSizeBytes(5120000L) + .totalDurationMs(250L) + .attempts(1) + .principalName("test-user") + .requestId("req-456") + .otelTraceId("trace-def") + .otelSpanId("span-uvw") + .reportTraceId("report-trace-2") + .build(); + + // Should not throw + persistence.writeCommitMetricsReport(report); + } + + @Test + void testWriteMultipleScanReports() { + for (int i = 0; i < 10; i++) { + ModelScanMetricsReport report = + ImmutableModelScanMetricsReport.builder() + .reportId(UUID.randomUUID().toString()) + .realmId("TEST_REALM") + .catalogId("test-catalog") + .catalogName("test-catalog") + .namespace("db.schema") + .tableName("table_" + i) + .timestampMs(System.currentTimeMillis()) + .resultDataFiles((long) (i * 10)) + .resultDeleteFiles(0L) + .totalFileSizeBytes((long) (i * 1024)) + .totalDataManifests(1L) + .totalDeleteManifests(0L) + .scannedDataManifests(1L) + .scannedDeleteManifests(0L) + .skippedDataManifests(0L) + .skippedDeleteManifests(0L) + .skippedDataFiles(0L) + .skippedDeleteFiles(0L) + .totalPlanningDurationMs((long) (i * 10)) + .equalityDeleteFiles(0L) + .positionalDeleteFiles(0L) + .indexedDeleteFiles(0L) + .totalDeleteFileSizeBytes(0L) + .build(); + + persistence.writeScanMetricsReport(report); + } + } + + @Test + void testWriteReportWithNullOptionalFields() { + ModelScanMetricsReport report = + ImmutableModelScanMetricsReport.builder() + .reportId(UUID.randomUUID().toString()) + .realmId("TEST_REALM") + .catalogId("test-catalog") + .catalogName("test-catalog") + .namespace("db") + .tableName("minimal_table") + .timestampMs(System.currentTimeMillis()) + // All optional fields left as null + .resultDataFiles(1L) + .resultDeleteFiles(0L) + .totalFileSizeBytes(100L) + .totalDataManifests(1L) + .totalDeleteManifests(0L) + .scannedDataManifests(1L) + .scannedDeleteManifests(0L) + .skippedDataManifests(0L) + .skippedDeleteManifests(0L) + .skippedDataFiles(0L) + .skippedDeleteFiles(0L) + .totalPlanningDurationMs(10L) + .equalityDeleteFiles(0L) + .positionalDeleteFiles(0L) + .indexedDeleteFiles(0L) + .totalDeleteFileSizeBytes(0L) + .build(); + + // Should not throw even with null optional fields + persistence.writeScanMetricsReport(report); + } + + @Test + void testQueryScanMetricsReportsByTable() { + long baseTime = System.currentTimeMillis(); + + // Write multiple reports for the same table + for (int i = 0; i < 5; i++) { + ModelScanMetricsReport report = + ImmutableModelScanMetricsReport.builder() + .reportId(UUID.randomUUID().toString()) + .realmId("TEST_REALM") + .catalogId("test-catalog") + .catalogName("test-catalog") + .namespace("db.schema") + .tableName("query_test_table") + .timestampMs(baseTime + i * 1000) + .resultDataFiles((long) i) + .resultDeleteFiles(0L) + .totalFileSizeBytes(100L) + .totalDataManifests(1L) + .totalDeleteManifests(0L) + .scannedDataManifests(1L) + .scannedDeleteManifests(0L) + .skippedDataManifests(0L) + .skippedDeleteManifests(0L) + .skippedDataFiles(0L) + .skippedDeleteFiles(0L) + .totalPlanningDurationMs(10L) + .equalityDeleteFiles(0L) + .positionalDeleteFiles(0L) + .indexedDeleteFiles(0L) + .totalDeleteFileSizeBytes(0L) + .build(); + persistence.writeScanMetricsReport(report); + } + + // Query all reports for the table + var results = + persistence.queryScanMetricsReports( + "test-catalog", "db.schema", "query_test_table", null, null, 10); + assertThat(results).hasSize(5); + + // Query with time range + var rangeResults = + persistence.queryScanMetricsReports( + "test-catalog", "db.schema", "query_test_table", baseTime + 1000, baseTime + 4000, 10); + assertThat(rangeResults).hasSize(3); + + // Query with limit + var limitedResults = + persistence.queryScanMetricsReports( + "test-catalog", "db.schema", "query_test_table", null, null, 2); + assertThat(limitedResults).hasSize(2); + } + + @Test + void testQueryScanMetricsReportsByTraceId() { + String traceId = "test-trace-" + UUID.randomUUID(); + + // Write a report with trace ID + ModelScanMetricsReport report = + ImmutableModelScanMetricsReport.builder() + .reportId(UUID.randomUUID().toString()) + .realmId("TEST_REALM") + .catalogId("test-catalog") + .catalogName("test-catalog") + .namespace("db") + .tableName("trace_test_table") + .timestampMs(System.currentTimeMillis()) + .otelTraceId(traceId) + .resultDataFiles(1L) + .resultDeleteFiles(0L) + .totalFileSizeBytes(100L) + .totalDataManifests(1L) + .totalDeleteManifests(0L) + .scannedDataManifests(1L) + .scannedDeleteManifests(0L) + .skippedDataManifests(0L) + .skippedDeleteManifests(0L) + .skippedDataFiles(0L) + .skippedDeleteFiles(0L) + .totalPlanningDurationMs(10L) + .equalityDeleteFiles(0L) + .positionalDeleteFiles(0L) + .indexedDeleteFiles(0L) + .totalDeleteFileSizeBytes(0L) + .build(); + persistence.writeScanMetricsReport(report); + + // Query by trace ID + var results = persistence.queryScanMetricsReportsByTraceId(traceId); + assertThat(results).hasSize(1); + assertThat(results.get(0).getOtelTraceId()).isEqualTo(traceId); + } + + @Test + void testDeleteOldScanMetricsReports() { + // Create reports with different timestamps + long now = System.currentTimeMillis(); + long oneHourAgo = now - 3600_000; + long twoDaysAgo = now - 2 * 24 * 3600_000; + + // Create an old report (2 days ago) + ModelScanMetricsReport oldReport = + ImmutableModelScanMetricsReport.builder() + .reportId("old-report-" + UUID.randomUUID()) + .realmId("TEST_REALM") + .catalogId("catalog1") + .catalogName("test_catalog") + .namespace("test_namespace") + .tableName("test_table") + .timestampMs(twoDaysAgo) + .resultDataFiles(10L) + .resultDeleteFiles(0L) + .totalFileSizeBytes(1000L) + .totalDataManifests(1L) + .totalDeleteManifests(0L) + .scannedDataManifests(1L) + .scannedDeleteManifests(0L) + .skippedDataManifests(0L) + .skippedDeleteManifests(0L) + .skippedDataFiles(0L) + .skippedDeleteFiles(0L) + .totalPlanningDurationMs(10L) + .equalityDeleteFiles(0L) + .positionalDeleteFiles(0L) + .indexedDeleteFiles(0L) + .totalDeleteFileSizeBytes(0L) + .build(); + persistence.writeScanMetricsReport(oldReport); + + // Create a recent report (1 hour ago) + ModelScanMetricsReport recentReport = + ImmutableModelScanMetricsReport.builder() + .reportId("recent-report-" + UUID.randomUUID()) + .realmId("TEST_REALM") + .catalogId("catalog1") + .catalogName("test_catalog") + .namespace("test_namespace") + .tableName("test_table") + .timestampMs(oneHourAgo) + .resultDataFiles(10L) + .resultDeleteFiles(0L) + .totalFileSizeBytes(1000L) + .totalDataManifests(1L) + .totalDeleteManifests(0L) + .scannedDataManifests(1L) + .scannedDeleteManifests(0L) + .skippedDataManifests(0L) + .skippedDeleteManifests(0L) + .skippedDataFiles(0L) + .skippedDeleteFiles(0L) + .totalPlanningDurationMs(10L) + .equalityDeleteFiles(0L) + .positionalDeleteFiles(0L) + .indexedDeleteFiles(0L) + .totalDeleteFileSizeBytes(0L) + .build(); + persistence.writeScanMetricsReport(recentReport); + + // Delete reports older than 1 day + long oneDayAgo = now - 24 * 3600_000; + int deleted = persistence.deleteScanMetricsReportsOlderThan(oneDayAgo); + + // Should have deleted the old report + assertThat(deleted).isEqualTo(1); + + // Query to verify only recent report remains + var results = + persistence.queryScanMetricsReports( + "test_catalog", "test_namespace", "test_table", null, null, 10); + assertThat(results).hasSize(1); + assertThat(results.get(0).getReportId()).isEqualTo(recentReport.getReportId()); + } + + @Test + void testDeleteOldCommitMetricsReports() { + // Create reports with different timestamps + long now = System.currentTimeMillis(); + long oneHourAgo = now - 3600_000; + long twoDaysAgo = now - 2 * 24 * 3600_000; + + // Create an old report (2 days ago) + ModelCommitMetricsReport oldReport = + ImmutableModelCommitMetricsReport.builder() + .reportId("old-commit-" + UUID.randomUUID()) + .realmId("TEST_REALM") + .catalogId("catalog1") + .catalogName("test_catalog") + .namespace("test_namespace") + .tableName("test_table") + .timestampMs(twoDaysAgo) + .snapshotId(100L) + .sequenceNumber(1L) + .operation("append") + .addedDataFiles(5L) + .removedDataFiles(0L) + .totalDataFiles(5L) + .addedDeleteFiles(0L) + .removedDeleteFiles(0L) + .totalDeleteFiles(0L) + .addedEqualityDeleteFiles(0L) + .removedEqualityDeleteFiles(0L) + .addedPositionalDeleteFiles(0L) + .removedPositionalDeleteFiles(0L) + .addedRecords(1000L) + .removedRecords(0L) + .totalRecords(1000L) + .addedFileSizeBytes(10000L) + .removedFileSizeBytes(0L) + .totalFileSizeBytes(10000L) + .totalDurationMs(50L) + .attempts(1) + .build(); + persistence.writeCommitMetricsReport(oldReport); + + // Create a recent report (1 hour ago) + ModelCommitMetricsReport recentReport = + ImmutableModelCommitMetricsReport.builder() + .reportId("recent-commit-" + UUID.randomUUID()) + .realmId("TEST_REALM") + .catalogId("catalog1") + .catalogName("test_catalog") + .namespace("test_namespace") + .tableName("test_table") + .timestampMs(oneHourAgo) + .snapshotId(101L) + .sequenceNumber(2L) + .operation("append") + .addedDataFiles(3L) + .removedDataFiles(0L) + .totalDataFiles(8L) + .addedDeleteFiles(0L) + .removedDeleteFiles(0L) + .totalDeleteFiles(0L) + .addedEqualityDeleteFiles(0L) + .removedEqualityDeleteFiles(0L) + .addedPositionalDeleteFiles(0L) + .removedPositionalDeleteFiles(0L) + .addedRecords(500L) + .removedRecords(0L) + .totalRecords(1500L) + .addedFileSizeBytes(5000L) + .removedFileSizeBytes(0L) + .totalFileSizeBytes(15000L) + .totalDurationMs(30L) + .attempts(1) + .build(); + persistence.writeCommitMetricsReport(recentReport); + + // Delete reports older than 1 day + long oneDayAgo = now - 24 * 3600_000; + int deleted = persistence.deleteCommitMetricsReportsOlderThan(oneDayAgo); + + // Should have deleted the old report + assertThat(deleted).isEqualTo(1); + + // Query to verify only recent report remains + var results = + persistence.queryCommitMetricsReports( + "test_catalog", "test_namespace", "test_table", null, null, 10); + assertThat(results).hasSize(1); + assertThat(results.get(0).getReportId()).isEqualTo(recentReport.getReportId()); + } + + private static class TestJdbcConfiguration implements RelationalJdbcConfiguration { + @Override + public Optional maxRetries() { + return Optional.of(1); + } + + @Override + public Optional maxDurationInMs() { + return Optional.of(100L); + } + + @Override + public Optional initialDelayInMs() { + return Optional.of(10L); + } + } +} diff --git a/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/models/ModelCommitMetricsReportTest.java b/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/models/ModelCommitMetricsReportTest.java new file mode 100644 index 0000000000..8bb295c4d1 --- /dev/null +++ b/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/models/ModelCommitMetricsReportTest.java @@ -0,0 +1,213 @@ +/* + * 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.polaris.persistence.relational.jdbc.models; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.Map; +import org.apache.polaris.persistence.relational.jdbc.DatabaseType; +import org.junit.jupiter.api.Test; +import org.postgresql.util.PGobject; + +public class ModelCommitMetricsReportTest { + + private static final String TEST_REPORT_ID = "commit-report-123"; + private static final String TEST_REALM_ID = "realm-1"; + private static final String TEST_CATALOG_ID = "catalog-1"; + private static final String TEST_CATALOG_NAME = "my_catalog"; + private static final String TEST_NAMESPACE = "db.schema"; + private static final String TEST_TABLE_NAME = "my_table"; + private static final long TEST_TIMESTAMP_MS = 1704067200000L; + private static final String TEST_PRINCIPAL = "user@example.com"; + private static final String TEST_REQUEST_ID = "req-456"; + private static final String TEST_OTEL_TRACE_ID = "trace-789"; + private static final String TEST_OTEL_SPAN_ID = "span-012"; + private static final String TEST_REPORT_TRACE_ID = "report-trace-345"; + private static final long TEST_SNAPSHOT_ID = 987654321L; + private static final Long TEST_SEQUENCE_NUMBER = 5L; + private static final String TEST_OPERATION = "append"; + private static final long TEST_ADDED_DATA_FILES = 10L; + private static final long TEST_REMOVED_DATA_FILES = 2L; + private static final long TEST_TOTAL_DATA_FILES = 50L; + private static final long TEST_ADDED_DELETE_FILES = 1L; + private static final long TEST_REMOVED_DELETE_FILES = 0L; + private static final long TEST_TOTAL_DELETE_FILES = 3L; + private static final long TEST_ADDED_EQUALITY_DELETE_FILES = 1L; + private static final long TEST_REMOVED_EQUALITY_DELETE_FILES = 0L; + private static final long TEST_ADDED_POSITIONAL_DELETE_FILES = 0L; + private static final long TEST_REMOVED_POSITIONAL_DELETE_FILES = 0L; + private static final long TEST_ADDED_RECORDS = 1000L; + private static final long TEST_REMOVED_RECORDS = 50L; + private static final long TEST_TOTAL_RECORDS = 10000L; + private static final long TEST_ADDED_FILE_SIZE = 1024000L; + private static final long TEST_REMOVED_FILE_SIZE = 51200L; + private static final long TEST_TOTAL_FILE_SIZE = 10240000L; + private static final long TEST_TOTAL_DURATION = 250L; + private static final int TEST_ATTEMPTS = 1; + private static final String TEST_METADATA = "{\"commit\":\"info\"}"; + + @Test + public void testFromResultSet() throws SQLException { + ResultSet mockResultSet = mock(ResultSet.class); + when(mockResultSet.getString(ModelCommitMetricsReport.REPORT_ID)).thenReturn(TEST_REPORT_ID); + when(mockResultSet.getString(ModelCommitMetricsReport.REALM_ID)).thenReturn(TEST_REALM_ID); + when(mockResultSet.getString(ModelCommitMetricsReport.CATALOG_ID)).thenReturn(TEST_CATALOG_ID); + when(mockResultSet.getString(ModelCommitMetricsReport.CATALOG_NAME)) + .thenReturn(TEST_CATALOG_NAME); + when(mockResultSet.getString(ModelCommitMetricsReport.NAMESPACE)).thenReturn(TEST_NAMESPACE); + when(mockResultSet.getString(ModelCommitMetricsReport.TABLE_NAME_COL)) + .thenReturn(TEST_TABLE_NAME); + when(mockResultSet.getLong(ModelCommitMetricsReport.TIMESTAMP_MS)) + .thenReturn(TEST_TIMESTAMP_MS); + when(mockResultSet.getString(ModelCommitMetricsReport.PRINCIPAL_NAME)) + .thenReturn(TEST_PRINCIPAL); + when(mockResultSet.getString(ModelCommitMetricsReport.REQUEST_ID)).thenReturn(TEST_REQUEST_ID); + when(mockResultSet.getString(ModelCommitMetricsReport.OTEL_TRACE_ID)) + .thenReturn(TEST_OTEL_TRACE_ID); + when(mockResultSet.getString(ModelCommitMetricsReport.OTEL_SPAN_ID)) + .thenReturn(TEST_OTEL_SPAN_ID); + when(mockResultSet.getString(ModelCommitMetricsReport.REPORT_TRACE_ID)) + .thenReturn(TEST_REPORT_TRACE_ID); + when(mockResultSet.getLong(ModelCommitMetricsReport.SNAPSHOT_ID)).thenReturn(TEST_SNAPSHOT_ID); + when(mockResultSet.getObject(ModelCommitMetricsReport.SEQUENCE_NUMBER, Long.class)) + .thenReturn(TEST_SEQUENCE_NUMBER); + when(mockResultSet.getString(ModelCommitMetricsReport.OPERATION)).thenReturn(TEST_OPERATION); + when(mockResultSet.getLong(ModelCommitMetricsReport.ADDED_DATA_FILES)) + .thenReturn(TEST_ADDED_DATA_FILES); + when(mockResultSet.getLong(ModelCommitMetricsReport.REMOVED_DATA_FILES)) + .thenReturn(TEST_REMOVED_DATA_FILES); + when(mockResultSet.getLong(ModelCommitMetricsReport.TOTAL_DATA_FILES)) + .thenReturn(TEST_TOTAL_DATA_FILES); + when(mockResultSet.getLong(ModelCommitMetricsReport.ADDED_DELETE_FILES)) + .thenReturn(TEST_ADDED_DELETE_FILES); + when(mockResultSet.getLong(ModelCommitMetricsReport.REMOVED_DELETE_FILES)) + .thenReturn(TEST_REMOVED_DELETE_FILES); + when(mockResultSet.getLong(ModelCommitMetricsReport.TOTAL_DELETE_FILES)) + .thenReturn(TEST_TOTAL_DELETE_FILES); + when(mockResultSet.getLong(ModelCommitMetricsReport.ADDED_EQUALITY_DELETE_FILES)) + .thenReturn(TEST_ADDED_EQUALITY_DELETE_FILES); + when(mockResultSet.getLong(ModelCommitMetricsReport.REMOVED_EQUALITY_DELETE_FILES)) + .thenReturn(TEST_REMOVED_EQUALITY_DELETE_FILES); + when(mockResultSet.getLong(ModelCommitMetricsReport.ADDED_POSITIONAL_DELETE_FILES)) + .thenReturn(TEST_ADDED_POSITIONAL_DELETE_FILES); + when(mockResultSet.getLong(ModelCommitMetricsReport.REMOVED_POSITIONAL_DELETE_FILES)) + .thenReturn(TEST_REMOVED_POSITIONAL_DELETE_FILES); + when(mockResultSet.getLong(ModelCommitMetricsReport.ADDED_RECORDS)) + .thenReturn(TEST_ADDED_RECORDS); + when(mockResultSet.getLong(ModelCommitMetricsReport.REMOVED_RECORDS)) + .thenReturn(TEST_REMOVED_RECORDS); + when(mockResultSet.getLong(ModelCommitMetricsReport.TOTAL_RECORDS)) + .thenReturn(TEST_TOTAL_RECORDS); + when(mockResultSet.getLong(ModelCommitMetricsReport.ADDED_FILE_SIZE_BYTES)) + .thenReturn(TEST_ADDED_FILE_SIZE); + when(mockResultSet.getLong(ModelCommitMetricsReport.REMOVED_FILE_SIZE_BYTES)) + .thenReturn(TEST_REMOVED_FILE_SIZE); + when(mockResultSet.getLong(ModelCommitMetricsReport.TOTAL_FILE_SIZE_BYTES)) + .thenReturn(TEST_TOTAL_FILE_SIZE); + when(mockResultSet.getLong(ModelCommitMetricsReport.TOTAL_DURATION_MS)) + .thenReturn(TEST_TOTAL_DURATION); + when(mockResultSet.getInt(ModelCommitMetricsReport.ATTEMPTS)).thenReturn(TEST_ATTEMPTS); + when(mockResultSet.getString(ModelCommitMetricsReport.METADATA)).thenReturn(TEST_METADATA); + + ModelCommitMetricsReport result = + ModelCommitMetricsReport.CONVERTER.fromResultSet(mockResultSet); + + assertEquals(TEST_REPORT_ID, result.getReportId()); + assertEquals(TEST_REALM_ID, result.getRealmId()); + assertEquals(TEST_CATALOG_ID, result.getCatalogId()); + assertEquals(TEST_CATALOG_NAME, result.getCatalogName()); + assertEquals(TEST_NAMESPACE, result.getNamespace()); + assertEquals(TEST_TABLE_NAME, result.getTableName()); + assertEquals(TEST_TIMESTAMP_MS, result.getTimestampMs()); + assertEquals(TEST_SNAPSHOT_ID, result.getSnapshotId()); + assertEquals(TEST_OPERATION, result.getOperation()); + assertEquals(TEST_ADDED_DATA_FILES, result.getAddedDataFiles()); + assertEquals(TEST_ADDED_RECORDS, result.getAddedRecords()); + assertEquals(TEST_TOTAL_DURATION, result.getTotalDurationMs()); + assertEquals(TEST_ATTEMPTS, result.getAttempts()); + assertEquals(TEST_METADATA, result.getMetadata()); + } + + @Test + public void testToMapWithH2DatabaseType() { + ModelCommitMetricsReport report = createTestReport(); + + Map resultMap = report.toMap(DatabaseType.H2); + + assertEquals(TEST_REPORT_ID, resultMap.get(ModelCommitMetricsReport.REPORT_ID)); + assertEquals(TEST_REALM_ID, resultMap.get(ModelCommitMetricsReport.REALM_ID)); + assertEquals(TEST_SNAPSHOT_ID, resultMap.get(ModelCommitMetricsReport.SNAPSHOT_ID)); + assertEquals(TEST_OPERATION, resultMap.get(ModelCommitMetricsReport.OPERATION)); + assertEquals(TEST_ADDED_DATA_FILES, resultMap.get(ModelCommitMetricsReport.ADDED_DATA_FILES)); + assertEquals(TEST_METADATA, resultMap.get(ModelCommitMetricsReport.METADATA)); + } + + @Test + public void testToMapWithPostgresType() { + ModelCommitMetricsReport report = createTestReport(); + + Map resultMap = report.toMap(DatabaseType.POSTGRES); + + assertEquals(TEST_REPORT_ID, resultMap.get(ModelCommitMetricsReport.REPORT_ID)); + PGobject pgObject = (PGobject) resultMap.get(ModelCommitMetricsReport.METADATA); + assertEquals("jsonb", pgObject.getType()); + assertEquals(TEST_METADATA, pgObject.getValue()); + } + + private ModelCommitMetricsReport createTestReport() { + return ImmutableModelCommitMetricsReport.builder() + .reportId(TEST_REPORT_ID) + .realmId(TEST_REALM_ID) + .catalogId(TEST_CATALOG_ID) + .catalogName(TEST_CATALOG_NAME) + .namespace(TEST_NAMESPACE) + .tableName(TEST_TABLE_NAME) + .timestampMs(TEST_TIMESTAMP_MS) + .principalName(TEST_PRINCIPAL) + .requestId(TEST_REQUEST_ID) + .otelTraceId(TEST_OTEL_TRACE_ID) + .snapshotId(TEST_SNAPSHOT_ID) + .sequenceNumber(TEST_SEQUENCE_NUMBER) + .operation(TEST_OPERATION) + .addedDataFiles(TEST_ADDED_DATA_FILES) + .removedDataFiles(TEST_REMOVED_DATA_FILES) + .totalDataFiles(TEST_TOTAL_DATA_FILES) + .addedDeleteFiles(TEST_ADDED_DELETE_FILES) + .removedDeleteFiles(TEST_REMOVED_DELETE_FILES) + .totalDeleteFiles(TEST_TOTAL_DELETE_FILES) + .addedEqualityDeleteFiles(TEST_ADDED_EQUALITY_DELETE_FILES) + .removedEqualityDeleteFiles(TEST_REMOVED_EQUALITY_DELETE_FILES) + .addedPositionalDeleteFiles(TEST_ADDED_POSITIONAL_DELETE_FILES) + .removedPositionalDeleteFiles(TEST_REMOVED_POSITIONAL_DELETE_FILES) + .addedRecords(TEST_ADDED_RECORDS) + .removedRecords(TEST_REMOVED_RECORDS) + .totalRecords(TEST_TOTAL_RECORDS) + .addedFileSizeBytes(TEST_ADDED_FILE_SIZE) + .removedFileSizeBytes(TEST_REMOVED_FILE_SIZE) + .totalFileSizeBytes(TEST_TOTAL_FILE_SIZE) + .totalDurationMs(TEST_TOTAL_DURATION) + .attempts(TEST_ATTEMPTS) + .metadata(TEST_METADATA) + .build(); + } +} diff --git a/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/models/ModelScanMetricsReportTest.java b/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/models/ModelScanMetricsReportTest.java new file mode 100644 index 0000000000..0c8f26a2ed --- /dev/null +++ b/persistence/relational-jdbc/src/test/java/org/apache/polaris/persistence/relational/jdbc/models/ModelScanMetricsReportTest.java @@ -0,0 +1,212 @@ +/* + * 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.polaris.persistence.relational.jdbc.models; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.Map; +import org.apache.polaris.persistence.relational.jdbc.DatabaseType; +import org.junit.jupiter.api.Test; +import org.postgresql.util.PGobject; + +public class ModelScanMetricsReportTest { + + private static final String TEST_REPORT_ID = "report-123"; + private static final String TEST_REALM_ID = "realm-1"; + private static final String TEST_CATALOG_ID = "catalog-1"; + private static final String TEST_CATALOG_NAME = "my_catalog"; + private static final String TEST_NAMESPACE = "db.schema"; + private static final String TEST_TABLE_NAME = "my_table"; + private static final long TEST_TIMESTAMP_MS = 1704067200000L; + private static final String TEST_PRINCIPAL = "user@example.com"; + private static final String TEST_REQUEST_ID = "req-456"; + private static final String TEST_OTEL_TRACE_ID = "trace-789"; + private static final String TEST_OTEL_SPAN_ID = "span-012"; + private static final String TEST_REPORT_TRACE_ID = "report-trace-345"; + private static final Long TEST_SNAPSHOT_ID = 123456789L; + private static final Integer TEST_SCHEMA_ID = 1; + private static final String TEST_FILTER = "id > 100"; + private static final String TEST_PROJECTED_IDS = "1,2,3"; + private static final String TEST_PROJECTED_NAMES = "id,name,value"; + private static final long TEST_RESULT_DATA_FILES = 10L; + private static final long TEST_RESULT_DELETE_FILES = 2L; + private static final long TEST_TOTAL_FILE_SIZE = 1024000L; + private static final long TEST_TOTAL_DATA_MANIFESTS = 5L; + private static final long TEST_TOTAL_DELETE_MANIFESTS = 1L; + private static final long TEST_SCANNED_DATA_MANIFESTS = 3L; + private static final long TEST_SCANNED_DELETE_MANIFESTS = 1L; + private static final long TEST_SKIPPED_DATA_MANIFESTS = 2L; + private static final long TEST_SKIPPED_DELETE_MANIFESTS = 0L; + private static final long TEST_SKIPPED_DATA_FILES = 5L; + private static final long TEST_SKIPPED_DELETE_FILES = 0L; + private static final long TEST_PLANNING_DURATION = 150L; + private static final long TEST_EQUALITY_DELETE_FILES = 1L; + private static final long TEST_POSITIONAL_DELETE_FILES = 1L; + private static final long TEST_INDEXED_DELETE_FILES = 0L; + private static final long TEST_DELETE_FILE_SIZE = 2048L; + private static final String TEST_METADATA = "{\"custom\":\"value\"}"; + + @Test + public void testFromResultSet() throws SQLException { + ResultSet mockResultSet = mock(ResultSet.class); + when(mockResultSet.getString(ModelScanMetricsReport.REPORT_ID)).thenReturn(TEST_REPORT_ID); + when(mockResultSet.getString(ModelScanMetricsReport.REALM_ID)).thenReturn(TEST_REALM_ID); + when(mockResultSet.getString(ModelScanMetricsReport.CATALOG_ID)).thenReturn(TEST_CATALOG_ID); + when(mockResultSet.getString(ModelScanMetricsReport.CATALOG_NAME)) + .thenReturn(TEST_CATALOG_NAME); + when(mockResultSet.getString(ModelScanMetricsReport.NAMESPACE)).thenReturn(TEST_NAMESPACE); + when(mockResultSet.getString(ModelScanMetricsReport.TABLE_NAME_COL)) + .thenReturn(TEST_TABLE_NAME); + when(mockResultSet.getLong(ModelScanMetricsReport.TIMESTAMP_MS)).thenReturn(TEST_TIMESTAMP_MS); + when(mockResultSet.getString(ModelScanMetricsReport.PRINCIPAL_NAME)).thenReturn(TEST_PRINCIPAL); + when(mockResultSet.getString(ModelScanMetricsReport.REQUEST_ID)).thenReturn(TEST_REQUEST_ID); + when(mockResultSet.getString(ModelScanMetricsReport.OTEL_TRACE_ID)) + .thenReturn(TEST_OTEL_TRACE_ID); + when(mockResultSet.getString(ModelScanMetricsReport.OTEL_SPAN_ID)) + .thenReturn(TEST_OTEL_SPAN_ID); + when(mockResultSet.getString(ModelScanMetricsReport.REPORT_TRACE_ID)) + .thenReturn(TEST_REPORT_TRACE_ID); + when(mockResultSet.getObject(ModelScanMetricsReport.SNAPSHOT_ID, Long.class)) + .thenReturn(TEST_SNAPSHOT_ID); + when(mockResultSet.getObject(ModelScanMetricsReport.SCHEMA_ID, Integer.class)) + .thenReturn(TEST_SCHEMA_ID); + when(mockResultSet.getString(ModelScanMetricsReport.FILTER_EXPRESSION)).thenReturn(TEST_FILTER); + when(mockResultSet.getString(ModelScanMetricsReport.PROJECTED_FIELD_IDS)) + .thenReturn(TEST_PROJECTED_IDS); + when(mockResultSet.getString(ModelScanMetricsReport.PROJECTED_FIELD_NAMES)) + .thenReturn(TEST_PROJECTED_NAMES); + when(mockResultSet.getLong(ModelScanMetricsReport.RESULT_DATA_FILES)) + .thenReturn(TEST_RESULT_DATA_FILES); + when(mockResultSet.getLong(ModelScanMetricsReport.RESULT_DELETE_FILES)) + .thenReturn(TEST_RESULT_DELETE_FILES); + when(mockResultSet.getLong(ModelScanMetricsReport.TOTAL_FILE_SIZE_BYTES)) + .thenReturn(TEST_TOTAL_FILE_SIZE); + when(mockResultSet.getLong(ModelScanMetricsReport.TOTAL_DATA_MANIFESTS)) + .thenReturn(TEST_TOTAL_DATA_MANIFESTS); + when(mockResultSet.getLong(ModelScanMetricsReport.TOTAL_DELETE_MANIFESTS)) + .thenReturn(TEST_TOTAL_DELETE_MANIFESTS); + when(mockResultSet.getLong(ModelScanMetricsReport.SCANNED_DATA_MANIFESTS)) + .thenReturn(TEST_SCANNED_DATA_MANIFESTS); + when(mockResultSet.getLong(ModelScanMetricsReport.SCANNED_DELETE_MANIFESTS)) + .thenReturn(TEST_SCANNED_DELETE_MANIFESTS); + when(mockResultSet.getLong(ModelScanMetricsReport.SKIPPED_DATA_MANIFESTS)) + .thenReturn(TEST_SKIPPED_DATA_MANIFESTS); + when(mockResultSet.getLong(ModelScanMetricsReport.SKIPPED_DELETE_MANIFESTS)) + .thenReturn(TEST_SKIPPED_DELETE_MANIFESTS); + when(mockResultSet.getLong(ModelScanMetricsReport.SKIPPED_DATA_FILES)) + .thenReturn(TEST_SKIPPED_DATA_FILES); + when(mockResultSet.getLong(ModelScanMetricsReport.SKIPPED_DELETE_FILES)) + .thenReturn(TEST_SKIPPED_DELETE_FILES); + when(mockResultSet.getLong(ModelScanMetricsReport.TOTAL_PLANNING_DURATION_MS)) + .thenReturn(TEST_PLANNING_DURATION); + when(mockResultSet.getLong(ModelScanMetricsReport.EQUALITY_DELETE_FILES)) + .thenReturn(TEST_EQUALITY_DELETE_FILES); + when(mockResultSet.getLong(ModelScanMetricsReport.POSITIONAL_DELETE_FILES)) + .thenReturn(TEST_POSITIONAL_DELETE_FILES); + when(mockResultSet.getLong(ModelScanMetricsReport.INDEXED_DELETE_FILES)) + .thenReturn(TEST_INDEXED_DELETE_FILES); + when(mockResultSet.getLong(ModelScanMetricsReport.TOTAL_DELETE_FILE_SIZE_BYTES)) + .thenReturn(TEST_DELETE_FILE_SIZE); + when(mockResultSet.getString(ModelScanMetricsReport.METADATA)).thenReturn(TEST_METADATA); + + ModelScanMetricsReport result = ModelScanMetricsReport.CONVERTER.fromResultSet(mockResultSet); + + assertEquals(TEST_REPORT_ID, result.getReportId()); + assertEquals(TEST_REALM_ID, result.getRealmId()); + assertEquals(TEST_CATALOG_ID, result.getCatalogId()); + assertEquals(TEST_CATALOG_NAME, result.getCatalogName()); + assertEquals(TEST_NAMESPACE, result.getNamespace()); + assertEquals(TEST_TABLE_NAME, result.getTableName()); + assertEquals(TEST_TIMESTAMP_MS, result.getTimestampMs()); + assertEquals(TEST_PRINCIPAL, result.getPrincipalName()); + assertEquals(TEST_REQUEST_ID, result.getRequestId()); + assertEquals(TEST_OTEL_TRACE_ID, result.getOtelTraceId()); + assertEquals(TEST_SNAPSHOT_ID, result.getSnapshotId()); + assertEquals(TEST_RESULT_DATA_FILES, result.getResultDataFiles()); + assertEquals(TEST_TOTAL_FILE_SIZE, result.getTotalFileSizeBytes()); + assertEquals(TEST_PLANNING_DURATION, result.getTotalPlanningDurationMs()); + assertEquals(TEST_METADATA, result.getMetadata()); + } + + @Test + public void testToMapWithH2DatabaseType() { + ModelScanMetricsReport report = createTestReport(); + + Map resultMap = report.toMap(DatabaseType.H2); + + assertEquals(TEST_REPORT_ID, resultMap.get(ModelScanMetricsReport.REPORT_ID)); + assertEquals(TEST_REALM_ID, resultMap.get(ModelScanMetricsReport.REALM_ID)); + assertEquals(TEST_CATALOG_ID, resultMap.get(ModelScanMetricsReport.CATALOG_ID)); + assertEquals(TEST_CATALOG_NAME, resultMap.get(ModelScanMetricsReport.CATALOG_NAME)); + assertEquals(TEST_NAMESPACE, resultMap.get(ModelScanMetricsReport.NAMESPACE)); + assertEquals(TEST_TABLE_NAME, resultMap.get(ModelScanMetricsReport.TABLE_NAME_COL)); + assertEquals(TEST_TIMESTAMP_MS, resultMap.get(ModelScanMetricsReport.TIMESTAMP_MS)); + assertEquals(TEST_RESULT_DATA_FILES, resultMap.get(ModelScanMetricsReport.RESULT_DATA_FILES)); + assertEquals(TEST_METADATA, resultMap.get(ModelScanMetricsReport.METADATA)); + } + + @Test + public void testToMapWithPostgresType() { + ModelScanMetricsReport report = createTestReport(); + + Map resultMap = report.toMap(DatabaseType.POSTGRES); + + assertEquals(TEST_REPORT_ID, resultMap.get(ModelScanMetricsReport.REPORT_ID)); + PGobject pgObject = (PGobject) resultMap.get(ModelScanMetricsReport.METADATA); + assertEquals("jsonb", pgObject.getType()); + assertEquals(TEST_METADATA, pgObject.getValue()); + } + + private ModelScanMetricsReport createTestReport() { + return ImmutableModelScanMetricsReport.builder() + .reportId(TEST_REPORT_ID) + .realmId(TEST_REALM_ID) + .catalogId(TEST_CATALOG_ID) + .catalogName(TEST_CATALOG_NAME) + .namespace(TEST_NAMESPACE) + .tableName(TEST_TABLE_NAME) + .timestampMs(TEST_TIMESTAMP_MS) + .principalName(TEST_PRINCIPAL) + .requestId(TEST_REQUEST_ID) + .otelTraceId(TEST_OTEL_TRACE_ID) + .snapshotId(TEST_SNAPSHOT_ID) + .resultDataFiles(TEST_RESULT_DATA_FILES) + .resultDeleteFiles(TEST_RESULT_DELETE_FILES) + .totalFileSizeBytes(TEST_TOTAL_FILE_SIZE) + .totalDataManifests(TEST_TOTAL_DATA_MANIFESTS) + .totalDeleteManifests(TEST_TOTAL_DELETE_MANIFESTS) + .scannedDataManifests(TEST_SCANNED_DATA_MANIFESTS) + .scannedDeleteManifests(TEST_SCANNED_DELETE_MANIFESTS) + .skippedDataManifests(TEST_SKIPPED_DATA_MANIFESTS) + .skippedDeleteManifests(TEST_SKIPPED_DELETE_MANIFESTS) + .skippedDataFiles(TEST_SKIPPED_DATA_FILES) + .skippedDeleteFiles(TEST_SKIPPED_DELETE_FILES) + .totalPlanningDurationMs(TEST_PLANNING_DURATION) + .equalityDeleteFiles(TEST_EQUALITY_DELETE_FILES) + .positionalDeleteFiles(TEST_POSITIONAL_DELETE_FILES) + .indexedDeleteFiles(TEST_INDEXED_DELETE_FILES) + .totalDeleteFileSizeBytes(TEST_DELETE_FILE_SIZE) + .metadata(TEST_METADATA) + .build(); + } +} diff --git a/polaris-core/src/main/java/org/apache/polaris/core/storage/CredentialVendingContext.java b/polaris-core/src/main/java/org/apache/polaris/core/storage/CredentialVendingContext.java index 718062c414..7c3b214d32 100644 --- a/polaris-core/src/main/java/org/apache/polaris/core/storage/CredentialVendingContext.java +++ b/polaris-core/src/main/java/org/apache/polaris/core/storage/CredentialVendingContext.java @@ -20,6 +20,7 @@ import java.util.Optional; import org.apache.polaris.immutables.PolarisImmutable; +import org.immutables.value.Value; /** * Context information for credential vending operations. This context is used to provide metadata @@ -33,10 +34,11 @@ *
  • {@code namespace} - The namespace/database being accessed (e.g., "db.schema") *
  • {@code tableName} - The name of the table being accessed *
  • {@code activatedRoles} - Comma-separated list of activated principal roles + *
  • {@code traceId} - OpenTelemetry trace ID for end-to-end correlation * * - *

    These values appear in cloud provider audit logs (e.g., AWS CloudTrail), enabling correlation - * between catalog operations and data access events. + *

    These values appear in cloud provider audit logs (e.g., AWS CloudTrail), enabling + * deterministic correlation between catalog operations and data access events. */ @PolarisImmutable public interface CredentialVendingContext { @@ -48,6 +50,7 @@ public interface CredentialVendingContext { String TAG_KEY_TABLE = "polaris:table"; String TAG_KEY_PRINCIPAL = "polaris:principal"; String TAG_KEY_ROLES = "polaris:roles"; + String TAG_KEY_TRACE_ID = "polaris:trace_id"; /** The name of the catalog that is vending credentials. */ Optional catalogName(); @@ -67,6 +70,19 @@ public interface CredentialVendingContext { */ Optional activatedRoles(); + /** + * The OpenTelemetry trace ID for end-to-end correlation. This enables correlation between + * credential vending (CloudTrail), catalog operations (Polaris events), and metrics reports from + * compute engines. + * + *

    This field is marked as {@link Value.Auxiliary} to exclude it from {@code equals()} and + * {@code hashCode()} methods. This is critical for cache key comparison - including trace ID + * would prevent cache hits since every request has a unique trace ID. The trace ID is for + * correlation/audit purposes only and should not affect credential caching behavior. + */ + @Value.Auxiliary + Optional traceId(); + /** * Creates a new builder for CredentialVendingContext. * @@ -95,6 +111,8 @@ interface Builder { Builder activatedRoles(Optional activatedRoles); + Builder traceId(Optional traceId); + CredentialVendingContext build(); } } diff --git a/polaris-core/src/test/java/org/apache/polaris/core/storage/cache/StorageCredentialCacheTest.java b/polaris-core/src/test/java/org/apache/polaris/core/storage/cache/StorageCredentialCacheTest.java index 98464dc30d..7f60af7ed7 100644 --- a/polaris-core/src/test/java/org/apache/polaris/core/storage/cache/StorageCredentialCacheTest.java +++ b/polaris-core/src/test/java/org/apache/polaris/core/storage/cache/StorageCredentialCacheTest.java @@ -228,11 +228,11 @@ public void testCacheMissForAnotherPrincipal() { new PolarisConfigurationStore() { @SuppressWarnings("unchecked") @Override - public String getConfiguration(@Nonnull RealmContext ctx, String configName) { + public T getConfiguration(@Nonnull RealmContext ctx, String configName) { if (configName.equals( FeatureConfiguration.INCLUDE_PRINCIPAL_NAME_IN_SUBSCOPED_CREDENTIAL .key())) { - return "true"; + return (T) "true"; } return null; } diff --git a/runtime/service/build.gradle.kts b/runtime/service/build.gradle.kts index 9dcd8c5885..302a308816 100644 --- a/runtime/service/build.gradle.kts +++ b/runtime/service/build.gradle.kts @@ -31,7 +31,7 @@ dependencies { implementation(project(":polaris-api-iceberg-service")) implementation(project(":polaris-api-catalog-service")) - runtimeOnly(project(":polaris-relational-jdbc")) + implementation(project(":polaris-relational-jdbc")) implementation(project(":polaris-runtime-defaults")) implementation(project(":polaris-runtime-common")) @@ -56,6 +56,7 @@ dependencies { implementation("io.quarkus:quarkus-micrometer-registry-prometheus") implementation("io.quarkus:quarkus-oidc") implementation("io.quarkus:quarkus-opentelemetry") + implementation("io.quarkus:quarkus-scheduler") implementation("io.quarkus:quarkus-security") implementation("io.quarkus:quarkus-smallrye-context-propagation") implementation("io.quarkus:quarkus-smallrye-fault-tolerance") diff --git a/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogAdapter.java b/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogAdapter.java index 6c30afb9e7..b5c9109abe 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogAdapter.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergCatalogAdapter.java @@ -718,6 +718,9 @@ public Response reportMetrics( ReportMetricsRequest reportMetricsRequest, RealmContext realmContext, SecurityContext securityContext) { + // Validate that the caller is authenticated (consistent with other endpoints) + validatePrincipal(securityContext); + String catalogName = prefixParser.prefixToCatalogName(realmContext, prefix); Namespace ns = decodeNamespace(namespace); TableIdentifier tableIdentifier = TableIdentifier.of(ns, RESTUtil.decodeString(table)); diff --git a/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergRestCatalogEventServiceDelegator.java b/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergRestCatalogEventServiceDelegator.java index 29a298a288..0491edb58d 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergRestCatalogEventServiceDelegator.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/catalog/iceberg/IcebergRestCatalogEventServiceDelegator.java @@ -27,6 +27,7 @@ import jakarta.ws.rs.core.Response; import jakarta.ws.rs.core.SecurityContext; import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.rest.RESTUtil; import org.apache.iceberg.rest.requests.CommitTransactionRequest; import org.apache.iceberg.rest.requests.CreateNamespaceRequest; import org.apache.iceberg.rest.requests.CreateTableRequest; @@ -642,7 +643,16 @@ public Response commitTransaction( return resp; } - /** This API is currently a no-op in Polaris. */ + /** + * Handles metrics reports from compute engines (Spark, Trino, Flink, etc.). + * + *

    This endpoint receives ScanReport and CommitReport metrics from compute engines after they + * perform table operations. Events are emitted before and after processing to enable audit + * logging and observability. + * + *

    The metadata map in the report can contain trace context (e.g., trace_id) for correlation + * with other audit events such as loadTable and loadCredentials. + */ @Override public Response reportMetrics( String prefix, @@ -651,8 +661,28 @@ public Response reportMetrics( ReportMetricsRequest reportMetricsRequest, RealmContext realmContext, SecurityContext securityContext) { - return delegate.reportMetrics( - prefix, namespace, table, reportMetricsRequest, realmContext, securityContext); + String catalogName = prefixParser.prefixToCatalogName(realmContext, prefix); + Namespace namespaceObj = decodeNamespace(namespace); + // Decode the table name to ensure consistent identifiers in audit events + String decodedTable = RESTUtil.decodeString(table); + polarisEventListener.onBeforeReportMetrics( + new IcebergRestCatalogEvents.BeforeReportMetricsEvent( + eventMetadataFactory.create(), + catalogName, + namespaceObj, + decodedTable, + reportMetricsRequest)); + Response resp = + delegate.reportMetrics( + prefix, namespace, table, reportMetricsRequest, realmContext, securityContext); + polarisEventListener.onAfterReportMetrics( + new IcebergRestCatalogEvents.AfterReportMetricsEvent( + eventMetadataFactory.create(), + catalogName, + namespaceObj, + decodedTable, + reportMetricsRequest)); + return resp; } @Override diff --git a/runtime/service/src/main/java/org/apache/polaris/service/catalog/io/StorageAccessConfigProvider.java b/runtime/service/src/main/java/org/apache/polaris/service/catalog/io/StorageAccessConfigProvider.java index c92204fe73..8180ccac05 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/catalog/io/StorageAccessConfigProvider.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/catalog/io/StorageAccessConfigProvider.java @@ -19,6 +19,8 @@ package org.apache.polaris.service.catalog.io; +import io.opentelemetry.api.trace.Span; +import io.opentelemetry.api.trace.SpanContext; import jakarta.annotation.Nonnull; import jakarta.enterprise.context.RequestScoped; import jakarta.inject.Inject; @@ -187,6 +189,24 @@ private CredentialVendingContext buildCredentialVendingContext( builder.activatedRoles(Optional.of(rolesString)); } + // Extract OpenTelemetry trace ID for end-to-end correlation + // This enables correlation between credential vending (CloudTrail), catalog operations + // (Polaris events), and metrics reports from compute engines + builder.traceId(getCurrentTraceId()); + return builder.build(); } + + /** + * Extracts the current OpenTelemetry trace ID from the active span context. + * + * @return the trace ID if a valid span context exists, empty otherwise + */ + private Optional getCurrentTraceId() { + SpanContext spanContext = Span.current().getSpanContext(); + if (spanContext.isValid()) { + return Optional.of(spanContext.getTraceId()); + } + return Optional.empty(); + } } diff --git a/runtime/service/src/main/java/org/apache/polaris/service/config/ServiceProducers.java b/runtime/service/src/main/java/org/apache/polaris/service/config/ServiceProducers.java index 0042ac84e2..ce00fcf5c6 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/config/ServiceProducers.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/config/ServiceProducers.java @@ -31,6 +31,8 @@ import jakarta.enterprise.inject.Produces; import jakarta.inject.Singleton; import java.time.Clock; +import java.util.ArrayList; +import java.util.List; import java.util.stream.Collectors; import org.apache.polaris.core.PolarisCallContext; import org.apache.polaris.core.PolarisDefaultDiagServiceImpl; @@ -78,6 +80,7 @@ import org.apache.polaris.service.ratelimiter.RateLimiterFilterConfiguration; import org.apache.polaris.service.ratelimiter.TokenBucketConfiguration; import org.apache.polaris.service.ratelimiter.TokenBucketFactory; +import org.apache.polaris.service.reporting.CompositeMetricsReporter; import org.apache.polaris.service.reporting.MetricsReportingConfiguration; import org.apache.polaris.service.reporting.PolarisMetricsReporter; import org.apache.polaris.service.secrets.SecretsManagerConfiguration; @@ -433,6 +436,57 @@ public void closeTaskExecutor(@Disposes @Identifier("task-executor") ManagedExec @ApplicationScoped public PolarisMetricsReporter metricsReporter( MetricsReportingConfiguration config, @Any Instance reporters) { - return reporters.select(Identifier.Literal.of(config.type())).get(); + String type = config.type(); + + if ("composite".equals(type)) { + List targets = config.targets(); + if (targets == null || targets.isEmpty()) { + LOGGER.warn( + "Composite metrics reporter configured but no targets specified. " + + "Falling back to default reporter."); + return reporters.select(Identifier.Literal.of("default")).get(); + } + + List delegates = new ArrayList<>(); + for (String target : targets) { + if (target == null || target.isBlank()) { + continue; + } + String trimmedTarget = target.trim(); + // Avoid infinite recursion - don't allow composite as a target + if ("composite".equals(trimmedTarget)) { + LOGGER.warn("Ignoring 'composite' as a target - would cause infinite recursion"); + continue; + } + try { + PolarisMetricsReporter delegate = + reporters.select(Identifier.Literal.of(trimmedTarget)).get(); + delegates.add(delegate); + LOGGER.info("Added metrics reporter target: {}", trimmedTarget); + } catch (Exception e) { + LOGGER.error( + "Failed to instantiate metrics reporter for target '{}': {}", + trimmedTarget, + e.getMessage()); + } + } + + if (delegates.isEmpty()) { + LOGGER.warn("No valid targets for composite reporter. Falling back to default reporter."); + return reporters.select(Identifier.Literal.of("default")).get(); + } + + return new CompositeMetricsReporter(delegates); + } + + try { + return reporters.select(Identifier.Literal.of(type)).get(); + } catch (Exception e) { + LOGGER.error( + "Failed to instantiate metrics reporter for type '{}': {}. Falling back to default.", + type, + e.getMessage()); + return reporters.select(Identifier.Literal.of("default")).get(); + } } } diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/IcebergRestCatalogEvents.java b/runtime/service/src/main/java/org/apache/polaris/service/events/IcebergRestCatalogEvents.java index 610d716cf9..e1eacf8d73 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/IcebergRestCatalogEvents.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/IcebergRestCatalogEvents.java @@ -28,6 +28,7 @@ import org.apache.iceberg.rest.requests.CreateViewRequest; import org.apache.iceberg.rest.requests.RegisterTableRequest; import org.apache.iceberg.rest.requests.RenameTableRequest; +import org.apache.iceberg.rest.requests.ReportMetricsRequest; import org.apache.iceberg.rest.requests.UpdateNamespacePropertiesRequest; import org.apache.iceberg.rest.requests.UpdateTableRequest; import org.apache.iceberg.rest.responses.ConfigResponse; @@ -645,4 +646,55 @@ public PolarisEventType type() { return PolarisEventType.AFTER_REFRESH_VIEW; } } + + // Metrics Reporting Events + + /** + * Event emitted before a metrics report is processed. + * + * @param metadata Event metadata including timestamp, realm, user, and request context + * @param catalogName The name of the catalog + * @param namespace The namespace containing the table + * @param table The table name for which metrics are being reported + * @param reportMetricsRequest The metrics report request (ScanReport or CommitReport) + */ + public record BeforeReportMetricsEvent( + PolarisEventMetadata metadata, + String catalogName, + Namespace namespace, + String table, + ReportMetricsRequest reportMetricsRequest) + implements PolarisEvent { + @Override + public PolarisEventType type() { + return PolarisEventType.BEFORE_REPORT_METRICS; + } + } + + /** + * Event emitted after a metrics report has been processed. + * + *

    This event enables audit logging of compute engine metrics reports, including scan metrics + * (files scanned, bytes read, planning duration) and commit metrics (files added/removed, + * operation type). The metadata map in the report can contain trace context for correlation with + * other audit events. + * + * @param metadata Event metadata including timestamp, realm, user, and request context + * @param catalogName The name of the catalog + * @param namespace The namespace containing the table + * @param table The table name for which metrics were reported + * @param reportMetricsRequest The metrics report request (ScanReport or CommitReport) + */ + public record AfterReportMetricsEvent( + PolarisEventMetadata metadata, + String catalogName, + Namespace namespace, + String table, + ReportMetricsRequest reportMetricsRequest) + implements PolarisEvent { + @Override + public PolarisEventType type() { + return PolarisEventType.AFTER_REPORT_METRICS; + } + } } diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/PolarisEventType.java b/runtime/service/src/main/java/org/apache/polaris/service/events/PolarisEventType.java index 949b9cb24e..bb8345bcee 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/PolarisEventType.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/PolarisEventType.java @@ -200,4 +200,8 @@ public enum PolarisEventType { // Rate Limiting Events BEFORE_LIMIT_REQUEST_RATE, + + // Metrics Reporting Events + BEFORE_REPORT_METRICS, + AFTER_REPORT_METRICS, } diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/PolarisEventListener.java b/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/PolarisEventListener.java index 962e636231..a96406e50f 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/PolarisEventListener.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/PolarisEventListener.java @@ -424,4 +424,10 @@ default void onAfterAttemptTask(AfterAttemptTaskEvent event) {} // ============= Rate Limiting Events ============= default void onBeforeLimitRequestRate(BeforeLimitRequestRateEvent event) {} + + // ============= Metrics Reporting Events ============= + + default void onBeforeReportMetrics(IcebergRestCatalogEvents.BeforeReportMetricsEvent event) {} + + default void onAfterReportMetrics(IcebergRestCatalogEvents.AfterReportMetricsEvent event) {} } diff --git a/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/PolarisPersistenceEventListener.java b/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/PolarisPersistenceEventListener.java index 16f85589e6..c56a2803bf 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/PolarisPersistenceEventListener.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/events/listeners/PolarisPersistenceEventListener.java @@ -23,7 +23,15 @@ import java.util.Map; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableMetadataParser; +import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.metrics.CommitMetricsResult; +import org.apache.iceberg.metrics.CommitReport; +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.MetricsReport; +import org.apache.iceberg.metrics.ScanMetricsResult; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.iceberg.rest.requests.ReportMetricsRequest; import org.apache.polaris.core.auth.PolarisPrincipal; import org.apache.polaris.core.entity.PolarisEvent; import org.apache.polaris.service.events.CatalogsServiceEvents; @@ -74,5 +82,152 @@ public void onAfterCreateCatalog(CatalogsServiceEvents.AfterCreateCatalogEvent e processEvent(event.metadata().realmId(), polarisEvent); } + @Override + public void onAfterReportMetrics(IcebergRestCatalogEvents.AfterReportMetricsEvent event) { + // Build table identifier with null safety + Namespace namespace = event.namespace(); + String table = event.table(); + String tableIdentifierStr = + (namespace != null && table != null) + ? TableIdentifier.of(namespace, table).toString() + : (table != null ? table : "unknown"); + + PolarisEvent polarisEvent = + new PolarisEvent( + event.catalogName(), + event.metadata().eventId().toString(), + event.metadata().requestId().orElse(null), + event.getClass().getSimpleName(), + event.metadata().timestamp().toEpochMilli(), + event.metadata().user().map(PolarisPrincipal::getName).orElse(null), + PolarisEvent.ResourceType.TABLE, + tableIdentifierStr); + + // Build additional properties with metrics data and trace context + ImmutableMap.Builder additionalProperties = ImmutableMap.builder(); + + // Add OpenTelemetry context from HTTP headers + additionalProperties.putAll(event.metadata().openTelemetryContext()); + + // Extract metadata and metrics from the report + ReportMetricsRequest request = event.reportMetricsRequest(); + if (request != null) { + MetricsReport report = request.report(); + if (report instanceof ScanReport scanReport) { + extractScanReportData(scanReport, additionalProperties); + } else if (report instanceof CommitReport commitReport) { + extractCommitReportData(commitReport, additionalProperties); + } + } + + Map props = additionalProperties.build(); + if (!props.isEmpty()) { + polarisEvent.setAdditionalProperties(props); + } + processEvent(event.metadata().realmId(), polarisEvent); + } + + /** + * Extracts data from a ScanReport including metadata (trace-id) and key scan metrics. + * + * @param scanReport The scan report from the compute engine + * @param builder The builder to add properties to + */ + private void extractScanReportData( + ScanReport scanReport, ImmutableMap.Builder builder) { + builder.put("report_type", "scan"); + builder.put("snapshot_id", String.valueOf(scanReport.snapshotId())); + builder.put("schema_id", String.valueOf(scanReport.schemaId())); + + // Extract trace-id and other metadata from the report's metadata map + // This is where compute engines pass trace context for correlation + Map reportMetadata = scanReport.metadata(); + addReportMetadata(builder, reportMetadata); + + // Extract key scan metrics for audit purposes + ScanMetricsResult metrics = scanReport.scanMetrics(); + if (metrics != null) { + addCounterIfPresent(builder, "result_data_files", metrics.resultDataFiles()); + addCounterIfPresent(builder, "result_delete_files", metrics.resultDeleteFiles()); + addCounterIfPresent(builder, "total_file_size_bytes", metrics.totalFileSizeInBytes()); + addCounterIfPresent(builder, "scanned_data_manifests", metrics.scannedDataManifests()); + addCounterIfPresent(builder, "skipped_data_manifests", metrics.skippedDataManifests()); + } + } + + /** + * Extracts data from a CommitReport including metadata (trace-id) and key commit metrics. + * + * @param commitReport The commit report from the compute engine + * @param builder The builder to add properties to + */ + private void extractCommitReportData( + CommitReport commitReport, ImmutableMap.Builder builder) { + builder.put("report_type", "commit"); + builder.put("snapshot_id", String.valueOf(commitReport.snapshotId())); + // Null-safe handling of sequence number + Long sequenceNumber = commitReport.sequenceNumber(); + if (sequenceNumber != null) { + builder.put("sequence_number", String.valueOf(sequenceNumber)); + } + // Null-safe handling of operation - it may be null for some report types + String operation = commitReport.operation(); + if (operation != null) { + builder.put("operation", operation); + } + + // Extract trace-id and other metadata from the report's metadata map + // This is where compute engines pass trace context for correlation + Map reportMetadata = commitReport.metadata(); + addReportMetadata(builder, reportMetadata); + + // Extract key commit metrics for audit purposes + CommitMetricsResult metrics = commitReport.commitMetrics(); + if (metrics != null) { + addCounterIfPresent(builder, "added_data_files", metrics.addedDataFiles()); + addCounterIfPresent(builder, "removed_data_files", metrics.removedDataFiles()); + addCounterIfPresent(builder, "added_records", metrics.addedRecords()); + addCounterIfPresent(builder, "removed_records", metrics.removedRecords()); + addCounterIfPresent(builder, "added_file_size_bytes", metrics.addedFilesSizeInBytes()); + addCounterIfPresent(builder, "removed_file_size_bytes", metrics.removedFilesSizeInBytes()); + } + } + + /** + * Adds report metadata entries to the builder with null-safety checks. Entries with null keys or + * values are skipped to prevent ImmutableMap.Builder from throwing NPE. + * + * @param builder The builder to add properties to + * @param reportMetadata The metadata map from the report (may be null) + */ + private void addReportMetadata( + ImmutableMap.Builder builder, Map reportMetadata) { + if (reportMetadata != null && !reportMetadata.isEmpty()) { + for (Map.Entry entry : reportMetadata.entrySet()) { + String key = entry.getKey(); + String value = entry.getValue(); + // Skip entries with null keys or values to prevent ImmutableMap.Builder NPE + if (key != null && value != null) { + // Prefix with "report." to distinguish from OpenTelemetry context + builder.put("report." + key, value); + } + } + } + } + + /** + * Adds a counter value to the builder if the counter is present and has a value. + * + * @param builder The builder to add the property to + * @param key The property key + * @param counter The counter result (may be null) + */ + private void addCounterIfPresent( + ImmutableMap.Builder builder, String key, CounterResult counter) { + if (counter != null) { + builder.put(key, String.valueOf(counter.value())); + } + } + protected abstract void processEvent(String realmId, PolarisEvent event); } diff --git a/runtime/service/src/main/java/org/apache/polaris/service/reporting/CompositeMetricsReporter.java b/runtime/service/src/main/java/org/apache/polaris/service/reporting/CompositeMetricsReporter.java new file mode 100644 index 0000000000..fbad83128e --- /dev/null +++ b/runtime/service/src/main/java/org/apache/polaris/service/reporting/CompositeMetricsReporter.java @@ -0,0 +1,92 @@ +/* + * 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.polaris.service.reporting; + +import java.util.List; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.metrics.MetricsReport; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A composite metrics reporter that delegates to multiple child reporters. This allows metrics to + * be sent to multiple destinations simultaneously, such as both the events table and dedicated + * metrics tables. + * + *

    To enable this reporter, set the configuration: + * + *

    + * polaris:
    + *   iceberg-metrics:
    + *     reporting:
    + *       type: composite
    + *       targets:
    + *         - events       # Write to events table
    + *         - persistence  # Write to dedicated tables
    + * 
    + * + *

    The composite reporter will call each configured target reporter in order. If one reporter + * fails, the others will still be called. + */ +public class CompositeMetricsReporter implements PolarisMetricsReporter { + + private static final Logger LOGGER = LoggerFactory.getLogger(CompositeMetricsReporter.class); + + private final List delegates; + + /** + * Creates a composite reporter with the given delegate reporters. + * + * @param delegates the list of reporters to delegate to + */ + public CompositeMetricsReporter(List delegates) { + this.delegates = List.copyOf(delegates); + LOGGER.info( + "CompositeMetricsReporter initialized with {} delegate(s): {}", + delegates.size(), + delegates.stream().map(r -> r.getClass().getSimpleName()).toList()); + } + + @Override + public void reportMetric(String catalogName, TableIdentifier table, MetricsReport metricsReport) { + for (PolarisMetricsReporter delegate : delegates) { + try { + delegate.reportMetric(catalogName, table, metricsReport); + } catch (Exception e) { + LOGGER.error( + "Delegate reporter {} failed for table {}.{}: {}", + delegate.getClass().getSimpleName(), + catalogName, + table, + e.getMessage(), + e); + // Continue with other delegates even if one fails + } + } + } + + /** + * Returns the list of delegate reporters. + * + * @return unmodifiable list of delegates + */ + public List getDelegates() { + return delegates; + } +} diff --git a/runtime/service/src/main/java/org/apache/polaris/service/reporting/EventsMetricsReporter.java b/runtime/service/src/main/java/org/apache/polaris/service/reporting/EventsMetricsReporter.java new file mode 100644 index 0000000000..686f2dde82 --- /dev/null +++ b/runtime/service/src/main/java/org/apache/polaris/service/reporting/EventsMetricsReporter.java @@ -0,0 +1,206 @@ +/* + * 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.polaris.service.reporting; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.opentelemetry.api.trace.Span; +import io.opentelemetry.api.trace.SpanContext; +import io.quarkus.security.identity.SecurityIdentity; +import io.smallrye.common.annotation.Identifier; +import jakarta.enterprise.context.ApplicationScoped; +import jakarta.enterprise.inject.Instance; +import jakarta.inject.Inject; +import java.security.Principal; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.metrics.CommitReport; +import org.apache.iceberg.metrics.MetricsReport; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.polaris.core.context.RealmContext; +import org.apache.polaris.core.entity.PolarisEvent; +import org.apache.polaris.core.persistence.BasePersistence; +import org.apache.polaris.core.persistence.MetaStoreManagerFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A metrics reporter that persists scan and commit reports to the events table as JSON. This + * provides a unified audit trail where metrics are stored alongside other catalog events. + * + *

    To enable this reporter, set the configuration: + * + *

    + * polaris:
    + *   iceberg-metrics:
    + *     reporting:
    + *       type: events
    + * 
    + * + *

    Or use it as part of a composite reporter: + * + *

    + * polaris:
    + *   iceberg-metrics:
    + *     reporting:
    + *       type: composite
    + *       targets:
    + *         - events
    + *         - persistence
    + * 
    + */ +@ApplicationScoped +@Identifier("events") +public class EventsMetricsReporter implements PolarisMetricsReporter { + + private static final Logger LOGGER = LoggerFactory.getLogger(EventsMetricsReporter.class); + + public static final String EVENT_TYPE_SCAN_REPORT = "ScanReport"; + public static final String EVENT_TYPE_COMMIT_REPORT = "CommitReport"; + + private final MetaStoreManagerFactory metaStoreManagerFactory; + private final RealmContext realmContext; + private final ObjectMapper objectMapper; + private final Instance securityIdentityInstance; + + @Inject + public EventsMetricsReporter( + MetaStoreManagerFactory metaStoreManagerFactory, + RealmContext realmContext, + ObjectMapper objectMapper, + Instance securityIdentityInstance) { + this.metaStoreManagerFactory = metaStoreManagerFactory; + this.realmContext = realmContext; + this.objectMapper = objectMapper; + this.securityIdentityInstance = securityIdentityInstance; + } + + @Override + public void reportMetric(String catalogName, TableIdentifier table, MetricsReport metricsReport) { + try { + String eventType; + PolarisEvent.ResourceType resourceType = PolarisEvent.ResourceType.TABLE; + String resourceIdentifier = table.toString(); + + if (metricsReport instanceof ScanReport) { + eventType = EVENT_TYPE_SCAN_REPORT; + } else if (metricsReport instanceof CommitReport) { + eventType = EVENT_TYPE_COMMIT_REPORT; + } else { + LOGGER.warn("Unknown metrics report type: {}", metricsReport.getClass().getName()); + return; + } + + // Extract principal name from security context + String principalName = extractPrincipalName(); + + // Extract OpenTelemetry trace context + String otelTraceId = null; + String otelSpanId = null; + Span currentSpan = Span.current(); + if (currentSpan != null) { + SpanContext spanContext = currentSpan.getSpanContext(); + if (spanContext != null && spanContext.isValid()) { + otelTraceId = spanContext.getTraceId(); + otelSpanId = spanContext.getSpanId(); + } + } + + // Serialize the metrics report and add trace context to additional properties + Map additionalProps = new HashMap<>(); + additionalProps.put("metricsReport", serializeMetricsReportToMap(metricsReport)); + if (otelTraceId != null) { + additionalProps.put("otelTraceId", otelTraceId); + } + if (otelSpanId != null) { + additionalProps.put("otelSpanId", otelSpanId); + } + String additionalPropsJson = serializeToJson(additionalProps); + + PolarisEvent event = + new PolarisEvent( + catalogName, + UUID.randomUUID().toString(), + null, // requestId - could be extracted from context if available + eventType, + System.currentTimeMillis(), + principalName, + resourceType, + resourceIdentifier); + event.setAdditionalProperties(additionalPropsJson); + + // Get the persistence session for the current realm and write the event + BasePersistence session = metaStoreManagerFactory.getOrCreateSession(realmContext); + session.writeEvents(List.of(event)); + + LOGGER.debug("Persisted {} event for table {}.{}", eventType, catalogName, table); + } catch (Exception e) { + LOGGER.error( + "Failed to persist metrics event for table {}.{}: {}", + catalogName, + table, + e.getMessage(), + e); + } + } + + /** + * Extracts the principal name from the current security context. + * + * @return the principal name, or null if not available + */ + private String extractPrincipalName() { + try { + if (securityIdentityInstance.isResolvable()) { + SecurityIdentity identity = securityIdentityInstance.get(); + if (identity != null && !identity.isAnonymous()) { + Principal principal = identity.getPrincipal(); + if (principal != null) { + return principal.getName(); + } + } + } + } catch (Exception e) { + LOGGER.trace("Could not extract principal name from security context: {}", e.getMessage()); + } + return null; + } + + private Object serializeMetricsReportToMap(MetricsReport metricsReport) { + try { + String json = objectMapper.writeValueAsString(metricsReport); + return objectMapper.readValue(json, Object.class); + } catch (JsonProcessingException e) { + LOGGER.warn("Failed to serialize metrics report: {}", e.getMessage()); + return Map.of(); + } + } + + private String serializeToJson(Object obj) { + try { + return objectMapper.writeValueAsString(obj); + } catch (JsonProcessingException e) { + LOGGER.warn("Failed to serialize to JSON: {}", e.getMessage()); + return "{}"; + } + } +} diff --git a/runtime/service/src/main/java/org/apache/polaris/service/reporting/MetricsReportCleanupService.java b/runtime/service/src/main/java/org/apache/polaris/service/reporting/MetricsReportCleanupService.java new file mode 100644 index 0000000000..04bbca6365 --- /dev/null +++ b/runtime/service/src/main/java/org/apache/polaris/service/reporting/MetricsReportCleanupService.java @@ -0,0 +1,198 @@ +/* + * 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.polaris.service.reporting; + +import io.quarkus.runtime.Startup; +import io.quarkus.scheduler.Scheduled; +import jakarta.enterprise.context.ApplicationScoped; +import jakarta.inject.Inject; +import java.time.Duration; +import java.time.Instant; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.polaris.core.context.RealmContext; +import org.apache.polaris.core.persistence.BasePersistence; +import org.apache.polaris.core.persistence.MetaStoreManagerFactory; +import org.apache.polaris.persistence.relational.jdbc.JdbcBasePersistenceImpl; +import org.apache.polaris.service.context.RealmContextConfiguration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Scheduled service that cleans up old metrics reports based on the configured retention policy. + * + *

    This service runs periodically and deletes metrics reports that are older than the configured + * retention period. It only operates when the persistence backend is relational-jdbc. + * + *

    Configuration example: + * + *

    + * polaris:
    + *   iceberg-metrics:
    + *     reporting:
    + *       type: persistence
    + *       retention:
    + *         enabled: true
    + *         retention-period: P30D  # 30 days
    + *         cleanup-interval: PT6H  # every 6 hours
    + * 
    + */ +@ApplicationScoped +@Startup +public class MetricsReportCleanupService { + + private static final Logger LOGGER = LoggerFactory.getLogger(MetricsReportCleanupService.class); + + private final MetricsReportingConfiguration config; + private final MetaStoreManagerFactory metaStoreManagerFactory; + private final RealmContextConfiguration realmContextConfiguration; + private final AtomicBoolean running = new AtomicBoolean(false); + + @Inject + public MetricsReportCleanupService( + MetricsReportingConfiguration config, + MetaStoreManagerFactory metaStoreManagerFactory, + RealmContextConfiguration realmContextConfiguration) { + this.config = config; + this.metaStoreManagerFactory = metaStoreManagerFactory; + this.realmContextConfiguration = realmContextConfiguration; + + if (config.retention().enabled()) { + LOGGER.info( + "Metrics report cleanup enabled with retention period: {}, cleanup interval: {}", + config.retention().retentionPeriod(), + config.retention().cleanupInterval()); + } else { + LOGGER.debug("Metrics report cleanup is disabled"); + } + } + + /** + * Scheduled cleanup job that runs at the configured interval. The actual interval is configured + * via the retention.cleanup-interval property. + */ + @Scheduled(every = "${polaris.iceberg-metrics.reporting.retention.cleanup-interval:6h}") + public void cleanupOldMetricsReports() { + if (!config.retention().enabled()) { + LOGGER.trace("Metrics cleanup is disabled, skipping"); + return; + } + + // Prevent concurrent runs + if (!running.compareAndSet(false, true)) { + LOGGER.debug("Cleanup already in progress, skipping this run"); + return; + } + + try { + performCleanup(); + } finally { + running.set(false); + } + } + + private void performCleanup() { + Duration retentionPeriod = config.retention().retentionPeriod(); + long cutoffTimestamp = Instant.now().minus(retentionPeriod).toEpochMilli(); + List realmIds = realmContextConfiguration.realms(); + + LOGGER.info( + "Starting metrics report cleanup across {} realm(s). Deleting reports older than {} (cutoff: {})", + realmIds.size(), + retentionPeriod, + Instant.ofEpochMilli(cutoffTimestamp)); + + int totalDeleted = 0; + for (String realmId : realmIds) { + try { + int deletedCount = cleanupForRealm(realmId, cutoffTimestamp); + if (deletedCount > 0) { + LOGGER.info("Deleted {} old metrics reports from realm '{}'", deletedCount, realmId); + totalDeleted += deletedCount; + } else { + LOGGER.debug("No old metrics reports to delete in realm '{}'", realmId); + } + } catch (Exception e) { + LOGGER.error( + "Failed to cleanup old metrics reports for realm '{}': {}", realmId, e.getMessage(), e); + } + } + + if (totalDeleted > 0) { + LOGGER.info("Total deleted metrics reports across all realms: {}", totalDeleted); + } + } + + private int cleanupForRealm(String realmId, long cutoffTimestamp) { + RealmContext realmContext = () -> realmId; + BasePersistence session = metaStoreManagerFactory.getOrCreateSession(realmContext); + + if (!(session instanceof JdbcBasePersistenceImpl jdbcPersistence)) { + LOGGER.debug( + "Metrics cleanup is only supported with relational-jdbc backend. " + + "Current backend: {} for realm '{}'", + session.getClass().getSimpleName(), + realmId); + return 0; + } + + return jdbcPersistence.deleteAllMetricsReportsOlderThan(cutoffTimestamp); + } + + /** + * Manually trigger a cleanup across all realms. This can be called from an admin endpoint or for + * testing. + * + * @return the total number of reports deleted across all realms, or -1 if cleanup is disabled or + * failed + */ + public int triggerCleanup() { + if (!config.retention().enabled()) { + LOGGER.warn("Cannot trigger cleanup: retention is disabled"); + return -1; + } + + if (!running.compareAndSet(false, true)) { + LOGGER.warn("Cannot trigger cleanup: cleanup already in progress"); + return -1; + } + + try { + Duration retentionPeriod = config.retention().retentionPeriod(); + long cutoffTimestamp = Instant.now().minus(retentionPeriod).toEpochMilli(); + List realmIds = realmContextConfiguration.realms(); + + int totalDeleted = 0; + for (String realmId : realmIds) { + try { + int deletedCount = cleanupForRealm(realmId, cutoffTimestamp); + totalDeleted += deletedCount; + } catch (Exception e) { + LOGGER.error("Failed to cleanup metrics for realm '{}': {}", realmId, e.getMessage(), e); + } + } + return totalDeleted; + } catch (Exception e) { + LOGGER.error("Failed to trigger cleanup: {}", e.getMessage(), e); + return -1; + } finally { + running.set(false); + } + } +} diff --git a/runtime/service/src/main/java/org/apache/polaris/service/reporting/MetricsReportingConfiguration.java b/runtime/service/src/main/java/org/apache/polaris/service/reporting/MetricsReportingConfiguration.java index 3d60302ab3..a041b3170c 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/reporting/MetricsReportingConfiguration.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/reporting/MetricsReportingConfiguration.java @@ -20,9 +20,77 @@ import io.smallrye.config.ConfigMapping; import io.smallrye.config.WithDefault; +import java.time.Duration; +import java.util.List; @ConfigMapping(prefix = "polaris.iceberg-metrics.reporting") public interface MetricsReportingConfiguration { + /** + * The type of metrics reporter to use. Supported values: + * + *
      + *
    • {@code default} - Log metrics to console only (no persistence) + *
    • {@code events} - Persist metrics to the events table as JSON + *
    • {@code persistence} - Persist metrics to dedicated tables (scan_metrics_report, + * commit_metrics_report) + *
    • {@code composite} - Use multiple reporters based on the {@link #targets()} configuration + *
    + * + * @return the reporter type + */ @WithDefault("default") String type(); + + /** + * List of reporter targets to use when {@link #type()} is set to {@code composite}. Each target + * corresponds to a reporter type: {@code default}, {@code events}, or {@code persistence}. + * + *

    Example configuration: + * + *

    +   * polaris:
    +   *   iceberg-metrics:
    +   *     reporting:
    +   *       type: composite
    +   *       targets:
    +   *         - events
    +   *         - persistence
    +   * 
    + * + * @return list of reporter targets, empty if not using composite type + */ + default List targets() { + return List.of(); + } + + /** Configuration for metrics retention and cleanup. */ + RetentionConfig retention(); + + interface RetentionConfig { + /** + * Whether automatic cleanup of old metrics reports is enabled. Default is false (disabled). + * + * @return true if cleanup is enabled + */ + @WithDefault("false") + boolean enabled(); + + /** + * How long to retain metrics reports before they are eligible for cleanup. Default is 30 days. + * Supports ISO-8601 duration format (e.g., "P30D" for 30 days, "PT24H" for 24 hours). + * + * @return the retention period + */ + @WithDefault("P30D") + Duration retentionPeriod(); + + /** + * How often to run the cleanup job. Default is every 6 hours. Supports ISO-8601 duration + * format. + * + * @return the cleanup interval + */ + @WithDefault("PT6H") + Duration cleanupInterval(); + } } diff --git a/runtime/service/src/main/java/org/apache/polaris/service/reporting/PersistingMetricsReporter.java b/runtime/service/src/main/java/org/apache/polaris/service/reporting/PersistingMetricsReporter.java new file mode 100644 index 0000000000..136a84d045 --- /dev/null +++ b/runtime/service/src/main/java/org/apache/polaris/service/reporting/PersistingMetricsReporter.java @@ -0,0 +1,188 @@ +/* + * 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.polaris.service.reporting; + +import io.opentelemetry.api.trace.Span; +import io.opentelemetry.api.trace.SpanContext; +import io.quarkus.security.identity.SecurityIdentity; +import io.smallrye.common.annotation.Identifier; +import jakarta.enterprise.context.ApplicationScoped; +import jakarta.enterprise.inject.Instance; +import jakarta.inject.Inject; +import java.security.Principal; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.metrics.CommitReport; +import org.apache.iceberg.metrics.MetricsReport; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.polaris.core.context.RealmContext; +import org.apache.polaris.core.persistence.BasePersistence; +import org.apache.polaris.core.persistence.MetaStoreManagerFactory; +import org.apache.polaris.persistence.relational.jdbc.JdbcBasePersistenceImpl; +import org.apache.polaris.persistence.relational.jdbc.models.MetricsReportConverter; +import org.apache.polaris.persistence.relational.jdbc.models.ModelCommitMetricsReport; +import org.apache.polaris.persistence.relational.jdbc.models.ModelScanMetricsReport; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A metrics reporter that persists scan and commit reports as first-class entities in the database. + * This provides better queryability and analytics capabilities compared to storing metrics as + * generic events. + * + *

    To enable this reporter, set the configuration: + * + *

    + * polaris:
    + *   iceberg-metrics:
    + *     reporting:
    + *       type: persistence
    + * 
    + * + *

    Note: This reporter requires the relational-jdbc persistence backend. If a different + * persistence backend is configured, metrics will be logged but not persisted. + */ +@ApplicationScoped +@Identifier("persistence") +public class PersistingMetricsReporter implements PolarisMetricsReporter { + + private static final Logger LOGGER = LoggerFactory.getLogger(PersistingMetricsReporter.class); + + private final MetaStoreManagerFactory metaStoreManagerFactory; + private final RealmContext realmContext; + private final Instance securityIdentityInstance; + + @Inject + public PersistingMetricsReporter( + MetaStoreManagerFactory metaStoreManagerFactory, + RealmContext realmContext, + Instance securityIdentityInstance) { + this.metaStoreManagerFactory = metaStoreManagerFactory; + this.realmContext = realmContext; + this.securityIdentityInstance = securityIdentityInstance; + } + + @Override + public void reportMetric(String catalogName, TableIdentifier table, MetricsReport metricsReport) { + try { + String realmId = realmContext.getRealmIdentifier(); + String catalogId = catalogName; // Using catalog name as ID for now + String namespace = table.namespace().toString(); + + // Extract principal name from security context + String principalName = extractPrincipalName(); + String requestId = null; + + // Extract OpenTelemetry trace context from the current span + String otelTraceId = null; + String otelSpanId = null; + Span currentSpan = Span.current(); + if (currentSpan != null) { + SpanContext spanContext = currentSpan.getSpanContext(); + if (spanContext != null && spanContext.isValid()) { + otelTraceId = spanContext.getTraceId(); + otelSpanId = spanContext.getSpanId(); + LOGGER.trace( + "Captured OpenTelemetry context: traceId={}, spanId={}", otelTraceId, otelSpanId); + } + } + + // Get the persistence session for the current realm + BasePersistence session = metaStoreManagerFactory.getOrCreateSession(realmContext); + + // Check if the session is a JdbcBasePersistenceImpl (supports metrics persistence) + if (!(session instanceof JdbcBasePersistenceImpl jdbcPersistence)) { + LOGGER.warn( + "Metrics persistence is only supported with relational-jdbc backend. " + + "Current backend: {}. Logging metrics instead.", + session.getClass().getSimpleName()); + LOGGER.info("{}.{}: {}", catalogName, table, metricsReport); + return; + } + + if (metricsReport instanceof ScanReport scanReport) { + ModelScanMetricsReport modelReport = + MetricsReportConverter.fromScanReport( + scanReport, + realmId, + catalogId, + catalogName, + namespace, + principalName, + requestId, + otelTraceId, + otelSpanId); + jdbcPersistence.writeScanMetricsReport(modelReport); + LOGGER.debug( + "Persisted scan metrics report {} for table {}.{}", + modelReport.getReportId(), + catalogName, + table); + } else if (metricsReport instanceof CommitReport commitReport) { + ModelCommitMetricsReport modelReport = + MetricsReportConverter.fromCommitReport( + commitReport, + realmId, + catalogId, + catalogName, + namespace, + principalName, + requestId, + otelTraceId, + otelSpanId); + jdbcPersistence.writeCommitMetricsReport(modelReport); + LOGGER.debug( + "Persisted commit metrics report {} for table {}.{}", + modelReport.getReportId(), + catalogName, + table); + } else { + LOGGER.warn("Unknown metrics report type: {}", metricsReport.getClass().getName()); + } + } catch (Exception e) { + LOGGER.error( + "Failed to persist metrics report for table {}.{}: {}", + catalogName, + table, + e.getMessage(), + e); + } + } + + /** + * Extracts the principal name from the current security context. + * + * @return the principal name, or null if not available + */ + private String extractPrincipalName() { + try { + if (securityIdentityInstance.isResolvable()) { + SecurityIdentity identity = securityIdentityInstance.get(); + if (identity != null && !identity.isAnonymous()) { + Principal principal = identity.getPrincipal(); + if (principal != null) { + return principal.getName(); + } + } + } + } catch (Exception e) { + LOGGER.trace("Could not extract principal name from security context: {}", e.getMessage()); + } + return null; + } +} diff --git a/runtime/service/src/main/java/org/apache/polaris/service/reporting/PolarisMetricsReporter.java b/runtime/service/src/main/java/org/apache/polaris/service/reporting/PolarisMetricsReporter.java index 7ffd84f4d8..0acb9d8f4b 100644 --- a/runtime/service/src/main/java/org/apache/polaris/service/reporting/PolarisMetricsReporter.java +++ b/runtime/service/src/main/java/org/apache/polaris/service/reporting/PolarisMetricsReporter.java @@ -21,6 +21,33 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.metrics.MetricsReport; +/** + * Interface for reporting Iceberg metrics in Polaris. + * + *

    Implementations of this interface handle the persistence or forwarding of Iceberg metrics + * reports (such as {@link org.apache.iceberg.metrics.ScanReport} and {@link + * org.apache.iceberg.metrics.CommitReport}) to various backends. + * + *

    Available implementations: + * + *

      + *
    • {@code default} - Logs metrics to console only (no persistence) + *
    • {@code events} - Persists metrics to the events table as JSON + *
    • {@code persistence} - Persists metrics to dedicated tables (scan_metrics_report, + * commit_metrics_report) + *
    • {@code composite} - Delegates to multiple reporters based on configuration + *
    + * + * @see MetricsReportingConfiguration + */ public interface PolarisMetricsReporter { - public void reportMetric(String catalogName, TableIdentifier table, MetricsReport metricsReport); + + /** + * Reports a metrics event for a table operation. + * + * @param catalogName the name of the catalog containing the table + * @param table the identifier of the table the metrics are for + * @param metricsReport the Iceberg metrics report (ScanReport or CommitReport) + */ + void reportMetric(String catalogName, TableIdentifier table, MetricsReport metricsReport); } diff --git a/runtime/service/src/test/java/org/apache/polaris/service/events/listeners/PolarisPersistenceEventListenerTest.java b/runtime/service/src/test/java/org/apache/polaris/service/events/listeners/PolarisPersistenceEventListenerTest.java new file mode 100644 index 0000000000..972e018929 --- /dev/null +++ b/runtime/service/src/test/java/org/apache/polaris/service/events/listeners/PolarisPersistenceEventListenerTest.java @@ -0,0 +1,236 @@ +/* + * 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.polaris.service.events.listeners; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatCode; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import com.google.common.collect.ImmutableMap; +import java.time.Instant; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.metrics.CommitReport; +import org.apache.iceberg.metrics.ImmutableScanReport; +import org.apache.iceberg.metrics.ScanMetrics; +import org.apache.iceberg.metrics.ScanMetricsResult; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.iceberg.rest.requests.ReportMetricsRequest; +import org.apache.polaris.core.auth.PolarisPrincipal; +import org.apache.polaris.core.entity.PolarisEvent; +import org.apache.polaris.service.events.IcebergRestCatalogEvents; +import org.apache.polaris.service.events.PolarisEventMetadata; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +/** + * Unit tests for {@link PolarisPersistenceEventListener} focusing on null-safety in metrics + * extraction. + */ +class PolarisPersistenceEventListenerTest { + + private TestPolarisPersistenceEventListener listener; + + @BeforeEach + void setUp() { + listener = new TestPolarisPersistenceEventListener(); + } + + @Test + void testScanReportWithNullMetadataValues() { + // Use mocks to simulate a ScanReport with null values in metadata + // (Iceberg's ImmutableScanReport.Builder doesn't allow nulls, but JSON deserialization might) + Map metadataWithNull = new HashMap<>(); + metadataWithNull.put("trace-id", "valid-trace-id"); + metadataWithNull.put("null-value-key", null); + + ScanReport mockScanReport = mock(ScanReport.class); + when(mockScanReport.snapshotId()).thenReturn(123L); + when(mockScanReport.schemaId()).thenReturn(0); + when(mockScanReport.metadata()).thenReturn(metadataWithNull); + when(mockScanReport.scanMetrics()).thenReturn(null); + + ReportMetricsRequest mockRequest = mock(ReportMetricsRequest.class); + when(mockRequest.report()).thenReturn(mockScanReport); + + IcebergRestCatalogEvents.AfterReportMetricsEvent event = + createAfterReportMetricsEvent(mockRequest); + + // Should not throw NPE + assertThatCode(() -> listener.onAfterReportMetrics(event)).doesNotThrowAnyException(); + + // Verify the valid metadata entry was captured, nulls were skipped + PolarisEvent persistedEvent = listener.getLastEvent(); + assertThat(persistedEvent).isNotNull(); + Map additionalProps = persistedEvent.getAdditionalPropertiesAsMap(); + assertThat(additionalProps).containsEntry("report.trace-id", "valid-trace-id"); + assertThat(additionalProps).doesNotContainKey("report.null-value-key"); + } + + @Test + void testCommitReportWithNullOperation() { + // Use mock to simulate a CommitReport with null operation + CommitReport mockCommitReport = mock(CommitReport.class); + when(mockCommitReport.snapshotId()).thenReturn(456L); + when(mockCommitReport.sequenceNumber()).thenReturn(1L); + when(mockCommitReport.operation()).thenReturn(null); // null operation + when(mockCommitReport.metadata()).thenReturn(ImmutableMap.of()); + when(mockCommitReport.commitMetrics()).thenReturn(null); + + ReportMetricsRequest mockRequest = mock(ReportMetricsRequest.class); + when(mockRequest.report()).thenReturn(mockCommitReport); + + IcebergRestCatalogEvents.AfterReportMetricsEvent event = + createAfterReportMetricsEvent(mockRequest); + + // Should not throw NPE + assertThatCode(() -> listener.onAfterReportMetrics(event)).doesNotThrowAnyException(); + + // Verify operation is not in additional properties (since it was null) + PolarisEvent persistedEvent = listener.getLastEvent(); + assertThat(persistedEvent).isNotNull(); + Map additionalProps = persistedEvent.getAdditionalPropertiesAsMap(); + assertThat(additionalProps) + .containsEntry("report_type", "commit") + .containsEntry("snapshot_id", "456") + .doesNotContainKey("operation"); + } + + @Test + void testCommitReportWithNullMetadataValues() { + // Use mock to simulate a CommitReport with null values in metadata + Map metadataWithNull = new HashMap<>(); + metadataWithNull.put("trace-id", "commit-trace-id"); + metadataWithNull.put("null-value-key", null); + + CommitReport mockCommitReport = mock(CommitReport.class); + when(mockCommitReport.snapshotId()).thenReturn(789L); + when(mockCommitReport.sequenceNumber()).thenReturn(2L); + when(mockCommitReport.operation()).thenReturn("append"); + when(mockCommitReport.metadata()).thenReturn(metadataWithNull); + when(mockCommitReport.commitMetrics()).thenReturn(null); + + ReportMetricsRequest mockRequest = mock(ReportMetricsRequest.class); + when(mockRequest.report()).thenReturn(mockCommitReport); + + IcebergRestCatalogEvents.AfterReportMetricsEvent event = + createAfterReportMetricsEvent(mockRequest); + + // Should not throw NPE + assertThatCode(() -> listener.onAfterReportMetrics(event)).doesNotThrowAnyException(); + + // Verify valid entries are captured, nulls are skipped + PolarisEvent persistedEvent = listener.getLastEvent(); + assertThat(persistedEvent).isNotNull(); + Map additionalProps = persistedEvent.getAdditionalPropertiesAsMap(); + assertThat(additionalProps) + .containsEntry("report.trace-id", "commit-trace-id") + .containsEntry("operation", "append") + .doesNotContainKey("report.null-value-key"); + } + + @Test + void testScanReportWithEmptyMetadata() { + ImmutableScanReport scanReport = + ImmutableScanReport.builder() + .schemaId(0) + .tableName("test_ns.test_table") + .snapshotId(100L) + .addProjectedFieldIds(1) + .addProjectedFieldNames("id") + .filter(Expressions.alwaysTrue()) + .scanMetrics(ScanMetricsResult.fromScanMetrics(ScanMetrics.noop())) + // Empty metadata map + .build(); + + ReportMetricsRequest request = ReportMetricsRequest.of(scanReport); + IcebergRestCatalogEvents.AfterReportMetricsEvent event = createAfterReportMetricsEvent(request); + + // Should not throw any exception + assertThatCode(() -> listener.onAfterReportMetrics(event)).doesNotThrowAnyException(); + + PolarisEvent persistedEvent = listener.getLastEvent(); + assertThat(persistedEvent).isNotNull(); + Map additionalProps = persistedEvent.getAdditionalPropertiesAsMap(); + assertThat(additionalProps) + .containsEntry("report_type", "scan") + .containsEntry("snapshot_id", "100"); + } + + private IcebergRestCatalogEvents.AfterReportMetricsEvent createAfterReportMetricsEvent( + ReportMetricsRequest request) { + PolarisEventMetadata metadata = + new PolarisEventMetadata() { + @Override + public UUID eventId() { + return UUID.randomUUID(); + } + + @Override + public Instant timestamp() { + return Instant.now(); + } + + @Override + public Optional user() { + return Optional.empty(); + } + + @Override + public Optional requestId() { + return Optional.of("test-request-id"); + } + + @Override + public Map openTelemetryContext() { + return ImmutableMap.of(); + } + + @Override + public String realmId() { + return "test-realm"; + } + }; + + return new IcebergRestCatalogEvents.AfterReportMetricsEvent( + metadata, "test-catalog", Namespace.of("test_ns"), "test_table", request); + } + + /** Concrete test implementation that captures persisted events for verification. */ + private static class TestPolarisPersistenceEventListener extends PolarisPersistenceEventListener { + private final Map events = new ConcurrentHashMap<>(); + private PolarisEvent lastEvent; + + @Override + protected void processEvent(String realmId, PolarisEvent event) { + events.put(event.getId(), event); + lastEvent = event; + } + + public PolarisEvent getLastEvent() { + return lastEvent; + } + } +} diff --git a/runtime/service/src/test/java/org/apache/polaris/service/events/listeners/inmemory/InMemoryBufferEventListenerIntegrationTest.java b/runtime/service/src/test/java/org/apache/polaris/service/events/listeners/inmemory/InMemoryBufferEventListenerIntegrationTest.java index 2d8e0ec57f..380d99b033 100644 --- a/runtime/service/src/test/java/org/apache/polaris/service/events/listeners/inmemory/InMemoryBufferEventListenerIntegrationTest.java +++ b/runtime/service/src/test/java/org/apache/polaris/service/events/listeners/inmemory/InMemoryBufferEventListenerIntegrationTest.java @@ -49,8 +49,14 @@ import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.SessionCatalog; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.metrics.ImmutableScanReport; +import org.apache.iceberg.metrics.ScanMetrics; +import org.apache.iceberg.metrics.ScanMetricsResult; +import org.apache.iceberg.metrics.ScanReport; import org.apache.iceberg.rest.RESTSessionCatalog; import org.apache.iceberg.rest.auth.OAuth2Properties; +import org.apache.iceberg.rest.requests.ReportMetricsRequest; import org.apache.iceberg.types.Types; import org.apache.polaris.core.admin.model.Catalog; import org.apache.polaris.core.admin.model.CatalogProperties; @@ -65,6 +71,7 @@ import org.apache.polaris.service.it.env.RestApi; import org.apache.polaris.service.it.ext.PolarisIntegrationTestExtension; import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.extension.ExtendWith; @@ -92,6 +99,7 @@ public Map getConfigOverrides() { .put("polaris.event-listener.persistence-in-memory-buffer.buffer-time", "100ms") .put("polaris.features.\"ALLOW_INSECURE_STORAGE_TYPES\"", "true") .put("polaris.features.\"SUPPORTED_CATALOG_STORAGE_TYPES\"", "[\"FILE\",\"S3\"]") + .put("polaris.features.\"ALLOW_OVERLAPPING_CATALOG_URLS\"", "true") .put("polaris.readiness.ignore-severe-issues", "true") .build(); } @@ -117,10 +125,32 @@ public void setup( baseLocation = IntegrationTestsHelper.getTemporaryDirectory(tempDir).resolve(realm + "/"); } + /** + * Reset the database state before each test to ensure test isolation. The H2 in-memory database + * with DB_CLOSE_DELAY=-1 persists state across tests, so we need to clean up catalog-related + * entities while preserving the realm and principal entities set up in @BeforeAll. + */ + @BeforeEach + public void resetDatabaseState() { + if (dataSource.isResolvable()) { + try (Connection conn = dataSource.get().getConnection(); + Statement stmt = conn.createStatement()) { + // Set the schema first + stmt.execute("SET SCHEMA POLARIS_SCHEMA"); + // Only delete events - catalogs use unique names and locations so they don't conflict + stmt.execute("DELETE FROM EVENTS"); + } catch (Exception e) { + // Ignore errors - tables may not exist yet on first run + } + } + } + @Test void testCreateCatalogAndTable() throws IOException { String catalogName = client.newEntityName("testCreateCatalogAndTable"); + // Use a unique base location for this catalog to avoid overlap with other catalogs + URI catalogBaseLocation = baseLocation.resolve(catalogName + "/"); Catalog catalog = PolarisCatalog.builder() @@ -130,7 +160,7 @@ void testCreateCatalogAndTable() throws IOException { .setStorageConfigInfo( FileStorageConfigInfo.builder() .setStorageType(StorageConfigInfo.StorageTypeEnum.FILE) - .setAllowedLocations(List.of(baseLocation.toString())) + .setAllowedLocations(List.of(catalogBaseLocation.toString())) .build()) .build(); @@ -221,4 +251,351 @@ void testCreateCatalogAndTable() throws IOException { .hasEntrySatisfying("otel.trace_id", value -> assertThat(value).matches("[0-9a-f]{32}")) .hasEntrySatisfying("otel.span_id", value -> assertThat(value).matches("[0-9a-f]{16}")); } + + /** + * Tests that reportMetrics events are emitted with proper trace context for correlation. This + * verifies that compute engine metrics reports can be correlated with other catalog operations + * via the OpenTelemetry trace_id. + */ + @Test + void testReportMetricsEventWithTraceContext() throws IOException { + String catalogName = client.newEntityName("testReportMetrics"); + // Use a unique base location for this catalog to avoid overlap with other catalogs + URI catalogBaseLocation = baseLocation.resolve(catalogName + "/"); + + Catalog catalog = + PolarisCatalog.builder() + .setName(catalogName) + .setType(Catalog.TypeEnum.INTERNAL) + .setProperties(CatalogProperties.builder("file:///tmp/").build()) + .setStorageConfigInfo( + FileStorageConfigInfo.builder() + .setStorageType(StorageConfigInfo.StorageTypeEnum.FILE) + .setAllowedLocations(List.of(catalogBaseLocation.toString())) + .build()) + .build(); + + try (Response response = + managementApi + .request("v1/catalogs") + .header("X-Request-ID", "metrics-test") + .post(Entity.json(catalog))) { + assertThat(response).returns(Response.Status.CREATED.getStatusCode(), Response::getStatus); + } + + // Create a table first + try (RESTSessionCatalog sessionCatalog = new RESTSessionCatalog()) { + sessionCatalog.initialize( + "polaris_catalog_metrics_test", + ImmutableMap.builder() + .put("uri", endpoints.catalogApiEndpoint().toString()) + .put(OAuth2Properties.TOKEN, authToken) + .put("warehouse", catalogName) + .putAll(endpoints.extraHeaders("header.")) + .build()); + + SessionCatalog.SessionContext sessionContext = SessionCatalog.SessionContext.createEmpty(); + Namespace ns = Namespace.of("metrics_ns"); + sessionCatalog.createNamespace(sessionContext, ns); + + sessionCatalog + .buildTable( + sessionContext, + TableIdentifier.of(ns, "metrics_table"), + new Schema(List.of(Types.NestedField.required(1, "id", Types.IntegerType.get())))) + .withSortOrder(SortOrder.unsorted()) + .withPartitionSpec(PartitionSpec.unpartitioned()) + .create(); + } + + // Now send a metrics report via the REST API + // Build a minimal ScanReport for testing + ScanReport scanReport = + ImmutableScanReport.builder() + .schemaId(0) + .tableName("metrics_ns.metrics_table") + .snapshotId(-1L) + .addProjectedFieldIds(1) + .addProjectedFieldNames("id") + .filter(Expressions.alwaysTrue()) + .scanMetrics(ScanMetricsResult.fromScanMetrics(ScanMetrics.noop())) + .build(); + + ReportMetricsRequest metricsRequest = ReportMetricsRequest.of(scanReport); + + RestApi catalogApi = client.catalogApi(authToken); + try (Response response = + catalogApi + .request("v1/" + catalogName + "/namespaces/metrics_ns/tables/metrics_table/metrics") + .header("X-Request-ID", "metrics-report-123") + .post(Entity.json(metricsRequest))) { + assertThat(response).returns(204, Response::getStatus); + } + + // Query for the AfterReportMetricsEvent + String query = + "SELECT * FROM polaris_schema.events WHERE realm_id = '" + + realm + + "' AND event_type = 'AfterReportMetricsEvent' ORDER BY timestamp_ms DESC"; + + List metricsEvents = + await() + .atMost(Duration.ofSeconds(10)) + .until( + () -> { + ImmutableList.Builder e = ImmutableList.builder(); + try (Connection connection = dataSource.get().getConnection(); + Statement statement = connection.createStatement(); + ResultSet rs = statement.executeQuery(query)) { + while (rs.next()) { + PolarisEvent event = CONVERTER.fromResultSet(rs); + e.add(event); + } + } + return e.build(); + }, + e -> !e.isEmpty()); + + PolarisEvent metricsEvent = metricsEvents.getFirst(); + assertThat(metricsEvent.getCatalogId()).isEqualTo(catalogName); + assertThat(metricsEvent.getResourceType()).isEqualTo(PolarisEvent.ResourceType.TABLE); + assertThat(metricsEvent.getResourceIdentifier()).isEqualTo("metrics_ns.metrics_table"); + assertThat(metricsEvent.getEventType()).isEqualTo("AfterReportMetricsEvent"); + assertThat(metricsEvent.getPrincipalName()).isEqualTo("root"); + assertThat(metricsEvent.getRequestId()).isEqualTo("metrics-report-123"); + + // Verify OpenTelemetry trace context is present for correlation + assertThat(metricsEvent.getAdditionalPropertiesAsMap()) + .containsEntry("otel.trace_flags", "01") + .containsEntry("otel.sampled", "true") + .hasEntrySatisfying("otel.trace_id", value -> assertThat(value).matches("[0-9a-f]{32}")) + .hasEntrySatisfying("otel.span_id", value -> assertThat(value).matches("[0-9a-f]{16}")); + } + + /** + * Tests that ScanReport with trace-id in metadata is properly extracted and stored. This verifies + * that compute engines can pass trace context in the report's metadata map for correlation. + */ + @Test + void testReportMetricsWithTraceIdInMetadata() throws IOException { + String catalogName = client.newEntityName("testMetricsTraceId"); + // Use a unique base location for this catalog to avoid overlap with other catalogs + URI catalogBaseLocation = baseLocation.resolve(catalogName + "/"); + + Catalog catalog = + PolarisCatalog.builder() + .setName(catalogName) + .setType(Catalog.TypeEnum.INTERNAL) + .setProperties(CatalogProperties.builder("file:///tmp/").build()) + .setStorageConfigInfo( + FileStorageConfigInfo.builder() + .setStorageType(StorageConfigInfo.StorageTypeEnum.FILE) + .setAllowedLocations(List.of(catalogBaseLocation.toString())) + .build()) + .build(); + + try (Response response = managementApi.request("v1/catalogs").post(Entity.json(catalog))) { + assertThat(response).returns(Response.Status.CREATED.getStatusCode(), Response::getStatus); + } + + // Create a table first + try (RESTSessionCatalog sessionCatalog = new RESTSessionCatalog()) { + sessionCatalog.initialize( + "polaris_catalog_trace_test", + ImmutableMap.builder() + .put("uri", endpoints.catalogApiEndpoint().toString()) + .put(OAuth2Properties.TOKEN, authToken) + .put("warehouse", catalogName) + .putAll(endpoints.extraHeaders("header.")) + .build()); + + SessionCatalog.SessionContext sessionContext = SessionCatalog.SessionContext.createEmpty(); + Namespace ns = Namespace.of("trace_ns"); + sessionCatalog.createNamespace(sessionContext, ns); + + sessionCatalog + .buildTable( + sessionContext, + TableIdentifier.of(ns, "trace_table"), + new Schema(List.of(Types.NestedField.required(1, "id", Types.IntegerType.get())))) + .withSortOrder(SortOrder.unsorted()) + .withPartitionSpec(PartitionSpec.unpartitioned()) + .create(); + } + + // Build a ScanReport with trace-id in metadata (as compute engines would do) + String clientTraceId = "abcdef1234567890abcdef1234567890"; + ScanReport scanReport = + ImmutableScanReport.builder() + .schemaId(0) + .tableName("trace_ns.trace_table") + .snapshotId(123L) + .addProjectedFieldIds(1) + .addProjectedFieldNames("id") + .filter(Expressions.alwaysTrue()) + .scanMetrics(ScanMetricsResult.fromScanMetrics(ScanMetrics.noop())) + .putMetadata("trace-id", clientTraceId) + .putMetadata("client-app", "spark-test") + .build(); + + ReportMetricsRequest metricsRequest = ReportMetricsRequest.of(scanReport); + + RestApi catalogApi = client.catalogApi(authToken); + try (Response response = + catalogApi + .request("v1/" + catalogName + "/namespaces/trace_ns/tables/trace_table/metrics") + .header("X-Request-ID", "trace-test-456") + .post(Entity.json(metricsRequest))) { + assertThat(response).returns(204, Response::getStatus); + } + + // Query for the AfterReportMetricsEvent + String query = + "SELECT * FROM polaris_schema.events WHERE realm_id = '" + + realm + + "' AND event_type = 'AfterReportMetricsEvent' AND request_id = 'trace-test-456'"; + + List metricsEvents = + await() + .atMost(Duration.ofSeconds(10)) + .until( + () -> { + ImmutableList.Builder e = ImmutableList.builder(); + try (Connection connection = dataSource.get().getConnection(); + Statement statement = connection.createStatement(); + ResultSet rs = statement.executeQuery(query)) { + while (rs.next()) { + PolarisEvent event = CONVERTER.fromResultSet(rs); + e.add(event); + } + } + return e.build(); + }, + e -> !e.isEmpty()); + + PolarisEvent metricsEvent = metricsEvents.getFirst(); + assertThat(metricsEvent.getEventType()).isEqualTo("AfterReportMetricsEvent"); + + // Verify trace-id from report metadata is extracted with "report." prefix + Map additionalProps = metricsEvent.getAdditionalPropertiesAsMap(); + assertThat(additionalProps) + .containsEntry("report.trace-id", clientTraceId) + .containsEntry("report.client-app", "spark-test") + .containsEntry("report_type", "scan") + .containsEntry("snapshot_id", "123") + .containsEntry("schema_id", "0"); + } + + /** + * Tests that CommitReport metrics are properly extracted and stored. This verifies the commit + * metrics path including operation type, sequence number, and commit metrics. + */ + @Test + void testReportCommitMetrics() throws IOException { + String catalogName = client.newEntityName("testCommitMetrics"); + // Use a unique base location for this catalog to avoid overlap with other catalogs + URI catalogBaseLocation = baseLocation.resolve(catalogName + "/"); + + Catalog catalog = + PolarisCatalog.builder() + .setName(catalogName) + .setType(Catalog.TypeEnum.INTERNAL) + .setProperties(CatalogProperties.builder("file:///tmp/").build()) + .setStorageConfigInfo( + FileStorageConfigInfo.builder() + .setStorageType(StorageConfigInfo.StorageTypeEnum.FILE) + .setAllowedLocations(List.of(catalogBaseLocation.toString())) + .build()) + .build(); + + try (Response response = managementApi.request("v1/catalogs").post(Entity.json(catalog))) { + assertThat(response).returns(Response.Status.CREATED.getStatusCode(), Response::getStatus); + } + + // Create a table first + try (RESTSessionCatalog sessionCatalog = new RESTSessionCatalog()) { + sessionCatalog.initialize( + "polaris_catalog_commit_test", + ImmutableMap.builder() + .put("uri", endpoints.catalogApiEndpoint().toString()) + .put(OAuth2Properties.TOKEN, authToken) + .put("warehouse", catalogName) + .putAll(endpoints.extraHeaders("header.")) + .build()); + + SessionCatalog.SessionContext sessionContext = SessionCatalog.SessionContext.createEmpty(); + Namespace ns = Namespace.of("commit_ns"); + sessionCatalog.createNamespace(sessionContext, ns); + + sessionCatalog + .buildTable( + sessionContext, + TableIdentifier.of(ns, "commit_table"), + new Schema(List.of(Types.NestedField.required(1, "id", Types.IntegerType.get())))) + .withSortOrder(SortOrder.unsorted()) + .withPartitionSpec(PartitionSpec.unpartitioned()) + .create(); + } + + // Build a CommitReport + org.apache.iceberg.metrics.CommitReport commitReport = + org.apache.iceberg.metrics.ImmutableCommitReport.builder() + .tableName("commit_ns.commit_table") + .snapshotId(456L) + .sequenceNumber(1L) + .operation("append") + .commitMetrics( + org.apache.iceberg.metrics.CommitMetricsResult.from( + org.apache.iceberg.metrics.CommitMetrics.noop(), ImmutableMap.of())) + .putMetadata("trace-id", "commit-trace-123") + .build(); + + ReportMetricsRequest metricsRequest = ReportMetricsRequest.of(commitReport); + + RestApi catalogApi = client.catalogApi(authToken); + try (Response response = + catalogApi + .request("v1/" + catalogName + "/namespaces/commit_ns/tables/commit_table/metrics") + .header("X-Request-ID", "commit-test-789") + .post(Entity.json(metricsRequest))) { + assertThat(response).returns(204, Response::getStatus); + } + + // Query for the AfterReportMetricsEvent + String query = + "SELECT * FROM polaris_schema.events WHERE realm_id = '" + + realm + + "' AND event_type = 'AfterReportMetricsEvent' AND request_id = 'commit-test-789'"; + + List metricsEvents = + await() + .atMost(Duration.ofSeconds(10)) + .until( + () -> { + ImmutableList.Builder e = ImmutableList.builder(); + try (Connection connection = dataSource.get().getConnection(); + Statement statement = connection.createStatement(); + ResultSet rs = statement.executeQuery(query)) { + while (rs.next()) { + PolarisEvent event = CONVERTER.fromResultSet(rs); + e.add(event); + } + } + return e.build(); + }, + e -> !e.isEmpty()); + + PolarisEvent metricsEvent = metricsEvents.getFirst(); + assertThat(metricsEvent.getEventType()).isEqualTo("AfterReportMetricsEvent"); + assertThat(metricsEvent.getResourceIdentifier()).isEqualTo("commit_ns.commit_table"); + + // Verify commit report data is extracted + Map additionalProps = metricsEvent.getAdditionalPropertiesAsMap(); + assertThat(additionalProps) + .containsEntry("report_type", "commit") + .containsEntry("snapshot_id", "456") + .containsEntry("sequence_number", "1") + .containsEntry("operation", "append") + .containsEntry("report.trace-id", "commit-trace-123"); + } } diff --git a/runtime/service/src/test/java/org/apache/polaris/service/reporting/CompositeMetricsReporterTest.java b/runtime/service/src/test/java/org/apache/polaris/service/reporting/CompositeMetricsReporterTest.java new file mode 100644 index 0000000000..a4e67248be --- /dev/null +++ b/runtime/service/src/test/java/org/apache/polaris/service/reporting/CompositeMetricsReporterTest.java @@ -0,0 +1,133 @@ +/* + * 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.polaris.service.reporting; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; + +import java.util.List; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.metrics.ScanReport; +import org.junit.jupiter.api.Test; + +class CompositeMetricsReporterTest { + + @Test + void testDelegatesToAllReporters() { + PolarisMetricsReporter reporter1 = mock(PolarisMetricsReporter.class); + PolarisMetricsReporter reporter2 = mock(PolarisMetricsReporter.class); + PolarisMetricsReporter reporter3 = mock(PolarisMetricsReporter.class); + + CompositeMetricsReporter composite = + new CompositeMetricsReporter(List.of(reporter1, reporter2, reporter3)); + + ScanReport scanReport = mock(ScanReport.class); + TableIdentifier table = TableIdentifier.of("db", "test_table"); + + composite.reportMetric("test-catalog", table, scanReport); + + verify(reporter1).reportMetric("test-catalog", table, scanReport); + verify(reporter2).reportMetric("test-catalog", table, scanReport); + verify(reporter3).reportMetric("test-catalog", table, scanReport); + } + + @Test + void testContinuesOnDelegateFailure() { + PolarisMetricsReporter reporter1 = mock(PolarisMetricsReporter.class); + PolarisMetricsReporter reporter2 = mock(PolarisMetricsReporter.class); + PolarisMetricsReporter reporter3 = mock(PolarisMetricsReporter.class); + + // Make reporter2 throw an exception + doThrow(new RuntimeException("Reporter 2 failed")) + .when(reporter2) + .reportMetric(any(), any(), any()); + + CompositeMetricsReporter composite = + new CompositeMetricsReporter(List.of(reporter1, reporter2, reporter3)); + + ScanReport scanReport = mock(ScanReport.class); + TableIdentifier table = TableIdentifier.of("db", "test_table"); + + // Should not throw + composite.reportMetric("test-catalog", table, scanReport); + + // All reporters should still be called + verify(reporter1).reportMetric("test-catalog", table, scanReport); + verify(reporter2).reportMetric("test-catalog", table, scanReport); + verify(reporter3).reportMetric("test-catalog", table, scanReport); + } + + @Test + void testEmptyDelegatesList() { + CompositeMetricsReporter composite = new CompositeMetricsReporter(List.of()); + + ScanReport scanReport = mock(ScanReport.class); + TableIdentifier table = TableIdentifier.of("db", "test_table"); + + // Should not throw + composite.reportMetric("test-catalog", table, scanReport); + + assertThat(composite.getDelegates()).isEmpty(); + } + + @Test + void testSingleDelegate() { + PolarisMetricsReporter reporter = mock(PolarisMetricsReporter.class); + CompositeMetricsReporter composite = new CompositeMetricsReporter(List.of(reporter)); + + ScanReport scanReport = mock(ScanReport.class); + TableIdentifier table = TableIdentifier.of("db", "test_table"); + + composite.reportMetric("test-catalog", table, scanReport); + + verify(reporter).reportMetric("test-catalog", table, scanReport); + assertThat(composite.getDelegates()).hasSize(1); + } + + @Test + void testGetDelegatesReturnsUnmodifiableList() { + PolarisMetricsReporter reporter = mock(PolarisMetricsReporter.class); + CompositeMetricsReporter composite = new CompositeMetricsReporter(List.of(reporter)); + + List delegates = composite.getDelegates(); + + // Should be unmodifiable + assertThat(delegates).hasSize(1); + org.junit.jupiter.api.Assertions.assertThrows( + UnsupportedOperationException.class, + () -> delegates.add(mock(PolarisMetricsReporter.class))); + } + + @Test + void testNullMetricsReportDoesNotThrow() { + PolarisMetricsReporter reporter = mock(PolarisMetricsReporter.class); + CompositeMetricsReporter composite = new CompositeMetricsReporter(List.of(reporter)); + + TableIdentifier table = TableIdentifier.of("db", "test_table"); + + // Should not throw even with null report + composite.reportMetric("test-catalog", table, null); + + verify(reporter).reportMetric(eq("test-catalog"), eq(table), eq(null)); + } +} diff --git a/runtime/service/src/test/java/org/apache/polaris/service/reporting/EventsMetricsReporterTest.java b/runtime/service/src/test/java/org/apache/polaris/service/reporting/EventsMetricsReporterTest.java new file mode 100644 index 0000000000..c36d8a76c9 --- /dev/null +++ b/runtime/service/src/test/java/org/apache/polaris/service/reporting/EventsMetricsReporterTest.java @@ -0,0 +1,156 @@ +/* + * 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.polaris.service.reporting; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.quarkus.security.identity.SecurityIdentity; +import jakarta.enterprise.inject.Instance; +import java.util.List; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.metrics.CommitReport; +import org.apache.iceberg.metrics.MetricsReport; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.polaris.core.context.RealmContext; +import org.apache.polaris.core.entity.PolarisEvent; +import org.apache.polaris.core.persistence.BasePersistence; +import org.apache.polaris.core.persistence.MetaStoreManagerFactory; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; + +class EventsMetricsReporterTest { + + private MetaStoreManagerFactory metaStoreManagerFactory; + private RealmContext realmContext; + private BasePersistence persistence; + private ObjectMapper objectMapper; + + @SuppressWarnings("unchecked") + private Instance securityIdentityInstance = mock(Instance.class); + + private EventsMetricsReporter reporter; + + @BeforeEach + void setUp() { + metaStoreManagerFactory = mock(MetaStoreManagerFactory.class); + realmContext = mock(RealmContext.class); + persistence = mock(BasePersistence.class); + objectMapper = new ObjectMapper(); + + when(realmContext.getRealmIdentifier()).thenReturn("test-realm"); + when(metaStoreManagerFactory.getOrCreateSession(any())).thenReturn(persistence); + when(securityIdentityInstance.isResolvable()).thenReturn(false); + + reporter = + new EventsMetricsReporter( + metaStoreManagerFactory, realmContext, objectMapper, securityIdentityInstance); + } + + @Test + void testReportScanMetrics() { + ScanReport scanReport = mock(ScanReport.class); + TableIdentifier table = TableIdentifier.of("db", "test_table"); + + reporter.reportMetric("test-catalog", table, scanReport); + + @SuppressWarnings("unchecked") + ArgumentCaptor> captor = ArgumentCaptor.forClass(List.class); + verify(persistence).writeEvents(captor.capture()); + + List events = captor.getValue(); + assertThat(events).hasSize(1); + + PolarisEvent event = events.get(0); + assertThat(event.getEventType()).isEqualTo(EventsMetricsReporter.EVENT_TYPE_SCAN_REPORT); + assertThat(event.getCatalogId()).isEqualTo("test-catalog"); + assertThat(event.getResourceType()).isEqualTo(PolarisEvent.ResourceType.TABLE); + assertThat(event.getResourceIdentifier()).isEqualTo("db.test_table"); + } + + @Test + void testReportCommitMetrics() { + CommitReport commitReport = mock(CommitReport.class); + TableIdentifier table = TableIdentifier.of("db", "test_table"); + + reporter.reportMetric("test-catalog", table, commitReport); + + @SuppressWarnings("unchecked") + ArgumentCaptor> captor = ArgumentCaptor.forClass(List.class); + verify(persistence).writeEvents(captor.capture()); + + List events = captor.getValue(); + assertThat(events).hasSize(1); + + PolarisEvent event = events.get(0); + assertThat(event.getEventType()).isEqualTo(EventsMetricsReporter.EVENT_TYPE_COMMIT_REPORT); + assertThat(event.getCatalogId()).isEqualTo("test-catalog"); + assertThat(event.getResourceType()).isEqualTo(PolarisEvent.ResourceType.TABLE); + assertThat(event.getResourceIdentifier()).isEqualTo("db.test_table"); + } + + @Test + void testUnknownMetricsReportTypeIsIgnored() { + MetricsReport unknownReport = mock(MetricsReport.class); + TableIdentifier table = TableIdentifier.of("db", "test_table"); + + reporter.reportMetric("test-catalog", table, unknownReport); + + verify(persistence, never()).writeEvents(any()); + } + + @Test + void testEventContainsSerializedMetrics() { + // Create a mock ScanReport + ScanReport scanReport = mock(ScanReport.class); + when(scanReport.tableName()).thenReturn("test_table"); + when(scanReport.snapshotId()).thenReturn(12345L); + + TableIdentifier table = TableIdentifier.of("db", "test_table"); + + reporter.reportMetric("test-catalog", table, scanReport); + + @SuppressWarnings("unchecked") + ArgumentCaptor> captor = ArgumentCaptor.forClass(List.class); + verify(persistence).writeEvents(captor.capture()); + + PolarisEvent event = captor.getValue().get(0); + String additionalProps = event.getAdditionalProperties(); + // Should contain JSON (at minimum an empty object or serialized report) + assertThat(additionalProps).isNotNull(); + } + + @Test + void testPersistenceErrorDoesNotThrow() { + ScanReport scanReport = mock(ScanReport.class); + TableIdentifier table = TableIdentifier.of("db", "test_table"); + + when(metaStoreManagerFactory.getOrCreateSession(any())) + .thenThrow(new RuntimeException("Database error")); + + // Should not throw + reporter.reportMetric("test-catalog", table, scanReport); + } +} diff --git a/runtime/service/src/test/java/org/apache/polaris/service/reporting/PersistingMetricsReporterTest.java b/runtime/service/src/test/java/org/apache/polaris/service/reporting/PersistingMetricsReporterTest.java new file mode 100644 index 0000000000..64740c5bfa --- /dev/null +++ b/runtime/service/src/test/java/org/apache/polaris/service/reporting/PersistingMetricsReporterTest.java @@ -0,0 +1,164 @@ +/* + * 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.polaris.service.reporting; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import io.quarkus.security.identity.SecurityIdentity; +import jakarta.enterprise.inject.Instance; +import java.security.Principal; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.metrics.CommitReport; +import org.apache.iceberg.metrics.MetricsReport; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.polaris.core.context.RealmContext; +import org.apache.polaris.core.persistence.BasePersistence; +import org.apache.polaris.core.persistence.MetaStoreManagerFactory; +import org.apache.polaris.persistence.relational.jdbc.JdbcBasePersistenceImpl; +import org.apache.polaris.persistence.relational.jdbc.models.ModelCommitMetricsReport; +import org.apache.polaris.persistence.relational.jdbc.models.ModelScanMetricsReport; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; + +class PersistingMetricsReporterTest { + + private MetaStoreManagerFactory metaStoreManagerFactory; + private RealmContext realmContext; + private JdbcBasePersistenceImpl jdbcPersistence; + private BasePersistence nonJdbcPersistence; + + @SuppressWarnings("unchecked") + private Instance securityIdentityInstance = mock(Instance.class); + + private PersistingMetricsReporter reporter; + + @BeforeEach + void setUp() { + metaStoreManagerFactory = mock(MetaStoreManagerFactory.class); + realmContext = mock(RealmContext.class); + jdbcPersistence = mock(JdbcBasePersistenceImpl.class); + nonJdbcPersistence = mock(BasePersistence.class); + + when(realmContext.getRealmIdentifier()).thenReturn("test-realm"); + when(securityIdentityInstance.isResolvable()).thenReturn(false); + + reporter = + new PersistingMetricsReporter( + metaStoreManagerFactory, realmContext, securityIdentityInstance); + } + + @Test + void testReportScanMetricsWithJdbcBackend() { + when(metaStoreManagerFactory.getOrCreateSession(any())).thenReturn(jdbcPersistence); + + ScanReport scanReport = mock(ScanReport.class); + when(scanReport.tableName()).thenReturn("test_table"); + TableIdentifier table = TableIdentifier.of("db", "test_table"); + + reporter.reportMetric("test-catalog", table, scanReport); + + verify(jdbcPersistence).writeScanMetricsReport(any(ModelScanMetricsReport.class)); + } + + @Test + void testReportCommitMetricsWithJdbcBackend() { + when(metaStoreManagerFactory.getOrCreateSession(any())).thenReturn(jdbcPersistence); + + CommitReport commitReport = mock(CommitReport.class); + when(commitReport.tableName()).thenReturn("test_table"); + when(commitReport.snapshotId()).thenReturn(12345L); + when(commitReport.operation()).thenReturn("append"); + TableIdentifier table = TableIdentifier.of("db", "test_table"); + + reporter.reportMetric("test-catalog", table, commitReport); + + verify(jdbcPersistence).writeCommitMetricsReport(any(ModelCommitMetricsReport.class)); + } + + @Test + void testFallbackToLoggingWithNonJdbcBackend() { + when(metaStoreManagerFactory.getOrCreateSession(any())).thenReturn(nonJdbcPersistence); + + ScanReport scanReport = mock(ScanReport.class); + TableIdentifier table = TableIdentifier.of("db", "test_table"); + + // Should not throw, just log + reporter.reportMetric("test-catalog", table, scanReport); + + // Verify no JDBC methods were called + verify(jdbcPersistence, never()).writeScanMetricsReport(any()); + } + + @Test + void testUnknownMetricsReportTypeIsIgnored() { + when(metaStoreManagerFactory.getOrCreateSession(any())).thenReturn(jdbcPersistence); + + MetricsReport unknownReport = mock(MetricsReport.class); + TableIdentifier table = TableIdentifier.of("db", "test_table"); + + reporter.reportMetric("test-catalog", table, unknownReport); + + verify(jdbcPersistence, never()).writeScanMetricsReport(any()); + verify(jdbcPersistence, never()).writeCommitMetricsReport(any()); + } + + @Test + void testPrincipalNameExtraction() { + when(metaStoreManagerFactory.getOrCreateSession(any())).thenReturn(jdbcPersistence); + + // Set up security identity with a principal + SecurityIdentity identity = mock(SecurityIdentity.class); + Principal principal = mock(Principal.class); + when(principal.getName()).thenReturn("test-user"); + when(identity.isAnonymous()).thenReturn(false); + when(identity.getPrincipal()).thenReturn(principal); + when(securityIdentityInstance.isResolvable()).thenReturn(true); + when(securityIdentityInstance.get()).thenReturn(identity); + + ScanReport scanReport = mock(ScanReport.class); + when(scanReport.tableName()).thenReturn("test_table"); + TableIdentifier table = TableIdentifier.of("db", "test_table"); + + reporter.reportMetric("test-catalog", table, scanReport); + + ArgumentCaptor captor = + ArgumentCaptor.forClass(ModelScanMetricsReport.class); + verify(jdbcPersistence).writeScanMetricsReport(captor.capture()); + + // The principal name should be captured in the report + // Note: The actual assertion depends on how the model is built + } + + @Test + void testPersistenceErrorDoesNotThrow() { + when(metaStoreManagerFactory.getOrCreateSession(any())) + .thenThrow(new RuntimeException("Database error")); + + ScanReport scanReport = mock(ScanReport.class); + TableIdentifier table = TableIdentifier.of("db", "test_table"); + + // Should not throw + reporter.reportMetric("test-catalog", table, scanReport); + } +} diff --git a/runtime/service/src/testFixtures/java/org/apache/polaris/service/events/listeners/TestPolarisEventListener.java b/runtime/service/src/testFixtures/java/org/apache/polaris/service/events/listeners/TestPolarisEventListener.java index 643dfa3754..31e4fdb387 100644 --- a/runtime/service/src/testFixtures/java/org/apache/polaris/service/events/listeners/TestPolarisEventListener.java +++ b/runtime/service/src/testFixtures/java/org/apache/polaris/service/events/listeners/TestPolarisEventListener.java @@ -846,4 +846,14 @@ public void onAfterAttemptTask(AfterAttemptTaskEvent event) { public void onBeforeLimitRequestRate(BeforeLimitRequestRateEvent event) { recordEvent(event); } + + @Override + public void onBeforeReportMetrics(IcebergRestCatalogEvents.BeforeReportMetricsEvent event) { + recordEvent(event); + } + + @Override + public void onAfterReportMetrics(IcebergRestCatalogEvents.AfterReportMetricsEvent event) { + recordEvent(event); + } } diff --git a/site/content/in-dev/unreleased/telemetry.md b/site/content/in-dev/unreleased/telemetry.md index fc1a1336ce..7edcc2d603 100644 --- a/site/content/in-dev/unreleased/telemetry.md +++ b/site/content/in-dev/unreleased/telemetry.md @@ -191,6 +191,185 @@ polaris.log.mdc.region=us-west-2 MDC context is propagated across threads, including in `TaskExecutor` threads. +## Compute Client Audit Reporting + +Polaris supports end-to-end audit correlation between catalog operations, credential vending, and +compute engine metrics reports. This enables organizations to trace data access from the initial +catalog request through to actual S3/GCS/Azure storage access. + +### Metrics Reporting Endpoint + +Compute engines can report scan and commit metrics to Polaris using the standard Iceberg REST +Catalog metrics endpoint: + +``` +POST /v1/{prefix}/namespaces/{namespace}/tables/{table}/metrics +``` + +**Request Body**: A `ReportMetricsRequest` containing either a `ScanReport` or `CommitReport`: + +```json +{ + "report-type": "scan-report", + "table-name": "my_table", + "snapshot-id": 123456789, + "schema-id": 0, + "projected-field-ids": [1, 2, 3], + "projected-field-names": ["id", "name", "value"], + "filter": {"type": "always-true"}, + "metrics": { + "result-data-files": {"unit": "count", "value": 10}, + "total-file-size-bytes": {"unit": "bytes", "value": 1048576} + }, + "metadata": { + "trace-id": "abcdef1234567890abcdef1234567890", + "client-app": "spark-3.5" + } +} +``` + +**Response**: `204 No Content` on success. + +The `metadata` map in the report can contain a `trace-id` for correlation with other audit events. +This trace ID is extracted and stored in the event's `additional_properties` with a `report.` prefix. + +### Trace Correlation + +When OpenTelemetry is enabled, Polaris captures the `trace_id` at multiple points: + +1. **Catalog Operations**: Events like `loadTable`, `createTable` include the OpenTelemetry trace + context in their metadata. +2. **Credential Vending**: When AWS STS session tags are enabled, the `trace_id` is included as a + session tag (`polaris:trace_id`) in the vended credentials. This appears in AWS CloudTrail logs. +3. **Metrics Reports**: When compute engines report scan/commit metrics back to Polaris, the + `reportMetrics` events capture both the OpenTelemetry trace context from HTTP headers and any + `trace-id` passed in the report's `metadata` map. + +### Enabling Session Tags for AWS + +To enable session tags (including trace_id) in AWS STS credentials, set the following feature flag: + +```properties +polaris.features."INCLUDE_SESSION_TAGS_IN_SUBSCOPED_CREDENTIAL"=true +``` + +This adds the following tags to all STS AssumeRole requests: + +- `polaris:catalog` - The catalog name +- `polaris:namespace` - The namespace being accessed +- `polaris:table` - The table name +- `polaris:principal` - The authenticated principal +- `polaris:roles` - The activated principal roles +- `polaris:trace_id` - The OpenTelemetry trace ID + +These tags appear in AWS CloudTrail logs, enabling correlation with Polaris audit events. + +**Note**: Enabling session tags requires the IAM role trust policy to allow the `sts:TagSession` +action. This feature may also reduce credential caching effectiveness since credentials become +specific to each table/namespace/role combination. + +### Compute Engine Integration + +For end-to-end trace correlation, compute engines should propagate the W3C Trace Context headers +when making requests to Polaris. The standard headers are: + +- `traceparent`: Contains the trace ID, parent span ID, and trace flags +- `tracestate`: Optional vendor-specific trace information + +#### Apache Spark + +Spark can propagate trace context using the OpenTelemetry Java agent. Add the agent to your Spark +submit command: + +```bash +spark-submit \ + --conf "spark.driver.extraJavaOptions=-javaagent:/path/to/opentelemetry-javaagent.jar" \ + --conf "spark.executor.extraJavaOptions=-javaagent:/path/to/opentelemetry-javaagent.jar" \ + -Dotel.service.name=spark-app \ + -Dotel.exporter.otlp.endpoint=http://collector:4317 \ + your-application.jar +``` + +Alternatively, configure the agent via environment variables: + +```bash +export OTEL_SERVICE_NAME=spark-app +export OTEL_EXPORTER_OTLP_ENDPOINT=http://collector:4317 +export JAVA_TOOL_OPTIONS="-javaagent:/path/to/opentelemetry-javaagent.jar" +``` + +#### Trino + +Trino supports OpenTelemetry tracing with the following configuration in `config.properties`: + +```properties +tracing.enabled=true +tracing.exporter.endpoint=http://collector:4317 +``` + +#### Flink + +Flink can be configured with OpenTelemetry using the Java agent: + +```bash +-javaagent:/path/to/opentelemetry-javaagent.jar \ +-Dotel.service.name=flink-job \ +-Dotel.exporter.otlp.endpoint=http://collector:4317 +``` + +### Correlating Audit Events + +With trace correlation enabled, you can join events across systems: + +1. **Polaris Events**: Query the events table for operations with a specific `trace_id` +2. **CloudTrail Logs**: Filter by the `polaris:trace_id` session tag +3. **Compute Engine Logs**: Search for the same trace ID in engine logs + +Example queries to find all Polaris events for a trace: + +**PostgreSQL** (using JSON operators): +```sql +SELECT * FROM polaris_schema.events +WHERE additional_properties->>'otel.trace_id' = '' + OR additional_properties->>'report.trace-id' = '' +ORDER BY timestamp_ms; +``` + +**H2/Generic SQL** (using LIKE pattern matching): +```sql +SELECT * FROM polaris_schema.events +WHERE additional_properties LIKE '%%' +ORDER BY timestamp_ms; +``` + +### Metrics Event Data + +The `AfterReportMetricsEvent` captures the following data in `additional_properties`: + +**For ScanReports:** +- `report_type`: "scan" +- `snapshot_id`: The snapshot ID being scanned +- `schema_id`: The schema ID +- `result_data_files`: Number of data files in the scan result +- `result_delete_files`: Number of delete files in the scan result +- `total_file_size_bytes`: Total size of files scanned +- `scanned_data_manifests`: Number of data manifests scanned +- `skipped_data_manifests`: Number of data manifests skipped +- `report.*`: Any metadata from the report's metadata map (e.g., `report.trace-id`) + +**For CommitReports:** +- `report_type`: "commit" +- `snapshot_id`: The new snapshot ID +- `sequence_number`: The sequence number +- `operation`: The operation type (e.g., "append", "overwrite") +- `added_data_files`: Number of data files added +- `removed_data_files`: Number of data files removed +- `added_records`: Number of records added +- `removed_records`: Number of records removed +- `added_file_size_bytes`: Total size of files added +- `removed_file_size_bytes`: Total size of files removed +- `report.*`: Any metadata from the report's metadata map (e.g., `report.trace-id`) + ## Links Visit [Using Polaris with telemetry tools]({{% relref "getting-started/using-polaris/telemetry-tools" %}}) to see sample Polaris config with Prometheus and Jaeger.