Skip to content
Open
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
10 changes: 10 additions & 0 deletions docs/src/main/sphinx/connector/hudi.md
Original file line number Diff line number Diff line change
Expand Up @@ -170,6 +170,16 @@ GROUP BY dt;
(1 rows)
```

### Time travel queries

The connector offers the ability to query historical data. This allows viewing the state of the table at a previous snapshot.

The historical data of the table can be retrieved by specifying the last valid commit timestamp:
```sql
SELECT * FROM hudi.default.table_name FOR VERSION AS OF 20251027183851494;
```
Note: Hudi stores its table versions as a yyyyMMddHHmmssSSS representation of the commit time in the table's time zone.

### Schema and table management

Hudi supports [two types of tables](https://hudi.apache.org/docs/table_types)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import io.trino.spi.connector.ConnectorTableVersion;
import io.trino.spi.connector.Constraint;
import io.trino.spi.connector.ConstraintApplicationResult;
import io.trino.spi.connector.PointerType;
import io.trino.spi.connector.RelationColumnsMetadata;
import io.trino.spi.connector.SchemaTableName;
import io.trino.spi.connector.SchemaTablePrefix;
Expand Down Expand Up @@ -82,6 +83,8 @@
public class HudiMetadata
implements ConnectorMetadata
{
private static final int VERSION_LENGTH = "yyyyMMddHHmmssSSS".length();
private static final long MAX_VERSION = 99999999999999999L;
private final HiveMetastore metastore;
private final TrinoFileSystemFactory fileSystemFactory;
private final TypeManager typeManager;
Expand All @@ -104,10 +107,6 @@ public List<String> listSchemaNames(ConnectorSession session)
@Override
public HudiTableHandle getTableHandle(ConnectorSession session, SchemaTableName tableName, Optional<ConnectorTableVersion> startVersion, Optional<ConnectorTableVersion> endVersion)
{
if (startVersion.isPresent() || endVersion.isPresent()) {
throw new TrinoException(NOT_SUPPORTED, "This connector does not support versioned tables");
}

if (isHiveSystemSchema(tableName.getSchemaName())) {
return null;
}
Expand All @@ -130,7 +129,30 @@ public HudiTableHandle getTableHandle(ConnectorSession session, SchemaTableName
COPY_ON_WRITE,
getPartitionKeyColumnHandles(table.get(), typeManager),
TupleDomain.all(),
TupleDomain.all());
TupleDomain.all(),
getReadTimestamp(startVersion, endVersion));
}

static long getReadTimestamp(Optional<ConnectorTableVersion> startVersion, Optional<ConnectorTableVersion> endVersion)
{
if (startVersion.isPresent()) {
throw new TrinoException(NOT_SUPPORTED, "Read table with start version is not supported");
}

if (endVersion.isPresent() && endVersion.get().getPointerType().equals(PointerType.TEMPORAL)) {
throw new TrinoException(NOT_SUPPORTED, "Cannot read 'TIMESTAMP' of Hudi table, use 'VERSION' instead");
}

long readTimestamp = MAX_VERSION;
if (endVersion.isPresent()) {
if (endVersion.get().getVersion() instanceof Long versionNumber && String.valueOf(versionNumber).length() == VERSION_LENGTH) {
readTimestamp = versionNumber;
}
else {
throw new TrinoException(NOT_SUPPORTED, "Provided read timestamp must be a number in format yyyyMMddHHmmssSSS of the table's time zone");
}
}
return readTimestamp;
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,7 @@ public class HudiTableHandle
private final Set<HiveColumnHandle> constraintColumns;
private final TupleDomain<HiveColumnHandle> partitionPredicates;
private final TupleDomain<HiveColumnHandle> regularPredicates;
private final long readTimestamp;

@JsonCreator
public HudiTableHandle(
Expand All @@ -50,9 +51,10 @@ public HudiTableHandle(
@JsonProperty("tableType") HoodieTableType tableType,
@JsonProperty("partitionColumns") List<HiveColumnHandle> partitionColumns,
@JsonProperty("partitionPredicates") TupleDomain<HiveColumnHandle> partitionPredicates,
@JsonProperty("regularPredicates") TupleDomain<HiveColumnHandle> regularPredicates)
@JsonProperty("regularPredicates") TupleDomain<HiveColumnHandle> regularPredicates,
@JsonProperty("readAsOf") long readTimestamp)
{
this(schemaName, tableName, basePath, tableType, partitionColumns, ImmutableSet.of(), partitionPredicates, regularPredicates);
this(schemaName, tableName, basePath, tableType, partitionColumns, ImmutableSet.of(), partitionPredicates, regularPredicates, readTimestamp);
}

public HudiTableHandle(
Expand All @@ -63,7 +65,8 @@ public HudiTableHandle(
List<HiveColumnHandle> partitionColumns,
Set<HiveColumnHandle> constraintColumns,
TupleDomain<HiveColumnHandle> partitionPredicates,
TupleDomain<HiveColumnHandle> regularPredicates)
TupleDomain<HiveColumnHandle> regularPredicates,
long readTimestamp)
{
this.schemaName = requireNonNull(schemaName, "schemaName is null");
this.tableName = requireNonNull(tableName, "tableName is null");
Expand All @@ -73,6 +76,7 @@ public HudiTableHandle(
this.constraintColumns = requireNonNull(constraintColumns, "constraintColumns is null");
this.partitionPredicates = requireNonNull(partitionPredicates, "partitionPredicates is null");
this.regularPredicates = requireNonNull(regularPredicates, "regularPredicates is null");
this.readTimestamp = readTimestamp;
}

@JsonProperty
Expand Down Expand Up @@ -124,6 +128,12 @@ public TupleDomain<HiveColumnHandle> getRegularPredicates()
return regularPredicates;
}

@JsonProperty
public long getReadTimestamp()
{
return readTimestamp;
}

public SchemaTableName getSchemaTableName()
{
return schemaTableName(schemaName, tableName);
Expand All @@ -142,7 +152,8 @@ HudiTableHandle applyPredicates(
partitionColumns,
constraintColumns,
partitionPredicates.intersect(partitionTupleDomain),
regularPredicates.intersect(regularTupleDomain));
regularPredicates.intersect(regularTupleDomain),
readTimestamp);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
import java.util.Optional;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.Stream;

import static com.google.common.collect.ImmutableList.toImmutableList;
import static io.airlift.units.DataSize.Unit.MEGABYTE;
Expand All @@ -51,6 +52,7 @@ public class HudiReadOptimizedDirectoryLister
private final HoodieTableFileSystemView fileSystemView;
private final List<Column> partitionColumns;
private final Map<String, HudiPartitionInfo> allPartitionInfoMap;
private final HudiTableHandle tableHandle;

public HudiReadOptimizedDirectoryLister(
HudiTableHandle tableHandle,
Expand All @@ -76,13 +78,23 @@ public HudiReadOptimizedDirectoryLister(
tableHandle.getPartitionPredicates(),
hiveTable,
hiveMetastore)));
this.tableHandle = tableHandle;
}

private static StoragePathInfo getStoragePathInfo(HoodieBaseFile baseFile)
{
if (baseFile.getBootstrapBaseFile().isPresent()) {
return baseFile.getBootstrapBaseFile().get().getPathInfo();
}
return baseFile.getPathInfo();
}

@Override
public List<HudiFileStatus> listStatus(HudiPartitionInfo partitionInfo)
{
LOG.debug("List partition: partitionInfo=%s", partitionInfo);
return fileSystemView.getLatestBaseFiles(partitionInfo.getRelativePartitionPath())
Stream<HoodieBaseFile> baseFileStream = fileSystemView.getLatestBaseFilesBeforeOrOn(partitionInfo.getRelativePartitionPath(), String.valueOf(tableHandle.getReadTimestamp()));
return baseFileStream
.map(HudiReadOptimizedDirectoryLister::getStoragePathInfo)
.map(fileEntry -> new HudiFileStatus(
Location.of(fileEntry.getPath().toString()),
Expand All @@ -106,12 +118,4 @@ public void close()
fileSystemView.close();
}
}

private static StoragePathInfo getStoragePathInfo(HoodieBaseFile baseFile)
{
if (baseFile.getBootstrapBaseFile().isPresent()) {
return baseFile.getBootstrapBaseFile().get().getPathInfo();
}
return baseFile.getPathInfo();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -160,7 +160,7 @@ public static void main(String[] args)
hiveMinioDataLake.start();
QueryRunner queryRunner = builder(hiveMinioDataLake)
.addCoordinatorProperty("http-server.http.port", "8080")
.setDataLoader(new TpchHudiTablesInitializer(TpchTable.getTables()))
.setDataLoader(new TpchHudiTablesInitializer(TpchTable.getTables(), "0"))
.build();

log.info("======== SERVER STARTED ========");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,13 +25,15 @@
public class TestHudiConnectorTest
extends BaseConnectorTest
{
private static final long COMMIT_TIMESTAMP = 20251027183851494L;

@Override
protected QueryRunner createQueryRunner()
throws Exception
{
return HudiQueryRunner.builder()
.addConnectorProperty("hudi.columns-to-hide", COLUMNS_TO_HIDE)
.setDataLoader(new TpchHudiTablesInitializer(REQUIRED_TPCH_TABLES))
.setDataLoader(new TpchHudiTablesInitializer(REQUIRED_TPCH_TABLES, String.valueOf(COMMIT_TIMESTAMP)))
.build();
}

Expand Down Expand Up @@ -87,4 +89,23 @@ public void testHideHiveSysSchema()
assertThat(computeActual("SHOW SCHEMAS").getOnlyColumnAsSet()).doesNotContain("sys");
assertQueryFails("SHOW TABLES IN hudi.sys", ".*Schema 'sys' does not exist");
}

@Override
protected void verifyVersionedQueryFailurePermissible(Exception e)
{
assertThat(e)
.hasMessageMatching("Read table with start version is not supported|" +
"Cannot read 'TIMESTAMP' of Hudi table, use 'VERSION' instead|" +
"Provided read timestamp must be a number in format yyyyMMddHHmmssSSS of the table's time zone");
}

@Test
public void testSelectTableUsingTargetIdVersion()
{
assertQuery("SELECT nationkey FROM hudi.tests.nation", "VALUES 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24");
assertThat(query("SELECT nationkey FROM hudi.tests.nation FOR VERSION AS OF " + (COMMIT_TIMESTAMP - 1))).returnsEmptyResult();
assertQueryFails("SELECT nationkey FROM hudi.tests.nation FOR VERSION AS OF 0", "Provided read timestamp must be a number in format yyyyMMddHHmmssSSS of the table's time zone");
assertQueryFails("SELECT nationkey FROM hudi.tests.nation FOR VERSION AS OF 'version'", "Provided read timestamp must be a number in format yyyyMMddHHmmssSSS of the table's time zone");
assertQueryFails("SELECT nationkey FROM hudi.tests.nation FOR TIMESTAMP AS OF TIMESTAMP '2025-10-29 15:00:00'", "Cannot read 'TIMESTAMP' of Hudi table, use 'VERSION' instead");
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@ protected QueryRunner createQueryRunner()

return HudiQueryRunner.builder(hiveMinioDataLake)
.addConnectorProperty("hudi.columns-to-hide", COLUMNS_TO_HIDE)
.setDataLoader(new TpchHudiTablesInitializer(REQUIRED_TPCH_TABLES))
.setDataLoader(new TpchHudiTablesInitializer(REQUIRED_TPCH_TABLES, "0"))
.build();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,7 @@ protected QueryRunner createQueryRunner()

queryRunner.execute("CREATE SCHEMA hive.default");

TpchHudiTablesInitializer tpchHudiTablesInitializer = new TpchHudiTablesInitializer(List.of(NATION));
TpchHudiTablesInitializer tpchHudiTablesInitializer = new TpchHudiTablesInitializer(List.of(NATION), "0");
tpchHudiTablesInitializer.initializeTables(queryRunner, Location.of(dataDirectory.toString()), "default");

copyTpchTables(queryRunner, "tpch", TINY_SCHEMA_NAME, hiveSession, ImmutableList.of(TpchTable.REGION));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -113,10 +113,12 @@ public class TpchHudiTablesInitializer
private static final HdfsContext CONTEXT = new HdfsContext(SESSION);

private final List<TpchTable<?>> tpchTables;
private final String commitTimestamp;

public TpchHudiTablesInitializer(List<TpchTable<?>> tpchTables)
public TpchHudiTablesInitializer(List<TpchTable<?>> tpchTables, String commitTimestamp)
{
this.tpchTables = requireNonNull(tpchTables, "tpchTables is null");
this.commitTimestamp = commitTimestamp;
}

@Override
Expand Down Expand Up @@ -166,9 +168,8 @@ public void load(TpchTable<?> tpchTables, QueryRunner queryRunner, java.nio.file
.map(MaterializedRow::getFields)
.map(recordConverter::toRecord)
.collect(Collectors.toList());
String timestamp = "0";
writeClient.startCommitWithTime(timestamp);
writeClient.insert(records, timestamp);
writeClient.startCommitWithTime(commitTimestamp);
writeClient.insert(records, commitTimestamp);
}
}

Expand Down