Skip to content

Commit

Permalink
Add $transactions system table to Delta Lake
Browse files Browse the repository at this point in the history
  • Loading branch information
ebyhr committed Dec 4, 2024
1 parent b2dc49a commit c16cc73
Show file tree
Hide file tree
Showing 4 changed files with 247 additions and 0 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -4100,6 +4100,12 @@ private Optional<SystemTable> getRawSystemTable(ConnectorSession session, Schema
fileSystemFactory,
transactionLogAccess,
typeManager));
case TRANSACTIONS -> Optional.of(new DeltaLakeTransactionsTable(
systemTableName,
tableLocation,
fileSystemFactory,
transactionLogAccess,
typeManager));
case PROPERTIES -> Optional.of(new DeltaLakePropertiesTable(systemTableName, tableLocation, transactionLogAccess));
case PARTITIONS -> Optional.of(new DeltaLakePartitionsTable(session, systemTableName, tableLocation, transactionLogAccess, typeManager));
};
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ public enum DeltaLakeTableType
{
DATA,
HISTORY,
TRANSACTIONS,
PROPERTIES,
PARTITIONS,
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,213 @@
/*
* Licensed 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 io.trino.plugin.deltalake;

import com.google.common.collect.ImmutableList;
import io.airlift.json.JsonCodec;
import io.trino.filesystem.TrinoFileSystem;
import io.trino.filesystem.TrinoFileSystemFactory;
import io.trino.plugin.deltalake.transactionlog.DeltaLakeTransactionLogEntry;
import io.trino.plugin.deltalake.transactionlog.TableSnapshot;
import io.trino.plugin.deltalake.transactionlog.Transaction;
import io.trino.plugin.deltalake.transactionlog.TransactionLogAccess;
import io.trino.plugin.deltalake.util.PageListBuilder;
import io.trino.spi.Page;
import io.trino.spi.TrinoException;
import io.trino.spi.connector.ColumnMetadata;
import io.trino.spi.connector.ConnectorPageSource;
import io.trino.spi.connector.ConnectorSession;
import io.trino.spi.connector.ConnectorTableMetadata;
import io.trino.spi.connector.ConnectorTransactionHandle;
import io.trino.spi.connector.EmptyPageSource;
import io.trino.spi.connector.FixedPageSource;
import io.trino.spi.connector.SchemaTableName;
import io.trino.spi.connector.SystemTable;
import io.trino.spi.predicate.Domain;
import io.trino.spi.predicate.Range;
import io.trino.spi.predicate.TupleDomain;
import io.trino.spi.type.TypeManager;
import io.trino.spi.type.TypeSignature;

import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.IntStream;

import static com.google.common.collect.MoreCollectors.onlyElement;
import static io.airlift.json.JsonCodec.listJsonCodec;
import static io.trino.plugin.deltalake.DeltaLakeErrorCode.DELTA_LAKE_INVALID_SCHEMA;
import static io.trino.plugin.deltalake.transactionlog.TransactionLogUtil.getTransactionLogDir;
import static io.trino.plugin.deltalake.transactionlog.checkpoint.TransactionLogTail.getEntriesFromJson;
import static io.trino.spi.type.BigintType.BIGINT;
import static io.trino.spi.type.StandardTypes.JSON;
import static java.util.Objects.requireNonNull;

public class DeltaLakeTransactionsTable
implements SystemTable
{
private static final JsonCodec<List<DeltaLakeTransactionLogEntry>> TRANSACTION_LOG_ENTRIES_CODEC = listJsonCodec(DeltaLakeTransactionLogEntry.class);

private final SchemaTableName tableName;
private final String tableLocation;
private final TrinoFileSystemFactory fileSystemFactory;
private final TransactionLogAccess transactionLogAccess;
private final ConnectorTableMetadata tableMetadata;

public DeltaLakeTransactionsTable(
SchemaTableName tableName,
String tableLocation,
TrinoFileSystemFactory fileSystemFactory,
TransactionLogAccess transactionLogAccess,
TypeManager typeManager)
{
requireNonNull(typeManager, "typeManager is null");
this.tableName = requireNonNull(tableName, "tableName is null");
this.tableLocation = requireNonNull(tableLocation, "tableLocation is null");
this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null");
this.transactionLogAccess = requireNonNull(transactionLogAccess, "transactionLogAccess is null");

this.tableMetadata = new ConnectorTableMetadata(
requireNonNull(tableName, "tableName is null"),
ImmutableList.<ColumnMetadata>builder()
.add(new ColumnMetadata("version", BIGINT))
.add(new ColumnMetadata("transaction", typeManager.getType(new TypeSignature(JSON))))
.build());
}

@Override
public Distribution getDistribution()
{
return Distribution.SINGLE_COORDINATOR;
}

@Override
public ConnectorTableMetadata getTableMetadata()
{
return tableMetadata;
}

@Override
public ConnectorPageSource pageSource(ConnectorTransactionHandle transactionHandle, ConnectorSession session, TupleDomain<Integer> constraint)
{
long snapshotVersion;
try {
// Verify the transaction log is readable
SchemaTableName baseTableName = new SchemaTableName(tableName.getSchemaName(), DeltaLakeTableName.tableNameFrom(tableName.getTableName()));
TableSnapshot tableSnapshot = transactionLogAccess.loadSnapshot(session, baseTableName, tableLocation, Optional.empty());
snapshotVersion = tableSnapshot.getVersion();
transactionLogAccess.getMetadataEntry(session, tableSnapshot);
}
catch (IOException e) {
throw new TrinoException(DeltaLakeErrorCode.DELTA_LAKE_INVALID_SCHEMA, "Unable to load table metadata from location: " + tableLocation, e);
}

int versionColumnIndex = IntStream.range(0, tableMetadata.getColumns().size())
.filter(i -> tableMetadata.getColumns().get(i).getName().equals("version"))
.boxed()
.collect(onlyElement());

Optional<Long> startVersionExclusive = Optional.empty();
Optional<Long> endVersionInclusive = Optional.empty();

if (constraint.getDomains().isPresent()) {
Map<Integer, Domain> domains = constraint.getDomains().get();
if (domains.containsKey(versionColumnIndex)) {
Domain versionDomain = domains.get(versionColumnIndex); // The zero value here relies on the column ordering defined in the constructor
Range range = versionDomain.getValues().getRanges().getSpan();
if (range.isSingleValue()) {
long value = (long) range.getSingleValue();
startVersionExclusive = Optional.of(value - 1);
endVersionInclusive = Optional.of(value);
}
else {
Optional<Long> lowValue = range.getLowValue().map(Long.class::cast);
if (lowValue.isPresent()) {
startVersionExclusive = Optional.of(lowValue.get() - (range.isLowInclusive() ? 1 : 0));
}

Optional<Long> highValue = range.getHighValue().map(Long.class::cast);
if (highValue.isPresent()) {
endVersionInclusive = Optional.of(highValue.get() - (range.isHighInclusive() ? 0 : 1));
}
}
}
}

if (startVersionExclusive.isPresent() && endVersionInclusive.isPresent() && startVersionExclusive.get() >= endVersionInclusive.get()) {
return new EmptyPageSource();
}

if (endVersionInclusive.isEmpty()) {
endVersionInclusive = Optional.of(snapshotVersion);
}

TrinoFileSystem fileSystem = fileSystemFactory.create(session);
try {
List<Transaction> transactions = loadNewTailBackward(fileSystem, tableLocation, startVersionExclusive, endVersionInclusive.get()).reversed();
return new FixedPageSource(buildPages(transactions));
}
catch (TrinoException e) {
throw e;
}
catch (IOException | RuntimeException e) {
throw new TrinoException(DELTA_LAKE_INVALID_SCHEMA, "Error getting commit info entries from " + tableLocation, e);
}
}

// Load a section of the Transaction Log JSON entries. Optionally from a given end version (inclusive) through an start version (exclusive)
private static List<Transaction> loadNewTailBackward(
TrinoFileSystem fileSystem,
String tableLocation,
Optional<Long> startVersion,
long endVersion)
throws IOException
{
ImmutableList.Builder<Transaction> transactionsBuilder = ImmutableList.builder();
String transactionLogDir = getTransactionLogDir(tableLocation);

long version = endVersion;
long entryNumber = version;
boolean endOfHead = false;

while (!endOfHead) {
Optional<List<DeltaLakeTransactionLogEntry>> results = getEntriesFromJson(entryNumber, transactionLogDir, fileSystem);
if (results.isPresent()) {
transactionsBuilder.add(new Transaction(version, results.get()));
version = entryNumber;
entryNumber--;
}
else {
// When there is a gap in the transaction log version, indicate the end of the current head
endOfHead = true;
}
if ((startVersion.isPresent() && version == startVersion.get() + 1) || entryNumber < 0) {
endOfHead = true;
}
}
return transactionsBuilder.build();
}

private List<Page> buildPages(List<Transaction> transactions)
{
PageListBuilder pagesBuilder = PageListBuilder.forTable(tableMetadata);
for (Transaction transaction : transactions) {
pagesBuilder.beginRow();
pagesBuilder.appendBigint(transaction.transactionId());
pagesBuilder.appendVarchar(TRANSACTION_LOG_ENTRIES_CODEC.toJson(transaction.transactionEntries()));
pagesBuilder.endRow();
}
return pagesBuilder.build();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
import com.google.common.io.Resources;
import io.trino.testing.AbstractTestQueryFramework;
import io.trino.testing.QueryRunner;
import io.trino.testing.sql.TestTable;
import org.intellij.lang.annotations.Language;
import org.junit.jupiter.api.Test;

Expand All @@ -25,6 +26,8 @@
import java.nio.file.Path;

import static io.trino.plugin.deltalake.TestingDeltaLakeUtils.copyDirectoryContents;
import static io.trino.testing.TestingAccessControlManager.TestingPrivilegeType.SELECT_COLUMN;
import static io.trino.testing.TestingAccessControlManager.privilege;
import static io.trino.testing.TestingNames.randomNameSuffix;
import static org.assertj.core.api.Assertions.assertThat;

Expand Down Expand Up @@ -100,6 +103,30 @@ public void testHistoryTable()
}
}

@Test
void testTransactionsTable()
{
try (TestTable table = new TestTable(getQueryRunner()::execute, "test_transactions", "(col int)")) {
assertThat((String) computeScalar("SELECT transaction FROM \"" + table.getName() + "$transactions\""))
.contains("commitInfo", "protocol", "metaData");
}
}

@Test
public void testTransactionsTableAccessControl()
{
try (TestTable table = new TestTable(getQueryRunner()::execute, "test_transactions", "(col int)")) {
// TODO Disallow access to transactions table when the user can't access the base table
assertAccessAllowed(
"SELECT * FROM \"" + table.getName() + "$transactions\"",
privilege(table.getName(), SELECT_COLUMN));
assertAccessDenied(
"SELECT * FROM \"" + table.getName() + "$transactions\"",
"Cannot select from columns .*",
privilege(table.getName() + "$transactions", SELECT_COLUMN));
}
}

@Test
public void testPropertiesTable()
{
Expand Down

0 comments on commit c16cc73

Please sign in to comment.