Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
19 changes: 19 additions & 0 deletions docs/src/main/sphinx/connector/delta-lake.md
Original file line number Diff line number Diff line change
Expand Up @@ -662,6 +662,25 @@ EXECUTE <alter-table-execute>`.
```{include} optimize.fragment
```

Use a `WHERE` clause with [metadata columns](delta-lake-special-columns) to filter
which files are optimized.

```sql
ALTER TABLE test_table EXECUTE optimize
WHERE "$file_modified_time" > date_trunc('day', CURRENT_TIMESTAMP);
```

```sql
ALTER TABLE test_table EXECUTE optimize
WHERE "$path" <> 'skipping-file-path'
```

```sql
-- optimze files smaller than 1MB
ALTER TABLE test_table EXECUTE optimize
WHERE "$file_size" <= 1024 * 1024
```

(delta-lake-alter-table-rename-to)=
#### ALTER TABLE RENAME TO

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,8 @@ public record DeltaLakeColumnHandle(
field("position", BIGINT),
field("partition", VARCHAR));

// Note: update `isMetadataColumnHandle` method when adding new metadata columns

public static final String PATH_COLUMN_NAME = "$path";
public static final Type PATH_TYPE = VARCHAR;

Expand Down Expand Up @@ -161,4 +163,9 @@ public static DeltaLakeColumnHandle mergeRowIdColumnHandle()
{
return new DeltaLakeColumnHandle(ROW_ID_COLUMN_NAME, MERGE_ROW_ID_TYPE, OptionalInt.empty(), ROW_ID_COLUMN_NAME, MERGE_ROW_ID_TYPE, SYNTHESIZED, Optional.empty());
}

public static boolean isMetadataColumnHandle(DeltaLakeColumnHandle columnHandle)
Comment thread
chenjian2664 marked this conversation as resolved.
Outdated
{
return columnHandle.equals(fileModifiedTimeColumnHandle()) || columnHandle.equals(pathColumnHandle()) || columnHandle.equals(fileSizeColumnHandle());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -219,6 +219,7 @@
import static io.trino.plugin.deltalake.DeltaLakeColumnHandle.PATH_COLUMN_NAME;
import static io.trino.plugin.deltalake.DeltaLakeColumnHandle.fileModifiedTimeColumnHandle;
import static io.trino.plugin.deltalake.DeltaLakeColumnHandle.fileSizeColumnHandle;
import static io.trino.plugin.deltalake.DeltaLakeColumnHandle.isMetadataColumnHandle;
import static io.trino.plugin.deltalake.DeltaLakeColumnHandle.mergeRowIdColumnHandle;
import static io.trino.plugin.deltalake.DeltaLakeColumnHandle.pathColumnHandle;
import static io.trino.plugin.deltalake.DeltaLakeColumnType.PARTITION_KEY;
Expand All @@ -236,7 +237,7 @@
import static io.trino.plugin.deltalake.DeltaLakeSessionProperties.isQueryPartitionFilterRequired;
import static io.trino.plugin.deltalake.DeltaLakeSessionProperties.isStoreTableMetadataInMetastoreEnabled;
import static io.trino.plugin.deltalake.DeltaLakeSessionProperties.isTableStatisticsEnabled;
import static io.trino.plugin.deltalake.DeltaLakeSplitManager.partitionMatchesPredicate;
import static io.trino.plugin.deltalake.DeltaLakeSplitManager.buildSplitPath;
import static io.trino.plugin.deltalake.DeltaLakeTableProperties.CHANGE_DATA_FEED_ENABLED_PROPERTY;
import static io.trino.plugin.deltalake.DeltaLakeTableProperties.CHECKPOINT_INTERVAL_PROPERTY;
import static io.trino.plugin.deltalake.DeltaLakeTableProperties.COLUMN_MAPPING_MODE_PROPERTY;
Expand Down Expand Up @@ -293,6 +294,13 @@
import static io.trino.plugin.deltalake.transactionlog.TransactionLogUtil.getTransactionLogJsonEntryPath;
import static io.trino.plugin.deltalake.transactionlog.checkpoint.TransactionLogTail.getEntriesFromJson;
import static io.trino.plugin.deltalake.transactionlog.checkpoint.TransactionLogTail.loadNewTail;
import static io.trino.plugin.deltalake.util.DeltaLakeDomains.fileModifiedTimeMatchesPredicate;
import static io.trino.plugin.deltalake.util.DeltaLakeDomains.fileSizeMatchesPredicate;
import static io.trino.plugin.deltalake.util.DeltaLakeDomains.getFileModifiedTimeDomain;
import static io.trino.plugin.deltalake.util.DeltaLakeDomains.getFileSizeDomain;
import static io.trino.plugin.deltalake.util.DeltaLakeDomains.getPathDomain;
import static io.trino.plugin.deltalake.util.DeltaLakeDomains.partitionMatchesPredicate;
import static io.trino.plugin.deltalake.util.DeltaLakeDomains.pathMatchesPredicate;
import static io.trino.plugin.hive.HiveMetadata.TRINO_QUERY_ID_NAME;
import static io.trino.plugin.hive.TableType.EXTERNAL_TABLE;
import static io.trino.plugin.hive.TableType.MANAGED_TABLE;
Expand Down Expand Up @@ -3332,11 +3340,13 @@ public Optional<ConstraintApplicationResult<ConnectorTableHandle>> applyFilter(C

TupleDomain<DeltaLakeColumnHandle> newEnforcedConstraint;
TupleDomain<DeltaLakeColumnHandle> newUnenforcedConstraint;
TupleDomain<DeltaLakeColumnHandle> newRemainingConstraint;
Set<DeltaLakeColumnHandle> newConstraintColumns;
if (predicate.isNone()) {
// Engine does not pass none Constraint.summary. It can become none when combined with the expression and connector's domain knowledge.
newEnforcedConstraint = TupleDomain.none();
newUnenforcedConstraint = TupleDomain.all();
newRemainingConstraint = TupleDomain.all();
newConstraintColumns = constraint.getPredicateColumns().stream()
.flatMap(Collection::stream)
.map(DeltaLakeColumnHandle.class::cast)
Expand All @@ -3348,6 +3358,7 @@ public Optional<ConstraintApplicationResult<ConnectorTableHandle>> applyFilter(C

ImmutableMap.Builder<DeltaLakeColumnHandle, Domain> enforceableDomains = ImmutableMap.builder();
ImmutableMap.Builder<DeltaLakeColumnHandle, Domain> unenforceableDomains = ImmutableMap.builder();
ImmutableMap.Builder<DeltaLakeColumnHandle, Domain> remainingDomains = ImmutableMap.builder();
ImmutableSet.Builder<DeltaLakeColumnHandle> constraintColumns = ImmutableSet.builder();
// We need additional field to track partition columns used in queries as enforceDomains seem to be not catching
// cases when partition columns is used within complex filter as 'partitionColumn % 2 = 0'
Expand All @@ -3357,9 +3368,14 @@ public Optional<ConstraintApplicationResult<ConnectorTableHandle>> applyFilter(C
.forEach(constraintColumns::add);
for (Entry<ColumnHandle, Domain> domainEntry : constraintDomains.entrySet()) {
DeltaLakeColumnHandle column = (DeltaLakeColumnHandle) domainEntry.getKey();
if (!partitionColumns.contains(column)) {
if (isMetadataColumnHandle(column)) {
unenforceableDomains.put(column, domainEntry.getValue());
}
else if (!partitionColumns.contains(column)) {
unenforceableDomains.put(column, domainEntry.getValue());
// the column can not be pusheddown
remainingDomains.put(column, domainEntry.getValue());
}
else {
enforceableDomains.put(column, domainEntry.getValue());
}
Expand All @@ -3368,6 +3384,7 @@ public Optional<ConstraintApplicationResult<ConnectorTableHandle>> applyFilter(C

newEnforcedConstraint = TupleDomain.withColumnDomains(enforceableDomains.buildOrThrow());
newUnenforcedConstraint = TupleDomain.withColumnDomains(unenforceableDomains.buildOrThrow());
newRemainingConstraint = TupleDomain.withColumnDomains(remainingDomains.buildOrThrow());
newConstraintColumns = constraintColumns.build();
}

Expand Down Expand Up @@ -3405,7 +3422,7 @@ public Optional<ConstraintApplicationResult<ConnectorTableHandle>> applyFilter(C

return Optional.of(new ConstraintApplicationResult<>(
newHandle,
newUnenforcedConstraint.transformKeys(ColumnHandle.class::cast),
newRemainingConstraint.transformKeys(ColumnHandle.class::cast),
extractionResult.remainingExpression(),
false));
}
Expand Down Expand Up @@ -3831,7 +3848,7 @@ private void generateMissingFileStatistics(ConnectorSession session, DeltaLakeTa
|| addFileEntry.getStats().get().getNullCount().isEmpty())
.filter(addFileEntry -> !URI.create(addFileEntry.getPath()).isAbsolute()) // TODO: Support absolute paths https://github.com/trinodb/trino/issues/18277
// Statistics returns whole path to file build in DeltaLakeSplitManager, so we need to create corresponding map key for AddFileEntry.
.collect(toImmutableMap(addFileEntry -> DeltaLakeSplitManager.buildSplitPath(Location.of(tableHandle.getLocation()), addFileEntry).toString(), identity()));
.collect(toImmutableMap(addFileEntry -> buildSplitPath(Location.of(tableHandle.getLocation()), addFileEntry).toString(), identity()));
}
if (addFileEntriesWithNoStats.isEmpty()) {
return;
Expand Down Expand Up @@ -4202,12 +4219,30 @@ private CommitDeleteOperationResult commitDeleteOperation(
long commitVersion = currentVersion + 1;
transactionLogWriter.appendCommitInfoEntry(getCommitInfoEntry(session, isolationLevel, commitVersion, writeTimestamp, operation, tableHandle.getReadVersion(), false));

Domain pathDomain = getPathDomain(tableHandle.getNonPartitionConstraint());
Domain fileModifiedDomain = getFileModifiedTimeDomain(tableHandle.getNonPartitionConstraint());
Domain fileSizeDomain = getFileSizeDomain(tableHandle.getNonPartitionConstraint());

long deletedRecords = 0L;
boolean allDeletedFilesStatsPresent = true;
try (Stream<AddFileEntry> activeFiles = getAddFileEntriesMatchingEnforcedPartitionConstraint(session, tableHandle)) {
Iterator<AddFileEntry> addFileEntryIterator = activeFiles.iterator();
while (addFileEntryIterator.hasNext()) {
AddFileEntry addFileEntry = addFileEntryIterator.next();

String splitPath = buildSplitPath(Location.of(tableHandle.getLocation()), addFileEntry).toString();
if (!pathMatchesPredicate(pathDomain, splitPath)) {
continue;
}

if (!fileModifiedTimeMatchesPredicate(fileModifiedDomain, addFileEntry.getModificationTime())) {
continue;
}

if (!fileSizeMatchesPredicate(fileSizeDomain, addFileEntry.getSize())) {
continue;
}

transactionLogWriter.appendRemoveFileEntry(new RemoveFileEntry(addFileEntry.getPath(), addFileEntry.getPartitionValues(), writeTimestamp, true, Optional.empty()));

Optional<Long> fileRecords = addFileEntry.getStats().flatMap(DeltaLakeFileStatistics::getNumRecords);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -100,11 +100,11 @@
import static io.trino.plugin.deltalake.DeltaLakeSessionProperties.isParquetIgnoreStatistics;
import static io.trino.plugin.deltalake.DeltaLakeSessionProperties.isParquetUseColumnIndex;
import static io.trino.plugin.deltalake.DeltaLakeSessionProperties.isParquetVectorizedDecodingEnabled;
import static io.trino.plugin.deltalake.DeltaLakeSplitManager.partitionMatchesPredicate;
import static io.trino.plugin.deltalake.delete.DeletionVectors.readDeletionVectors;
import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.extractSchema;
import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.getColumnMappingMode;
import static io.trino.plugin.deltalake.transactionlog.TransactionLogParser.deserializePartitionValue;
import static io.trino.plugin.deltalake.util.DeltaLakeDomains.partitionMatchesPredicate;
import static io.trino.plugin.hive.parquet.ParquetPageSourceFactory.PARQUET_ROW_INDEX_COLUMN;
import static io.trino.spi.type.DateTimeEncoding.packDateTimeWithZone;
import static io.trino.spi.type.TimeZoneKey.UTC_KEY;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,14 +58,19 @@
import static com.google.common.collect.ImmutableList.toImmutableList;
import static com.google.common.collect.ImmutableMap.toImmutableMap;
import static com.google.common.collect.ImmutableSet.toImmutableSet;
import static io.airlift.slice.Slices.utf8Slice;
import static io.trino.plugin.deltalake.DeltaLakeAnalyzeProperties.AnalyzeMode.FULL_REFRESH;
import static io.trino.plugin.deltalake.DeltaLakeColumnHandle.pathColumnHandle;
import static io.trino.plugin.deltalake.DeltaLakeMetadata.createStatisticsPredicate;
import static io.trino.plugin.deltalake.DeltaLakeSessionProperties.getDynamicFilteringWaitTimeout;
import static io.trino.plugin.deltalake.DeltaLakeSessionProperties.getMaxSplitSize;
import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.extractSchema;
import static io.trino.plugin.deltalake.transactionlog.TransactionLogParser.deserializePartitionValue;
import static io.trino.plugin.deltalake.util.DeltaLakeDomains.fileModifiedTimeMatchesPredicate;
import static io.trino.plugin.deltalake.util.DeltaLakeDomains.fileSizeMatchesPredicate;
import static io.trino.plugin.deltalake.util.DeltaLakeDomains.getFileModifiedTimeDomain;
import static io.trino.plugin.deltalake.util.DeltaLakeDomains.getFileSizeDomain;
import static io.trino.plugin.deltalake.util.DeltaLakeDomains.getPathDomain;
import static io.trino.plugin.deltalake.util.DeltaLakeDomains.partitionMatchesPredicate;
import static io.trino.plugin.deltalake.util.DeltaLakeDomains.pathMatchesPredicate;
import static io.trino.spi.connector.FixedSplitSource.emptySplitSource;
import static java.lang.Math.clamp;
import static java.util.Objects.requireNonNull;
Expand Down Expand Up @@ -163,6 +168,8 @@ private Stream<DeltaLakeSplit> getSplits(
TupleDomain<DeltaLakeColumnHandle> enforcedPartitionConstraint = tableHandle.getEnforcedPartitionConstraint();
TupleDomain<DeltaLakeColumnHandle> nonPartitionConstraint = tableHandle.getNonPartitionConstraint();
Domain pathDomain = getPathDomain(nonPartitionConstraint);
Domain fileModifiedDomain = getFileModifiedTimeDomain(nonPartitionConstraint);
Domain fileSizeDomain = getFileSizeDomain(nonPartitionConstraint);

boolean splittable =
// Delta Lake handles updates and deletes by copying entire data files, minus updates/deletes. Because of this we can only have one Split/UpdatablePageSource
Expand Down Expand Up @@ -207,10 +214,18 @@ private Stream<DeltaLakeSplit> getSplits(
return Stream.empty();
}

if (!fileModifiedTimeMatchesPredicate(fileModifiedDomain, addAction.getModificationTime())) {
return Stream.empty();
}

if (addAction.getDeletionVector().isEmpty() && maxScannedFileSizeInBytes.isPresent() && addAction.getSize() > maxScannedFileSizeInBytes.get()) {
return Stream.empty();
}

if (!fileSizeMatchesPredicate(fileSizeDomain, addAction.getSize())) {
return Stream.empty();
}

Map<DeltaLakeColumnHandle, Domain> enforcedDomains = enforcedPartitionConstraint.getDomains().orElseThrow();
if (!partitionMatchesPredicate(addAction.getCanonicalPartitionValues(), enforcedDomains)) {
return Stream.empty();
Expand Down Expand Up @@ -281,30 +296,6 @@ private static boolean mayAnyDataColumnProjected(DeltaLakeTableHandle tableHandl
.anyMatch(DeltaLakeColumnType.REGULAR::equals);
}

public static boolean partitionMatchesPredicate(Map<String, Optional<String>> partitionKeys, Map<DeltaLakeColumnHandle, Domain> domains)
{
for (Map.Entry<DeltaLakeColumnHandle, Domain> enforcedDomainsEntry : domains.entrySet()) {
DeltaLakeColumnHandle partitionColumn = enforcedDomainsEntry.getKey();
Domain partitionDomain = enforcedDomainsEntry.getValue();
if (!partitionDomain.includesNullableValue(deserializePartitionValue(partitionColumn, partitionKeys.get(partitionColumn.basePhysicalColumnName())))) {
return false;
}
}
return true;
}

private static Domain getPathDomain(TupleDomain<DeltaLakeColumnHandle> effectivePredicate)
{
return effectivePredicate.getDomains()
.flatMap(domains -> Optional.ofNullable(domains.get(pathColumnHandle())))
.orElseGet(() -> Domain.all(pathColumnHandle().baseType()));
}

private static boolean pathMatchesPredicate(Domain pathDomain, String path)
{
return pathDomain.includesNullableValue(utf8Slice(path));
}

private List<DeltaLakeSplit> splitsForFile(
ConnectorSession session,
AddFileEntry addFileEntry,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@
import static com.google.common.util.concurrent.Futures.immediateFailedFuture;
import static com.google.common.util.concurrent.MoreExecutors.directExecutor;
import static io.airlift.concurrent.MoreFutures.toCompletableFuture;
import static io.trino.plugin.deltalake.DeltaLakeSplitManager.partitionMatchesPredicate;
import static io.trino.plugin.deltalake.util.DeltaLakeDomains.partitionMatchesPredicate;
import static io.trino.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR;
import static java.util.Objects.requireNonNull;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@

import com.google.common.collect.ImmutableSet;
import com.google.inject.Inject;
import io.trino.filesystem.Location;
import io.trino.plugin.deltalake.DeltaLakeColumnHandle;
import io.trino.plugin.deltalake.DeltaLakeColumnMetadata;
import io.trino.plugin.deltalake.DeltaLakeTableHandle;
Expand All @@ -25,6 +26,7 @@
import io.trino.plugin.deltalake.transactionlog.TransactionLogAccess;
import io.trino.plugin.deltalake.transactionlog.statistics.DeltaLakeFileStatistics;
import io.trino.spi.connector.ConnectorSession;
import io.trino.spi.predicate.Domain;
import io.trino.spi.predicate.TupleDomain;
import io.trino.spi.statistics.ColumnStatistics;
import io.trino.spi.statistics.DoubleRange;
Expand All @@ -48,7 +50,14 @@
import static io.trino.plugin.deltalake.DeltaLakeColumnType.REGULAR;
import static io.trino.plugin.deltalake.DeltaLakeMetadata.createStatisticsPredicate;
import static io.trino.plugin.deltalake.DeltaLakeSessionProperties.isExtendedStatisticsEnabled;
import static io.trino.plugin.deltalake.DeltaLakeSplitManager.partitionMatchesPredicate;
import static io.trino.plugin.deltalake.DeltaLakeSplitManager.buildSplitPath;
import static io.trino.plugin.deltalake.util.DeltaLakeDomains.fileModifiedTimeMatchesPredicate;
import static io.trino.plugin.deltalake.util.DeltaLakeDomains.fileSizeMatchesPredicate;
import static io.trino.plugin.deltalake.util.DeltaLakeDomains.getFileModifiedTimeDomain;
import static io.trino.plugin.deltalake.util.DeltaLakeDomains.getFileSizeDomain;
import static io.trino.plugin.deltalake.util.DeltaLakeDomains.getPathDomain;
import static io.trino.plugin.deltalake.util.DeltaLakeDomains.partitionMatchesPredicate;
import static io.trino.plugin.deltalake.util.DeltaLakeDomains.pathMatchesPredicate;
import static io.trino.spi.statistics.StatsUtil.toStatsRepresentation;
import static java.lang.Double.NEGATIVE_INFINITY;
import static java.lang.Double.NaN;
Expand Down Expand Up @@ -113,6 +122,9 @@ public TableStatistics getTableStatistics(ConnectorSession session, DeltaLakeTab
.filter(column -> predicatedColumnNames.contains(column.name()))
.collect(toImmutableList());

Domain pathDomain = getPathDomain(tableHandle.getNonPartitionConstraint());
Domain fileModifiedDomain = getFileModifiedTimeDomain(tableHandle.getNonPartitionConstraint());
Domain fileSizeDomain = getFileSizeDomain(tableHandle.getNonPartitionConstraint());
try (Stream<AddFileEntry> addEntries = transactionLogAccess.getActiveFiles(
session,
tableSnapshot,
Expand All @@ -133,6 +145,19 @@ public TableStatistics getTableStatistics(ConnectorSession session, DeltaLakeTab
continue;
}

String splitPath = buildSplitPath(Location.of(tableHandle.getLocation()), addEntry).toString();
if (!pathMatchesPredicate(pathDomain, splitPath)) {
continue;
}

if (!fileModifiedTimeMatchesPredicate(fileModifiedDomain, addEntry.getModificationTime())) {
continue;
}

if (!fileSizeMatchesPredicate(fileSizeDomain, addEntry.getSize())) {
continue;
}

TupleDomain<DeltaLakeColumnHandle> statisticsPredicate = createStatisticsPredicate(
addEntry,
predicatedColumns,
Expand Down
Loading