diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java index e90aeb4f72bf..9c3b1e34ec6e 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java @@ -251,8 +251,6 @@ import static io.trino.plugin.iceberg.PartitionFields.parsePartitionFields; import static io.trino.plugin.iceberg.PartitionFields.toPartitionFields; import static io.trino.plugin.iceberg.SortFieldUtils.parseSortFields; -import static io.trino.plugin.iceberg.TableStatisticsReader.TRINO_STATS_COLUMN_ID_PATTERN; -import static io.trino.plugin.iceberg.TableStatisticsReader.TRINO_STATS_PREFIX; import static io.trino.plugin.iceberg.TableStatisticsWriter.StatsUpdateMode.INCREMENTAL_UPDATE; import static io.trino.plugin.iceberg.TableStatisticsWriter.StatsUpdateMode.REPLACE; import static io.trino.plugin.iceberg.TableType.DATA; @@ -1499,13 +1497,6 @@ private void executeDropExtendedStats(ConnectorSession session, IcebergTableExec updateStatistics.removeStatistics(statisticsFile.snapshotId()); } updateStatistics.commit(); - UpdateProperties updateProperties = transaction.updateProperties(); - for (String key : transaction.table().properties().keySet()) { - if (key.startsWith(TRINO_STATS_PREFIX)) { - updateProperties.remove(key); - } - } - updateProperties.commit(); transaction.commitTransaction(); transaction = null; } @@ -2174,41 +2165,12 @@ public void finishStatisticsCollection(ConnectorSession session, ConnectorTableH "Unexpected computed statistics that cannot be attached to a snapshot because none exists: %s", computedStatistics); - // TODO (https://github.com/trinodb/trino/issues/15397): remove support for Trino-specific statistics properties - // Drop all stats. Empty table needs none - UpdateProperties updateProperties = transaction.updateProperties(); - table.properties().keySet().stream() - .filter(key -> key.startsWith(TRINO_STATS_PREFIX)) - .forEach(updateProperties::remove); - updateProperties.commit(); - transaction.commitTransaction(); transaction = null; return; } long snapshotId = handle.getSnapshotId().orElseThrow(); - Set columnIds = table.schema().columns().stream() - .map(Types.NestedField::fieldId) - .collect(toImmutableSet()); - - // TODO (https://github.com/trinodb/trino/issues/15397): remove support for Trino-specific statistics properties - // Drop stats for obsolete columns - UpdateProperties updateProperties = transaction.updateProperties(); - table.properties().keySet().stream() - .filter(key -> { - if (!key.startsWith(TRINO_STATS_PREFIX)) { - return false; - } - Matcher matcher = TRINO_STATS_COLUMN_ID_PATTERN.matcher(key); - if (!matcher.matches()) { - return false; - } - return !columnIds.contains(Integer.parseInt(matcher.group("columnId"))); - }) - .forEach(updateProperties::remove); - updateProperties.commit(); - CollectedStatistics collectedStatistics = processComputedTableStatistics(table, computedStatistics); StatisticsFile statisticsFile = tableStatisticsWriter.writeStatisticsFile( session, diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/TableStatisticsReader.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/TableStatisticsReader.java index 8e00d7e0c3da..0b4947959e0e 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/TableStatisticsReader.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/TableStatisticsReader.java @@ -51,8 +51,6 @@ import java.util.Objects; import java.util.Optional; import java.util.Set; -import java.util.regex.Matcher; -import java.util.regex.Pattern; import static com.google.common.base.Verify.verifyNotNull; import static com.google.common.collect.ImmutableMap.toImmutableMap; @@ -77,19 +75,6 @@ private TableStatisticsReader() {} private static final Logger log = Logger.get(TableStatisticsReader.class); - // TODO (https://github.com/trinodb/trino/issues/15397): remove support for Trino-specific statistics properties - @Deprecated - public static final String TRINO_STATS_PREFIX = "trino.stats.ndv."; - // TODO (https://github.com/trinodb/trino/issues/15397): remove support for Trino-specific statistics properties - @Deprecated - public static final String TRINO_STATS_NDV_FORMAT = TRINO_STATS_PREFIX + "%d.ndv"; - // TODO (https://github.com/trinodb/trino/issues/15397): remove support for Trino-specific statistics properties - @Deprecated - public static final Pattern TRINO_STATS_COLUMN_ID_PATTERN = Pattern.compile(Pattern.quote(TRINO_STATS_PREFIX) + "(?\\d+)\\..*"); - // TODO (https://github.com/trinodb/trino/issues/15397): remove support for Trino-specific statistics properties - @Deprecated - public static final Pattern TRINO_STATS_NDV_PATTERN = Pattern.compile(Pattern.quote(TRINO_STATS_PREFIX) + "(?\\d+)\\.ndv"); - public static final String APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY = "ndv"; public static TableStatistics getTableStatistics(TypeManager typeManager, ConnectorSession session, IcebergTableHandle tableHandle, Table icebergTable) @@ -256,24 +241,6 @@ private static Map readNdvs(Table icebergTable, long snapshotId, } }); - // TODO (https://github.com/trinodb/trino/issues/15397): remove support for Trino-specific statistics properties - Iterator> properties = icebergTable.properties().entrySet().iterator(); - while (!remainingColumnIds.isEmpty() && properties.hasNext()) { - Entry entry = properties.next(); - String key = entry.getKey(); - String value = entry.getValue(); - if (key.startsWith(TRINO_STATS_PREFIX)) { - Matcher matcher = TRINO_STATS_NDV_PATTERN.matcher(key); - if (matcher.matches()) { - int columnId = Integer.parseInt(matcher.group("columnId")); - if (remainingColumnIds.remove(columnId)) { - long ndv = parseLong(value); - ndvByColumnId.put(columnId, ndv); - } - } - } - } - return ndvByColumnId.buildOrThrow(); }