Skip to content

Commit 3a85001

Browse files
committed
Remove support for pre-Puffin Iceberg stats
Before Iceberg library gained Puffin stats, Trino would store table stats in Iceberg table properties. This became obsoleted in Trino 405 (released in Dec 2022). Dropping support for old stats format means need to re-analyze tables that haven't been written to by Trino 405 or newer.
1 parent 031bd9f commit 3a85001

File tree

2 files changed

+0
-71
lines changed

2 files changed

+0
-71
lines changed

plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java

Lines changed: 0 additions & 38 deletions
Original file line numberDiff line numberDiff line change
@@ -251,8 +251,6 @@
251251
import static io.trino.plugin.iceberg.PartitionFields.parsePartitionFields;
252252
import static io.trino.plugin.iceberg.PartitionFields.toPartitionFields;
253253
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;
256254
import static io.trino.plugin.iceberg.TableStatisticsWriter.StatsUpdateMode.INCREMENTAL_UPDATE;
257255
import static io.trino.plugin.iceberg.TableStatisticsWriter.StatsUpdateMode.REPLACE;
258256
import static io.trino.plugin.iceberg.TableType.DATA;
@@ -1499,13 +1497,6 @@ private void executeDropExtendedStats(ConnectorSession session, IcebergTableExec
14991497
updateStatistics.removeStatistics(statisticsFile.snapshotId());
15001498
}
15011499
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();
15091500
transaction.commitTransaction();
15101501
transaction = null;
15111502
}
@@ -2174,41 +2165,12 @@ public void finishStatisticsCollection(ConnectorSession session, ConnectorTableH
21742165
"Unexpected computed statistics that cannot be attached to a snapshot because none exists: %s",
21752166
computedStatistics);
21762167

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-
21852168
transaction.commitTransaction();
21862169
transaction = null;
21872170
return;
21882171
}
21892172
long snapshotId = handle.getSnapshotId().orElseThrow();
21902173

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-
22122174
CollectedStatistics collectedStatistics = processComputedTableStatistics(table, computedStatistics);
22132175
StatisticsFile statisticsFile = tableStatisticsWriter.writeStatisticsFile(
22142176
session,

plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/TableStatisticsReader.java

Lines changed: 0 additions & 33 deletions
Original file line numberDiff line numberDiff line change
@@ -51,8 +51,6 @@
5151
import java.util.Objects;
5252
import java.util.Optional;
5353
import java.util.Set;
54-
import java.util.regex.Matcher;
55-
import java.util.regex.Pattern;
5654

5755
import static com.google.common.base.Verify.verifyNotNull;
5856
import static com.google.common.collect.ImmutableMap.toImmutableMap;
@@ -77,19 +75,6 @@ private TableStatisticsReader() {}
7775

7876
private static final Logger log = Logger.get(TableStatisticsReader.class);
7977

80-
// TODO (https://github.com/trinodb/trino/issues/15397): remove support for Trino-specific statistics properties
81-
@Deprecated
82-
public static final String TRINO_STATS_PREFIX = "trino.stats.ndv.";
83-
// TODO (https://github.com/trinodb/trino/issues/15397): remove support for Trino-specific statistics properties
84-
@Deprecated
85-
public static final String TRINO_STATS_NDV_FORMAT = TRINO_STATS_PREFIX + "%d.ndv";
86-
// TODO (https://github.com/trinodb/trino/issues/15397): remove support for Trino-specific statistics properties
87-
@Deprecated
88-
public static final Pattern TRINO_STATS_COLUMN_ID_PATTERN = Pattern.compile(Pattern.quote(TRINO_STATS_PREFIX) + "(?<columnId>\\d+)\\..*");
89-
// TODO (https://github.com/trinodb/trino/issues/15397): remove support for Trino-specific statistics properties
90-
@Deprecated
91-
public static final Pattern TRINO_STATS_NDV_PATTERN = Pattern.compile(Pattern.quote(TRINO_STATS_PREFIX) + "(?<columnId>\\d+)\\.ndv");
92-
9378
public static final String APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY = "ndv";
9479

9580
public static TableStatistics getTableStatistics(TypeManager typeManager, ConnectorSession session, IcebergTableHandle tableHandle, Table icebergTable)
@@ -256,24 +241,6 @@ private static Map<Integer, Long> readNdvs(Table icebergTable, long snapshotId,
256241
}
257242
});
258243

259-
// TODO (https://github.com/trinodb/trino/issues/15397): remove support for Trino-specific statistics properties
260-
Iterator<Entry<String, String>> properties = icebergTable.properties().entrySet().iterator();
261-
while (!remainingColumnIds.isEmpty() && properties.hasNext()) {
262-
Entry<String, String> entry = properties.next();
263-
String key = entry.getKey();
264-
String value = entry.getValue();
265-
if (key.startsWith(TRINO_STATS_PREFIX)) {
266-
Matcher matcher = TRINO_STATS_NDV_PATTERN.matcher(key);
267-
if (matcher.matches()) {
268-
int columnId = Integer.parseInt(matcher.group("columnId"));
269-
if (remainingColumnIds.remove(columnId)) {
270-
long ndv = parseLong(value);
271-
ndvByColumnId.put(columnId, ndv);
272-
}
273-
}
274-
}
275-
}
276-
277244
return ndvByColumnId.buildOrThrow();
278245
}
279246

0 commit comments

Comments
 (0)