Skip to content

Commit

Permalink
Read delta lake table history when initial transaction logs are deleted
Browse files Browse the repository at this point in the history
  • Loading branch information
krvikash committed Sep 13, 2023
1 parent fd5b247 commit 1922381
Show file tree
Hide file tree
Showing 3 changed files with 77 additions and 6 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@
import io.trino.plugin.deltalake.transactionlog.DeltaLakeTransactionLogEntry;
import io.trino.plugin.deltalake.transactionlog.TableSnapshot;
import io.trino.plugin.deltalake.transactionlog.TransactionLogAccess;
import io.trino.plugin.deltalake.transactionlog.checkpoint.TransactionLogTail;
import io.trino.plugin.deltalake.util.PageListBuilder;
import io.trino.spi.Page;
import io.trino.spi.TrinoException;
Expand Down Expand Up @@ -49,6 +48,8 @@
import static com.google.common.collect.ImmutableList.toImmutableList;
import static com.google.common.collect.MoreCollectors.onlyElement;
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.BooleanType.BOOLEAN;
import static io.trino.spi.type.TimestampWithTimeZoneType.TIMESTAMP_TZ_MILLIS;
Expand Down Expand Up @@ -110,10 +111,12 @@ public ConnectorTableMetadata getTableMetadata()
@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(baseTableName, tableLocation, session);
snapshotVersion = tableSnapshot.getVersion();
transactionLogAccess.getMetadataEntry(tableSnapshot, session);
}
catch (IOException e) {
Expand Down Expand Up @@ -156,12 +159,17 @@ public ConnectorPageSource pageSource(ConnectorTransactionHandle transactionHand
return new EmptyPageSource();
}

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

TrinoFileSystem fileSystem = fileSystemFactory.create(session);
try {
List<CommitInfoEntry> commitInfoEntries = TransactionLogTail.loadNewTail(fileSystem, tableLocation, startVersionExclusive, endVersionInclusive).getFileEntries().stream()
List<CommitInfoEntry> commitInfoEntries = loadNewTailBackward(fileSystem, tableLocation, startVersionExclusive, endVersionInclusive.get()).stream()
.map(DeltaLakeTransactionLogEntry::getCommitInfo)
.filter(Objects::nonNull)
.collect(toImmutableList());
.collect(toImmutableList())
.reverse();
return new FixedPageSource(buildPages(session, commitInfoEntries));
}
catch (TrinoException e) {
Expand All @@ -172,6 +180,39 @@ public ConnectorPageSource pageSource(ConnectorTransactionHandle transactionHand
}
}

// Load a section of the Transaction Log JSON entries. Optionally from a given end version (inclusive) through an start version (exclusive)
private static List<DeltaLakeTransactionLogEntry> loadNewTailBackward(
TrinoFileSystem fileSystem,
String tableLocation,
Optional<Long> startVersion,
long endVersion)
throws IOException
{
ImmutableList.Builder<DeltaLakeTransactionLogEntry> entriesBuilder = 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()) {
entriesBuilder.addAll(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 entriesBuilder.build();
}

private List<Page> buildPages(ConnectorSession session, List<CommitInfoEntry> commitInfoEntries)
{
PageListBuilder pagesBuilder = PageListBuilder.forTable(tableMetadata);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1959,6 +1959,36 @@ public void testHistoryTable()
}
}

@Test
public void testHistoryTableWithDeletedTransactionLog()
{
try (TestTable table = new TestTable(
getQueryRunner()::execute,
"test_history_table_with_deleted_transaction_log",
"(int_col INTEGER) WITH (checkpoint_interval = 3)")) {
assertUpdate("INSERT INTO " + table.getName() + " VALUES 1, 2, 3", 3);
assertUpdate("INSERT INTO " + table.getName() + " VALUES 4, 5, 6", 3);
assertUpdate("DELETE FROM " + table.getName() + " WHERE int_col = 1", 1);
assertUpdate("UPDATE " + table.getName() + " SET int_col = int_col * 2 WHERE int_col = 6", 1);

String tableLocation = getTableLocation(table.getName());
// Remove first two transaction logs to mimic log retention duration exceeds
String key = tableLocation.substring(bucketUrl().length());
MinioClient minio = hiveMinioDataLake.getMinioClient();
minio.removeObject(bucketName, "%s/_delta_log/%020d.json".formatted(key, 0));
minio.removeObject(bucketName, "%s/_delta_log/%020d.json".formatted(key, 1));

assertQuery("SELECT version, operation FROM \"" + table.getName() + "$history\"", "VALUES (2, 'WRITE'), (3, 'MERGE'), (4, 'MERGE')");
assertThat(query("SELECT version, operation FROM \"" + table.getName() + "$history\" WHERE version = 1")).returnsEmptyResult();
assertQuery("SELECT version, operation FROM \"" + table.getName() + "$history\" WHERE version = 3", "VALUES (3, 'MERGE')");
assertQuery("SELECT version, operation FROM \"" + table.getName() + "$history\" WHERE version > 3", "VALUES (4, 'MERGE')");
assertQuery("SELECT version, operation FROM \"" + table.getName() + "$history\" WHERE version < 3", "VALUES (2, 'WRITE')");
assertQuery("SELECT version, operation FROM \"" + table.getName() + "$history\" WHERE version >= 3 OR version = 1", "VALUES (3, 'MERGE'), (4, 'MERGE')");
assertQuery("SELECT version, operation FROM \"" + table.getName() + "$history\" WHERE version >= 1 AND version < 3", "VALUES (2, 'WRITE')");
assertThat(query("SELECT version, operation FROM \"" + table.getName() + "$history\" WHERE version > 1 AND version < 2")).returnsEmptyResult();
}
}

/**
* @see BaseDeltaLakeRegisterTableProcedureTest for more detailed tests
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -346,7 +346,7 @@ public void testHistorySystemTable()
.addCopies(new FileOperation(TRANSACTION_LOG_JSON, "00000000000000000002.json", INPUT_FILE_NEW_STREAM), 2)
.addCopies(new FileOperation(TRANSACTION_LOG_JSON, "00000000000000000003.json", INPUT_FILE_NEW_STREAM), 2)
.addCopies(new FileOperation(TRANSACTION_LOG_JSON, "00000000000000000004.json", INPUT_FILE_NEW_STREAM), 2)
.addCopies(new FileOperation(TRANSACTION_LOG_JSON, "00000000000000000005.json", INPUT_FILE_NEW_STREAM), 2)
.add(new FileOperation(TRANSACTION_LOG_JSON, "00000000000000000005.json", INPUT_FILE_NEW_STREAM))
.add(new FileOperation(LAST_CHECKPOINT, "_last_checkpoint", INPUT_FILE_NEW_STREAM))
.build());

Expand All @@ -368,7 +368,7 @@ public void testHistorySystemTable()
.add(new FileOperation(TRANSACTION_LOG_JSON, "00000000000000000002.json", INPUT_FILE_NEW_STREAM))
.add(new FileOperation(TRANSACTION_LOG_JSON, "00000000000000000003.json", INPUT_FILE_NEW_STREAM))
.addCopies(new FileOperation(TRANSACTION_LOG_JSON, "00000000000000000004.json", INPUT_FILE_NEW_STREAM), 2)
.addCopies(new FileOperation(TRANSACTION_LOG_JSON, "00000000000000000005.json", INPUT_FILE_NEW_STREAM), 2)
.add(new FileOperation(TRANSACTION_LOG_JSON, "00000000000000000005.json", INPUT_FILE_NEW_STREAM))
.add(new FileOperation(LAST_CHECKPOINT, "_last_checkpoint", INPUT_FILE_NEW_STREAM))
.build());

Expand All @@ -379,7 +379,7 @@ public void testHistorySystemTable()
.addCopies(new FileOperation(TRANSACTION_LOG_JSON, "00000000000000000002.json", INPUT_FILE_NEW_STREAM), 2)
.addCopies(new FileOperation(TRANSACTION_LOG_JSON, "00000000000000000003.json", INPUT_FILE_NEW_STREAM), 2)
.addCopies(new FileOperation(TRANSACTION_LOG_JSON, "00000000000000000004.json", INPUT_FILE_NEW_STREAM), 2)
.addCopies(new FileOperation(TRANSACTION_LOG_JSON, "00000000000000000005.json", INPUT_FILE_NEW_STREAM), 2)
.add(new FileOperation(TRANSACTION_LOG_JSON, "00000000000000000005.json", INPUT_FILE_NEW_STREAM))
.add(new FileOperation(LAST_CHECKPOINT, "_last_checkpoint", INPUT_FILE_NEW_STREAM))
.build());

Expand Down

0 comments on commit 1922381

Please sign in to comment.