diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/HudiQueryRunner.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/HudiQueryRunner.java index 46963c397335..25c40dbe6069 100644 --- a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/HudiQueryRunner.java +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/HudiQueryRunner.java @@ -22,8 +22,8 @@ import io.trino.metastore.HiveMetastoreFactory; import io.trino.plugin.base.util.Closables; import io.trino.plugin.hive.containers.Hive3MinioDataLake; +import io.trino.plugin.hudi.testing.DynamicHudiTablesInitializer; import io.trino.plugin.hudi.testing.HudiTablesInitializer; -import io.trino.plugin.hudi.testing.ResourceHudiTablesInitializer; import io.trino.plugin.hudi.testing.TpchHudiTablesInitializer; import io.trino.spi.security.PrincipalType; import io.trino.testing.DistributedQueryRunner; @@ -138,7 +138,7 @@ static void main() QueryRunner queryRunner = builder() .addCoordinatorProperty("http-server.http.port", "8080") - .setDataLoader(new ResourceHudiTablesInitializer()) + .setDataLoader(new DynamicHudiTablesInitializer()) .build(); log.info("======== SERVER STARTED ========"); diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiConnectorParquetColumnNamesTest.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiConnectorParquetColumnNamesTest.java index 7e938ba3aed1..de833ffae731 100644 --- a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiConnectorParquetColumnNamesTest.java +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiConnectorParquetColumnNamesTest.java @@ -13,7 +13,7 @@ */ package io.trino.plugin.hudi; -import io.trino.plugin.hudi.testing.ResourceHudiTablesInitializer; +import io.trino.plugin.hudi.testing.DynamicHudiTablesInitializer; import io.trino.testing.QueryRunner; public class TestHudiConnectorParquetColumnNamesTest @@ -25,7 +25,7 @@ protected QueryRunner createQueryRunner() { return HudiQueryRunner.builder() .addConnectorProperty("hudi.parquet.use-column-names", "false") - .setDataLoader(new ResourceHudiTablesInitializer()) + .setDataLoader(new DynamicHudiTablesInitializer()) .build(); } } diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiSmokeTest.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiSmokeTest.java index 24abd8b5509c..235a84b64573 100644 --- a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiSmokeTest.java +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiSmokeTest.java @@ -17,7 +17,7 @@ import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.TrinoInputFile; -import io.trino.plugin.hudi.testing.ResourceHudiTablesInitializer; +import io.trino.plugin.hudi.testing.DynamicHudiTablesInitializer; import io.trino.spi.security.ConnectorIdentity; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.QueryRunner; @@ -26,21 +26,22 @@ import java.time.ZonedDateTime; -import static io.trino.plugin.hudi.testing.ResourceHudiTablesInitializer.TestingTable.HUDI_COW_PT_TBL; -import static io.trino.plugin.hudi.testing.ResourceHudiTablesInitializer.TestingTable.HUDI_NON_PART_COW; -import static io.trino.plugin.hudi.testing.ResourceHudiTablesInitializer.TestingTable.STOCK_TICKS_COW; -import static io.trino.plugin.hudi.testing.ResourceHudiTablesInitializer.TestingTable.STOCK_TICKS_MOR; import static org.assertj.core.api.Assertions.assertThat; public class TestHudiSmokeTest extends AbstractTestQueryFramework { + private static final String HUDI_NON_PART_COW = "hudi_non_part_cow"; + private static final String HUDI_COW_PT_TBL = "hudi_cow_pt_tbl"; + private static final String STOCK_TICKS_COW = "stock_ticks_cow"; + private static final String STOCK_TICKS_MOR = "stock_ticks_mor"; + @Override protected QueryRunner createQueryRunner() throws Exception { return HudiQueryRunner.builder() - .setDataLoader(new ResourceHudiTablesInitializer()) + .setDataLoader(new DynamicHudiTablesInitializer()) .build(); } @@ -125,13 +126,21 @@ public void testShowCreateTable() @Test public void testMetaColumns() { - assertQuery("SELECT _hoodie_commit_time FROM hudi_cow_pt_tbl", "VALUES ('20220906063435640'), ('20220906063456550')"); - assertQuery("SELECT _hoodie_commit_seqno FROM hudi_cow_pt_tbl", "VALUES ('20220906063435640_0_0'), ('20220906063456550_0_0')"); - assertQuery("SELECT _hoodie_record_key FROM hudi_cow_pt_tbl", "VALUES ('id:1'), ('id:2')"); - assertQuery("SELECT _hoodie_partition_path FROM hudi_cow_pt_tbl", "VALUES ('dt=2021-12-09/hh=10'), ('dt=2021-12-09/hh=11')"); - assertQuery( - "SELECT _hoodie_file_name FROM hudi_cow_pt_tbl", - "VALUES ('719c3273-2805-4124-b1ac-e980dada85bf-0_0-27-1215_20220906063435640.parquet'), ('4a3fcb9b-65eb-4f6e-acf9-7b0764bb4dd1-0_0-70-2444_20220906063456550.parquet')"); + // Verify commit times exist and there are 2 records + assertQuery("SELECT count(*) FROM hudi_cow_pt_tbl WHERE _hoodie_commit_time IS NOT NULL", "VALUES 2"); + + // Verify commit sequences exist + assertQuery("SELECT count(*) FROM hudi_cow_pt_tbl WHERE _hoodie_commit_seqno IS NOT NULL", "VALUES 2"); + + // Verify record keys are correct + assertQuery("SELECT _hoodie_record_key FROM hudi_cow_pt_tbl ORDER BY _hoodie_record_key", "VALUES ('id:1'), ('id:2')"); + + // Verify partition paths are correct + assertQuery("SELECT _hoodie_partition_path FROM hudi_cow_pt_tbl ORDER BY _hoodie_partition_path", + "VALUES ('dt=2021-12-09/hh=10'), ('dt=2021-12-09/hh=11')"); + + // Verify file names exist and contain .parquet extension + assertQuery("SELECT count(*) FROM hudi_cow_pt_tbl WHERE _hoodie_file_name LIKE '%.parquet'", "VALUES 2"); } @Test @@ -178,7 +187,7 @@ public void testPartitionFilterRequired() assertQueryFails( session, "SELECT * FROM " + HUDI_COW_PT_TBL, - "Filter required on tests." + HUDI_COW_PT_TBL.getTableName() + " for at least one of the partition columns: dt, hh"); + "Filter required on tests." + HUDI_COW_PT_TBL + " for at least one of the partition columns: dt, hh"); } @Test @@ -189,7 +198,7 @@ public void testPartitionFilterRequiredPredicateOnNonPartitionColumn() assertQueryFails( session, "SELECT * FROM " + HUDI_COW_PT_TBL + " WHERE id = 1", - "Filter required on tests." + HUDI_COW_PT_TBL.getTableName() + " for at least one of the partition columns: dt, hh"); + "Filter required on tests." + HUDI_COW_PT_TBL + " for at least one of the partition columns: dt, hh"); } @Test @@ -224,14 +233,14 @@ public void testPartitionFilterRequiredFilterRemovedByPlanner() assertQueryFails( session, "SELECT id FROM " + HUDI_COW_PT_TBL + " WHERE dt IS NOT null OR true", - "Filter required on tests." + HUDI_COW_PT_TBL.getTableName() + " for at least one of the partition columns: dt, hh"); + "Filter required on tests." + HUDI_COW_PT_TBL + " for at least one of the partition columns: dt, hh"); } @Test public void testPartitionFilterRequiredOnJoin() { Session session = withPartitionFilterRequired(getSession()); - @Language("RegExp") String errorMessage = "Filter required on tests." + HUDI_COW_PT_TBL.getTableName() + " for at least one of the partition columns: dt, hh"; + @Language("RegExp") String errorMessage = "Filter required on tests." + HUDI_COW_PT_TBL + " for at least one of the partition columns: dt, hh"; // ON with partition column assertQueryFails( @@ -290,7 +299,7 @@ public void testPartitionFilterRequiredOnJoinBothTablePartitioned() assertQueryFails( session, "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_COW_PT_TBL + " t2 ON (t1.dt = t2.dt)", - "Filter required on tests." + HUDI_COW_PT_TBL.getTableName() + " for at least one of the partition columns: dt, hh"); + "Filter required on tests." + HUDI_COW_PT_TBL + " for at least one of the partition columns: dt, hh"); // ON with partition column and WHERE with same left table's partition column assertQuery( session, @@ -302,7 +311,7 @@ public void testPartitionFilterRequiredOnJoinBothTablePartitioned() "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_COW_PT_TBL + " t2 ON (t1.dt = t2.dt) WHERE t2.dt = '2021-12-09'", "VALUES ('a1', 'a1'), ('a2', 'a2'), ('a1', 'a2'), ('a2', 'a1')"); - @Language("RegExp") String errorMessage = "Filter required on tests." + HUDI_COW_PT_TBL.getTableName() + " for at least one of the partition columns: dt, hh"; + @Language("RegExp") String errorMessage = "Filter required on tests." + HUDI_COW_PT_TBL + " for at least one of the partition columns: dt, hh"; // ON with partition column and WHERE with different left table's partition column assertQueryFails(session, "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_COW_PT_TBL + " t2 ON (t1.dt = t2.dt) WHERE t1.hh = '10'", errorMessage); // ON with partition column and WHERE with different right table's partition column @@ -325,7 +334,7 @@ public void testPartitionFilterRequiredWithLike() assertQueryFails( session, "SELECT name FROM " + HUDI_COW_PT_TBL + " WHERE name LIKE '%1'", - "Filter required on tests." + HUDI_COW_PT_TBL.getTableName() + " for at least one of the partition columns: dt, hh"); + "Filter required on tests." + HUDI_COW_PT_TBL + " for at least one of the partition columns: dt, hh"); } @Test @@ -346,7 +355,7 @@ public void testPartitionFilterRequiredFilterIncluded() assertQuery(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE hh = '12' AND dt = '2021-12-19'", "VALUES 0"); // Predicate which could not be translated into tuple domain - @Language("RegExp") String errorMessage = "Filter required on tests." + HUDI_COW_PT_TBL.getTableName() + " for at least one of the partition columns: dt, hh"; + @Language("RegExp") String errorMessage = "Filter required on tests." + HUDI_COW_PT_TBL + " for at least one of the partition columns: dt, hh"; assertQueryFails(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE CAST(hh AS INTEGER) % 2 = 0", errorMessage); assertQueryFails(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE CAST(hh AS INTEGER) - 11 = 0", errorMessage); assertQueryFails(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE CAST(hh AS INTEGER) * 2 = 20", errorMessage); diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiSystemTables.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiSystemTables.java index 1bca5dfaa0df..b2613143d15a 100644 --- a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiSystemTables.java +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiSystemTables.java @@ -13,7 +13,7 @@ */ package io.trino.plugin.hudi; -import io.trino.plugin.hudi.testing.ResourceHudiTablesInitializer; +import io.trino.plugin.hudi.testing.DynamicHudiTablesInitializer; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.QueryRunner; import org.junit.jupiter.api.Test; @@ -26,7 +26,7 @@ protected QueryRunner createQueryRunner() throws Exception { return HudiQueryRunner.builder() - .setDataLoader(new ResourceHudiTablesInitializer()) + .setDataLoader(new DynamicHudiTablesInitializer()) .build(); } @@ -38,10 +38,10 @@ public void testTimelineTable() "('action', 'varchar', '', '')," + "('state', 'varchar', '', '')"); - assertQuery("SELECT timestamp, action, state FROM tests.\"hudi_cow_pt_tbl$timeline\"", - "VALUES ('20220906063435640', 'commit', 'COMPLETED'), ('20220906063456550', 'commit', 'COMPLETED')"); + assertQuery("SELECT action, state FROM tests.\"hudi_cow_pt_tbl$timeline\"", + "VALUES ('commit', 'COMPLETED'), ('commit', 'COMPLETED')"); - assertQueryFails("SELECT timestamp, action, state FROM tests.\"non_existing$timeline\"", + assertQueryFails("SELECT action, state FROM tests.\"non_existing$timeline\"", ".*Table 'hudi.tests.\"non_existing\\$timeline\"' does not exist"); } diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/DynamicHudiTablesInitializer.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/DynamicHudiTablesInitializer.java new file mode 100644 index 000000000000..e8495c192ffa --- /dev/null +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/DynamicHudiTablesInitializer.java @@ -0,0 +1,244 @@ +/* + * 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.hudi.testing; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.airlift.log.Logger; +import io.trino.filesystem.Location; +import io.trino.filesystem.TrinoFileSystem; +import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.hdfs.HdfsContext; +import io.trino.metastore.Column; +import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; +import io.trino.metastore.Partition; +import io.trino.metastore.PartitionStatistics; +import io.trino.metastore.PartitionWithStatistics; +import io.trino.metastore.PrincipalPrivileges; +import io.trino.metastore.StorageFormat; +import io.trino.metastore.Table; +import io.trino.plugin.hudi.HudiConnector; +import io.trino.spi.security.ConnectorIdentity; +import io.trino.testing.QueryRunner; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.client.HoodieJavaWriteClient; +import org.apache.hudi.client.common.HoodieJavaEngineContext; +import org.apache.hudi.common.bootstrap.index.NoOpBootstrapIndex; +import org.apache.hudi.common.model.HoodieAvroPayload; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.HoodieTableVersion; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.storage.StorageConfiguration; +import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static com.google.common.io.MoreFiles.deleteRecursively; +import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; +import static io.trino.hive.formats.HiveClassNames.HUDI_PARQUET_INPUT_FORMAT; +import static io.trino.hive.formats.HiveClassNames.MAPRED_PARQUET_OUTPUT_FORMAT_CLASS; +import static io.trino.hive.formats.HiveClassNames.PARQUET_HIVE_SERDE_CLASS; +import static io.trino.plugin.hive.HivePartitionManager.extractPartitionValues; +import static io.trino.plugin.hive.HiveTestUtils.HDFS_ENVIRONMENT; +import static io.trino.plugin.hive.TableType.EXTERNAL_TABLE; +import static io.trino.testing.TestingConnectorSession.SESSION; +import static java.nio.file.Files.createTempDirectory; + +/** + * Initializes Hudi test tables by dynamically creating them using Hudi's Java client, + * rather than unzipping pre-existing table artifacts. + *

+ * This approach provides several benefits: + *

+ */ +public class DynamicHudiTablesInitializer + implements HudiTablesInitializer +{ + private static final Logger log = Logger.get(DynamicHudiTablesInitializer.class); + private static final HdfsContext HDFS_CONTEXT = new HdfsContext(SESSION); + + private final List tableDefinitions; + + public DynamicHudiTablesInitializer() + { + this.tableDefinitions = + ImmutableList.of( + new HudiNonPartCowTableDefinition(), + new HudiCowPtTblTableDefinition(), + new StockTicksCowTableDefinition(), + new StockTicksMorTableDefinition()); + } + + @Override + public void initializeTables(QueryRunner queryRunner, Location externalLocation, String schemaName) + throws Exception + { + TrinoFileSystem fileSystem = getFileSystem(queryRunner); + HiveMetastore metastore = getMetastore(queryRunner); + Location baseLocation = externalLocation.appendSuffix(schemaName); + + java.nio.file.Path tempDir = createTempDirectory("hudi-script-based-tables"); + try { + for (HudiTableDefinition tableDefinition : tableDefinitions) { + log.info("Creating Hudi table: %s", tableDefinition.getTableName()); + createTable(tableDefinition, tempDir, fileSystem, baseLocation, metastore, schemaName); + } + } + finally { + deleteRecursively(tempDir, ALLOW_INSECURE); + } + } + + /** + * Creates a single Hudi table based on the provided definition. + */ + private void createTable( + HudiTableDefinition tableDefinition, + java.nio.file.Path tempDir, + TrinoFileSystem fileSystem, + Location baseLocation, + HiveMetastore metastore, + String schemaName) + throws Exception + { + String tableName = tableDefinition.getTableName(); + java.nio.file.Path tableDir = tempDir.resolve(tableName); + + // Create Hudi write client + HoodieWriteConfig config = tableDefinition.createWriteConfig(tableDir.toString()); + try (HoodieJavaWriteClient client = createWriteClient(config, tableDir)) { + // Execute all commits defined for this table + tableDefinition.executeCommits(client); + } + + // Copy table data to target location + Location tableLocation = baseLocation.appendPath(tableName); + ResourceHudiTablesInitializer.copyDir(tableDir, fileSystem, tableLocation); + + // Register table with metastore + registerTableWithMetastore( + metastore, + schemaName, + tableName, + tableLocation, + tableDefinition.getDataColumns(), + tableDefinition.getPartitionColumns(), + tableDefinition.getPartitions()); + + log.info("Successfully created Hudi table: %s at %s", tableName, tableLocation); + } + + /** + * Creates a HoodieJavaWriteClient for writing data to a Hudi table. + */ + private HoodieJavaWriteClient createWriteClient( + HoodieWriteConfig config, + java.nio.file.Path tableDir) + { + StorageConfiguration storageConfig = new HadoopStorageConfiguration( + HDFS_ENVIRONMENT.getConfiguration(HDFS_CONTEXT, new Path(tableDir.toUri()))); + + // Initialize Hudi table metadata + try { + HoodieTableMetaClient.newTableBuilder() + .fromProperties(config.getProps()) + .setTableType(config.getTableType()) + .setTableName(config.getTableName()) + .setTableVersion(HoodieTableVersion.SIX.versionCode()) + .setBootstrapIndexClass(NoOpBootstrapIndex.class.getName()) + .setPayloadClassName(HoodieAvroPayload.class.getName()) + .initTable(storageConfig, config.getBasePath()); + } + catch (IOException e) { + throw new RuntimeException("Failed to initialize Hudi table: " + config.getTableName(), e); + } + + return new HoodieJavaWriteClient<>(new HoodieJavaEngineContext(storageConfig), config); + } + + /** + * Registers a Hudi table with the Hive metastore. + */ + private void registerTableWithMetastore( + HiveMetastore metastore, + String schemaName, + String tableName, + Location tableLocation, + List dataColumns, + List partitionColumns, + Map partitions) + { + StorageFormat storageFormat = StorageFormat.create( + PARQUET_HIVE_SERDE_CLASS, + HUDI_PARQUET_INPUT_FORMAT, + MAPRED_PARQUET_OUTPUT_FORMAT_CLASS); + + Table table = Table.builder() + .setDatabaseName(schemaName) + .setTableName(tableName) + .setTableType(EXTERNAL_TABLE.name()) + .setOwner(Optional.of("public")) + .setDataColumns(dataColumns) + .setPartitionColumns(partitionColumns) + .setParameters(ImmutableMap.of("serialization.format", "1", "EXTERNAL", "TRUE")) + .withStorage(storageBuilder -> storageBuilder + .setStorageFormat(storageFormat) + .setLocation(tableLocation.toString())) + .build(); + + metastore.createTable(table, PrincipalPrivileges.NO_PRIVILEGES); + + // Register partitions if this is a partitioned table + if (!partitions.isEmpty()) { + List partitionsToAdd = new ArrayList<>(); + partitions.forEach((partitionName, partitionPath) -> { + Partition partition = Partition.builder() + .setDatabaseName(schemaName) + .setTableName(tableName) + .setValues(extractPartitionValues(partitionName)) + .withStorage(storageBuilder -> storageBuilder + .setStorageFormat(storageFormat) + .setLocation(tableLocation.appendPath(partitionPath).toString())) + .setColumns(dataColumns) + .build(); + partitionsToAdd.add(new PartitionWithStatistics(partition, partitionName, PartitionStatistics.empty())); + }); + metastore.addPartitions(schemaName, tableName, partitionsToAdd); + } + } + + private TrinoFileSystem getFileSystem(QueryRunner queryRunner) + { + return ((HudiConnector) queryRunner.getCoordinator().getConnector("hudi")).getInjector() + .getInstance(TrinoFileSystemFactory.class) + .create(ConnectorIdentity.ofUser("test")); + } + + private HiveMetastore getMetastore(QueryRunner queryRunner) + { + return ((HudiConnector) queryRunner.getCoordinator().getConnector("hudi")).getInjector() + .getInstance(HiveMetastoreFactory.class) + .createMetastore(Optional.empty()); + } +} diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/HudiCowPtTblTableDefinition.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/HudiCowPtTblTableDefinition.java new file mode 100644 index 000000000000..f88d540e317d --- /dev/null +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/HudiCowPtTblTableDefinition.java @@ -0,0 +1,129 @@ +/* + * 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.hudi.testing; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.trino.metastore.Column; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.hudi.client.HoodieJavaWriteClient; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.model.HoodieAvroPayload; +import org.apache.hudi.common.model.HoodieRecord; + +import java.util.List; +import java.util.Map; + +import static io.trino.metastore.HiveType.HIVE_LONG; +import static io.trino.metastore.HiveType.HIVE_STRING; +import static io.trino.plugin.hudi.testing.HudiRecordFactory.fieldValues; +import static org.apache.hudi.common.model.HoodieTableType.COPY_ON_WRITE; + +/** + * Defines the hudi_cow_pt_tbl test table - a multi-level partitioned Copy-on-Write table. + */ +public class HudiCowPtTblTableDefinition + extends HudiTableDefinition +{ + private static final String TABLE_NAME = "hudi_cow_pt_tbl"; + private static final String RECORD_KEY_FIELD = "id"; + + public HudiCowPtTblTableDefinition() + { + super( + TABLE_NAME, + COPY_ON_WRITE, + createRegularColumns(), + createPartitionColumns(), + RECORD_KEY_FIELD, + null, // No pre-combine field for COW + createPartitions()); + } + + private static List createRegularColumns() + { + return ImmutableList.of( + column("id", HIVE_LONG), + column("name", HIVE_STRING), + column("ts", HIVE_LONG)); + } + + private static List createPartitionColumns() + { + return ImmutableList.of( + column("dt", HIVE_STRING), + column("hh", HIVE_STRING)); + } + + private static Map createPartitions() + { + return ImmutableMap.of( + "dt=2021-12-09/hh=10", "dt=2021-12-09/hh=10", + "dt=2021-12-09/hh=11", "dt=2021-12-09/hh=11"); + } + + @Override + protected Schema getAvroSchema() + { + return SchemaBuilder.record(TABLE_NAME) + .fields() + .requiredLong("id") + .requiredString("name") + .requiredLong("ts") + .requiredString("dt") + .requiredString("hh") + .endRecord(); + } + + @Override + public void executeCommits(HoodieJavaWriteClient client) + { + HudiRecordFactory recordFactory = new HudiRecordFactory(getAvroSchema()); + + // Commit 1: Insert record into partition dt=2021-12-09/hh=10 + List> firstCommitRecords = ImmutableList.of( + recordFactory.createRecord( + "id:1", + "dt=2021-12-09/hh=10", + fieldValues() + .longField("id", 1L) + .stringField("name", "a1") + .longField("ts", 1000L) + .stringField("dt", "2021-12-09") + .stringField("hh", "10") + .build())); + + String instant1 = client.startCommit(); + List statuses1 = client.insert(firstCommitRecords, instant1); + client.commit(instant1, statuses1); + + // Commit 2: Insert record into partition dt=2021-12-09/hh=11 + List> secondCommitRecords = ImmutableList.of( + recordFactory.createRecord( + "id:2", + "dt=2021-12-09/hh=11", + fieldValues() + .longField("id", 2L) + .stringField("name", "a2") + .longField("ts", 1000L) + .stringField("dt", "2021-12-09") + .stringField("hh", "11") + .build())); + + String instant2 = client.startCommit(); + List statuses2 = client.insert(secondCommitRecords, instant2); + client.commit(instant2, statuses2); + } +} diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/HudiNonPartCowTableDefinition.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/HudiNonPartCowTableDefinition.java new file mode 100644 index 000000000000..baab7938f03d --- /dev/null +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/HudiNonPartCowTableDefinition.java @@ -0,0 +1,103 @@ +/* + * 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.hudi.testing; + +import com.google.common.collect.ImmutableList; +import io.trino.metastore.Column; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.hudi.client.HoodieJavaWriteClient; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.model.HoodieAvroPayload; +import org.apache.hudi.common.model.HoodieRecord; + +import java.util.List; + +import static io.trino.metastore.HiveType.HIVE_LONG; +import static io.trino.metastore.HiveType.HIVE_STRING; +import static io.trino.plugin.hudi.testing.HudiRecordFactory.fieldValues; +import static org.apache.hudi.common.model.HoodieTableType.COPY_ON_WRITE; + +/** + * Defines the hudi_non_part_cow test table - a non-partitioned Copy-on-Write table. + */ +public class HudiNonPartCowTableDefinition + extends HudiTableDefinition +{ + private static final String TABLE_NAME = "hudi_non_part_cow"; + private static final String RECORD_KEY_FIELD = "id"; + + public HudiNonPartCowTableDefinition() + { + super( + TABLE_NAME, + COPY_ON_WRITE, + createRegularColumns(), + RECORD_KEY_FIELD, + null); // No pre-combine field for COW + } + + private static List createRegularColumns() + { + return ImmutableList.of( + column("id", HIVE_LONG), + column("name", HIVE_STRING), + column("ts", HIVE_LONG), + column("dt", HIVE_STRING), + column("hh", HIVE_STRING)); + } + + @Override + protected Schema getAvroSchema() + { + return SchemaBuilder.record(TABLE_NAME) + .fields() + .requiredLong("id") + .requiredString("name") + .requiredLong("ts") + .requiredString("dt") + .requiredString("hh") + .endRecord(); + } + + @Override + public void executeCommits(HoodieJavaWriteClient client) + { + HudiRecordFactory recordFactory = new HudiRecordFactory(getAvroSchema()); + + List> records = ImmutableList.of( + recordFactory.createRecord( + "id:1", + fieldValues() + .longField("id", 1L) + .stringField("name", "a1") + .longField("ts", 1000L) + .stringField("dt", "2021-12-09") + .stringField("hh", "10") + .build()), + recordFactory.createRecord( + "id:2", + fieldValues() + .longField("id", 2L) + .stringField("name", "a2") + .longField("ts", 1000L) + .stringField("dt", "2021-12-09") + .stringField("hh", "11") + .build())); + + String instant = client.startCommit(); + List statuses = client.insert(records, instant); + client.commit(instant, statuses); + } +} diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/HudiNonPartMorTableDefinition.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/HudiNonPartMorTableDefinition.java new file mode 100644 index 000000000000..365b19901c65 --- /dev/null +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/HudiNonPartMorTableDefinition.java @@ -0,0 +1,171 @@ +/* + * 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.hudi.testing; + +import com.google.common.collect.ImmutableList; +import io.trino.metastore.Column; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.hudi.client.HoodieJavaWriteClient; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.model.HoodieAvroPayload; +import org.apache.hudi.common.model.HoodieRecord; + +import java.util.List; + +import static io.trino.metastore.HiveType.HIVE_INT; +import static io.trino.metastore.HiveType.HIVE_STRING; +import static io.trino.plugin.hudi.testing.HudiRecordFactory.fieldValues; +import static org.apache.hudi.common.model.HoodieTableType.MERGE_ON_READ; + +/** + * Defines the hudi_non_part_mor test table - a non-partitioned Merge-on-Read table. + *

+ * This table demonstrates: + *

    + *
  • Non-partitioned MOR table
  • + *
  • Multiple commits (bulk_insert + upsert)
  • + *
  • Record updates via upsert operation
  • + *
+ *

+ * Schema: + *

    + *
  • id (String) - record key
  • + *
  • name (String) - pre-combine field
  • + *
  • age (Integer)
  • + *
+ *

+ * Data Evolution: + *

    + *
  • Commit 1 (bulk_insert): (1, Alice, 30), (2, Bob, 25)
  • + *
  • Commit 2 (upsert): (1, Cathy, 30), (2, David, 25)
  • + *
  • Final state: (1, Cathy, 30), (2, David, 25)
  • + *
+ */ +public class HudiNonPartMorTableDefinition + extends HudiTableDefinition +{ + private static final String TABLE_NAME = "hudi_non_part_mor"; + private static final String RECORD_KEY_FIELD = "id"; + private static final String PRE_COMBINE_FIELD = "name"; + + public HudiNonPartMorTableDefinition() + { + super( + TABLE_NAME, + MERGE_ON_READ, + createRegularColumns(), + RECORD_KEY_FIELD, + PRE_COMBINE_FIELD); + } + + /** + * Defines the regular (non-Hudi metadata) columns for this table. + */ + private static List createRegularColumns() + { + return ImmutableList.of( + column("id", HIVE_STRING), + column("name", HIVE_STRING), + column("age", HIVE_INT)); + } + + /** + * Creates the Avro schema for this table. + */ + private Schema createAvroSchema() + { + return SchemaBuilder.record(TABLE_NAME) + .fields() + .requiredString("id") + .requiredString("name") + .requiredInt("age") + .endRecord(); + } + + @Override + protected Schema getAvroSchema() + { + return createAvroSchema(); + } + + @Override + public void executeCommits(HoodieJavaWriteClient client) + throws Exception + { + HudiRecordFactory recordFactory = new HudiRecordFactory(getAvroSchema()); + + // Commit 1: bulk_insert with Alice and Bob + executeCommit1BulkInsert(client, recordFactory); + + // Commit 2: upsert to replace with Cathy and David + executeCommit2Upsert(client, recordFactory); + } + + /** + * First commit: Bulk insert of initial records (Alice and Bob). + */ + private void executeCommit1BulkInsert( + HoodieJavaWriteClient client, + HudiRecordFactory recordFactory) + { + List> records = ImmutableList.of( + recordFactory.createRecord( + "1", + fieldValues() + .stringField("id", "1") + .stringField("name", "Alice") + .intField("age", 30) + .build()), + recordFactory.createRecord( + "2", + fieldValues() + .stringField("id", "2") + .stringField("name", "Bob") + .intField("age", 25) + .build())); + + String instant = client.startCommit(); + List statuses = client.insert(records, instant); + client.commit(instant, statuses); + } + + /** + * Second commit: Upsert to replace Alice with Cathy and Bob with David. + */ + private void executeCommit2Upsert( + HoodieJavaWriteClient client, + HudiRecordFactory recordFactory) + { + List> records = ImmutableList.of( + recordFactory.createRecord( + "1", + fieldValues() + .stringField("id", "1") + .stringField("name", "Cathy") + .intField("age", 30) + .build()), + recordFactory.createRecord( + "2", + fieldValues() + .stringField("id", "2") + .stringField("name", "David") + .intField("age", 25) + .build())); + + String instant = client.startCommit(); + List statuses = client.upsert(records, instant); + client.commit(instant, statuses); + } +} diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/HudiRecordFactory.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/HudiRecordFactory.java new file mode 100644 index 000000000000..6af620b3e2fb --- /dev/null +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/HudiRecordFactory.java @@ -0,0 +1,211 @@ +/* + * 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.hudi.testing; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.hudi.common.model.HoodieAvroPayload; +import org.apache.hudi.common.model.HoodieAvroRecord; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.Option; + +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Random; +import java.util.UUID; + +import static java.util.Objects.requireNonNull; + +/** + * Factory for creating Hudi records with deterministic properties to ensure + * reproducible test results. + *

+ * This factory provides utilities for: + *

    + *
  • Creating records with predictable record keys
  • + *
  • Generating deterministic UUIDs based on seeds
  • + *
  • Building Avro records from field values
  • + *
+ */ +public class HudiRecordFactory +{ + private final Schema schema; + private final Random random; + + /** + * Creates a factory with the specified Avro schema. + * + * @param schema the Avro schema for the records + */ + public HudiRecordFactory(Schema schema) + { + this(schema, 12345L); + } + + /** + * Creates a factory with the specified Avro schema and random seed. + * + * @param schema the Avro schema for the records + * @param seed the seed for deterministic random number generation + */ + public HudiRecordFactory(Schema schema, long seed) + { + this.schema = requireNonNull(schema, "schema is null"); + this.random = new Random(seed); + } + + /** + * Creates a Hudi record with the specified record key, partition path, and field values. + * + * @param recordKey the record key (e.g., "id:1") + * @param partitionPath the partition path (e.g., "dt=2021-12-09/hh=10" or "" for non-partitioned) + * @param fieldValues map of field names to values + * @return a new HoodieRecord + */ + public HoodieRecord createRecord( + String recordKey, + String partitionPath, + Map fieldValues) + { + GenericRecord record = new GenericData.Record(schema); + + // Set all field values + for (Map.Entry entry : fieldValues.entrySet()) { + String fieldName = entry.getKey(); + Object value = entry.getValue(); + + // Verify field exists in schema + if (schema.getField(fieldName) == null) { + throw new IllegalArgumentException( + "Field '" + fieldName + "' does not exist in schema. Available fields: " + schema.getFields()); + } + + record.put(fieldName, value); + } + + HoodieKey key = new HoodieKey(recordKey, partitionPath); + HoodieAvroPayload payload = new HoodieAvroPayload(Option.of(record)); + return new HoodieAvroRecord<>(key, payload, null); + } + + /** + * Creates a Hudi record for a non-partitioned table. + * + * @param recordKey the record key + * @param fieldValues map of field names to values + * @return a new HoodieRecord + */ + public HoodieRecord createRecord(String recordKey, Map fieldValues) + { + return createRecord(recordKey, "", fieldValues); + } + + /** + * Generates a deterministic UUID based on the current random state. + *

+ * This ensures file IDs and other UUID-based identifiers are reproducible + * across test runs. + * + * @return a deterministic UUID + */ + public UUID generateDeterministicUuid() + { + long mostSigBits = random.nextLong(); + long leastSigBits = random.nextLong(); + return new UUID(mostSigBits, leastSigBits); + } + + /** + * Builder for creating field value maps with a fluent API. + */ + public static class FieldValuesBuilder + { + private final Map values = new LinkedHashMap<>(); + + /** + * Adds a field with the specified name and value. + */ + public FieldValuesBuilder field(String name, Object value) + { + values.put(name, value); + return this; + } + + /** + * Adds a string field. + */ + public FieldValuesBuilder stringField(String name, String value) + { + return field(name, value); + } + + /** + * Adds a long field. + */ + public FieldValuesBuilder longField(String name, long value) + { + return field(name, value); + } + + /** + * Adds an integer field. + */ + public FieldValuesBuilder intField(String name, int value) + { + return field(name, value); + } + + /** + * Adds a double field. + */ + public FieldValuesBuilder doubleField(String name, double value) + { + return field(name, value); + } + + /** + * Adds a boolean field. + */ + public FieldValuesBuilder booleanField(String name, boolean value) + { + return field(name, value); + } + + /** + * Builds the field values map. + */ + public Map build() + { + return new LinkedHashMap<>(values); + } + } + + /** + * Creates a new field values builder. + */ + public static FieldValuesBuilder fieldValues() + { + return new FieldValuesBuilder(); + } + + /** + * Returns the schema used by this factory. + */ + public Schema getSchema() + { + return schema; + } +} diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/HudiTableDefinition.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/HudiTableDefinition.java new file mode 100644 index 000000000000..80d50e7bab86 --- /dev/null +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/HudiTableDefinition.java @@ -0,0 +1,229 @@ +/* + * 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.hudi.testing; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.trino.metastore.Column; +import io.trino.metastore.HiveType; +import org.apache.avro.Schema; +import org.apache.hudi.client.HoodieJavaWriteClient; +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.HoodieAvroPayload; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableVersion; +import org.apache.hudi.common.table.marker.MarkerType; +import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.index.HoodieIndex; +import org.assertj.core.util.Strings; + +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Stream; + +import static io.trino.metastore.HiveType.HIVE_STRING; + +/** + * Defines the specification for a Hudi test table, including its schema, + * partitioning, table type, and the data/commits to create. + *

+ * Subclasses should implement the {@link #executeCommits(HoodieJavaWriteClient)} + * method to define what data is written to the table and in what order. + */ +public abstract class HudiTableDefinition +{ + /** + * Hudi metadata columns that are automatically added to all Hudi tables. + */ + protected static final List HUDI_META_COLUMNS = ImmutableList.of( + new Column("_hoodie_commit_time", HIVE_STRING, Optional.empty(), Map.of()), + new Column("_hoodie_commit_seqno", HIVE_STRING, Optional.empty(), Map.of()), + new Column("_hoodie_record_key", HIVE_STRING, Optional.empty(), Map.of()), + new Column("_hoodie_partition_path", HIVE_STRING, Optional.empty(), Map.of()), + new Column("_hoodie_file_name", HIVE_STRING, Optional.empty(), Map.of())); + + private final String tableName; + private final HoodieTableType tableType; + private final List regularColumns; + private final List partitionColumns; + private final String recordKeyField; + private final String preCombineField; + private final Map partitions; + + protected HudiTableDefinition( + String tableName, + HoodieTableType tableType, + List regularColumns, + List partitionColumns, + String recordKeyField, + String preCombineField, + Map partitions) + { + this.tableName = tableName; + this.tableType = tableType; + this.regularColumns = ImmutableList.copyOf(regularColumns); + this.partitionColumns = ImmutableList.copyOf(partitionColumns); + this.recordKeyField = recordKeyField; + this.preCombineField = preCombineField; + this.partitions = ImmutableMap.copyOf(partitions); + } + + /** + * Constructor for non-partitioned tables. + */ + protected HudiTableDefinition( + String tableName, + HoodieTableType tableType, + List regularColumns, + String recordKeyField, + String preCombineField) + { + this(tableName, tableType, regularColumns, ImmutableList.of(), recordKeyField, preCombineField, ImmutableMap.of()); + } + + /** + * Returns the name of this table. + */ + public String getTableName() + { + return tableName; + } + + /** + * Returns the Hudi table type (COPY_ON_WRITE or MERGE_ON_READ). + */ + public HoodieTableType getTableType() + { + return tableType; + } + + /** + * Returns all data columns (Hudi metadata columns + regular columns). + */ + public List getDataColumns() + { + return Stream.of(HUDI_META_COLUMNS, regularColumns) + .flatMap(Collection::stream) + .toList(); + } + + /** + * Returns the partition columns for this table. + */ + public List getPartitionColumns() + { + return partitionColumns; + } + + /** + * Returns the partitions to register with the metastore. + * Map key is the partition name (e.g., "dt=2021-12-09/hh=10"), + * Map value is the partition path (e.g., "dt=2021-12-09/hh=10"). + */ + public Map getPartitions() + { + return partitions; + } + + /** + * Returns the record key field name. + */ + public String getRecordKeyField() + { + return recordKeyField; + } + + /** + * Returns the pre-combine field name (required for MERGE_ON_READ tables). + */ + public Optional getPreCombineField() + { + return Optional.ofNullable(preCombineField); + } + + /** + * Creates the HoodieWriteConfig for this table with deterministic settings + * to ensure reproducible test results. + */ + @SuppressWarnings("deprecation") + public HoodieWriteConfig createWriteConfig(String basePath) + { + TypedProperties properties = new TypedProperties(); + properties.setProperty("hoodie.table.type", tableType.name()); + + // Add partition fields if this is a partitioned table + if (!partitionColumns.isEmpty()) { + String[] partitionFields = partitionColumns.stream() + .map(Column::getName) + .toArray(String[]::new); + properties.put("hoodie.datasource.write.partitionpath.field", Strings.concat(partitionFields, ",")); + } + HoodieWriteConfig.Builder builder = HoodieWriteConfig.newBuilder() + .withPath(basePath) + .withSchema(getAvroSchema().toString()) + .forTable(tableName) + .withProps(properties) + .withEmbeddedTimelineServerEnabled(false) + .withMarkersType(MarkerType.DIRECT.name()) + .withWriteTableVersion(HoodieTableVersion.SIX.versionCode()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder() + .enable(false) + .build()) + .withIndexConfig(HoodieIndexConfig.newBuilder() + .withIndexType(HoodieIndex.IndexType.INMEMORY) + .build()); + + if (preCombineField != null) { + builder.withPreCombineField(preCombineField); + } + + return builder.build(); + } + + /** + * Returns the Avro schema for this table. + *

+ * The schema defines the structure of the data records and is required + * by Hudi for writing data. + * + * @return the Avro schema + */ + protected abstract Schema getAvroSchema(); + + /** + * Executes the commits that populate this table with data. + *

+ * Subclasses must implement this method to define what data is written + * and in what order (e.g., initial insert followed by updates). + *

+ * Use deterministic commit timestamps (via {@link HoodieJavaWriteClient}) + * to ensure reproducible test results. + * + * @param client the Hudi write client to use for writing data + */ + public abstract void executeCommits(HoodieJavaWriteClient client) + throws Exception; + + /** + * Helper method to create a Column with the specified name and type. + */ + protected static Column column(String name, HiveType type) + { + return new Column(name, type, Optional.empty(), Map.of()); + } +} diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/HudiTableUnzipper.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/HudiTableUnzipper.java new file mode 100644 index 000000000000..e8216d1b4d0c --- /dev/null +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/HudiTableUnzipper.java @@ -0,0 +1,127 @@ +/* + * 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.hudi.testing; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.net.URISyntaxException; +import java.net.URL; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Comparator; +import java.util.stream.Stream; +import java.util.zip.ZipEntry; +import java.util.zip.ZipInputStream; + +import static com.google.common.io.Resources.getResource; +import static java.util.Objects.requireNonNull; + +public class HudiTableUnzipper +{ + private static final String ZIP_EXT = ".zip"; + + private HudiTableUnzipper() {} + + public static void unzipAllItemsInResource(String resourceName) + throws IOException, URISyntaxException + { + requireNonNull(resourceName, "Resource name cannot be null or empty."); + + URL resourceUrl = HudiTableUnzipper.class.getClassLoader().getResource(resourceName); + if (resourceUrl == null) { + throw new IOException("Resource not found: " + resourceName); + } + + for (File file : Path.of(getResource(resourceName).toURI()).toFile().listFiles()) { + if (file.isFile() && file.getName().endsWith(ZIP_EXT)) { + // Only handle zip files + unzipFile(file.toURI().toURL(), Path.of(file.getParent())); + } + } + } + + private static void unzipFile(URL resourceUrl, Path targetDirectory) + throws IOException + { + try (InputStream is = resourceUrl.openStream(); ZipInputStream zis = new ZipInputStream(is)) { + ZipEntry zipEntry = zis.getNextEntry(); + byte[] buffer = new byte[1024]; + + while (zipEntry != null) { + Path newFilePath = targetDirectory.resolve(zipEntry.getName()); + + // Prevent Zip Slip vulnerability (Do not want files written outside the our target dir) + if (!newFilePath.normalize().startsWith(targetDirectory.normalize())) { + throw new IOException("Bad zip entry: " + zipEntry.getName()); + } + + if (zipEntry.isDirectory()) { + // Handle directories + if (!Files.exists(newFilePath)) { + Files.createDirectories(newFilePath); + } + } + else { + // Ensure parent directory exists before handling files + Path parentDir = newFilePath.getParent(); + if (parentDir != null && !Files.exists(parentDir)) { + Files.createDirectories(parentDir); + } + + try (FileOutputStream fos = new FileOutputStream(newFilePath.toFile())) { + int len; + while ((len = zis.read(buffer)) > 0) { + fos.write(buffer, 0, len); + } + } + } + zis.closeEntry(); + zipEntry = zis.getNextEntry(); + } + } + } + + public static void deleteInflatedFiles(String resourceName) + throws URISyntaxException, IOException + { + requireNonNull(resourceName, "Resource name cannot be null or empty."); + Path directoryPath = Path.of(getResource(resourceName).toURI()); + + for (File file : directoryPath.toFile().listFiles()) { + // Ignore all zip files + if (file.isFile() && file.getName().endsWith(ZIP_EXT)) { + continue; + } + // Not really required, as we are in the test-classes directory + // Ensure that we are only deleting deflated folders of zip + if (directoryPath.resolve(file.getName() + ZIP_EXT).toFile().exists()) { + deleteFilesInDirectory(file.toPath()); + } + } + } + + private static void deleteFilesInDirectory(Path pathToDelete) + throws IOException + { + // Recursively delete all files in path + try (Stream filesInDir = Files.walk(pathToDelete)) { + // Reverse order to delete all children before parent + filesInDir.sorted(Comparator.reverseOrder()) + .map(Path::toFile) + .forEach(File::delete); + } + } +} diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/ResourceHudiTablesInitializer.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/ResourceHudiTablesInitializer.java index 55d7c78b22fb..8c57be528f24 100644 --- a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/ResourceHudiTablesInitializer.java +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/ResourceHudiTablesInitializer.java @@ -32,9 +32,9 @@ import io.trino.spi.security.ConnectorIdentity; import io.trino.testing.QueryRunner; -import java.io.File; import java.io.IOException; import java.io.OutputStream; +import java.net.URISyntaxException; import java.nio.file.Files; import java.nio.file.Path; import java.util.ArrayList; @@ -61,15 +61,18 @@ public class ResourceHudiTablesInitializer implements HudiTablesInitializer { + private static final String TEST_RESOURCE_NAME = "hudi-testing-data"; + @Override public void initializeTables(QueryRunner queryRunner, Location externalLocation, String schemaName) throws Exception { + HudiTableUnzipper.unzipAllItemsInResource(TEST_RESOURCE_NAME); TrinoFileSystem fileSystem = ((HudiConnector) queryRunner.getCoordinator().getConnector("hudi")).getInjector() .getInstance(TrinoFileSystemFactory.class) .create(ConnectorIdentity.ofUser("test")); Location baseLocation = externalLocation.appendSuffix(schemaName); - copyDir(new File(getResource("hudi-testing-data").toURI()).toPath(), fileSystem, baseLocation); + copyDir(Path.of(getResource(TEST_RESOURCE_NAME).toURI()), fileSystem, baseLocation); for (TestingTable table : TestingTable.values()) { String tableName = table.getTableName(); @@ -85,6 +88,18 @@ public void initializeTables(QueryRunner queryRunner, Location externalLocation, } } + /** + * Deletes the test resource directory specified by the {@code TEST_RESOURCE_NAME} constant. + * + * @throws IOException if an I/O error occurs during the deletion. + * @throws URISyntaxException if the resource URI, derived from the lookup, is invalid. + */ + public void deleteTestResources() + throws IOException, URISyntaxException + { + HudiTableUnzipper.deleteInflatedFiles(TEST_RESOURCE_NAME); + } + private void createTable( QueryRunner queryRunner, String schemaName, diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/StockTicksCowTableDefinition.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/StockTicksCowTableDefinition.java new file mode 100644 index 000000000000..7f730d853df8 --- /dev/null +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/StockTicksCowTableDefinition.java @@ -0,0 +1,144 @@ +/* + * 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.hudi.testing; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.trino.metastore.Column; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.hudi.client.HoodieJavaWriteClient; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.model.HoodieAvroPayload; +import org.apache.hudi.common.model.HoodieRecord; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static io.trino.metastore.HiveType.HIVE_DOUBLE; +import static io.trino.metastore.HiveType.HIVE_INT; +import static io.trino.metastore.HiveType.HIVE_LONG; +import static io.trino.metastore.HiveType.HIVE_STRING; +import static io.trino.plugin.hudi.testing.HudiRecordFactory.fieldValues; +import static org.apache.hudi.common.model.HoodieTableType.COPY_ON_WRITE; + +/** + * Defines the stock_ticks_cow test table - a partitioned Copy-on-Write table with stock ticker data. + */ +public class StockTicksCowTableDefinition + extends HudiTableDefinition +{ + private static final String TABLE_NAME = "stock_ticks_cow"; + private static final String RECORD_KEY_FIELD = "key"; + private static final int RECORD_COUNT = 99; + + public StockTicksCowTableDefinition() + { + super( + TABLE_NAME, + COPY_ON_WRITE, + createRegularColumns(), + createPartitionColumns(), + RECORD_KEY_FIELD, + null, // No pre-combine field for COW + createPartitions()); + } + + private static List createRegularColumns() + { + return ImmutableList.of( + column("volume", HIVE_LONG), + column("ts", HIVE_STRING), + column("symbol", HIVE_STRING), + column("year", HIVE_INT), + column("month", HIVE_STRING), + column("high", HIVE_DOUBLE), + column("low", HIVE_DOUBLE), + column("key", HIVE_STRING), + column("date", HIVE_STRING), + column("close", HIVE_DOUBLE), + column("open", HIVE_DOUBLE), + column("day", HIVE_STRING)); + } + + private static List createPartitionColumns() + { + return ImmutableList.of(column("dt", HIVE_STRING)); + } + + private static Map createPartitions() + { + return ImmutableMap.of("dt=2018-08-31", "2018/08/31"); + } + + @Override + protected Schema getAvroSchema() + { + return SchemaBuilder.record(TABLE_NAME) + .fields() + .requiredLong("volume") + .requiredString("ts") + .requiredString("symbol") + .requiredInt("year") + .requiredString("month") + .requiredDouble("high") + .requiredDouble("low") + .requiredString("key") + .requiredString("date") + .requiredDouble("close") + .requiredDouble("open") + .requiredString("day") + .requiredString("dt") + .endRecord(); + } + + @Override + public void executeCommits(HoodieJavaWriteClient client) + { + HudiRecordFactory recordFactory = new HudiRecordFactory(getAvroSchema(), 42L); + + List> records = new ArrayList<>(); + String[] symbols = {"GOOG", "AAPL", "MSFT", "AMZN", "META"}; + + for (int i = 0; i < RECORD_COUNT; i++) { + String symbol = symbols[(symbols.length - 1 - (i % symbols.length)) % symbols.length]; + int minute = i % 60; + String timestamp = String.format("2018-08-31 10:%02d:00", minute); + + records.add(recordFactory.createRecord( + symbol + "_2018-08-31_" + i, + "2018/08/31", + fieldValues() + .longField("volume", 1000L + (i * 100L)) + .stringField("ts", timestamp) + .stringField("symbol", symbol) + .intField("year", 2018) + .stringField("month", "08") + .doubleField("high", 100.0 + (i * 0.5)) + .doubleField("low", 90.0 + (i * 0.5)) + .stringField("key", symbol + "_2018-08-31_" + i) + .stringField("date", "2018-08-31") + .doubleField("close", 95.0 + (i * 0.5)) + .doubleField("open", 92.0 + (i * 0.5)) + .stringField("day", "31") + .stringField("dt", "2018-08-31") + .build())); + } + + String instant = client.startCommit(); + List statuses = client.insert(records, instant); + client.commit(instant, statuses); + } +} diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/StockTicksMorTableDefinition.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/StockTicksMorTableDefinition.java new file mode 100644 index 000000000000..56e728c059c2 --- /dev/null +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/StockTicksMorTableDefinition.java @@ -0,0 +1,181 @@ +/* + * 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.hudi.testing; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.trino.metastore.Column; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.hudi.client.HoodieJavaWriteClient; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.model.HoodieAvroPayload; +import org.apache.hudi.common.model.HoodieRecord; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static io.trino.metastore.HiveType.HIVE_DOUBLE; +import static io.trino.metastore.HiveType.HIVE_INT; +import static io.trino.metastore.HiveType.HIVE_LONG; +import static io.trino.metastore.HiveType.HIVE_STRING; +import static io.trino.plugin.hudi.testing.HudiRecordFactory.fieldValues; +import static org.apache.hudi.common.model.HoodieTableType.MERGE_ON_READ; + +/** + * Defines the stock_ticks_mor test table - a partitioned Merge-on-Read table with stock ticker data. + */ +public class StockTicksMorTableDefinition + extends HudiTableDefinition +{ + private static final String TABLE_NAME = "stock_ticks_mor"; + private static final String RECORD_KEY_FIELD = "key"; + private static final String PRE_COMBINE_FIELD = "ts"; + private static final int RECORD_COUNT = 99; + + public StockTicksMorTableDefinition() + { + super( + TABLE_NAME, + MERGE_ON_READ, + createRegularColumns(), + createPartitionColumns(), + RECORD_KEY_FIELD, + PRE_COMBINE_FIELD, + createPartitions()); + } + + private static List createRegularColumns() + { + return ImmutableList.of( + column("volume", HIVE_LONG), + column("ts", HIVE_STRING), + column("symbol", HIVE_STRING), + column("year", HIVE_INT), + column("month", HIVE_STRING), + column("high", HIVE_DOUBLE), + column("low", HIVE_DOUBLE), + column("key", HIVE_STRING), + column("date", HIVE_STRING), + column("close", HIVE_DOUBLE), + column("open", HIVE_DOUBLE), + column("day", HIVE_STRING)); + } + + private static List createPartitionColumns() + { + return ImmutableList.of(column("dt", HIVE_STRING)); + } + + private static Map createPartitions() + { + return ImmutableMap.of("dt=2018-08-31", "2018/08/31"); + } + + @Override + protected Schema getAvroSchema() + { + return SchemaBuilder.record(TABLE_NAME) + .fields() + .requiredLong("volume") + .requiredString("ts") + .requiredString("symbol") + .requiredInt("year") + .requiredString("month") + .requiredDouble("high") + .requiredDouble("low") + .requiredString("key") + .requiredString("date") + .requiredDouble("close") + .requiredDouble("open") + .requiredString("day") + .requiredString("dt") + .endRecord(); + } + + @Override + public void executeCommits(HoodieJavaWriteClient client) + { + HudiRecordFactory recordFactory = new HudiRecordFactory(getAvroSchema(), 42L); + + // First commit: Initial bulk insert + List> records = new ArrayList<>(); + String[] symbols = {"GOOG", "AAPL", "MSFT", "AMZN", "META"}; + + for (int i = 0; i < RECORD_COUNT; i++) { + String symbol = symbols[(symbols.length - 1 - (i % symbols.length)) % symbols.length]; + int minute = i % 60; + String timestamp = String.format("2018-08-31 10:%02d:00", minute); + + records.add(recordFactory.createRecord( + symbol + "_2018-08-31_" + i, + "2018/08/31", + fieldValues() + .longField("volume", 1000L + (i * 100L)) + .stringField("ts", timestamp) + .stringField("symbol", symbol) + .intField("year", 2018) + .stringField("month", "08") + .doubleField("high", 100.0 + (i * 0.5)) + .doubleField("low", 90.0 + (i * 0.5)) + .stringField("key", symbol + "_2018-08-31_" + i) + .stringField("date", "2018-08-31") + .doubleField("close", 95.0 + (i * 0.5)) + .doubleField("open", 92.0 + (i * 0.5)) + .stringField("day", "31") + .stringField("dt", "2018-08-31") + .build())); + } + + // For MOR tables, use bulkInsert to create base parquet files directly + // instead of log files. This matches the original test data structure. + String instant = client.startCommit(); + List statuses = client.bulkInsert(records, instant); + client.commit(instant, statuses); + + // Second commit: Update all records from the first commit + List> updateRecords = new ArrayList<>(); + + for (int i = 0; i < RECORD_COUNT; i++) { + String symbol = symbols[(symbols.length - 1 - (i % symbols.length)) % symbols.length]; + int minute = i % 60; + // Update timestamp to be later (11:xx instead of 10:xx) for pre-combine field + String updatedTimestamp = String.format("2018-08-31 11:%02d:00", minute); + + updateRecords.add(recordFactory.createRecord( + symbol + "_2018-08-31_" + i, + "2018/08/31", + fieldValues() + .longField("volume", 2000L + (i * 100L)) // Updated volume + .stringField("ts", updatedTimestamp) + .stringField("symbol", symbol) + .intField("year", 2018) + .stringField("month", "08") + .doubleField("high", 110.0 + (i * 0.5)) // Updated high + .doubleField("low", 100.0 + (i * 0.5)) // Updated low + .stringField("key", symbol + "_2018-08-31_" + i) + .stringField("date", "2018-08-31") + .doubleField("close", 105.0 + (i * 0.5)) // Updated close + .doubleField("open", 102.0 + (i * 0.5)) // Updated open + .stringField("day", "31") + .stringField("dt", "2018-08-31") + .build())); + } + + String updateInstant = client.startCommit(); + List updateStatuses = client.upsert(updateRecords, updateInstant); + client.commit(updateInstant, updateStatuses); + } +} diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_cow_pt_tbl/.hoodie/20220906063435640.commit b/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_cow_pt_tbl/.hoodie/20220906063435640.commit deleted file mode 100644 index f7993d9cbb86..000000000000 --- a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_cow_pt_tbl/.hoodie/20220906063435640.commit +++ /dev/null @@ -1,73 +0,0 @@ -{ - "partitionToWriteStats" : { - "dt=2021-12-09/hh=10" : [ { - "fileId" : "719c3273-2805-4124-b1ac-e980dada85bf-0", - "path" : "dt=2021-12-09/hh=10/719c3273-2805-4124-b1ac-e980dada85bf-0_0-27-1215_20220906063435640.parquet", - "prevCommit" : "null", - "numWrites" : 1, - "numDeletes" : 0, - "numUpdateWrites" : 0, - "numInserts" : 1, - "totalWriteBytes" : 435204, - "totalWriteErrors" : 0, - "tempPath" : null, - "partitionPath" : "dt=2021-12-09/hh=10", - "totalLogRecords" : 0, - "totalLogFilesCompacted" : 0, - "totalLogSizeCompacted" : 0, - "totalUpdatedRecordsCompacted" : 0, - "totalLogBlocks" : 0, - "totalCorruptLogBlock" : 0, - "totalRollbackBlocks" : 0, - "fileSizeInBytes" : 435204, - "minEventTime" : null, - "maxEventTime" : null - } ] - }, - "compacted" : false, - "extraMetadata" : { - "schema" : "{\"type\":\"record\",\"name\":\"hudi_cow_pt_tbl_record\",\"namespace\":\"hoodie.hudi_cow_pt_tbl\",\"fields\":[{\"name\":\"id\",\"type\":\"long\"},{\"name\":\"name\",\"type\":\"string\"},{\"name\":\"ts\",\"type\":\"long\"},{\"name\":\"dt\",\"type\":\"string\"},{\"name\":\"hh\",\"type\":\"string\"}]}" - }, - "operationType" : "UPSERT", - "writePartitionPaths" : [ "dt=2021-12-09/hh=10" ], - "fileIdAndRelativePaths" : { - "719c3273-2805-4124-b1ac-e980dada85bf-0" : "dt=2021-12-09/hh=10/719c3273-2805-4124-b1ac-e980dada85bf-0_0-27-1215_20220906063435640.parquet" - }, - "totalRecordsDeleted" : 0, - "totalLogRecordsCompacted" : 0, - "totalLogFilesCompacted" : 0, - "totalCompactedRecordsUpdated" : 0, - "totalLogFilesSize" : 0, - "totalScanTime" : 0, - "totalCreateTime" : 512, - "totalUpsertTime" : 0, - "minAndMaxEventTime" : { - "Optional.empty" : { - "val" : null, - "present" : false - } - }, - "writeStats" : [ { - "fileId" : "719c3273-2805-4124-b1ac-e980dada85bf-0", - "path" : "dt=2021-12-09/hh=10/719c3273-2805-4124-b1ac-e980dada85bf-0_0-27-1215_20220906063435640.parquet", - "prevCommit" : "null", - "numWrites" : 1, - "numDeletes" : 0, - "numUpdateWrites" : 0, - "numInserts" : 1, - "totalWriteBytes" : 435204, - "totalWriteErrors" : 0, - "tempPath" : null, - "partitionPath" : "dt=2021-12-09/hh=10", - "totalLogRecords" : 0, - "totalLogFilesCompacted" : 0, - "totalLogSizeCompacted" : 0, - "totalUpdatedRecordsCompacted" : 0, - "totalLogBlocks" : 0, - "totalCorruptLogBlock" : 0, - "totalRollbackBlocks" : 0, - "fileSizeInBytes" : 435204, - "minEventTime" : null, - "maxEventTime" : null - } ] -} \ No newline at end of file diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_cow_pt_tbl/.hoodie/20220906063435640.commit.requested b/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_cow_pt_tbl/.hoodie/20220906063435640.commit.requested deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_cow_pt_tbl/.hoodie/20220906063435640.inflight b/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_cow_pt_tbl/.hoodie/20220906063435640.inflight deleted file mode 100644 index 5bb738a80f74..000000000000 --- a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_cow_pt_tbl/.hoodie/20220906063435640.inflight +++ /dev/null @@ -1,71 +0,0 @@ -{ - "partitionToWriteStats" : { - "dt=2021-12-09/hh=10" : [ { - "fileId" : "", - "path" : null, - "prevCommit" : "null", - "numWrites" : 0, - "numDeletes" : 0, - "numUpdateWrites" : 0, - "numInserts" : 1, - "totalWriteBytes" : 0, - "totalWriteErrors" : 0, - "tempPath" : null, - "partitionPath" : null, - "totalLogRecords" : 0, - "totalLogFilesCompacted" : 0, - "totalLogSizeCompacted" : 0, - "totalUpdatedRecordsCompacted" : 0, - "totalLogBlocks" : 0, - "totalCorruptLogBlock" : 0, - "totalRollbackBlocks" : 0, - "fileSizeInBytes" : 0, - "minEventTime" : null, - "maxEventTime" : null - } ] - }, - "compacted" : false, - "extraMetadata" : { }, - "operationType" : "UPSERT", - "writePartitionPaths" : [ "dt=2021-12-09/hh=10" ], - "fileIdAndRelativePaths" : { - "" : null - }, - "totalRecordsDeleted" : 0, - "totalLogRecordsCompacted" : 0, - "totalLogFilesCompacted" : 0, - "totalCompactedRecordsUpdated" : 0, - "totalLogFilesSize" : 0, - "totalScanTime" : 0, - "totalCreateTime" : 0, - "totalUpsertTime" : 0, - "minAndMaxEventTime" : { - "Optional.empty" : { - "val" : null, - "present" : false - } - }, - "writeStats" : [ { - "fileId" : "", - "path" : null, - "prevCommit" : "null", - "numWrites" : 0, - "numDeletes" : 0, - "numUpdateWrites" : 0, - "numInserts" : 1, - "totalWriteBytes" : 0, - "totalWriteErrors" : 0, - "tempPath" : null, - "partitionPath" : null, - "totalLogRecords" : 0, - "totalLogFilesCompacted" : 0, - "totalLogSizeCompacted" : 0, - "totalUpdatedRecordsCompacted" : 0, - "totalLogBlocks" : 0, - "totalCorruptLogBlock" : 0, - "totalRollbackBlocks" : 0, - "fileSizeInBytes" : 0, - "minEventTime" : null, - "maxEventTime" : null - } ] -} \ No newline at end of file diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_cow_pt_tbl/.hoodie/20220906063456550.commit b/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_cow_pt_tbl/.hoodie/20220906063456550.commit deleted file mode 100644 index 0b2cacafaf52..000000000000 --- a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_cow_pt_tbl/.hoodie/20220906063456550.commit +++ /dev/null @@ -1,73 +0,0 @@ -{ - "partitionToWriteStats" : { - "dt=2021-12-09/hh=11" : [ { - "fileId" : "4a3fcb9b-65eb-4f6e-acf9-7b0764bb4dd1-0", - "path" : "dt=2021-12-09/hh=11/4a3fcb9b-65eb-4f6e-acf9-7b0764bb4dd1-0_0-70-2444_20220906063456550.parquet", - "prevCommit" : "null", - "numWrites" : 1, - "numDeletes" : 0, - "numUpdateWrites" : 0, - "numInserts" : 1, - "totalWriteBytes" : 435204, - "totalWriteErrors" : 0, - "tempPath" : null, - "partitionPath" : "dt=2021-12-09/hh=11", - "totalLogRecords" : 0, - "totalLogFilesCompacted" : 0, - "totalLogSizeCompacted" : 0, - "totalUpdatedRecordsCompacted" : 0, - "totalLogBlocks" : 0, - "totalCorruptLogBlock" : 0, - "totalRollbackBlocks" : 0, - "fileSizeInBytes" : 435204, - "minEventTime" : null, - "maxEventTime" : null - } ] - }, - "compacted" : false, - "extraMetadata" : { - "schema" : "{\"type\":\"record\",\"name\":\"hudi_cow_pt_tbl_record\",\"namespace\":\"hoodie.hudi_cow_pt_tbl\",\"fields\":[{\"name\":\"id\",\"type\":\"long\"},{\"name\":\"name\",\"type\":\"string\"},{\"name\":\"ts\",\"type\":\"long\"},{\"name\":\"dt\",\"type\":\"string\"},{\"name\":\"hh\",\"type\":\"string\"}]}" - }, - "operationType" : "UPSERT", - "writePartitionPaths" : [ "dt=2021-12-09/hh=11" ], - "fileIdAndRelativePaths" : { - "4a3fcb9b-65eb-4f6e-acf9-7b0764bb4dd1-0" : "dt=2021-12-09/hh=11/4a3fcb9b-65eb-4f6e-acf9-7b0764bb4dd1-0_0-70-2444_20220906063456550.parquet" - }, - "totalRecordsDeleted" : 0, - "totalLogRecordsCompacted" : 0, - "totalLogFilesCompacted" : 0, - "totalCompactedRecordsUpdated" : 0, - "totalLogFilesSize" : 0, - "totalScanTime" : 0, - "totalCreateTime" : 72, - "totalUpsertTime" : 0, - "minAndMaxEventTime" : { - "Optional.empty" : { - "val" : null, - "present" : false - } - }, - "writeStats" : [ { - "fileId" : "4a3fcb9b-65eb-4f6e-acf9-7b0764bb4dd1-0", - "path" : "dt=2021-12-09/hh=11/4a3fcb9b-65eb-4f6e-acf9-7b0764bb4dd1-0_0-70-2444_20220906063456550.parquet", - "prevCommit" : "null", - "numWrites" : 1, - "numDeletes" : 0, - "numUpdateWrites" : 0, - "numInserts" : 1, - "totalWriteBytes" : 435204, - "totalWriteErrors" : 0, - "tempPath" : null, - "partitionPath" : "dt=2021-12-09/hh=11", - "totalLogRecords" : 0, - "totalLogFilesCompacted" : 0, - "totalLogSizeCompacted" : 0, - "totalUpdatedRecordsCompacted" : 0, - "totalLogBlocks" : 0, - "totalCorruptLogBlock" : 0, - "totalRollbackBlocks" : 0, - "fileSizeInBytes" : 435204, - "minEventTime" : null, - "maxEventTime" : null - } ] -} \ No newline at end of file diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_cow_pt_tbl/.hoodie/20220906063456550.commit.requested b/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_cow_pt_tbl/.hoodie/20220906063456550.commit.requested deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_cow_pt_tbl/.hoodie/20220906063456550.inflight b/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_cow_pt_tbl/.hoodie/20220906063456550.inflight deleted file mode 100644 index 8ef87d4ed5f6..000000000000 --- a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_cow_pt_tbl/.hoodie/20220906063456550.inflight +++ /dev/null @@ -1,71 +0,0 @@ -{ - "partitionToWriteStats" : { - "dt=2021-12-09/hh=11" : [ { - "fileId" : "", - "path" : null, - "prevCommit" : "null", - "numWrites" : 0, - "numDeletes" : 0, - "numUpdateWrites" : 0, - "numInserts" : 1, - "totalWriteBytes" : 0, - "totalWriteErrors" : 0, - "tempPath" : null, - "partitionPath" : null, - "totalLogRecords" : 0, - "totalLogFilesCompacted" : 0, - "totalLogSizeCompacted" : 0, - "totalUpdatedRecordsCompacted" : 0, - "totalLogBlocks" : 0, - "totalCorruptLogBlock" : 0, - "totalRollbackBlocks" : 0, - "fileSizeInBytes" : 0, - "minEventTime" : null, - "maxEventTime" : null - } ] - }, - "compacted" : false, - "extraMetadata" : { }, - "operationType" : "UPSERT", - "writePartitionPaths" : [ "dt=2021-12-09/hh=11" ], - "fileIdAndRelativePaths" : { - "" : null - }, - "totalRecordsDeleted" : 0, - "totalLogRecordsCompacted" : 0, - "totalLogFilesCompacted" : 0, - "totalCompactedRecordsUpdated" : 0, - "totalLogFilesSize" : 0, - "totalScanTime" : 0, - "totalCreateTime" : 0, - "totalUpsertTime" : 0, - "minAndMaxEventTime" : { - "Optional.empty" : { - "val" : null, - "present" : false - } - }, - "writeStats" : [ { - "fileId" : "", - "path" : null, - "prevCommit" : "null", - "numWrites" : 0, - "numDeletes" : 0, - "numUpdateWrites" : 0, - "numInserts" : 1, - "totalWriteBytes" : 0, - "totalWriteErrors" : 0, - "tempPath" : null, - "partitionPath" : null, - "totalLogRecords" : 0, - "totalLogFilesCompacted" : 0, - "totalLogSizeCompacted" : 0, - "totalUpdatedRecordsCompacted" : 0, - "totalLogBlocks" : 0, - "totalCorruptLogBlock" : 0, - "totalRollbackBlocks" : 0, - "fileSizeInBytes" : 0, - "minEventTime" : null, - "maxEventTime" : null - } ] -} \ No newline at end of file diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_cow_pt_tbl/.hoodie/hoodie.properties b/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_cow_pt_tbl/.hoodie/hoodie.properties deleted file mode 100644 index 4d3a2d67cfc6..000000000000 --- a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_cow_pt_tbl/.hoodie/hoodie.properties +++ /dev/null @@ -1,18 +0,0 @@ -#Updated at 2022-09-06T06:34:40.904Z -#Tue Sep 06 06:34:40 UTC 2022 -hoodie.table.precombine.field=ts -hoodie.datasource.write.drop.partition.columns=false -hoodie.table.partition.fields=dt,hh -hoodie.table.type=COPY_ON_WRITE -hoodie.archivelog.folder=archived -hoodie.timeline.layout.version=1 -hoodie.table.version=5 -hoodie.table.metadata.partitions=files -hoodie.table.recordkey.fields=id -hoodie.database.name=default -hoodie.datasource.write.partitionpath.urlencode=false -hoodie.table.keygenerator.class=org.apache.hudi.keygen.ComplexKeyGenerator -hoodie.table.name=hudi_cow_pt_tbl -hoodie.datasource.write.hive_style_partitioning=true -hoodie.table.checksum=1395413629 -hoodie.table.create.schema={"type"\:"record","name"\:"hudi_cow_pt_tbl_record","namespace"\:"hoodie.hudi_cow_pt_tbl","fields"\:[{"name"\:"_hoodie_commit_time","type"\:["string","null"]},{"name"\:"_hoodie_commit_seqno","type"\:["string","null"]},{"name"\:"_hoodie_record_key","type"\:["string","null"]},{"name"\:"_hoodie_partition_path","type"\:["string","null"]},{"name"\:"_hoodie_file_name","type"\:["string","null"]},{"name"\:"id","type"\:["long","null"]},{"name"\:"name","type"\:["string","null"]},{"name"\:"ts","type"\:["long","null"]},{"name"\:"dt","type"\:["string","null"]},{"name"\:"hh","type"\:["string","null"]}]} diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_cow_pt_tbl/dt=2021-12-09/hh=10/.hoodie_partition_metadata b/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_cow_pt_tbl/dt=2021-12-09/hh=10/.hoodie_partition_metadata deleted file mode 100644 index 92ce82c886b7..000000000000 --- a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_cow_pt_tbl/dt=2021-12-09/hh=10/.hoodie_partition_metadata +++ /dev/null @@ -1,4 +0,0 @@ -#partition metadata -#Tue Sep 06 06:34:49 UTC 2022 -commitTime=20220906063435640 -partitionDepth=2 diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_cow_pt_tbl/dt=2021-12-09/hh=10/719c3273-2805-4124-b1ac-e980dada85bf-0_0-27-1215_20220906063435640.parquet b/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_cow_pt_tbl/dt=2021-12-09/hh=10/719c3273-2805-4124-b1ac-e980dada85bf-0_0-27-1215_20220906063435640.parquet deleted file mode 100644 index 4785e8c09095..000000000000 Binary files a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_cow_pt_tbl/dt=2021-12-09/hh=10/719c3273-2805-4124-b1ac-e980dada85bf-0_0-27-1215_20220906063435640.parquet and /dev/null differ diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_cow_pt_tbl/dt=2021-12-09/hh=11/.hoodie_partition_metadata b/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_cow_pt_tbl/dt=2021-12-09/hh=11/.hoodie_partition_metadata deleted file mode 100644 index ed8c619a1362..000000000000 --- a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_cow_pt_tbl/dt=2021-12-09/hh=11/.hoodie_partition_metadata +++ /dev/null @@ -1,4 +0,0 @@ -#partition metadata -#Tue Sep 06 06:35:03 UTC 2022 -commitTime=20220906063456550 -partitionDepth=2 diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_cow_pt_tbl/dt=2021-12-09/hh=11/4a3fcb9b-65eb-4f6e-acf9-7b0764bb4dd1-0_0-70-2444_20220906063456550.parquet b/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_cow_pt_tbl/dt=2021-12-09/hh=11/4a3fcb9b-65eb-4f6e-acf9-7b0764bb4dd1-0_0-70-2444_20220906063456550.parquet deleted file mode 100644 index f2918fae851c..000000000000 Binary files a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_cow_pt_tbl/dt=2021-12-09/hh=11/4a3fcb9b-65eb-4f6e-acf9-7b0764bb4dd1-0_0-70-2444_20220906063456550.parquet and /dev/null differ diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/20231127051653361.commit b/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/20231127051653361.commit deleted file mode 100644 index 9fc9470ff41a..000000000000 --- a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/20231127051653361.commit +++ /dev/null @@ -1,37 +0,0 @@ -{ - "partitionToWriteStats" : { - "" : [ { - "fileId" : "05b0f4ec-00fb-49f2-a1e2-7f510f3da93b-0", - "path" : "05b0f4ec-00fb-49f2-a1e2-7f510f3da93b-0_0-27-28_20231127051653361.parquet", - "prevCommit" : "null", - "numWrites" : 2, - "numDeletes" : 0, - "numUpdateWrites" : 0, - "numInserts" : 2, - "totalWriteBytes" : 435338, - "totalWriteErrors" : 0, - "tempPath" : null, - "partitionPath" : "", - "totalLogRecords" : 0, - "totalLogFilesCompacted" : 0, - "totalLogSizeCompacted" : 0, - "totalUpdatedRecordsCompacted" : 0, - "totalLogBlocks" : 0, - "totalCorruptLogBlock" : 0, - "totalRollbackBlocks" : 0, - "fileSizeInBytes" : 435338, - "minEventTime" : null, - "maxEventTime" : null, - "runtimeStats" : { - "totalScanTime" : 0, - "totalUpsertTime" : 0, - "totalCreateTime" : 856 - } - } ] - }, - "compacted" : false, - "extraMetadata" : { - "schema" : "{\"type\":\"record\",\"name\":\"hudi_non_part_cow_record\",\"namespace\":\"hoodie.hudi_non_part_cow\",\"fields\":[{\"name\":\"id\",\"type\":[\"null\",\"long\"],\"default\":null},{\"name\":\"name\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"ts\",\"type\":[\"null\",\"long\"],\"default\":null},{\"name\":\"dt\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"hh\",\"type\":[\"null\",\"string\"],\"default\":null}]}" - }, - "operationType" : "UPSERT" -} \ No newline at end of file diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/20231127051653361.commit.requested b/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/20231127051653361.commit.requested deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/20231127051653361.inflight b/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/20231127051653361.inflight deleted file mode 100644 index dad745d91bd7..000000000000 --- a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/20231127051653361.inflight +++ /dev/null @@ -1,31 +0,0 @@ -{ - "partitionToWriteStats" : { - "" : [ { - "fileId" : "", - "path" : null, - "prevCommit" : "null", - "numWrites" : 0, - "numDeletes" : 0, - "numUpdateWrites" : 0, - "numInserts" : 2, - "totalWriteBytes" : 0, - "totalWriteErrors" : 0, - "tempPath" : null, - "partitionPath" : null, - "totalLogRecords" : 0, - "totalLogFilesCompacted" : 0, - "totalLogSizeCompacted" : 0, - "totalUpdatedRecordsCompacted" : 0, - "totalLogBlocks" : 0, - "totalCorruptLogBlock" : 0, - "totalRollbackBlocks" : 0, - "fileSizeInBytes" : 0, - "minEventTime" : null, - "maxEventTime" : null, - "runtimeStats" : null - } ] - }, - "compacted" : false, - "extraMetadata" : { }, - "operationType" : "UPSERT" -} \ No newline at end of file diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/hoodie.properties b/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/hoodie.properties deleted file mode 100644 index aa323696b299..000000000000 --- a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/hoodie.properties +++ /dev/null @@ -1,17 +0,0 @@ -#Updated at 2023-11-27T05:16:58.380652Z -#Mon Nov 27 05:16:58 UTC 2023 -hoodie.table.type=COPY_ON_WRITE -hoodie.table.metadata.partitions=files -hoodie.table.precombine.field=ts -hoodie.archivelog.folder=archived -hoodie.table.create.schema={"type"\:"record","name"\:"hudi_non_part_cow_record","namespace"\:"hoodie.hudi_non_part_cow","fields"\:[{"name"\:"_hoodie_commit_time","type"\:["string","null"]},{"name"\:"_hoodie_commit_seqno","type"\:["string","null"]},{"name"\:"_hoodie_record_key","type"\:["string","null"]},{"name"\:"_hoodie_partition_path","type"\:["string","null"]},{"name"\:"_hoodie_file_name","type"\:["string","null"]},{"name"\:"id","type"\:["long","null"]},{"name"\:"name","type"\:["string","null"]},{"name"\:"ts","type"\:["long","null"]},{"name"\:"dt","type"\:["string","null"]},{"name"\:"hh","type"\:["string","null"]}]} -hoodie.timeline.layout.version=1 -hoodie.table.checksum=2968816715 -hoodie.datasource.write.drop.partition.columns=false -hoodie.table.recordkey.fields=id -hoodie.table.name=hudi_non_part_cow -hoodie.datasource.write.hive_style_partitioning=true -hoodie.table.keygenerator.class=org.apache.hudi.keygen.NonpartitionedKeyGenerator -hoodie.database.name=default -hoodie.datasource.write.partitionpath.urlencode=false -hoodie.table.version=5 diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie_partition_metadata b/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie_partition_metadata deleted file mode 100644 index e9de1b96c3ff..000000000000 --- a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie_partition_metadata +++ /dev/null @@ -1,4 +0,0 @@ -#partition metadata -#Mon Nov 27 05:16:59 UTC 2023 -commitTime=20231127051653361 -partitionDepth=0 diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/05b0f4ec-00fb-49f2-a1e2-7f510f3da93b-0_0-27-28_20231127051653361.parquet b/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/05b0f4ec-00fb-49f2-a1e2-7f510f3da93b-0_0-27-28_20231127051653361.parquet deleted file mode 100644 index c8368bd7c33c..000000000000 Binary files a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/05b0f4ec-00fb-49f2-a1e2-7f510f3da93b-0_0-27-28_20231127051653361.parquet and /dev/null differ diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow/.hoodie/20211216071453747.commit b/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow/.hoodie/20211216071453747.commit deleted file mode 100644 index 18cf55cc1bfd..000000000000 --- a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow/.hoodie/20211216071453747.commit +++ /dev/null @@ -1,51 +0,0 @@ -{ - "partitionToWriteStats" : { - "2018/08/31" : [ { - "fileId" : "871677fb-e0e3-46f8-9cc1-fe497e317216-0", - "path" : "2018/08/31/871677fb-e0e3-46f8-9cc1-fe497e317216-0_0-28-26_20211216071453747.parquet", - "prevCommit" : "null", - "numWrites" : 99, - "numDeletes" : 0, - "numUpdateWrites" : 0, - "numInserts" : 99, - "totalWriteBytes" : 440747, - "totalWriteErrors" : 0, - "tempPath" : null, - "partitionPath" : "2018/08/31", - "totalLogRecords" : 0, - "totalLogFilesCompacted" : 0, - "totalLogSizeCompacted" : 0, - "totalUpdatedRecordsCompacted" : 0, - "totalLogBlocks" : 0, - "totalCorruptLogBlock" : 0, - "totalRollbackBlocks" : 0, - "fileSizeInBytes" : 440747, - "minEventTime" : null, - "maxEventTime" : null - } ] - }, - "compacted" : false, - "extraMetadata" : { - "schema" : "{\"type\":\"record\",\"name\":\"stock_ticks\",\"fields\":[{\"name\":\"volume\",\"type\":\"long\"},{\"name\":\"ts\",\"type\":\"string\"},{\"name\":\"symbol\",\"type\":\"string\"},{\"name\":\"year\",\"type\":\"int\"},{\"name\":\"month\",\"type\":\"string\"},{\"name\":\"high\",\"type\":\"double\"},{\"name\":\"low\",\"type\":\"double\"},{\"name\":\"key\",\"type\":\"string\"},{\"name\":\"date\",\"type\":\"string\"},{\"name\":\"close\",\"type\":\"double\"},{\"name\":\"open\",\"type\":\"double\"},{\"name\":\"day\",\"type\":\"string\"}]}", - "deltastreamer.checkpoint.key" : "stock_ticks,0:1668" - }, - "operationType" : "UPSERT", - "fileIdAndRelativePaths" : { - "871677fb-e0e3-46f8-9cc1-fe497e317216-0" : "2018/08/31/871677fb-e0e3-46f8-9cc1-fe497e317216-0_0-28-26_20211216071453747.parquet" - }, - "totalRecordsDeleted" : 0, - "totalLogRecordsCompacted" : 0, - "totalLogFilesCompacted" : 0, - "totalCompactedRecordsUpdated" : 0, - "totalLogFilesSize" : 0, - "totalScanTime" : 0, - "totalCreateTime" : 750, - "totalUpsertTime" : 0, - "minAndMaxEventTime" : { - "Optional.empty" : { - "val" : null, - "present" : false - } - }, - "writePartitionPaths" : [ "2018/08/31" ] -} \ No newline at end of file diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow/.hoodie/20211216071453747.commit.requested b/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow/.hoodie/20211216071453747.commit.requested deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow/.hoodie/20211216071453747.inflight b/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow/.hoodie/20211216071453747.inflight deleted file mode 100644 index 6dc689a285d9..000000000000 --- a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow/.hoodie/20211216071453747.inflight +++ /dev/null @@ -1,48 +0,0 @@ -{ - "partitionToWriteStats" : { - "2018/08/31" : [ { - "fileId" : "", - "path" : null, - "prevCommit" : "null", - "numWrites" : 0, - "numDeletes" : 0, - "numUpdateWrites" : 0, - "numInserts" : 99, - "totalWriteBytes" : 0, - "totalWriteErrors" : 0, - "tempPath" : null, - "partitionPath" : null, - "totalLogRecords" : 0, - "totalLogFilesCompacted" : 0, - "totalLogSizeCompacted" : 0, - "totalUpdatedRecordsCompacted" : 0, - "totalLogBlocks" : 0, - "totalCorruptLogBlock" : 0, - "totalRollbackBlocks" : 0, - "fileSizeInBytes" : 0, - "minEventTime" : null, - "maxEventTime" : null - } ] - }, - "compacted" : false, - "extraMetadata" : { }, - "operationType" : "UPSERT", - "fileIdAndRelativePaths" : { - "" : null - }, - "totalRecordsDeleted" : 0, - "totalLogRecordsCompacted" : 0, - "totalLogFilesCompacted" : 0, - "totalCompactedRecordsUpdated" : 0, - "totalLogFilesSize" : 0, - "totalScanTime" : 0, - "totalCreateTime" : 0, - "totalUpsertTime" : 0, - "minAndMaxEventTime" : { - "Optional.empty" : { - "val" : null, - "present" : false - } - }, - "writePartitionPaths" : [ "2018/08/31" ] -} \ No newline at end of file diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow/.hoodie/hoodie.properties b/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow/.hoodie/hoodie.properties deleted file mode 100644 index 4754c1c23eb2..000000000000 --- a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow/.hoodie/hoodie.properties +++ /dev/null @@ -1,13 +0,0 @@ -#Properties saved on Thu Dec 16 07:14:51 UTC 2021 -#Thu Dec 16 07:14:51 UTC 2021 -hoodie.table.precombine.field=ts -hoodie.table.partition.fields=date -hoodie.table.type=COPY_ON_WRITE -hoodie.archivelog.folder=archived -hoodie.populate.meta.fields=true -hoodie.timeline.layout.version=1 -hoodie.table.version=3 -hoodie.table.recordkey.fields=key -hoodie.table.base.file.format=PARQUET -hoodie.table.keygenerator.class=org.apache.hudi.keygen.SimpleKeyGenerator -hoodie.table.name=stock_ticks_cow diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow/2018/08/31/.hoodie_partition_metadata b/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow/2018/08/31/.hoodie_partition_metadata deleted file mode 100644 index 1aaf9e64d933..000000000000 --- a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow/2018/08/31/.hoodie_partition_metadata +++ /dev/null @@ -1,4 +0,0 @@ -#partition metadata -#Thu Dec 16 07:14:56 UTC 2021 -commitTime=20211216071453747 -partitionDepth=3 diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow/2018/08/31/871677fb-e0e3-46f8-9cc1-fe497e317216-0_0-28-26_20211216071453747.parquet b/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow/2018/08/31/871677fb-e0e3-46f8-9cc1-fe497e317216-0_0-28-26_20211216071453747.parquet deleted file mode 100644 index b97391697e62..000000000000 Binary files a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_cow/2018/08/31/871677fb-e0e3-46f8-9cc1-fe497e317216-0_0-28-26_20211216071453747.parquet and /dev/null differ diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_mor/.hoodie/20211221030120532.deltacommit b/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_mor/.hoodie/20211221030120532.deltacommit deleted file mode 100644 index f9e28873d524..000000000000 --- a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_mor/.hoodie/20211221030120532.deltacommit +++ /dev/null @@ -1,51 +0,0 @@ -{ - "partitionToWriteStats" : { - "2018/08/31" : [ { - "fileId" : "167a0e3e-9b94-444f-a178-242230cdb5a2-0", - "path" : "2018/08/31/167a0e3e-9b94-444f-a178-242230cdb5a2-0_0-28-26_20211221030120532.parquet", - "prevCommit" : "null", - "numWrites" : 99, - "numDeletes" : 0, - "numUpdateWrites" : 0, - "numInserts" : 99, - "totalWriteBytes" : 440746, - "totalWriteErrors" : 0, - "tempPath" : null, - "partitionPath" : "2018/08/31", - "totalLogRecords" : 0, - "totalLogFilesCompacted" : 0, - "totalLogSizeCompacted" : 0, - "totalUpdatedRecordsCompacted" : 0, - "totalLogBlocks" : 0, - "totalCorruptLogBlock" : 0, - "totalRollbackBlocks" : 0, - "fileSizeInBytes" : 440746, - "minEventTime" : null, - "maxEventTime" : null - } ] - }, - "compacted" : false, - "extraMetadata" : { - "schema" : "{\"type\":\"record\",\"name\":\"stock_ticks\",\"fields\":[{\"name\":\"volume\",\"type\":\"long\"},{\"name\":\"ts\",\"type\":\"string\"},{\"name\":\"symbol\",\"type\":\"string\"},{\"name\":\"year\",\"type\":\"int\"},{\"name\":\"month\",\"type\":\"string\"},{\"name\":\"high\",\"type\":\"double\"},{\"name\":\"low\",\"type\":\"double\"},{\"name\":\"key\",\"type\":\"string\"},{\"name\":\"date\",\"type\":\"string\"},{\"name\":\"close\",\"type\":\"double\"},{\"name\":\"open\",\"type\":\"double\"},{\"name\":\"day\",\"type\":\"string\"}]}", - "deltastreamer.checkpoint.key" : "stock_ticks,0:1668" - }, - "operationType" : "UPSERT", - "fileIdAndRelativePaths" : { - "167a0e3e-9b94-444f-a178-242230cdb5a2-0" : "2018/08/31/167a0e3e-9b94-444f-a178-242230cdb5a2-0_0-28-26_20211221030120532.parquet" - }, - "totalRecordsDeleted" : 0, - "totalLogRecordsCompacted" : 0, - "totalLogFilesCompacted" : 0, - "totalCompactedRecordsUpdated" : 0, - "totalLogFilesSize" : 0, - "totalScanTime" : 0, - "totalCreateTime" : 1402, - "totalUpsertTime" : 0, - "minAndMaxEventTime" : { - "Optional.empty" : { - "val" : null, - "present" : false - } - }, - "writePartitionPaths" : [ "2018/08/31" ] -} \ No newline at end of file diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_mor/.hoodie/20211221030120532.deltacommit.inflight b/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_mor/.hoodie/20211221030120532.deltacommit.inflight deleted file mode 100644 index 6dc689a285d9..000000000000 --- a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_mor/.hoodie/20211221030120532.deltacommit.inflight +++ /dev/null @@ -1,48 +0,0 @@ -{ - "partitionToWriteStats" : { - "2018/08/31" : [ { - "fileId" : "", - "path" : null, - "prevCommit" : "null", - "numWrites" : 0, - "numDeletes" : 0, - "numUpdateWrites" : 0, - "numInserts" : 99, - "totalWriteBytes" : 0, - "totalWriteErrors" : 0, - "tempPath" : null, - "partitionPath" : null, - "totalLogRecords" : 0, - "totalLogFilesCompacted" : 0, - "totalLogSizeCompacted" : 0, - "totalUpdatedRecordsCompacted" : 0, - "totalLogBlocks" : 0, - "totalCorruptLogBlock" : 0, - "totalRollbackBlocks" : 0, - "fileSizeInBytes" : 0, - "minEventTime" : null, - "maxEventTime" : null - } ] - }, - "compacted" : false, - "extraMetadata" : { }, - "operationType" : "UPSERT", - "fileIdAndRelativePaths" : { - "" : null - }, - "totalRecordsDeleted" : 0, - "totalLogRecordsCompacted" : 0, - "totalLogFilesCompacted" : 0, - "totalCompactedRecordsUpdated" : 0, - "totalLogFilesSize" : 0, - "totalScanTime" : 0, - "totalCreateTime" : 0, - "totalUpsertTime" : 0, - "minAndMaxEventTime" : { - "Optional.empty" : { - "val" : null, - "present" : false - } - }, - "writePartitionPaths" : [ "2018/08/31" ] -} \ No newline at end of file diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_mor/.hoodie/20211221030120532.deltacommit.requested b/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_mor/.hoodie/20211221030120532.deltacommit.requested deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_mor/.hoodie/20211227092838847.deltacommit b/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_mor/.hoodie/20211227092838847.deltacommit deleted file mode 100644 index f1cc26fecc7b..000000000000 --- a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_mor/.hoodie/20211227092838847.deltacommit +++ /dev/null @@ -1,55 +0,0 @@ -{ - "partitionToWriteStats" : { - "2018/08/31" : [ { - "fileId" : "167a0e3e-9b94-444f-a178-242230cdb5a2-0", - "path" : "2018/08/31/.167a0e3e-9b94-444f-a178-242230cdb5a2-0_20211221030120532.log.1_0-28-29", - "prevCommit" : "20211221030120532", - "numWrites" : 99, - "numDeletes" : 0, - "numUpdateWrites" : 99, - "numInserts" : 0, - "totalWriteBytes" : 22220, - "totalWriteErrors" : 0, - "tempPath" : null, - "partitionPath" : "2018/08/31", - "totalLogRecords" : 0, - "totalLogFilesCompacted" : 0, - "totalLogSizeCompacted" : 0, - "totalUpdatedRecordsCompacted" : 0, - "totalLogBlocks" : 0, - "totalCorruptLogBlock" : 0, - "totalRollbackBlocks" : 0, - "fileSizeInBytes" : 22220, - "minEventTime" : null, - "maxEventTime" : null, - "logVersion" : 1, - "logOffset" : 0, - "baseFile" : "167a0e3e-9b94-444f-a178-242230cdb5a2-0_0-28-26_20211221030120532.parquet", - "logFiles" : [ ".167a0e3e-9b94-444f-a178-242230cdb5a2-0_20211221030120532.log.1_0-28-29" ] - } ] - }, - "compacted" : false, - "extraMetadata" : { - "schema" : "{\"type\":\"record\",\"name\":\"stock_ticks\",\"fields\":[{\"name\":\"volume\",\"type\":\"long\"},{\"name\":\"ts\",\"type\":\"string\"},{\"name\":\"symbol\",\"type\":\"string\"},{\"name\":\"year\",\"type\":\"int\"},{\"name\":\"month\",\"type\":\"string\"},{\"name\":\"high\",\"type\":\"double\"},{\"name\":\"low\",\"type\":\"double\"},{\"name\":\"key\",\"type\":\"string\"},{\"name\":\"date\",\"type\":\"string\"},{\"name\":\"close\",\"type\":\"double\"},{\"name\":\"open\",\"type\":\"double\"},{\"name\":\"day\",\"type\":\"string\"}]}", - "deltastreamer.checkpoint.key" : "stock_ticks,0:3336" - }, - "operationType" : "UPSERT", - "totalRecordsDeleted" : 0, - "totalLogRecordsCompacted" : 0, - "totalLogFilesCompacted" : 0, - "totalCompactedRecordsUpdated" : 0, - "totalLogFilesSize" : 0, - "totalScanTime" : 0, - "totalCreateTime" : 0, - "totalUpsertTime" : 187, - "minAndMaxEventTime" : { - "Optional.empty" : { - "val" : null, - "present" : false - } - }, - "writePartitionPaths" : [ "2018/08/31" ], - "fileIdAndRelativePaths" : { - "167a0e3e-9b94-444f-a178-242230cdb5a2-0" : "2018/08/31/.167a0e3e-9b94-444f-a178-242230cdb5a2-0_20211221030120532.log.1_0-28-29" - } -} \ No newline at end of file diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_mor/.hoodie/20211227092838847.deltacommit.inflight b/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_mor/.hoodie/20211227092838847.deltacommit.inflight deleted file mode 100644 index 724ce56ff0d6..000000000000 --- a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_mor/.hoodie/20211227092838847.deltacommit.inflight +++ /dev/null @@ -1,71 +0,0 @@ -{ - "partitionToWriteStats" : { - "2018/08/31" : [ { - "fileId" : "", - "path" : null, - "prevCommit" : "null", - "numWrites" : 0, - "numDeletes" : 0, - "numUpdateWrites" : 0, - "numInserts" : 0, - "totalWriteBytes" : 0, - "totalWriteErrors" : 0, - "tempPath" : null, - "partitionPath" : null, - "totalLogRecords" : 0, - "totalLogFilesCompacted" : 0, - "totalLogSizeCompacted" : 0, - "totalUpdatedRecordsCompacted" : 0, - "totalLogBlocks" : 0, - "totalCorruptLogBlock" : 0, - "totalRollbackBlocks" : 0, - "fileSizeInBytes" : 0, - "minEventTime" : null, - "maxEventTime" : null - }, { - "fileId" : "167a0e3e-9b94-444f-a178-242230cdb5a2-0", - "path" : null, - "prevCommit" : "20211221030120532", - "numWrites" : 0, - "numDeletes" : 0, - "numUpdateWrites" : 99, - "numInserts" : 0, - "totalWriteBytes" : 0, - "totalWriteErrors" : 0, - "tempPath" : null, - "partitionPath" : null, - "totalLogRecords" : 0, - "totalLogFilesCompacted" : 0, - "totalLogSizeCompacted" : 0, - "totalUpdatedRecordsCompacted" : 0, - "totalLogBlocks" : 0, - "totalCorruptLogBlock" : 0, - "totalRollbackBlocks" : 0, - "fileSizeInBytes" : 0, - "minEventTime" : null, - "maxEventTime" : null - } ] - }, - "compacted" : false, - "extraMetadata" : { }, - "operationType" : "UPSERT", - "totalRecordsDeleted" : 0, - "totalLogRecordsCompacted" : 0, - "totalLogFilesCompacted" : 0, - "totalCompactedRecordsUpdated" : 0, - "totalLogFilesSize" : 0, - "totalScanTime" : 0, - "totalCreateTime" : 0, - "totalUpsertTime" : 0, - "minAndMaxEventTime" : { - "Optional.empty" : { - "val" : null, - "present" : false - } - }, - "writePartitionPaths" : [ "2018/08/31" ], - "fileIdAndRelativePaths" : { - "" : null, - "167a0e3e-9b94-444f-a178-242230cdb5a2-0" : null - } -} \ No newline at end of file diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_mor/.hoodie/20211227092838847.deltacommit.requested b/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_mor/.hoodie/20211227092838847.deltacommit.requested deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_mor/.hoodie/hoodie.properties b/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_mor/.hoodie/hoodie.properties deleted file mode 100644 index 33392aa182f2..000000000000 --- a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_mor/.hoodie/hoodie.properties +++ /dev/null @@ -1,14 +0,0 @@ -#Properties saved on Tue Dec 21 03:01:13 UTC 2021 -#Tue Dec 21 03:01:13 UTC 2021 -hoodie.table.precombine.field=ts -hoodie.table.partition.fields=date -hoodie.table.type=MERGE_ON_READ -hoodie.archivelog.folder=archived -hoodie.populate.meta.fields=true -hoodie.compaction.payload.class=org.apache.hudi.common.model.OverwriteWithLatestAvroPayload -hoodie.timeline.layout.version=1 -hoodie.table.version=3 -hoodie.table.recordkey.fields=key -hoodie.table.base.file.format=PARQUET -hoodie.table.keygenerator.class=org.apache.hudi.keygen.SimpleKeyGenerator -hoodie.table.name=stock_ticks_mor diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_mor/2018/08/31/.167a0e3e-9b94-444f-a178-242230cdb5a2-0_20211221030120532.log.1_0-28-29 b/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_mor/2018/08/31/.167a0e3e-9b94-444f-a178-242230cdb5a2-0_20211221030120532.log.1_0-28-29 deleted file mode 100644 index da3c7bc07ee1..000000000000 Binary files a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_mor/2018/08/31/.167a0e3e-9b94-444f-a178-242230cdb5a2-0_20211221030120532.log.1_0-28-29 and /dev/null differ diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_mor/2018/08/31/.hoodie_partition_metadata b/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_mor/2018/08/31/.hoodie_partition_metadata deleted file mode 100644 index 340533d6e680..000000000000 --- a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_mor/2018/08/31/.hoodie_partition_metadata +++ /dev/null @@ -1,4 +0,0 @@ -#partition metadata -#Tue Dec 21 03:01:25 UTC 2021 -commitTime=20211221030120532 -partitionDepth=3 diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_mor/2018/08/31/167a0e3e-9b94-444f-a178-242230cdb5a2-0_0-28-26_20211221030120532.parquet b/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_mor/2018/08/31/167a0e3e-9b94-444f-a178-242230cdb5a2-0_0-28-26_20211221030120532.parquet deleted file mode 100644 index 9fe2112d09bb..000000000000 Binary files a/plugin/trino-hudi/src/test/resources/hudi-testing-data/stock_ticks_mor/2018/08/31/167a0e3e-9b94-444f-a178-242230cdb5a2-0_0-28-26_20211221030120532.parquet and /dev/null differ