|
251 | 251 | import static io.trino.plugin.iceberg.PartitionFields.parsePartitionFields; |
252 | 252 | import static io.trino.plugin.iceberg.PartitionFields.toPartitionFields; |
253 | 253 | import static io.trino.plugin.iceberg.SortFieldUtils.parseSortFields; |
254 | | -import static io.trino.plugin.iceberg.TableStatisticsReader.TRINO_STATS_COLUMN_ID_PATTERN; |
255 | | -import static io.trino.plugin.iceberg.TableStatisticsReader.TRINO_STATS_PREFIX; |
256 | 254 | import static io.trino.plugin.iceberg.TableStatisticsWriter.StatsUpdateMode.INCREMENTAL_UPDATE; |
257 | 255 | import static io.trino.plugin.iceberg.TableStatisticsWriter.StatsUpdateMode.REPLACE; |
258 | 256 | import static io.trino.plugin.iceberg.TableType.DATA; |
@@ -1499,13 +1497,6 @@ private void executeDropExtendedStats(ConnectorSession session, IcebergTableExec |
1499 | 1497 | updateStatistics.removeStatistics(statisticsFile.snapshotId()); |
1500 | 1498 | } |
1501 | 1499 | updateStatistics.commit(); |
1502 | | - UpdateProperties updateProperties = transaction.updateProperties(); |
1503 | | - for (String key : transaction.table().properties().keySet()) { |
1504 | | - if (key.startsWith(TRINO_STATS_PREFIX)) { |
1505 | | - updateProperties.remove(key); |
1506 | | - } |
1507 | | - } |
1508 | | - updateProperties.commit(); |
1509 | 1500 | transaction.commitTransaction(); |
1510 | 1501 | transaction = null; |
1511 | 1502 | } |
@@ -2174,41 +2165,12 @@ public void finishStatisticsCollection(ConnectorSession session, ConnectorTableH |
2174 | 2165 | "Unexpected computed statistics that cannot be attached to a snapshot because none exists: %s", |
2175 | 2166 | computedStatistics); |
2176 | 2167 |
|
2177 | | - // TODO (https://github.com/trinodb/trino/issues/15397): remove support for Trino-specific statistics properties |
2178 | | - // Drop all stats. Empty table needs none |
2179 | | - UpdateProperties updateProperties = transaction.updateProperties(); |
2180 | | - table.properties().keySet().stream() |
2181 | | - .filter(key -> key.startsWith(TRINO_STATS_PREFIX)) |
2182 | | - .forEach(updateProperties::remove); |
2183 | | - updateProperties.commit(); |
2184 | | - |
2185 | 2168 | transaction.commitTransaction(); |
2186 | 2169 | transaction = null; |
2187 | 2170 | return; |
2188 | 2171 | } |
2189 | 2172 | long snapshotId = handle.getSnapshotId().orElseThrow(); |
2190 | 2173 |
|
2191 | | - Set<Integer> columnIds = table.schema().columns().stream() |
2192 | | - .map(Types.NestedField::fieldId) |
2193 | | - .collect(toImmutableSet()); |
2194 | | - |
2195 | | - // TODO (https://github.com/trinodb/trino/issues/15397): remove support for Trino-specific statistics properties |
2196 | | - // Drop stats for obsolete columns |
2197 | | - UpdateProperties updateProperties = transaction.updateProperties(); |
2198 | | - table.properties().keySet().stream() |
2199 | | - .filter(key -> { |
2200 | | - if (!key.startsWith(TRINO_STATS_PREFIX)) { |
2201 | | - return false; |
2202 | | - } |
2203 | | - Matcher matcher = TRINO_STATS_COLUMN_ID_PATTERN.matcher(key); |
2204 | | - if (!matcher.matches()) { |
2205 | | - return false; |
2206 | | - } |
2207 | | - return !columnIds.contains(Integer.parseInt(matcher.group("columnId"))); |
2208 | | - }) |
2209 | | - .forEach(updateProperties::remove); |
2210 | | - updateProperties.commit(); |
2211 | | - |
2212 | 2174 | CollectedStatistics collectedStatistics = processComputedTableStatistics(table, computedStatistics); |
2213 | 2175 | StatisticsFile statisticsFile = tableStatisticsWriter.writeStatisticsFile( |
2214 | 2176 | session, |
|
0 commit comments