diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveClientConfig.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveClientConfig.java index f4d9c7e00a1d..f0207269a59f 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveClientConfig.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveClientConfig.java @@ -190,6 +190,7 @@ public class HiveClientConfig private boolean verboseRuntimeStatsEnabled; private boolean useRecordPageSourceForCustomSplit = true; private boolean hudiMetadataEnabled; + private String hudiTablesUseMergedView; private boolean sizeBasedSplitWeightsEnabled = true; private double minimumAssignedSplitWeight = 0.05; @@ -1647,6 +1648,19 @@ public boolean isHudiMetadataEnabled() return this.hudiMetadataEnabled; } + @Config("hive.hudi-tables-use-merged-view") + @ConfigDescription("For Hudi tables prefer to fetch the list of files from the merged file system view") + public HiveClientConfig setHudiTablesUseMergedView(String hudiTablesUseMergedView) + { + this.hudiTablesUseMergedView = hudiTablesUseMergedView; + return this; + } + + public String getHudiTablesUseMergedView() + { + return this.hudiTablesUseMergedView; + } + @Config("hive.quick-stats.enabled") @ConfigDescription("Use quick stats to resolve stats") public HiveClientConfig setQuickStatsEnabled(boolean quickStatsEnabled) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveSessionProperties.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveSessionProperties.java index f175a109636c..9231b1754ba4 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveSessionProperties.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveSessionProperties.java @@ -125,6 +125,7 @@ public final class HiveSessionProperties public static final String MAX_INITIAL_SPLITS = "max_initial_splits"; public static final String FILE_SPLITTABLE = "file_splittable"; private static final String HUDI_METADATA_ENABLED = "hudi_metadata_enabled"; + private static final String HUDI_TABLES_USE_MERGED_VIEW = "hudi_tables_use_merged_view"; private static final String READ_TABLE_CONSTRAINTS = "read_table_constraints"; public static final String PARALLEL_PARSING_OF_PARTITION_VALUES_ENABLED = "parallel_parsing_of_partition_values_enabled"; public static final String QUICK_STATS_ENABLED = "quick_stats_enabled"; @@ -608,6 +609,11 @@ public HiveSessionProperties(HiveClientConfig hiveClientConfig, OrcFileWriterCon "For Hudi tables prefer to fetch the list of file names, sizes and other metadata from the internal metadata table rather than storage", hiveClientConfig.isHudiMetadataEnabled(), false), + stringProperty( + HUDI_TABLES_USE_MERGED_VIEW, + "For Hudi tables, use merged view to read data", + hiveClientConfig.getHudiTablesUseMergedView(), + false), booleanProperty( PARALLEL_PARSING_OF_PARTITION_VALUES_ENABLED, "Enables parallel parsing of partition values from partition names using thread pool", @@ -1101,6 +1107,12 @@ public static boolean isHudiMetadataEnabled(ConnectorSession session) return session.getProperty(HUDI_METADATA_ENABLED, Boolean.class); } + public static String getHudiTablesUseMergedView(ConnectorSession session) + { + String hudiTablesUseMergedView = session.getProperty(HUDI_TABLES_USE_MERGED_VIEW, String.class); + return hudiTablesUseMergedView == null ? "" : hudiTablesUseMergedView; + } + public static boolean isReadTableConstraints(ConnectorSession session) { return session.getProperty(READ_TABLE_CONSTRAINTS, Boolean.class); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HudiDirectoryLister.java b/presto-hive/src/main/java/com/facebook/presto/hive/HudiDirectoryLister.java index 72f9802ddad7..aa8a70644b71 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HudiDirectoryLister.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HudiDirectoryLister.java @@ -20,6 +20,7 @@ import com.facebook.presto.hive.metastore.Table; import com.facebook.presto.hive.util.HiveFileIterator; import com.facebook.presto.spi.ConnectorSession; +import com.google.common.base.Splitter; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.BlockLocation; import org.apache.hadoop.fs.FileStatus; @@ -30,16 +31,23 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.engine.HoodieLocalEngineContext; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.view.FileSystemViewManager; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +import org.apache.hudi.common.util.Option; import java.io.IOException; import java.util.Iterator; import java.util.Optional; +import java.util.stream.Stream; import static com.facebook.presto.hive.HiveFileInfo.createHiveFileInfo; +import static com.facebook.presto.hive.HiveSessionProperties.getHudiTablesUseMergedView; import static com.facebook.presto.hive.HiveSessionProperties.isHudiMetadataEnabled; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DEFAULT_PORT; @@ -47,15 +55,19 @@ public class HudiDirectoryLister implements DirectoryLister { private static final Logger log = Logger.get(HudiDirectoryLister.class); + private static final Splitter SPLITTER = Splitter.on(',').trimResults().omitEmptyStrings(); private final HoodieTableFileSystemView fileSystemView; private final HoodieTableMetaClient metaClient; private final boolean metadataEnabled; + private final String latestInstant; + private final boolean shouldUseMergedView; public HudiDirectoryLister(Configuration conf, ConnectorSession session, Table table) { log.info("Using Hudi Directory Lister."); this.metadataEnabled = isHudiMetadataEnabled(session); + this.shouldUseMergedView = SPLITTER.splitToList(getHudiTablesUseMergedView(session)).contains(table.getSchemaTableName().toString()); Configuration actualConfig = ((CachingJobConf) conf).getConfig(); /* WrapperJobConf acts as a wrapper on top of the actual Configuration object. If `hive.copy-on-first-write-configuration-enabled` @@ -68,6 +80,12 @@ public HudiDirectoryLister(Configuration conf, ConnectorSession session, Table t .setConf(actualConfig) .setBasePath(table.getStorage().getLocation()) .build(); + this.latestInstant = metaClient.getActiveTimeline() + .getCommitsAndCompactionTimeline() + .filterCompletedInstants() + .filter(instant -> !HoodieTableType.MERGE_ON_READ.equals(metaClient.getTableType()) || instant.getAction().equals(HoodieTimeline.COMMIT_ACTION)) + .lastInstant() + .map(HoodieInstant::getTimestamp).orElseThrow(() -> new RuntimeException("No active instant found")); HoodieEngineContext engineContext = new HoodieLocalEngineContext(actualConfig); HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder() .enable(metadataEnabled) @@ -96,7 +114,10 @@ public Iterator list( fileSystemView, metadataEnabled ? Optional.empty() : Optional.of(fileSystem.listStatus(p)), table.getStorage().getLocation(), - p), + p, + metaClient.getTableType(), + latestInstant, + shouldUseMergedView), namenodeStats, hiveDirectoryContext.getNestedDirectoryPolicy(), hiveDirectoryContext.isSkipEmptyFilesEnabled()); @@ -111,7 +132,10 @@ public HudiFileInfoIterator( HoodieTableFileSystemView fileSystemView, Optional fileStatuses, String tablePath, - Path directory) + Path directory, + HoodieTableType tableType, + String latestInstant, + boolean shouldUseMergedView) { String partition = FSUtils.getRelativePartitionPath(new Path(tablePath), directory); if (fileStatuses.isPresent()) { @@ -119,7 +143,15 @@ public HudiFileInfoIterator( this.hoodieBaseFileIterator = fileSystemView.fetchLatestBaseFiles(partition).iterator(); } else { - this.hoodieBaseFileIterator = fileSystemView.getLatestBaseFiles(partition).iterator(); + if (shouldUseMergedView) { + Stream fileSlices = HoodieTableType.MERGE_ON_READ.equals(tableType) ? + fileSystemView.getLatestMergedFileSlicesBeforeOrOn(partition, latestInstant) : + fileSystemView.getLatestFileSlicesBeforeOrOn(partition, latestInstant, false); + this.hoodieBaseFileIterator = fileSlices.map(FileSlice::getBaseFile).filter(Option::isPresent).map(Option::get).iterator(); + } + else { + this.hoodieBaseFileIterator = fileSystemView.getLatestBaseFiles(partition).iterator(); + } } } diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHudiDirectoryLister.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHudiDirectoryLister.java index 5dd727627711..d67805ab7083 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHudiDirectoryLister.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHudiDirectoryLister.java @@ -18,7 +18,9 @@ import com.facebook.presto.hive.filesystem.ExtendedFileSystem; import com.facebook.presto.hive.metastore.Storage; import com.facebook.presto.hive.metastore.Table; +import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.security.ConnectorIdentity; +import com.facebook.presto.testing.TestingConnectorSession; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.hadoop.conf.Configuration; @@ -38,10 +40,13 @@ import static com.facebook.presto.hive.BucketFunctionType.HIVE_COMPATIBLE; import static com.facebook.presto.hive.HiveStorageFormat.PARQUET; import static com.facebook.presto.hive.HiveTestUtils.SESSION; +import static com.facebook.presto.hive.HiveTestUtils.TEST_CLIENT_TAGS; +import static com.facebook.presto.hive.HiveTestUtils.getAllSessionProperties; import static com.facebook.presto.hive.NestedDirectoryPolicy.IGNORED; import static com.facebook.presto.hive.metastore.PrestoTableType.EXTERNAL_TABLE; import static com.facebook.presto.hive.metastore.StorageFormat.fromHiveStorageFormat; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertThrows; import static org.testng.Assert.assertTrue; @@ -107,6 +112,71 @@ private Table getMockTable() Optional.empty()); } + private Table getMockMORTableWithPartition() + { + return new Table( + "schema", + "hudi_mor_part_update", + "user", + EXTERNAL_TABLE, + new Storage(fromHiveStorageFormat(PARQUET), + getTableBasePath("hudi_mor_part_update"), + Optional.of(new HiveBucketProperty( + ImmutableList.of(), + 1, + ImmutableList.of(), + HIVE_COMPATIBLE, + Optional.empty())), + false, + ImmutableMap.of(), + ImmutableMap.of()), + ImmutableList.of(), + ImmutableList.of(), + ImmutableMap.of(), + Optional.empty(), + Optional.empty()); + } + + @Test + public void testDirectoryListerForMORTableWithPartitionUpdates() + throws IOException + { + Table mockTable = getMockMORTableWithPartition(); + Configuration hadoopConf = getHadoopConfWithCopyOnFirstWriteDisabled(); + try { + ConnectorSession session = new TestingConnectorSession( + getAllSessionProperties( + new HiveClientConfig() + .setHudiMetadataEnabled(true) + .setHudiTablesUseMergedView(mockTable.getSchemaTableName().toString()), + new HiveCommonClientConfig()), + TEST_CLIENT_TAGS); + HudiDirectoryLister directoryLister = new HudiDirectoryLister(hadoopConf, session, mockTable); + HoodieTableMetaClient metaClient = directoryLister.getMetaClient(); + assertEquals(metaClient.getBasePath(), mockTable.getStorage().getLocation()); + Path path = new Path(mockTable.getStorage().getLocation()); + ExtendedFileSystem fs = (ExtendedFileSystem) path.getFileSystem(hadoopConf); + Iterator fileInfoIterator = directoryLister.list(fs, mockTable, path, Optional.empty(), new NamenodeStats(), new HiveDirectoryContext( + IGNORED, + false, + false, + new ConnectorIdentity("test", Optional.empty(), Optional.empty()), + ImmutableMap.of(), + new RuntimeStats())); + while (fileInfoIterator.hasNext()) { + HiveFileInfo fileInfo = fileInfoIterator.next(); + String fileName = fileInfo.getPath().getName(); + // expected to have the latest base file in p1 and p2 partitions + assertTrue(fileName.startsWith("37c2b860-eea6-4142-8bda-257b2562e4b4-0_1-338-594") || fileName.startsWith("7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_1-149-341")); + // not expected to have the older version of the base file in p1 + assertFalse(fileName.startsWith("c0bbff31-67b3-4660-99ba-d388b8bb8c3c-0_0-32-192")); + } + } + finally { + hadoopConf = null; + } + } + @Test public void testDirectoryListerForHudiTable() throws IOException diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000000000.deltacommit.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000000000.deltacommit.crc new file mode 100644 index 000000000000..6faac7ab3cc5 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000000000.deltacommit.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000000000.deltacommit.inflight.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000000000.deltacommit.inflight.crc new file mode 100644 index 000000000000..b4cea9f0b229 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000000000.deltacommit.inflight.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000000000.deltacommit.requested.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000000000.deltacommit.requested.crc new file mode 100644 index 000000000000..3b7b044936a8 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000000000.deltacommit.requested.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000005000.deltacommit.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000005000.deltacommit.crc new file mode 100644 index 000000000000..450a4a10e65e Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000005000.deltacommit.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000005000.deltacommit.inflight.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000005000.deltacommit.inflight.crc new file mode 100644 index 000000000000..9d61269475a2 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000005000.deltacommit.inflight.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000005000.deltacommit.requested.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000005000.deltacommit.requested.crc new file mode 100644 index 000000000000..3b7b044936a8 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000005000.deltacommit.requested.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000006000.deltacommit.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000006000.deltacommit.crc new file mode 100644 index 000000000000..a1cc3a7d667e Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000006000.deltacommit.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000006000.deltacommit.inflight.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000006000.deltacommit.inflight.crc new file mode 100644 index 000000000000..852037172ee4 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000006000.deltacommit.inflight.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000006000.deltacommit.requested.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000006000.deltacommit.requested.crc new file mode 100644 index 000000000000..3b7b044936a8 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000006000.deltacommit.requested.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000007000.deltacommit.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000007000.deltacommit.crc new file mode 100644 index 000000000000..02f244959194 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000007000.deltacommit.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000007000.deltacommit.inflight.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000007000.deltacommit.inflight.crc new file mode 100644 index 000000000000..e1eeb2db56ae Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000007000.deltacommit.inflight.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000007000.deltacommit.requested.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000007000.deltacommit.requested.crc new file mode 100644 index 000000000000..3b7b044936a8 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000007000.deltacommit.requested.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000008000.deltacommit.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000008000.deltacommit.crc new file mode 100644 index 000000000000..56f35963896c Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000008000.deltacommit.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000008000.deltacommit.inflight.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000008000.deltacommit.inflight.crc new file mode 100644 index 000000000000..6545c9045b9d Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000008000.deltacommit.inflight.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000008000.deltacommit.requested.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000008000.deltacommit.requested.crc new file mode 100644 index 000000000000..3b7b044936a8 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000008000.deltacommit.requested.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000009000.deltacommit.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000009000.deltacommit.crc new file mode 100644 index 000000000000..7e32e845f94f Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000009000.deltacommit.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000009000.deltacommit.inflight.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000009000.deltacommit.inflight.crc new file mode 100644 index 000000000000..a31d3f866025 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000009000.deltacommit.inflight.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000009000.deltacommit.requested.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000009000.deltacommit.requested.crc new file mode 100644 index 000000000000..3b7b044936a8 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.19700101000009000.deltacommit.requested.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.hoodie.properties.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.hoodie.properties.crc new file mode 100644 index 000000000000..5ee6d1bbebd1 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/.hoodie.properties.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000000000.deltacommit b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000000000.deltacommit new file mode 100644 index 000000000000..85604703f33f --- /dev/null +++ b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000000000.deltacommit @@ -0,0 +1,38 @@ +{ + "partitionToWriteStats" : { + "p1" : [ { + "fileId" : "c0bbff31-67b3-4660-99ba-d388b8bb8c3c-0", + "path" : "p1/c0bbff31-67b3-4660-99ba-d388b8bb8c3c-0_0-32-192_19700101000000000.parquet", + "cdcStats" : null, + "prevCommit" : "null", + "numWrites" : 4, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 4, + "totalWriteBytes" : 435904, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "p1", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 435904, + "minEventTime" : null, + "maxEventTime" : null, + "runtimeStats" : { + "totalScanTime" : 0, + "totalUpsertTime" : 0, + "totalCreateTime" : 1553 + } + } ] + }, + "compacted" : false, + "extraMetadata" : { + "schema" : "{\"type\":\"record\",\"name\":\"PayloadAdaptableRecord\",\"namespace\":\"hoodie\",\"fields\":[{\"name\":\"id\",\"type\":\"int\"},{\"name\":\"pt\",\"type\":\"string\"},{\"name\":\"ts\",\"type\":[\"null\",\"long\"],\"default\":null},{\"name\":\"Op\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"_change_operation_type\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"_hoodie_is_deleted\",\"type\":[\"null\",\"boolean\"],\"default\":null},{\"name\":\"_event_seq\",\"type\":[\"null\",\"long\"],\"default\":null},{\"name\":\"_event_lsn\",\"type\":[\"null\",\"long\"],\"default\":null}]}" + }, + "operationType" : "UPSERT" +} \ No newline at end of file diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000000000.deltacommit.inflight b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000000000.deltacommit.inflight new file mode 100644 index 000000000000..76a12cb5dc52 --- /dev/null +++ b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000000000.deltacommit.inflight @@ -0,0 +1,32 @@ +{ + "partitionToWriteStats" : { + "p1" : [ { + "fileId" : "", + "path" : null, + "cdcStats" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 4, + "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/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000000000.deltacommit.requested b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000000000.deltacommit.requested new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000005000.deltacommit b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000005000.deltacommit new file mode 100644 index 000000000000..efba53c4ba40 --- /dev/null +++ b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000005000.deltacommit @@ -0,0 +1,45 @@ +{ + "partitionToWriteStats" : { + "p1" : [ { + "fileId" : "c0bbff31-67b3-4660-99ba-d388b8bb8c3c-0", + "path" : "p1/.c0bbff31-67b3-4660-99ba-d388b8bb8c3c-0_19700101000000000.log.1_0-86-264", + "cdcStats" : null, + "prevCommit" : "19700101000000000", + "numWrites" : 4, + "numDeletes" : 0, + "numUpdateWrites" : 4, + "numInserts" : 0, + "totalWriteBytes" : 1413, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "p1", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 1413, + "minEventTime" : null, + "maxEventTime" : null, + "runtimeStats" : { + "totalScanTime" : 0, + "totalUpsertTime" : 109, + "totalCreateTime" : 0 + }, + "logVersion" : 1, + "logOffset" : 0, + "baseFile" : "c0bbff31-67b3-4660-99ba-d388b8bb8c3c-0_0-32-192_19700101000000000.parquet", + "logFiles" : [ ".c0bbff31-67b3-4660-99ba-d388b8bb8c3c-0_19700101000000000.log.1_0-86-264" ], + "recordsStats" : { + "val" : null + } + } ] + }, + "compacted" : false, + "extraMetadata" : { + "schema" : "{\"type\":\"record\",\"name\":\"PayloadAdaptableRecord\",\"namespace\":\"hoodie\",\"fields\":[{\"name\":\"id\",\"type\":\"int\"},{\"name\":\"pt\",\"type\":\"string\"},{\"name\":\"ts\",\"type\":[\"null\",\"long\"],\"default\":null},{\"name\":\"Op\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"_change_operation_type\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"_hoodie_is_deleted\",\"type\":[\"null\",\"boolean\"],\"default\":null},{\"name\":\"_event_seq\",\"type\":[\"null\",\"long\"],\"default\":null},{\"name\":\"_event_lsn\",\"type\":[\"null\",\"long\"],\"default\":null}]}" + }, + "operationType" : "UPSERT" +} \ No newline at end of file diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000005000.deltacommit.inflight b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000005000.deltacommit.inflight new file mode 100644 index 000000000000..9b464f445e61 --- /dev/null +++ b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000005000.deltacommit.inflight @@ -0,0 +1,56 @@ +{ + "partitionToWriteStats" : { + "p1" : [ { + "fileId" : "", + "path" : null, + "cdcStats" : 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, + "runtimeStats" : null + }, { + "fileId" : "c0bbff31-67b3-4660-99ba-d388b8bb8c3c-0", + "path" : null, + "cdcStats" : null, + "prevCommit" : "19700101000000000", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 4, + "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, + "runtimeStats" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { }, + "operationType" : "UPSERT" +} \ No newline at end of file diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000005000.deltacommit.requested b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000005000.deltacommit.requested new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000006000.deltacommit b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000006000.deltacommit new file mode 100644 index 000000000000..beb8503cf75a --- /dev/null +++ b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000006000.deltacommit @@ -0,0 +1,74 @@ +{ + "partitionToWriteStats" : { + "p1" : [ { + "fileId" : "c0bbff31-67b3-4660-99ba-d388b8bb8c3c-0", + "path" : "p1/.c0bbff31-67b3-4660-99ba-d388b8bb8c3c-0_19700101000000000.log.2_0-149-340", + "cdcStats" : null, + "prevCommit" : "19700101000000000", + "numWrites" : 0, + "numDeletes" : 4, + "numUpdateWrites" : 0, + "numInserts" : 0, + "totalWriteBytes" : 1035, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "p1", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 1035, + "minEventTime" : null, + "maxEventTime" : null, + "runtimeStats" : { + "totalScanTime" : 0, + "totalUpsertTime" : 90, + "totalCreateTime" : 0 + }, + "logVersion" : 2, + "logOffset" : 0, + "baseFile" : "c0bbff31-67b3-4660-99ba-d388b8bb8c3c-0_0-32-192_19700101000000000.parquet", + "logFiles" : [ ".c0bbff31-67b3-4660-99ba-d388b8bb8c3c-0_19700101000000000.log.1_0-86-264", ".c0bbff31-67b3-4660-99ba-d388b8bb8c3c-0_19700101000000000.log.2_0-149-340" ], + "recordsStats" : { + "val" : null + } + } ], + "p2" : [ { + "fileId" : "7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0", + "path" : "p2/7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_1-149-341_19700101000006000.parquet", + "cdcStats" : null, + "prevCommit" : "null", + "numWrites" : 4, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 4, + "totalWriteBytes" : 435917, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "p2", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 435917, + "minEventTime" : null, + "maxEventTime" : null, + "runtimeStats" : { + "totalScanTime" : 0, + "totalUpsertTime" : 0, + "totalCreateTime" : 139 + } + } ] + }, + "compacted" : false, + "extraMetadata" : { + "schema" : "{\"type\":\"record\",\"name\":\"PayloadAdaptableRecord\",\"namespace\":\"hoodie\",\"fields\":[{\"name\":\"id\",\"type\":\"int\"},{\"name\":\"pt\",\"type\":\"string\"},{\"name\":\"ts\",\"type\":[\"null\",\"long\"],\"default\":null},{\"name\":\"Op\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"_change_operation_type\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"_hoodie_is_deleted\",\"type\":[\"null\",\"boolean\"],\"default\":null},{\"name\":\"_event_seq\",\"type\":[\"null\",\"long\"],\"default\":null},{\"name\":\"_event_lsn\",\"type\":[\"null\",\"long\"],\"default\":null}]}" + }, + "operationType" : "UPSERT" +} \ No newline at end of file diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000006000.deltacommit.inflight b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000006000.deltacommit.inflight new file mode 100644 index 000000000000..ed48bd2b4c07 --- /dev/null +++ b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000006000.deltacommit.inflight @@ -0,0 +1,81 @@ +{ + "partitionToWriteStats" : { + "p1" : [ { + "fileId" : "", + "path" : null, + "cdcStats" : 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, + "runtimeStats" : null + }, { + "fileId" : "c0bbff31-67b3-4660-99ba-d388b8bb8c3c-0", + "path" : null, + "cdcStats" : null, + "prevCommit" : "19700101000000000", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 4, + "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, + "runtimeStats" : null + } ], + "p2" : [ { + "fileId" : "", + "path" : null, + "cdcStats" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 4, + "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/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000006000.deltacommit.requested b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000006000.deltacommit.requested new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000007000.deltacommit b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000007000.deltacommit new file mode 100644 index 000000000000..5677c0da2235 --- /dev/null +++ b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000007000.deltacommit @@ -0,0 +1,45 @@ +{ + "partitionToWriteStats" : { + "p2" : [ { + "fileId" : "7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0", + "path" : "p2/.7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_19700101000006000.log.1_0-212-431", + "cdcStats" : null, + "prevCommit" : "19700101000006000", + "numWrites" : 0, + "numDeletes" : 2, + "numUpdateWrites" : 0, + "numInserts" : 0, + "totalWriteBytes" : 1017, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "p2", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 1017, + "minEventTime" : null, + "maxEventTime" : null, + "runtimeStats" : { + "totalScanTime" : 0, + "totalUpsertTime" : 111, + "totalCreateTime" : 0 + }, + "logVersion" : 1, + "logOffset" : 0, + "baseFile" : "7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_1-149-341_19700101000006000.parquet", + "logFiles" : [ ".7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_19700101000006000.log.1_0-212-431" ], + "recordsStats" : { + "val" : null + } + } ] + }, + "compacted" : false, + "extraMetadata" : { + "schema" : "{\"type\":\"record\",\"name\":\"PayloadAdaptableRecord\",\"namespace\":\"hoodie\",\"fields\":[{\"name\":\"id\",\"type\":\"int\"},{\"name\":\"pt\",\"type\":\"string\"},{\"name\":\"ts\",\"type\":[\"null\",\"long\"],\"default\":null},{\"name\":\"Op\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"_change_operation_type\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"_hoodie_is_deleted\",\"type\":[\"null\",\"boolean\"],\"default\":null},{\"name\":\"_event_seq\",\"type\":[\"null\",\"long\"],\"default\":null},{\"name\":\"_event_lsn\",\"type\":[\"null\",\"long\"],\"default\":null}]}" + }, + "operationType" : "UPSERT" +} \ No newline at end of file diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000007000.deltacommit.inflight b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000007000.deltacommit.inflight new file mode 100644 index 000000000000..a5c052f81628 --- /dev/null +++ b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000007000.deltacommit.inflight @@ -0,0 +1,56 @@ +{ + "partitionToWriteStats" : { + "p2" : [ { + "fileId" : "", + "path" : null, + "cdcStats" : 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, + "runtimeStats" : null + }, { + "fileId" : "7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0", + "path" : null, + "cdcStats" : null, + "prevCommit" : "19700101000006000", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 2, + "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, + "runtimeStats" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { }, + "operationType" : "UPSERT" +} \ No newline at end of file diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000007000.deltacommit.requested b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000007000.deltacommit.requested new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000008000.deltacommit b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000008000.deltacommit new file mode 100644 index 000000000000..bfb7ad0011b3 --- /dev/null +++ b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000008000.deltacommit @@ -0,0 +1,45 @@ +{ + "partitionToWriteStats" : { + "p2" : [ { + "fileId" : "7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0", + "path" : "p2/.7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_19700101000006000.log.2_0-275-512", + "cdcStats" : null, + "prevCommit" : "19700101000006000", + "numWrites" : 0, + "numDeletes" : 1, + "numUpdateWrites" : 0, + "numInserts" : 0, + "totalWriteBytes" : 1008, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "p2", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 1008, + "minEventTime" : null, + "maxEventTime" : null, + "runtimeStats" : { + "totalScanTime" : 0, + "totalUpsertTime" : 89, + "totalCreateTime" : 0 + }, + "logVersion" : 2, + "logOffset" : 0, + "baseFile" : "7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_1-149-341_19700101000006000.parquet", + "logFiles" : [ ".7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_19700101000006000.log.1_0-212-431", ".7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_19700101000006000.log.2_0-275-512" ], + "recordsStats" : { + "val" : null + } + } ] + }, + "compacted" : false, + "extraMetadata" : { + "schema" : "{\"type\":\"record\",\"name\":\"PayloadAdaptableRecord\",\"namespace\":\"hoodie\",\"fields\":[{\"name\":\"id\",\"type\":\"int\"},{\"name\":\"pt\",\"type\":\"string\"},{\"name\":\"ts\",\"type\":[\"null\",\"long\"],\"default\":null},{\"name\":\"Op\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"_change_operation_type\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"_hoodie_is_deleted\",\"type\":[\"null\",\"boolean\"],\"default\":null},{\"name\":\"_event_seq\",\"type\":[\"null\",\"long\"],\"default\":null},{\"name\":\"_event_lsn\",\"type\":[\"null\",\"long\"],\"default\":null}]}" + }, + "operationType" : "UPSERT" +} \ No newline at end of file diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000008000.deltacommit.inflight b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000008000.deltacommit.inflight new file mode 100644 index 000000000000..5a870be96af3 --- /dev/null +++ b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000008000.deltacommit.inflight @@ -0,0 +1,56 @@ +{ + "partitionToWriteStats" : { + "p2" : [ { + "fileId" : "", + "path" : null, + "cdcStats" : 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, + "runtimeStats" : null + }, { + "fileId" : "7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0", + "path" : null, + "cdcStats" : null, + "prevCommit" : "19700101000006000", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "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, + "runtimeStats" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { }, + "operationType" : "UPSERT" +} \ No newline at end of file diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000008000.deltacommit.requested b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000008000.deltacommit.requested new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000009000.deltacommit b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000009000.deltacommit new file mode 100644 index 000000000000..9b75fcac051b --- /dev/null +++ b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000009000.deltacommit @@ -0,0 +1,74 @@ +{ + "partitionToWriteStats" : { + "p1" : [ { + "fileId" : "37c2b860-eea6-4142-8bda-257b2562e4b4-0", + "path" : "p1/37c2b860-eea6-4142-8bda-257b2562e4b4-0_1-338-594_19700101000009000.parquet", + "cdcStats" : null, + "prevCommit" : "null", + "numWrites" : 4, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 4, + "totalWriteBytes" : 435913, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "p1", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 435913, + "minEventTime" : null, + "maxEventTime" : null, + "runtimeStats" : { + "totalScanTime" : 0, + "totalUpsertTime" : 0, + "totalCreateTime" : 113 + } + } ], + "p2" : [ { + "fileId" : "7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0", + "path" : "p2/.7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_19700101000006000.log.3_0-338-593", + "cdcStats" : null, + "prevCommit" : "19700101000006000", + "numWrites" : 0, + "numDeletes" : 1, + "numUpdateWrites" : 0, + "numInserts" : 0, + "totalWriteBytes" : 1008, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "p2", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 1008, + "minEventTime" : null, + "maxEventTime" : null, + "runtimeStats" : { + "totalScanTime" : 0, + "totalUpsertTime" : 100, + "totalCreateTime" : 0 + }, + "logVersion" : 3, + "logOffset" : 0, + "baseFile" : "7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_1-149-341_19700101000006000.parquet", + "logFiles" : [ ".7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_19700101000006000.log.2_0-275-512", ".7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_19700101000006000.log.1_0-212-431", ".7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_19700101000006000.log.3_0-338-593" ], + "recordsStats" : { + "val" : null + } + } ] + }, + "compacted" : false, + "extraMetadata" : { + "schema" : "{\"type\":\"record\",\"name\":\"PayloadAdaptableRecord\",\"namespace\":\"hoodie\",\"fields\":[{\"name\":\"id\",\"type\":\"int\"},{\"name\":\"pt\",\"type\":\"string\"},{\"name\":\"ts\",\"type\":[\"null\",\"long\"],\"default\":null},{\"name\":\"Op\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"_change_operation_type\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"_hoodie_is_deleted\",\"type\":[\"null\",\"boolean\"],\"default\":null},{\"name\":\"_event_seq\",\"type\":[\"null\",\"long\"],\"default\":null},{\"name\":\"_event_lsn\",\"type\":[\"null\",\"long\"],\"default\":null}]}" + }, + "operationType" : "UPSERT" +} \ No newline at end of file diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000009000.deltacommit.inflight b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000009000.deltacommit.inflight new file mode 100644 index 000000000000..ee41cd93ab4a --- /dev/null +++ b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000009000.deltacommit.inflight @@ -0,0 +1,81 @@ +{ + "partitionToWriteStats" : { + "p1" : [ { + "fileId" : "", + "path" : null, + "cdcStats" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 4, + "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 + } ], + "p2" : [ { + "fileId" : "", + "path" : null, + "cdcStats" : 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, + "runtimeStats" : null + }, { + "fileId" : "7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0", + "path" : null, + "cdcStats" : null, + "prevCommit" : "19700101000006000", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "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, + "runtimeStats" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { }, + "operationType" : "UPSERT" +} \ No newline at end of file diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000009000.deltacommit.requested b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/19700101000009000.deltacommit.requested new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/hoodie.properties b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/hoodie.properties new file mode 100644 index 000000000000..4ef3bf0fc45e --- /dev/null +++ b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/hoodie.properties @@ -0,0 +1,18 @@ +#Updated at 2024-10-31T12:17:28.084Z +#Thu Oct 31 17:47:28 IST 2024 +hoodie.table.precombine.field=ts +hoodie.datasource.write.drop.partition.columns=false +hoodie.table.partition.fields=pt +hoodie.table.type=MERGE_ON_READ +hoodie.archivelog.folder=archived +hoodie.compaction.payload.class=org.apache.hudi.common.model.DefaultHoodieRecordPayload +hoodie.timeline.layout.version=1 +hoodie.table.version=6 +hoodie.bootstrap.index.class=org.apache.hudi.common.bootstrap.index.NoOpBootstrapIndex +hoodie.table.base.file.format=PARQUET +hoodie.table.recordkey.fields=id +hoodie.table.metadata.partitions=files,record_index +hoodie.table.name=test-trip-table +hoodie.compaction.record.merger.strategy=eeb8d96f-b1e4-49fd-bbf8-28ac514178e5 +hoodie.table.metadata.partitions.inflight= +hoodie.table.checksum=3139451808 diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.00000000000000010.deltacommit.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.00000000000000010.deltacommit.crc new file mode 100644 index 000000000000..1ad8e2f46e9c Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.00000000000000010.deltacommit.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.00000000000000010.deltacommit.inflight.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.00000000000000010.deltacommit.inflight.crc new file mode 100644 index 000000000000..3b7b044936a8 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.00000000000000010.deltacommit.inflight.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.00000000000000010.deltacommit.requested.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.00000000000000010.deltacommit.requested.crc new file mode 100644 index 000000000000..3b7b044936a8 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.00000000000000010.deltacommit.requested.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.00000000000000011.deltacommit.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.00000000000000011.deltacommit.crc new file mode 100644 index 000000000000..c257d8c88337 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.00000000000000011.deltacommit.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.00000000000000011.deltacommit.inflight.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.00000000000000011.deltacommit.inflight.crc new file mode 100644 index 000000000000..3b7b044936a8 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.00000000000000011.deltacommit.inflight.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.00000000000000011.deltacommit.requested.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.00000000000000011.deltacommit.requested.crc new file mode 100644 index 000000000000..3b7b044936a8 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.00000000000000011.deltacommit.requested.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000000000.deltacommit.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000000000.deltacommit.crc new file mode 100644 index 000000000000..6f467a83e193 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000000000.deltacommit.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000000000.deltacommit.inflight.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000000000.deltacommit.inflight.crc new file mode 100644 index 000000000000..8e0c2d7a8922 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000000000.deltacommit.inflight.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000000000.deltacommit.requested.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000000000.deltacommit.requested.crc new file mode 100644 index 000000000000..3b7b044936a8 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000000000.deltacommit.requested.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000005000.deltacommit.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000005000.deltacommit.crc new file mode 100644 index 000000000000..27b3610228b6 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000005000.deltacommit.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000005000.deltacommit.inflight.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000005000.deltacommit.inflight.crc new file mode 100644 index 000000000000..dd3ef205a05d Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000005000.deltacommit.inflight.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000005000.deltacommit.requested.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000005000.deltacommit.requested.crc new file mode 100644 index 000000000000..3b7b044936a8 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000005000.deltacommit.requested.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000006000.deltacommit.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000006000.deltacommit.crc new file mode 100644 index 000000000000..41bec2c82d9b Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000006000.deltacommit.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000006000.deltacommit.inflight.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000006000.deltacommit.inflight.crc new file mode 100644 index 000000000000..1e177126bdde Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000006000.deltacommit.inflight.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000006000.deltacommit.requested.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000006000.deltacommit.requested.crc new file mode 100644 index 000000000000..3b7b044936a8 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000006000.deltacommit.requested.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000007000.deltacommit.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000007000.deltacommit.crc new file mode 100644 index 000000000000..a478306d06b8 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000007000.deltacommit.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000007000.deltacommit.inflight.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000007000.deltacommit.inflight.crc new file mode 100644 index 000000000000..6ed6b4b615cd Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000007000.deltacommit.inflight.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000007000.deltacommit.requested.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000007000.deltacommit.requested.crc new file mode 100644 index 000000000000..3b7b044936a8 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000007000.deltacommit.requested.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000008000.deltacommit.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000008000.deltacommit.crc new file mode 100644 index 000000000000..4d0e6ca7f7c0 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000008000.deltacommit.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000008000.deltacommit.inflight.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000008000.deltacommit.inflight.crc new file mode 100644 index 000000000000..695c86e7a433 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000008000.deltacommit.inflight.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000008000.deltacommit.requested.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000008000.deltacommit.requested.crc new file mode 100644 index 000000000000..3b7b044936a8 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000008000.deltacommit.requested.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000009000.deltacommit.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000009000.deltacommit.crc new file mode 100644 index 000000000000..8d2db079e0f6 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000009000.deltacommit.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000009000.deltacommit.inflight.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000009000.deltacommit.inflight.crc new file mode 100644 index 000000000000..1e177126bdde Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000009000.deltacommit.inflight.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000009000.deltacommit.requested.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000009000.deltacommit.requested.crc new file mode 100644 index 000000000000..3b7b044936a8 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.19700101000009000.deltacommit.requested.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.hoodie.properties.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.hoodie.properties.crc new file mode 100644 index 000000000000..886bf02f0140 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/.hoodie.properties.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/00000000000000010.deltacommit b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/00000000000000010.deltacommit new file mode 100644 index 000000000000..831a5ac6bae3 --- /dev/null +++ b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/00000000000000010.deltacommit @@ -0,0 +1,38 @@ +{ + "partitionToWriteStats" : { + "files" : [ { + "fileId" : "files-0000-0", + "path" : "files/files-0000-0_0-5-4_00000000000000010.hfile", + "cdcStats" : null, + "prevCommit" : "null", + "numWrites" : 1, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 1, + "totalWriteBytes" : 6807, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "files", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 6807, + "minEventTime" : null, + "maxEventTime" : null, + "runtimeStats" : { + "totalScanTime" : 0, + "totalUpsertTime" : 0, + "totalCreateTime" : 5672 + } + } ] + }, + "compacted" : false, + "extraMetadata" : { + "schema" : "{\"type\":\"record\",\"name\":\"HoodieMetadataRecord\",\"namespace\":\"org.apache.hudi.avro.model\",\"doc\":\"A record saved within the Metadata Table\",\"fields\":[{\"name\":\"key\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}},{\"name\":\"type\",\"type\":\"int\",\"doc\":\"Type of the metadata record\"},{\"name\":\"filesystemMetadata\",\"type\":[\"null\",{\"type\":\"map\",\"values\":{\"type\":\"record\",\"name\":\"HoodieMetadataFileInfo\",\"fields\":[{\"name\":\"size\",\"type\":\"long\",\"doc\":\"Size of the file\"},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"True if this file has been deleted\"}]},\"avro.java.string\":\"String\"}],\"doc\":\"Contains information about partitions and files within the dataset\"},{\"name\":\"BloomFilterMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieMetadataBloomFilter\",\"doc\":\"Data file bloom filter details\",\"fields\":[{\"name\":\"type\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"},\"doc\":\"Bloom filter type code\"},{\"name\":\"timestamp\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"},\"doc\":\"Instant timestamp when this metadata was created/updated\"},{\"name\":\"bloomFilter\",\"type\":\"bytes\",\"doc\":\"Bloom filter binary byte array\"},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"Bloom filter entry valid/deleted flag\"}]}],\"doc\":\"Metadata Index of bloom filters for all data files in the user table\",\"default\":null},{\"name\":\"ColumnStatsMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieMetadataColumnStats\",\"doc\":\"Data file column statistics\",\"fields\":[{\"name\":\"fileName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"File name for which this column statistics applies\",\"default\":null},{\"name\":\"columnName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"Column name for which this column statistics applies\",\"default\":null},{\"name\":\"minValue\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"BooleanWrapper\",\"doc\":\"A record wrapping boolean type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"boolean\"}]},{\"type\":\"record\",\"name\":\"IntWrapper\",\"doc\":\"A record wrapping int type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"int\"}]},{\"type\":\"record\",\"name\":\"LongWrapper\",\"doc\":\"A record wrapping long type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"long\"}]},{\"type\":\"record\",\"name\":\"FloatWrapper\",\"doc\":\"A record wrapping float type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"float\"}]},{\"type\":\"record\",\"name\":\"DoubleWrapper\",\"doc\":\"A record wrapping double type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"double\"}]},{\"type\":\"record\",\"name\":\"BytesWrapper\",\"doc\":\"A record wrapping bytes type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"bytes\"}]},{\"type\":\"record\",\"name\":\"StringWrapper\",\"doc\":\"A record wrapping string type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}]},{\"type\":\"record\",\"name\":\"DateWrapper\",\"doc\":\"A record wrapping Date logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"int\"}]},{\"type\":\"record\",\"name\":\"DecimalWrapper\",\"doc\":\"A record wrapping Decimal logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"bytes\",\"logicalType\":\"decimal\",\"precision\":30,\"scale\":15}}]},{\"type\":\"record\",\"name\":\"TimeMicrosWrapper\",\"doc\":\"A record wrapping Time-micros logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"long\",\"logicalType\":\"time-micros\"}}]},{\"type\":\"record\",\"name\":\"TimestampMicrosWrapper\",\"doc\":\"A record wrapping Timestamp-micros logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"long\"}]}],\"doc\":\"Minimum value in the range. Based on user data table schema, we can convert this to appropriate type\",\"default\":null},{\"name\":\"maxValue\",\"type\":[\"null\",\"BooleanWrapper\",\"IntWrapper\",\"LongWrapper\",\"FloatWrapper\",\"DoubleWrapper\",\"BytesWrapper\",\"StringWrapper\",\"DateWrapper\",\"DecimalWrapper\",\"TimeMicrosWrapper\",\"TimestampMicrosWrapper\"],\"doc\":\"Maximum value in the range. Based on user data table schema, we can convert it to appropriate type\",\"default\":null},{\"name\":\"valueCount\",\"type\":[\"null\",\"long\"],\"doc\":\"Total count of values\",\"default\":null},{\"name\":\"nullCount\",\"type\":[\"null\",\"long\"],\"doc\":\"Total count of null values\",\"default\":null},{\"name\":\"totalSize\",\"type\":[\"null\",\"long\"],\"doc\":\"Total storage size on disk\",\"default\":null},{\"name\":\"totalUncompressedSize\",\"type\":[\"null\",\"long\"],\"doc\":\"Total uncompressed storage size on disk\",\"default\":null},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"Column range entry valid/deleted flag\"}]}],\"doc\":\"Metadata Index of column statistics for all data files in the user table\",\"default\":null},{\"name\":\"recordIndexMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieRecordIndexInfo\",\"fields\":[{\"name\":\"partitionName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"Refers to the partition name the record belongs to\",\"default\":null},{\"name\":\"fileIdHighBits\",\"type\":[\"null\",\"long\"],\"doc\":\"Refers to high 64 bits if the fileId is based on UUID format. \\nA UUID based fileId is stored as 3 pieces in RLI (fileIdHighBits, fileIdLowBits and fileIndex). \\nFileID format is {UUID}-{fileIndex}.\",\"default\":null},{\"name\":\"fileIdLowBits\",\"type\":[\"null\",\"long\"],\"doc\":\"Refers to low 64 bits if the fileId is based on UUID format. \\nA UUID based fileId is stored as 3 pieces in RLI (fileIdHighBits, fileIdLowBits and fileIndex). \\nFileID format is {UUID}-{fileIndex}.\",\"default\":null},{\"name\":\"fileIndex\",\"type\":[\"null\",\"int\"],\"doc\":\"Index representing file index which is used to re-construct UUID based fileID. Applicable when the fileId is based on UUID format. \\nA UUID based fileId is stored as 3 pieces in RLI (fileIdHighBits, fileIdLowBits and fileIndex). \\nFileID format is {UUID}-{fileIndex}.\",\"default\":null},{\"name\":\"fileId\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"Represents fileId of the location where record belongs to. When the encoding is 1, fileID is stored in raw string format.\",\"default\":null},{\"name\":\"instantTime\",\"type\":[\"null\",\"long\"],\"doc\":\"Epoch time in millisecond representing the commit time at which record was added\",\"default\":null},{\"name\":\"fileIdEncoding\",\"type\":\"int\",\"doc\":\"Represents fileId encoding. Possible values are 0 and 1. O represents UUID based fileID, and 1 represents raw string format of the fileId. \\nWhen the encoding is 0, reader can deduce fileID from fileIdLowBits, fileIdLowBits and fileIndex.\",\"default\":0}]}],\"doc\":\"Metadata Index that contains information about record keys and their location in the dataset\",\"default\":null}]}" + }, + "operationType" : "BULK_INSERT" +} \ No newline at end of file diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/00000000000000010.deltacommit.inflight b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/00000000000000010.deltacommit.inflight new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/00000000000000010.deltacommit.requested b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/00000000000000010.deltacommit.requested new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/00000000000000011.deltacommit b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/00000000000000011.deltacommit new file mode 100644 index 000000000000..1e9574803ad6 --- /dev/null +++ b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/00000000000000011.deltacommit @@ -0,0 +1,8 @@ +{ + "partitionToWriteStats" : { }, + "compacted" : false, + "extraMetadata" : { + "schema" : "{\"type\":\"record\",\"name\":\"HoodieMetadataRecord\",\"namespace\":\"org.apache.hudi.avro.model\",\"doc\":\"A record saved within the Metadata Table\",\"fields\":[{\"name\":\"key\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}},{\"name\":\"type\",\"type\":\"int\",\"doc\":\"Type of the metadata record\"},{\"name\":\"filesystemMetadata\",\"type\":[\"null\",{\"type\":\"map\",\"values\":{\"type\":\"record\",\"name\":\"HoodieMetadataFileInfo\",\"fields\":[{\"name\":\"size\",\"type\":\"long\",\"doc\":\"Size of the file\"},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"True if this file has been deleted\"}]},\"avro.java.string\":\"String\"}],\"doc\":\"Contains information about partitions and files within the dataset\"},{\"name\":\"BloomFilterMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieMetadataBloomFilter\",\"doc\":\"Data file bloom filter details\",\"fields\":[{\"name\":\"type\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"},\"doc\":\"Bloom filter type code\"},{\"name\":\"timestamp\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"},\"doc\":\"Instant timestamp when this metadata was created/updated\"},{\"name\":\"bloomFilter\",\"type\":\"bytes\",\"doc\":\"Bloom filter binary byte array\"},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"Bloom filter entry valid/deleted flag\"}]}],\"doc\":\"Metadata Index of bloom filters for all data files in the user table\",\"default\":null},{\"name\":\"ColumnStatsMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieMetadataColumnStats\",\"doc\":\"Data file column statistics\",\"fields\":[{\"name\":\"fileName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"File name for which this column statistics applies\",\"default\":null},{\"name\":\"columnName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"Column name for which this column statistics applies\",\"default\":null},{\"name\":\"minValue\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"BooleanWrapper\",\"doc\":\"A record wrapping boolean type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"boolean\"}]},{\"type\":\"record\",\"name\":\"IntWrapper\",\"doc\":\"A record wrapping int type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"int\"}]},{\"type\":\"record\",\"name\":\"LongWrapper\",\"doc\":\"A record wrapping long type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"long\"}]},{\"type\":\"record\",\"name\":\"FloatWrapper\",\"doc\":\"A record wrapping float type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"float\"}]},{\"type\":\"record\",\"name\":\"DoubleWrapper\",\"doc\":\"A record wrapping double type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"double\"}]},{\"type\":\"record\",\"name\":\"BytesWrapper\",\"doc\":\"A record wrapping bytes type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"bytes\"}]},{\"type\":\"record\",\"name\":\"StringWrapper\",\"doc\":\"A record wrapping string type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}]},{\"type\":\"record\",\"name\":\"DateWrapper\",\"doc\":\"A record wrapping Date logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"int\"}]},{\"type\":\"record\",\"name\":\"DecimalWrapper\",\"doc\":\"A record wrapping Decimal logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"bytes\",\"logicalType\":\"decimal\",\"precision\":30,\"scale\":15}}]},{\"type\":\"record\",\"name\":\"TimeMicrosWrapper\",\"doc\":\"A record wrapping Time-micros logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"long\",\"logicalType\":\"time-micros\"}}]},{\"type\":\"record\",\"name\":\"TimestampMicrosWrapper\",\"doc\":\"A record wrapping Timestamp-micros logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"long\"}]}],\"doc\":\"Minimum value in the range. Based on user data table schema, we can convert this to appropriate type\",\"default\":null},{\"name\":\"maxValue\",\"type\":[\"null\",\"BooleanWrapper\",\"IntWrapper\",\"LongWrapper\",\"FloatWrapper\",\"DoubleWrapper\",\"BytesWrapper\",\"StringWrapper\",\"DateWrapper\",\"DecimalWrapper\",\"TimeMicrosWrapper\",\"TimestampMicrosWrapper\"],\"doc\":\"Maximum value in the range. Based on user data table schema, we can convert it to appropriate type\",\"default\":null},{\"name\":\"valueCount\",\"type\":[\"null\",\"long\"],\"doc\":\"Total count of values\",\"default\":null},{\"name\":\"nullCount\",\"type\":[\"null\",\"long\"],\"doc\":\"Total count of null values\",\"default\":null},{\"name\":\"totalSize\",\"type\":[\"null\",\"long\"],\"doc\":\"Total storage size on disk\",\"default\":null},{\"name\":\"totalUncompressedSize\",\"type\":[\"null\",\"long\"],\"doc\":\"Total uncompressed storage size on disk\",\"default\":null},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"Column range entry valid/deleted flag\"}]}],\"doc\":\"Metadata Index of column statistics for all data files in the user table\",\"default\":null},{\"name\":\"recordIndexMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieRecordIndexInfo\",\"fields\":[{\"name\":\"partitionName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"Refers to the partition name the record belongs to\",\"default\":null},{\"name\":\"fileIdHighBits\",\"type\":[\"null\",\"long\"],\"doc\":\"Refers to high 64 bits if the fileId is based on UUID format. \\nA UUID based fileId is stored as 3 pieces in RLI (fileIdHighBits, fileIdLowBits and fileIndex). \\nFileID format is {UUID}-{fileIndex}.\",\"default\":null},{\"name\":\"fileIdLowBits\",\"type\":[\"null\",\"long\"],\"doc\":\"Refers to low 64 bits if the fileId is based on UUID format. \\nA UUID based fileId is stored as 3 pieces in RLI (fileIdHighBits, fileIdLowBits and fileIndex). \\nFileID format is {UUID}-{fileIndex}.\",\"default\":null},{\"name\":\"fileIndex\",\"type\":[\"null\",\"int\"],\"doc\":\"Index representing file index which is used to re-construct UUID based fileID. Applicable when the fileId is based on UUID format. \\nA UUID based fileId is stored as 3 pieces in RLI (fileIdHighBits, fileIdLowBits and fileIndex). \\nFileID format is {UUID}-{fileIndex}.\",\"default\":null},{\"name\":\"fileId\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"Represents fileId of the location where record belongs to. When the encoding is 1, fileID is stored in raw string format.\",\"default\":null},{\"name\":\"instantTime\",\"type\":[\"null\",\"long\"],\"doc\":\"Epoch time in millisecond representing the commit time at which record was added\",\"default\":null},{\"name\":\"fileIdEncoding\",\"type\":\"int\",\"doc\":\"Represents fileId encoding. Possible values are 0 and 1. O represents UUID based fileID, and 1 represents raw string format of the fileId. \\nWhen the encoding is 0, reader can deduce fileID from fileIdLowBits, fileIdLowBits and fileIndex.\",\"default\":0}]}],\"doc\":\"Metadata Index that contains information about record keys and their location in the dataset\",\"default\":null}]}" + }, + "operationType" : "BULK_INSERT" +} \ No newline at end of file diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/00000000000000011.deltacommit.inflight b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/00000000000000011.deltacommit.inflight new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/00000000000000011.deltacommit.requested b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/00000000000000011.deltacommit.requested new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000000000.deltacommit b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000000000.deltacommit new file mode 100644 index 000000000000..f8e15603b703 --- /dev/null +++ b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000000000.deltacommit @@ -0,0 +1,186 @@ +{ + "partitionToWriteStats" : { + "record_index" : [ { + "fileId" : "record-index-0000-0", + "path" : "record_index/.record-index-0000-0_00000000000000011.log.2_0-52-201", + "cdcStats" : null, + "prevCommit" : "00000000000000011", + "numWrites" : 1, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "numInserts" : 0, + "totalWriteBytes" : 13282, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "record_index", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 13282, + "minEventTime" : null, + "maxEventTime" : null, + "runtimeStats" : { + "totalScanTime" : 0, + "totalUpsertTime" : 143, + "totalCreateTime" : 0 + }, + "logVersion" : 2, + "logOffset" : 0, + "baseFile" : "", + "logFiles" : [ ".record-index-0000-0_00000000000000011.log.1_0-0-0", ".record-index-0000-0_00000000000000011.log.2_0-52-201" ], + "recordsStats" : { + "val" : null + } + }, { + "fileId" : "record-index-0001-0", + "path" : "record_index/.record-index-0001-0_00000000000000011.log.2_1-52-202", + "cdcStats" : null, + "prevCommit" : "00000000000000011", + "numWrites" : 1, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "numInserts" : 0, + "totalWriteBytes" : 13282, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "record_index", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 13282, + "minEventTime" : null, + "maxEventTime" : null, + "runtimeStats" : { + "totalScanTime" : 0, + "totalUpsertTime" : 135, + "totalCreateTime" : 0 + }, + "logVersion" : 2, + "logOffset" : 0, + "baseFile" : "", + "logFiles" : [ ".record-index-0001-0_00000000000000011.log.1_0-0-0", ".record-index-0001-0_00000000000000011.log.2_1-52-202" ], + "recordsStats" : { + "val" : null + } + }, { + "fileId" : "record-index-0008-0", + "path" : "record_index/.record-index-0008-0_00000000000000011.log.2_2-52-203", + "cdcStats" : null, + "prevCommit" : "00000000000000011", + "numWrites" : 1, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "numInserts" : 0, + "totalWriteBytes" : 13281, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "record_index", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 13281, + "minEventTime" : null, + "maxEventTime" : null, + "runtimeStats" : { + "totalScanTime" : 0, + "totalUpsertTime" : 135, + "totalCreateTime" : 0 + }, + "logVersion" : 2, + "logOffset" : 0, + "baseFile" : "", + "logFiles" : [ ".record-index-0008-0_00000000000000011.log.1_0-0-0", ".record-index-0008-0_00000000000000011.log.2_2-52-203" ], + "recordsStats" : { + "val" : null + } + }, { + "fileId" : "record-index-0009-0", + "path" : "record_index/.record-index-0009-0_00000000000000011.log.2_3-52-204", + "cdcStats" : null, + "prevCommit" : "00000000000000011", + "numWrites" : 1, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "numInserts" : 0, + "totalWriteBytes" : 13281, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "record_index", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 13281, + "minEventTime" : null, + "maxEventTime" : null, + "runtimeStats" : { + "totalScanTime" : 0, + "totalUpsertTime" : 143, + "totalCreateTime" : 0 + }, + "logVersion" : 2, + "logOffset" : 0, + "baseFile" : "", + "logFiles" : [ ".record-index-0009-0_00000000000000011.log.1_0-0-0", ".record-index-0009-0_00000000000000011.log.2_3-52-204" ], + "recordsStats" : { + "val" : null + } + } ], + "files" : [ { + "fileId" : "files-0000-0", + "path" : "files/.files-0000-0_00000000000000010.log.2_4-52-205", + "cdcStats" : null, + "prevCommit" : "00000000000000010", + "numWrites" : 2, + "numDeletes" : 0, + "numUpdateWrites" : 2, + "numInserts" : 0, + "totalWriteBytes" : 13369, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "files", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 13369, + "minEventTime" : null, + "maxEventTime" : null, + "runtimeStats" : { + "totalScanTime" : 0, + "totalUpsertTime" : 135, + "totalCreateTime" : 0 + }, + "logVersion" : 2, + "logOffset" : 0, + "baseFile" : "files-0000-0_0-5-4_00000000000000010.hfile", + "logFiles" : [ ".files-0000-0_00000000000000010.log.1_0-0-0", ".files-0000-0_00000000000000010.log.2_4-52-205" ], + "recordsStats" : { + "val" : null + } + } ] + }, + "compacted" : false, + "extraMetadata" : { + "schema" : "{\"type\":\"record\",\"name\":\"HoodieMetadataRecord\",\"namespace\":\"org.apache.hudi.avro.model\",\"doc\":\"A record saved within the Metadata Table\",\"fields\":[{\"name\":\"key\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}},{\"name\":\"type\",\"type\":\"int\",\"doc\":\"Type of the metadata record\"},{\"name\":\"filesystemMetadata\",\"type\":[\"null\",{\"type\":\"map\",\"values\":{\"type\":\"record\",\"name\":\"HoodieMetadataFileInfo\",\"fields\":[{\"name\":\"size\",\"type\":\"long\",\"doc\":\"Size of the file\"},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"True if this file has been deleted\"}]},\"avro.java.string\":\"String\"}],\"doc\":\"Contains information about partitions and files within the dataset\"},{\"name\":\"BloomFilterMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieMetadataBloomFilter\",\"doc\":\"Data file bloom filter details\",\"fields\":[{\"name\":\"type\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"},\"doc\":\"Bloom filter type code\"},{\"name\":\"timestamp\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"},\"doc\":\"Instant timestamp when this metadata was created/updated\"},{\"name\":\"bloomFilter\",\"type\":\"bytes\",\"doc\":\"Bloom filter binary byte array\"},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"Bloom filter entry valid/deleted flag\"}]}],\"doc\":\"Metadata Index of bloom filters for all data files in the user table\",\"default\":null},{\"name\":\"ColumnStatsMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieMetadataColumnStats\",\"doc\":\"Data file column statistics\",\"fields\":[{\"name\":\"fileName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"File name for which this column statistics applies\",\"default\":null},{\"name\":\"columnName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"Column name for which this column statistics applies\",\"default\":null},{\"name\":\"minValue\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"BooleanWrapper\",\"doc\":\"A record wrapping boolean type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"boolean\"}]},{\"type\":\"record\",\"name\":\"IntWrapper\",\"doc\":\"A record wrapping int type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"int\"}]},{\"type\":\"record\",\"name\":\"LongWrapper\",\"doc\":\"A record wrapping long type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"long\"}]},{\"type\":\"record\",\"name\":\"FloatWrapper\",\"doc\":\"A record wrapping float type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"float\"}]},{\"type\":\"record\",\"name\":\"DoubleWrapper\",\"doc\":\"A record wrapping double type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"double\"}]},{\"type\":\"record\",\"name\":\"BytesWrapper\",\"doc\":\"A record wrapping bytes type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"bytes\"}]},{\"type\":\"record\",\"name\":\"StringWrapper\",\"doc\":\"A record wrapping string type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}]},{\"type\":\"record\",\"name\":\"DateWrapper\",\"doc\":\"A record wrapping Date logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"int\"}]},{\"type\":\"record\",\"name\":\"DecimalWrapper\",\"doc\":\"A record wrapping Decimal logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"bytes\",\"logicalType\":\"decimal\",\"precision\":30,\"scale\":15}}]},{\"type\":\"record\",\"name\":\"TimeMicrosWrapper\",\"doc\":\"A record wrapping Time-micros logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"long\",\"logicalType\":\"time-micros\"}}]},{\"type\":\"record\",\"name\":\"TimestampMicrosWrapper\",\"doc\":\"A record wrapping Timestamp-micros logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"long\"}]}],\"doc\":\"Minimum value in the range. Based on user data table schema, we can convert this to appropriate type\",\"default\":null},{\"name\":\"maxValue\",\"type\":[\"null\",\"BooleanWrapper\",\"IntWrapper\",\"LongWrapper\",\"FloatWrapper\",\"DoubleWrapper\",\"BytesWrapper\",\"StringWrapper\",\"DateWrapper\",\"DecimalWrapper\",\"TimeMicrosWrapper\",\"TimestampMicrosWrapper\"],\"doc\":\"Maximum value in the range. Based on user data table schema, we can convert it to appropriate type\",\"default\":null},{\"name\":\"valueCount\",\"type\":[\"null\",\"long\"],\"doc\":\"Total count of values\",\"default\":null},{\"name\":\"nullCount\",\"type\":[\"null\",\"long\"],\"doc\":\"Total count of null values\",\"default\":null},{\"name\":\"totalSize\",\"type\":[\"null\",\"long\"],\"doc\":\"Total storage size on disk\",\"default\":null},{\"name\":\"totalUncompressedSize\",\"type\":[\"null\",\"long\"],\"doc\":\"Total uncompressed storage size on disk\",\"default\":null},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"Column range entry valid/deleted flag\"}]}],\"doc\":\"Metadata Index of column statistics for all data files in the user table\",\"default\":null},{\"name\":\"recordIndexMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieRecordIndexInfo\",\"fields\":[{\"name\":\"partitionName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"Refers to the partition name the record belongs to\",\"default\":null},{\"name\":\"fileIdHighBits\",\"type\":[\"null\",\"long\"],\"doc\":\"Refers to high 64 bits if the fileId is based on UUID format. \\nA UUID based fileId is stored as 3 pieces in RLI (fileIdHighBits, fileIdLowBits and fileIndex). \\nFileID format is {UUID}-{fileIndex}.\",\"default\":null},{\"name\":\"fileIdLowBits\",\"type\":[\"null\",\"long\"],\"doc\":\"Refers to low 64 bits if the fileId is based on UUID format. \\nA UUID based fileId is stored as 3 pieces in RLI (fileIdHighBits, fileIdLowBits and fileIndex). \\nFileID format is {UUID}-{fileIndex}.\",\"default\":null},{\"name\":\"fileIndex\",\"type\":[\"null\",\"int\"],\"doc\":\"Index representing file index which is used to re-construct UUID based fileID. Applicable when the fileId is based on UUID format. \\nA UUID based fileId is stored as 3 pieces in RLI (fileIdHighBits, fileIdLowBits and fileIndex). \\nFileID format is {UUID}-{fileIndex}.\",\"default\":null},{\"name\":\"fileId\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"Represents fileId of the location where record belongs to. When the encoding is 1, fileID is stored in raw string format.\",\"default\":null},{\"name\":\"instantTime\",\"type\":[\"null\",\"long\"],\"doc\":\"Epoch time in millisecond representing the commit time at which record was added\",\"default\":null},{\"name\":\"fileIdEncoding\",\"type\":\"int\",\"doc\":\"Represents fileId encoding. Possible values are 0 and 1. O represents UUID based fileID, and 1 represents raw string format of the fileId. \\nWhen the encoding is 0, reader can deduce fileID from fileIdLowBits, fileIdLowBits and fileIndex.\",\"default\":0}]}],\"doc\":\"Metadata Index that contains information about record keys and their location in the dataset\",\"default\":null}]}" + }, + "operationType" : "UPSERT_PREPPED" +} \ No newline at end of file diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000000000.deltacommit.inflight b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000000000.deltacommit.inflight new file mode 100644 index 000000000000..f344dac34fdc --- /dev/null +++ b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000000000.deltacommit.inflight @@ -0,0 +1,177 @@ +{ + "partitionToWriteStats" : { + "record_index" : [ { + "fileId" : "", + "path" : null, + "cdcStats" : 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, + "runtimeStats" : null + }, { + "fileId" : "record-index-0000-0", + "path" : null, + "cdcStats" : null, + "prevCommit" : "00000000000000011", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "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, + "runtimeStats" : null + }, { + "fileId" : "record-index-0001-0", + "path" : null, + "cdcStats" : null, + "prevCommit" : "00000000000000011", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "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, + "runtimeStats" : null + }, { + "fileId" : "record-index-0008-0", + "path" : null, + "cdcStats" : null, + "prevCommit" : "00000000000000011", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "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, + "runtimeStats" : null + }, { + "fileId" : "record-index-0009-0", + "path" : null, + "cdcStats" : null, + "prevCommit" : "00000000000000011", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "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, + "runtimeStats" : null + } ], + "files" : [ { + "fileId" : "", + "path" : null, + "cdcStats" : 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, + "runtimeStats" : null + }, { + "fileId" : "files-0000-0", + "path" : null, + "cdcStats" : null, + "prevCommit" : "00000000000000010", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 2, + "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, + "runtimeStats" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { }, + "operationType" : "UPSERT_PREPPED" +} \ No newline at end of file diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000000000.deltacommit.requested b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000000000.deltacommit.requested new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000005000.deltacommit b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000005000.deltacommit new file mode 100644 index 000000000000..e3daadfb6313 --- /dev/null +++ b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000005000.deltacommit @@ -0,0 +1,45 @@ +{ + "partitionToWriteStats" : { + "files" : [ { + "fileId" : "files-0000-0", + "path" : "files/.files-0000-0_00000000000000010.log.3_0-108-273", + "cdcStats" : null, + "prevCommit" : "00000000000000010", + "numWrites" : 2, + "numDeletes" : 0, + "numUpdateWrites" : 2, + "numInserts" : 0, + "totalWriteBytes" : 13367, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "files", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 13367, + "minEventTime" : null, + "maxEventTime" : null, + "runtimeStats" : { + "totalScanTime" : 0, + "totalUpsertTime" : 54, + "totalCreateTime" : 0 + }, + "logVersion" : 3, + "logOffset" : 0, + "baseFile" : "files-0000-0_0-5-4_00000000000000010.hfile", + "logFiles" : [ ".files-0000-0_00000000000000010.log.2_4-52-205", ".files-0000-0_00000000000000010.log.1_0-0-0", ".files-0000-0_00000000000000010.log.3_0-108-273" ], + "recordsStats" : { + "val" : null + } + } ] + }, + "compacted" : false, + "extraMetadata" : { + "schema" : "{\"type\":\"record\",\"name\":\"HoodieMetadataRecord\",\"namespace\":\"org.apache.hudi.avro.model\",\"doc\":\"A record saved within the Metadata Table\",\"fields\":[{\"name\":\"key\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}},{\"name\":\"type\",\"type\":\"int\",\"doc\":\"Type of the metadata record\"},{\"name\":\"filesystemMetadata\",\"type\":[\"null\",{\"type\":\"map\",\"values\":{\"type\":\"record\",\"name\":\"HoodieMetadataFileInfo\",\"fields\":[{\"name\":\"size\",\"type\":\"long\",\"doc\":\"Size of the file\"},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"True if this file has been deleted\"}]},\"avro.java.string\":\"String\"}],\"doc\":\"Contains information about partitions and files within the dataset\"},{\"name\":\"BloomFilterMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieMetadataBloomFilter\",\"doc\":\"Data file bloom filter details\",\"fields\":[{\"name\":\"type\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"},\"doc\":\"Bloom filter type code\"},{\"name\":\"timestamp\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"},\"doc\":\"Instant timestamp when this metadata was created/updated\"},{\"name\":\"bloomFilter\",\"type\":\"bytes\",\"doc\":\"Bloom filter binary byte array\"},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"Bloom filter entry valid/deleted flag\"}]}],\"doc\":\"Metadata Index of bloom filters for all data files in the user table\",\"default\":null},{\"name\":\"ColumnStatsMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieMetadataColumnStats\",\"doc\":\"Data file column statistics\",\"fields\":[{\"name\":\"fileName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"File name for which this column statistics applies\",\"default\":null},{\"name\":\"columnName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"Column name for which this column statistics applies\",\"default\":null},{\"name\":\"minValue\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"BooleanWrapper\",\"doc\":\"A record wrapping boolean type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"boolean\"}]},{\"type\":\"record\",\"name\":\"IntWrapper\",\"doc\":\"A record wrapping int type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"int\"}]},{\"type\":\"record\",\"name\":\"LongWrapper\",\"doc\":\"A record wrapping long type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"long\"}]},{\"type\":\"record\",\"name\":\"FloatWrapper\",\"doc\":\"A record wrapping float type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"float\"}]},{\"type\":\"record\",\"name\":\"DoubleWrapper\",\"doc\":\"A record wrapping double type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"double\"}]},{\"type\":\"record\",\"name\":\"BytesWrapper\",\"doc\":\"A record wrapping bytes type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"bytes\"}]},{\"type\":\"record\",\"name\":\"StringWrapper\",\"doc\":\"A record wrapping string type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}]},{\"type\":\"record\",\"name\":\"DateWrapper\",\"doc\":\"A record wrapping Date logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"int\"}]},{\"type\":\"record\",\"name\":\"DecimalWrapper\",\"doc\":\"A record wrapping Decimal logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"bytes\",\"logicalType\":\"decimal\",\"precision\":30,\"scale\":15}}]},{\"type\":\"record\",\"name\":\"TimeMicrosWrapper\",\"doc\":\"A record wrapping Time-micros logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"long\",\"logicalType\":\"time-micros\"}}]},{\"type\":\"record\",\"name\":\"TimestampMicrosWrapper\",\"doc\":\"A record wrapping Timestamp-micros logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"long\"}]}],\"doc\":\"Minimum value in the range. Based on user data table schema, we can convert this to appropriate type\",\"default\":null},{\"name\":\"maxValue\",\"type\":[\"null\",\"BooleanWrapper\",\"IntWrapper\",\"LongWrapper\",\"FloatWrapper\",\"DoubleWrapper\",\"BytesWrapper\",\"StringWrapper\",\"DateWrapper\",\"DecimalWrapper\",\"TimeMicrosWrapper\",\"TimestampMicrosWrapper\"],\"doc\":\"Maximum value in the range. Based on user data table schema, we can convert it to appropriate type\",\"default\":null},{\"name\":\"valueCount\",\"type\":[\"null\",\"long\"],\"doc\":\"Total count of values\",\"default\":null},{\"name\":\"nullCount\",\"type\":[\"null\",\"long\"],\"doc\":\"Total count of null values\",\"default\":null},{\"name\":\"totalSize\",\"type\":[\"null\",\"long\"],\"doc\":\"Total storage size on disk\",\"default\":null},{\"name\":\"totalUncompressedSize\",\"type\":[\"null\",\"long\"],\"doc\":\"Total uncompressed storage size on disk\",\"default\":null},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"Column range entry valid/deleted flag\"}]}],\"doc\":\"Metadata Index of column statistics for all data files in the user table\",\"default\":null},{\"name\":\"recordIndexMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieRecordIndexInfo\",\"fields\":[{\"name\":\"partitionName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"Refers to the partition name the record belongs to\",\"default\":null},{\"name\":\"fileIdHighBits\",\"type\":[\"null\",\"long\"],\"doc\":\"Refers to high 64 bits if the fileId is based on UUID format. \\nA UUID based fileId is stored as 3 pieces in RLI (fileIdHighBits, fileIdLowBits and fileIndex). \\nFileID format is {UUID}-{fileIndex}.\",\"default\":null},{\"name\":\"fileIdLowBits\",\"type\":[\"null\",\"long\"],\"doc\":\"Refers to low 64 bits if the fileId is based on UUID format. \\nA UUID based fileId is stored as 3 pieces in RLI (fileIdHighBits, fileIdLowBits and fileIndex). \\nFileID format is {UUID}-{fileIndex}.\",\"default\":null},{\"name\":\"fileIndex\",\"type\":[\"null\",\"int\"],\"doc\":\"Index representing file index which is used to re-construct UUID based fileID. Applicable when the fileId is based on UUID format. \\nA UUID based fileId is stored as 3 pieces in RLI (fileIdHighBits, fileIdLowBits and fileIndex). \\nFileID format is {UUID}-{fileIndex}.\",\"default\":null},{\"name\":\"fileId\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"Represents fileId of the location where record belongs to. When the encoding is 1, fileID is stored in raw string format.\",\"default\":null},{\"name\":\"instantTime\",\"type\":[\"null\",\"long\"],\"doc\":\"Epoch time in millisecond representing the commit time at which record was added\",\"default\":null},{\"name\":\"fileIdEncoding\",\"type\":\"int\",\"doc\":\"Represents fileId encoding. Possible values are 0 and 1. O represents UUID based fileID, and 1 represents raw string format of the fileId. \\nWhen the encoding is 0, reader can deduce fileID from fileIdLowBits, fileIdLowBits and fileIndex.\",\"default\":0}]}],\"doc\":\"Metadata Index that contains information about record keys and their location in the dataset\",\"default\":null}]}" + }, + "operationType" : "UPSERT_PREPPED" +} \ No newline at end of file diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000005000.deltacommit.inflight b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000005000.deltacommit.inflight new file mode 100644 index 000000000000..6f718ee1a47d --- /dev/null +++ b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000005000.deltacommit.inflight @@ -0,0 +1,56 @@ +{ + "partitionToWriteStats" : { + "files" : [ { + "fileId" : "", + "path" : null, + "cdcStats" : 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, + "runtimeStats" : null + }, { + "fileId" : "files-0000-0", + "path" : null, + "cdcStats" : null, + "prevCommit" : "00000000000000010", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 2, + "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, + "runtimeStats" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { }, + "operationType" : "UPSERT_PREPPED" +} \ No newline at end of file diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000005000.deltacommit.requested b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000005000.deltacommit.requested new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000006000.deltacommit b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000006000.deltacommit new file mode 100644 index 000000000000..91414fdba33e --- /dev/null +++ b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000006000.deltacommit @@ -0,0 +1,186 @@ +{ + "partitionToWriteStats" : { + "record_index" : [ { + "fileId" : "record-index-0000-0", + "path" : "record_index/.record-index-0000-0_00000000000000011.log.3_0-171-352", + "cdcStats" : null, + "prevCommit" : "00000000000000011", + "numWrites" : 1, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "numInserts" : 0, + "totalWriteBytes" : 13283, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "record_index", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 13283, + "minEventTime" : null, + "maxEventTime" : null, + "runtimeStats" : { + "totalScanTime" : 0, + "totalUpsertTime" : 61, + "totalCreateTime" : 0 + }, + "logVersion" : 3, + "logOffset" : 0, + "baseFile" : "", + "logFiles" : [ ".record-index-0000-0_00000000000000011.log.2_0-52-201", ".record-index-0000-0_00000000000000011.log.1_0-0-0", ".record-index-0000-0_00000000000000011.log.3_0-171-352" ], + "recordsStats" : { + "val" : null + } + }, { + "fileId" : "record-index-0001-0", + "path" : "record_index/.record-index-0001-0_00000000000000011.log.3_1-171-353", + "cdcStats" : null, + "prevCommit" : "00000000000000011", + "numWrites" : 1, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "numInserts" : 0, + "totalWriteBytes" : 13283, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "record_index", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 13283, + "minEventTime" : null, + "maxEventTime" : null, + "runtimeStats" : { + "totalScanTime" : 0, + "totalUpsertTime" : 59, + "totalCreateTime" : 0 + }, + "logVersion" : 3, + "logOffset" : 0, + "baseFile" : "", + "logFiles" : [ ".record-index-0001-0_00000000000000011.log.2_1-52-202", ".record-index-0001-0_00000000000000011.log.1_0-0-0", ".record-index-0001-0_00000000000000011.log.3_1-171-353" ], + "recordsStats" : { + "val" : null + } + }, { + "fileId" : "record-index-0008-0", + "path" : "record_index/.record-index-0008-0_00000000000000011.log.3_2-171-354", + "cdcStats" : null, + "prevCommit" : "00000000000000011", + "numWrites" : 1, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "numInserts" : 0, + "totalWriteBytes" : 13283, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "record_index", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 13283, + "minEventTime" : null, + "maxEventTime" : null, + "runtimeStats" : { + "totalScanTime" : 0, + "totalUpsertTime" : 64, + "totalCreateTime" : 0 + }, + "logVersion" : 3, + "logOffset" : 0, + "baseFile" : "", + "logFiles" : [ ".record-index-0008-0_00000000000000011.log.2_2-52-203", ".record-index-0008-0_00000000000000011.log.1_0-0-0", ".record-index-0008-0_00000000000000011.log.3_2-171-354" ], + "recordsStats" : { + "val" : null + } + }, { + "fileId" : "record-index-0009-0", + "path" : "record_index/.record-index-0009-0_00000000000000011.log.3_3-171-355", + "cdcStats" : null, + "prevCommit" : "00000000000000011", + "numWrites" : 1, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "numInserts" : 0, + "totalWriteBytes" : 13283, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "record_index", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 13283, + "minEventTime" : null, + "maxEventTime" : null, + "runtimeStats" : { + "totalScanTime" : 0, + "totalUpsertTime" : 67, + "totalCreateTime" : 0 + }, + "logVersion" : 3, + "logOffset" : 0, + "baseFile" : "", + "logFiles" : [ ".record-index-0009-0_00000000000000011.log.2_3-52-204", ".record-index-0009-0_00000000000000011.log.1_0-0-0", ".record-index-0009-0_00000000000000011.log.3_3-171-355" ], + "recordsStats" : { + "val" : null + } + } ], + "files" : [ { + "fileId" : "files-0000-0", + "path" : "files/.files-0000-0_00000000000000010.log.4_4-171-356", + "cdcStats" : null, + "prevCommit" : "00000000000000010", + "numWrites" : 3, + "numDeletes" : 0, + "numUpdateWrites" : 3, + "numInserts" : 0, + "totalWriteBytes" : 13428, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "files", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 13428, + "minEventTime" : null, + "maxEventTime" : null, + "runtimeStats" : { + "totalScanTime" : 0, + "totalUpsertTime" : 67, + "totalCreateTime" : 0 + }, + "logVersion" : 4, + "logOffset" : 0, + "baseFile" : "files-0000-0_0-5-4_00000000000000010.hfile", + "logFiles" : [ ".files-0000-0_00000000000000010.log.3_0-108-273", ".files-0000-0_00000000000000010.log.2_4-52-205", ".files-0000-0_00000000000000010.log.1_0-0-0", ".files-0000-0_00000000000000010.log.4_4-171-356" ], + "recordsStats" : { + "val" : null + } + } ] + }, + "compacted" : false, + "extraMetadata" : { + "schema" : "{\"type\":\"record\",\"name\":\"HoodieMetadataRecord\",\"namespace\":\"org.apache.hudi.avro.model\",\"doc\":\"A record saved within the Metadata Table\",\"fields\":[{\"name\":\"key\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}},{\"name\":\"type\",\"type\":\"int\",\"doc\":\"Type of the metadata record\"},{\"name\":\"filesystemMetadata\",\"type\":[\"null\",{\"type\":\"map\",\"values\":{\"type\":\"record\",\"name\":\"HoodieMetadataFileInfo\",\"fields\":[{\"name\":\"size\",\"type\":\"long\",\"doc\":\"Size of the file\"},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"True if this file has been deleted\"}]},\"avro.java.string\":\"String\"}],\"doc\":\"Contains information about partitions and files within the dataset\"},{\"name\":\"BloomFilterMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieMetadataBloomFilter\",\"doc\":\"Data file bloom filter details\",\"fields\":[{\"name\":\"type\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"},\"doc\":\"Bloom filter type code\"},{\"name\":\"timestamp\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"},\"doc\":\"Instant timestamp when this metadata was created/updated\"},{\"name\":\"bloomFilter\",\"type\":\"bytes\",\"doc\":\"Bloom filter binary byte array\"},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"Bloom filter entry valid/deleted flag\"}]}],\"doc\":\"Metadata Index of bloom filters for all data files in the user table\",\"default\":null},{\"name\":\"ColumnStatsMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieMetadataColumnStats\",\"doc\":\"Data file column statistics\",\"fields\":[{\"name\":\"fileName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"File name for which this column statistics applies\",\"default\":null},{\"name\":\"columnName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"Column name for which this column statistics applies\",\"default\":null},{\"name\":\"minValue\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"BooleanWrapper\",\"doc\":\"A record wrapping boolean type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"boolean\"}]},{\"type\":\"record\",\"name\":\"IntWrapper\",\"doc\":\"A record wrapping int type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"int\"}]},{\"type\":\"record\",\"name\":\"LongWrapper\",\"doc\":\"A record wrapping long type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"long\"}]},{\"type\":\"record\",\"name\":\"FloatWrapper\",\"doc\":\"A record wrapping float type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"float\"}]},{\"type\":\"record\",\"name\":\"DoubleWrapper\",\"doc\":\"A record wrapping double type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"double\"}]},{\"type\":\"record\",\"name\":\"BytesWrapper\",\"doc\":\"A record wrapping bytes type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"bytes\"}]},{\"type\":\"record\",\"name\":\"StringWrapper\",\"doc\":\"A record wrapping string type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}]},{\"type\":\"record\",\"name\":\"DateWrapper\",\"doc\":\"A record wrapping Date logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"int\"}]},{\"type\":\"record\",\"name\":\"DecimalWrapper\",\"doc\":\"A record wrapping Decimal logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"bytes\",\"logicalType\":\"decimal\",\"precision\":30,\"scale\":15}}]},{\"type\":\"record\",\"name\":\"TimeMicrosWrapper\",\"doc\":\"A record wrapping Time-micros logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"long\",\"logicalType\":\"time-micros\"}}]},{\"type\":\"record\",\"name\":\"TimestampMicrosWrapper\",\"doc\":\"A record wrapping Timestamp-micros logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"long\"}]}],\"doc\":\"Minimum value in the range. Based on user data table schema, we can convert this to appropriate type\",\"default\":null},{\"name\":\"maxValue\",\"type\":[\"null\",\"BooleanWrapper\",\"IntWrapper\",\"LongWrapper\",\"FloatWrapper\",\"DoubleWrapper\",\"BytesWrapper\",\"StringWrapper\",\"DateWrapper\",\"DecimalWrapper\",\"TimeMicrosWrapper\",\"TimestampMicrosWrapper\"],\"doc\":\"Maximum value in the range. Based on user data table schema, we can convert it to appropriate type\",\"default\":null},{\"name\":\"valueCount\",\"type\":[\"null\",\"long\"],\"doc\":\"Total count of values\",\"default\":null},{\"name\":\"nullCount\",\"type\":[\"null\",\"long\"],\"doc\":\"Total count of null values\",\"default\":null},{\"name\":\"totalSize\",\"type\":[\"null\",\"long\"],\"doc\":\"Total storage size on disk\",\"default\":null},{\"name\":\"totalUncompressedSize\",\"type\":[\"null\",\"long\"],\"doc\":\"Total uncompressed storage size on disk\",\"default\":null},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"Column range entry valid/deleted flag\"}]}],\"doc\":\"Metadata Index of column statistics for all data files in the user table\",\"default\":null},{\"name\":\"recordIndexMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieRecordIndexInfo\",\"fields\":[{\"name\":\"partitionName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"Refers to the partition name the record belongs to\",\"default\":null},{\"name\":\"fileIdHighBits\",\"type\":[\"null\",\"long\"],\"doc\":\"Refers to high 64 bits if the fileId is based on UUID format. \\nA UUID based fileId is stored as 3 pieces in RLI (fileIdHighBits, fileIdLowBits and fileIndex). \\nFileID format is {UUID}-{fileIndex}.\",\"default\":null},{\"name\":\"fileIdLowBits\",\"type\":[\"null\",\"long\"],\"doc\":\"Refers to low 64 bits if the fileId is based on UUID format. \\nA UUID based fileId is stored as 3 pieces in RLI (fileIdHighBits, fileIdLowBits and fileIndex). \\nFileID format is {UUID}-{fileIndex}.\",\"default\":null},{\"name\":\"fileIndex\",\"type\":[\"null\",\"int\"],\"doc\":\"Index representing file index which is used to re-construct UUID based fileID. Applicable when the fileId is based on UUID format. \\nA UUID based fileId is stored as 3 pieces in RLI (fileIdHighBits, fileIdLowBits and fileIndex). \\nFileID format is {UUID}-{fileIndex}.\",\"default\":null},{\"name\":\"fileId\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"Represents fileId of the location where record belongs to. When the encoding is 1, fileID is stored in raw string format.\",\"default\":null},{\"name\":\"instantTime\",\"type\":[\"null\",\"long\"],\"doc\":\"Epoch time in millisecond representing the commit time at which record was added\",\"default\":null},{\"name\":\"fileIdEncoding\",\"type\":\"int\",\"doc\":\"Represents fileId encoding. Possible values are 0 and 1. O represents UUID based fileID, and 1 represents raw string format of the fileId. \\nWhen the encoding is 0, reader can deduce fileID from fileIdLowBits, fileIdLowBits and fileIndex.\",\"default\":0}]}],\"doc\":\"Metadata Index that contains information about record keys and their location in the dataset\",\"default\":null}]}" + }, + "operationType" : "UPSERT_PREPPED" +} \ No newline at end of file diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000006000.deltacommit.inflight b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000006000.deltacommit.inflight new file mode 100644 index 000000000000..29e4b7ac0002 --- /dev/null +++ b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000006000.deltacommit.inflight @@ -0,0 +1,177 @@ +{ + "partitionToWriteStats" : { + "record_index" : [ { + "fileId" : "", + "path" : null, + "cdcStats" : 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, + "runtimeStats" : null + }, { + "fileId" : "record-index-0000-0", + "path" : null, + "cdcStats" : null, + "prevCommit" : "00000000000000011", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "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, + "runtimeStats" : null + }, { + "fileId" : "record-index-0001-0", + "path" : null, + "cdcStats" : null, + "prevCommit" : "00000000000000011", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "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, + "runtimeStats" : null + }, { + "fileId" : "record-index-0008-0", + "path" : null, + "cdcStats" : null, + "prevCommit" : "00000000000000011", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "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, + "runtimeStats" : null + }, { + "fileId" : "record-index-0009-0", + "path" : null, + "cdcStats" : null, + "prevCommit" : "00000000000000011", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "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, + "runtimeStats" : null + } ], + "files" : [ { + "fileId" : "", + "path" : null, + "cdcStats" : 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, + "runtimeStats" : null + }, { + "fileId" : "files-0000-0", + "path" : null, + "cdcStats" : null, + "prevCommit" : "00000000000000010", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 3, + "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, + "runtimeStats" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { }, + "operationType" : "UPSERT_PREPPED" +} \ No newline at end of file diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000006000.deltacommit.requested b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000006000.deltacommit.requested new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000007000.deltacommit b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000007000.deltacommit new file mode 100644 index 000000000000..3c112257a476 --- /dev/null +++ b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000007000.deltacommit @@ -0,0 +1,116 @@ +{ + "partitionToWriteStats" : { + "record_index" : [ { + "fileId" : "record-index-0008-0", + "path" : "record_index/.record-index-0008-0_00000000000000011.log.4_0-234-440", + "cdcStats" : null, + "prevCommit" : "00000000000000011", + "numWrites" : 0, + "numDeletes" : 1, + "numUpdateWrites" : 0, + "numInserts" : 0, + "totalWriteBytes" : 7264, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "record_index", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 7264, + "minEventTime" : null, + "maxEventTime" : null, + "runtimeStats" : { + "totalScanTime" : 0, + "totalUpsertTime" : 41, + "totalCreateTime" : 0 + }, + "logVersion" : 4, + "logOffset" : 0, + "baseFile" : "", + "logFiles" : [ ".record-index-0008-0_00000000000000011.log.3_2-171-354", ".record-index-0008-0_00000000000000011.log.2_2-52-203", ".record-index-0008-0_00000000000000011.log.1_0-0-0", ".record-index-0008-0_00000000000000011.log.4_0-234-440" ], + "recordsStats" : { + "val" : null + } + }, { + "fileId" : "record-index-0009-0", + "path" : "record_index/.record-index-0009-0_00000000000000011.log.4_1-234-441", + "cdcStats" : null, + "prevCommit" : "00000000000000011", + "numWrites" : 0, + "numDeletes" : 1, + "numUpdateWrites" : 0, + "numInserts" : 0, + "totalWriteBytes" : 7264, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "record_index", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 7264, + "minEventTime" : null, + "maxEventTime" : null, + "runtimeStats" : { + "totalScanTime" : 0, + "totalUpsertTime" : 35, + "totalCreateTime" : 0 + }, + "logVersion" : 4, + "logOffset" : 0, + "baseFile" : "", + "logFiles" : [ ".record-index-0009-0_00000000000000011.log.3_3-171-355", ".record-index-0009-0_00000000000000011.log.2_3-52-204", ".record-index-0009-0_00000000000000011.log.1_0-0-0", ".record-index-0009-0_00000000000000011.log.4_1-234-441" ], + "recordsStats" : { + "val" : null + } + } ], + "files" : [ { + "fileId" : "files-0000-0", + "path" : "files/.files-0000-0_00000000000000010.log.5_2-234-442", + "cdcStats" : null, + "prevCommit" : "00000000000000010", + "numWrites" : 2, + "numDeletes" : 0, + "numUpdateWrites" : 2, + "numInserts" : 0, + "totalWriteBytes" : 13371, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "files", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 13371, + "minEventTime" : null, + "maxEventTime" : null, + "runtimeStats" : { + "totalScanTime" : 0, + "totalUpsertTime" : 56, + "totalCreateTime" : 0 + }, + "logVersion" : 5, + "logOffset" : 0, + "baseFile" : "files-0000-0_0-5-4_00000000000000010.hfile", + "logFiles" : [ ".files-0000-0_00000000000000010.log.4_4-171-356", ".files-0000-0_00000000000000010.log.3_0-108-273", ".files-0000-0_00000000000000010.log.2_4-52-205", ".files-0000-0_00000000000000010.log.1_0-0-0", ".files-0000-0_00000000000000010.log.5_2-234-442" ], + "recordsStats" : { + "val" : null + } + } ] + }, + "compacted" : false, + "extraMetadata" : { + "schema" : "{\"type\":\"record\",\"name\":\"HoodieMetadataRecord\",\"namespace\":\"org.apache.hudi.avro.model\",\"doc\":\"A record saved within the Metadata Table\",\"fields\":[{\"name\":\"key\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}},{\"name\":\"type\",\"type\":\"int\",\"doc\":\"Type of the metadata record\"},{\"name\":\"filesystemMetadata\",\"type\":[\"null\",{\"type\":\"map\",\"values\":{\"type\":\"record\",\"name\":\"HoodieMetadataFileInfo\",\"fields\":[{\"name\":\"size\",\"type\":\"long\",\"doc\":\"Size of the file\"},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"True if this file has been deleted\"}]},\"avro.java.string\":\"String\"}],\"doc\":\"Contains information about partitions and files within the dataset\"},{\"name\":\"BloomFilterMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieMetadataBloomFilter\",\"doc\":\"Data file bloom filter details\",\"fields\":[{\"name\":\"type\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"},\"doc\":\"Bloom filter type code\"},{\"name\":\"timestamp\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"},\"doc\":\"Instant timestamp when this metadata was created/updated\"},{\"name\":\"bloomFilter\",\"type\":\"bytes\",\"doc\":\"Bloom filter binary byte array\"},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"Bloom filter entry valid/deleted flag\"}]}],\"doc\":\"Metadata Index of bloom filters for all data files in the user table\",\"default\":null},{\"name\":\"ColumnStatsMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieMetadataColumnStats\",\"doc\":\"Data file column statistics\",\"fields\":[{\"name\":\"fileName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"File name for which this column statistics applies\",\"default\":null},{\"name\":\"columnName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"Column name for which this column statistics applies\",\"default\":null},{\"name\":\"minValue\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"BooleanWrapper\",\"doc\":\"A record wrapping boolean type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"boolean\"}]},{\"type\":\"record\",\"name\":\"IntWrapper\",\"doc\":\"A record wrapping int type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"int\"}]},{\"type\":\"record\",\"name\":\"LongWrapper\",\"doc\":\"A record wrapping long type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"long\"}]},{\"type\":\"record\",\"name\":\"FloatWrapper\",\"doc\":\"A record wrapping float type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"float\"}]},{\"type\":\"record\",\"name\":\"DoubleWrapper\",\"doc\":\"A record wrapping double type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"double\"}]},{\"type\":\"record\",\"name\":\"BytesWrapper\",\"doc\":\"A record wrapping bytes type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"bytes\"}]},{\"type\":\"record\",\"name\":\"StringWrapper\",\"doc\":\"A record wrapping string type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}]},{\"type\":\"record\",\"name\":\"DateWrapper\",\"doc\":\"A record wrapping Date logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"int\"}]},{\"type\":\"record\",\"name\":\"DecimalWrapper\",\"doc\":\"A record wrapping Decimal logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"bytes\",\"logicalType\":\"decimal\",\"precision\":30,\"scale\":15}}]},{\"type\":\"record\",\"name\":\"TimeMicrosWrapper\",\"doc\":\"A record wrapping Time-micros logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"long\",\"logicalType\":\"time-micros\"}}]},{\"type\":\"record\",\"name\":\"TimestampMicrosWrapper\",\"doc\":\"A record wrapping Timestamp-micros logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"long\"}]}],\"doc\":\"Minimum value in the range. Based on user data table schema, we can convert this to appropriate type\",\"default\":null},{\"name\":\"maxValue\",\"type\":[\"null\",\"BooleanWrapper\",\"IntWrapper\",\"LongWrapper\",\"FloatWrapper\",\"DoubleWrapper\",\"BytesWrapper\",\"StringWrapper\",\"DateWrapper\",\"DecimalWrapper\",\"TimeMicrosWrapper\",\"TimestampMicrosWrapper\"],\"doc\":\"Maximum value in the range. Based on user data table schema, we can convert it to appropriate type\",\"default\":null},{\"name\":\"valueCount\",\"type\":[\"null\",\"long\"],\"doc\":\"Total count of values\",\"default\":null},{\"name\":\"nullCount\",\"type\":[\"null\",\"long\"],\"doc\":\"Total count of null values\",\"default\":null},{\"name\":\"totalSize\",\"type\":[\"null\",\"long\"],\"doc\":\"Total storage size on disk\",\"default\":null},{\"name\":\"totalUncompressedSize\",\"type\":[\"null\",\"long\"],\"doc\":\"Total uncompressed storage size on disk\",\"default\":null},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"Column range entry valid/deleted flag\"}]}],\"doc\":\"Metadata Index of column statistics for all data files in the user table\",\"default\":null},{\"name\":\"recordIndexMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieRecordIndexInfo\",\"fields\":[{\"name\":\"partitionName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"Refers to the partition name the record belongs to\",\"default\":null},{\"name\":\"fileIdHighBits\",\"type\":[\"null\",\"long\"],\"doc\":\"Refers to high 64 bits if the fileId is based on UUID format. \\nA UUID based fileId is stored as 3 pieces in RLI (fileIdHighBits, fileIdLowBits and fileIndex). \\nFileID format is {UUID}-{fileIndex}.\",\"default\":null},{\"name\":\"fileIdLowBits\",\"type\":[\"null\",\"long\"],\"doc\":\"Refers to low 64 bits if the fileId is based on UUID format. \\nA UUID based fileId is stored as 3 pieces in RLI (fileIdHighBits, fileIdLowBits and fileIndex). \\nFileID format is {UUID}-{fileIndex}.\",\"default\":null},{\"name\":\"fileIndex\",\"type\":[\"null\",\"int\"],\"doc\":\"Index representing file index which is used to re-construct UUID based fileID. Applicable when the fileId is based on UUID format. \\nA UUID based fileId is stored as 3 pieces in RLI (fileIdHighBits, fileIdLowBits and fileIndex). \\nFileID format is {UUID}-{fileIndex}.\",\"default\":null},{\"name\":\"fileId\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"Represents fileId of the location where record belongs to. When the encoding is 1, fileID is stored in raw string format.\",\"default\":null},{\"name\":\"instantTime\",\"type\":[\"null\",\"long\"],\"doc\":\"Epoch time in millisecond representing the commit time at which record was added\",\"default\":null},{\"name\":\"fileIdEncoding\",\"type\":\"int\",\"doc\":\"Represents fileId encoding. Possible values are 0 and 1. O represents UUID based fileID, and 1 represents raw string format of the fileId. \\nWhen the encoding is 0, reader can deduce fileID from fileIdLowBits, fileIdLowBits and fileIndex.\",\"default\":0}]}],\"doc\":\"Metadata Index that contains information about record keys and their location in the dataset\",\"default\":null}]}" + }, + "operationType" : "UPSERT_PREPPED" +} \ No newline at end of file diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000007000.deltacommit.inflight b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000007000.deltacommit.inflight new file mode 100644 index 000000000000..e6011815660e --- /dev/null +++ b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000007000.deltacommit.inflight @@ -0,0 +1,129 @@ +{ + "partitionToWriteStats" : { + "record_index" : [ { + "fileId" : "", + "path" : null, + "cdcStats" : 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, + "runtimeStats" : null + }, { + "fileId" : "record-index-0008-0", + "path" : null, + "cdcStats" : null, + "prevCommit" : "00000000000000011", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "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, + "runtimeStats" : null + }, { + "fileId" : "record-index-0009-0", + "path" : null, + "cdcStats" : null, + "prevCommit" : "00000000000000011", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "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, + "runtimeStats" : null + } ], + "files" : [ { + "fileId" : "", + "path" : null, + "cdcStats" : 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, + "runtimeStats" : null + }, { + "fileId" : "files-0000-0", + "path" : null, + "cdcStats" : null, + "prevCommit" : "00000000000000010", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 2, + "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, + "runtimeStats" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { }, + "operationType" : "UPSERT_PREPPED" +} \ No newline at end of file diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000007000.deltacommit.requested b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000007000.deltacommit.requested new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000008000.deltacommit b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000008000.deltacommit new file mode 100644 index 000000000000..569861b24477 --- /dev/null +++ b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000008000.deltacommit @@ -0,0 +1,81 @@ +{ + "partitionToWriteStats" : { + "record_index" : [ { + "fileId" : "record-index-0000-0", + "path" : "record_index/.record-index-0000-0_00000000000000011.log.4_0-297-521", + "cdcStats" : null, + "prevCommit" : "00000000000000011", + "numWrites" : 0, + "numDeletes" : 1, + "numUpdateWrites" : 0, + "numInserts" : 0, + "totalWriteBytes" : 7264, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "record_index", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 7264, + "minEventTime" : null, + "maxEventTime" : null, + "runtimeStats" : { + "totalScanTime" : 0, + "totalUpsertTime" : 38, + "totalCreateTime" : 0 + }, + "logVersion" : 4, + "logOffset" : 0, + "baseFile" : "", + "logFiles" : [ ".record-index-0000-0_00000000000000011.log.3_0-171-352", ".record-index-0000-0_00000000000000011.log.2_0-52-201", ".record-index-0000-0_00000000000000011.log.1_0-0-0", ".record-index-0000-0_00000000000000011.log.4_0-297-521" ], + "recordsStats" : { + "val" : null + } + } ], + "files" : [ { + "fileId" : "files-0000-0", + "path" : "files/.files-0000-0_00000000000000010.log.6_1-297-522", + "cdcStats" : null, + "prevCommit" : "00000000000000010", + "numWrites" : 2, + "numDeletes" : 0, + "numUpdateWrites" : 2, + "numInserts" : 0, + "totalWriteBytes" : 13371, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "files", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 13371, + "minEventTime" : null, + "maxEventTime" : null, + "runtimeStats" : { + "totalScanTime" : 0, + "totalUpsertTime" : 57, + "totalCreateTime" : 0 + }, + "logVersion" : 6, + "logOffset" : 0, + "baseFile" : "files-0000-0_0-5-4_00000000000000010.hfile", + "logFiles" : [ ".files-0000-0_00000000000000010.log.5_2-234-442", ".files-0000-0_00000000000000010.log.4_4-171-356", ".files-0000-0_00000000000000010.log.3_0-108-273", ".files-0000-0_00000000000000010.log.2_4-52-205", ".files-0000-0_00000000000000010.log.1_0-0-0", ".files-0000-0_00000000000000010.log.6_1-297-522" ], + "recordsStats" : { + "val" : null + } + } ] + }, + "compacted" : false, + "extraMetadata" : { + "schema" : "{\"type\":\"record\",\"name\":\"HoodieMetadataRecord\",\"namespace\":\"org.apache.hudi.avro.model\",\"doc\":\"A record saved within the Metadata Table\",\"fields\":[{\"name\":\"key\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}},{\"name\":\"type\",\"type\":\"int\",\"doc\":\"Type of the metadata record\"},{\"name\":\"filesystemMetadata\",\"type\":[\"null\",{\"type\":\"map\",\"values\":{\"type\":\"record\",\"name\":\"HoodieMetadataFileInfo\",\"fields\":[{\"name\":\"size\",\"type\":\"long\",\"doc\":\"Size of the file\"},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"True if this file has been deleted\"}]},\"avro.java.string\":\"String\"}],\"doc\":\"Contains information about partitions and files within the dataset\"},{\"name\":\"BloomFilterMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieMetadataBloomFilter\",\"doc\":\"Data file bloom filter details\",\"fields\":[{\"name\":\"type\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"},\"doc\":\"Bloom filter type code\"},{\"name\":\"timestamp\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"},\"doc\":\"Instant timestamp when this metadata was created/updated\"},{\"name\":\"bloomFilter\",\"type\":\"bytes\",\"doc\":\"Bloom filter binary byte array\"},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"Bloom filter entry valid/deleted flag\"}]}],\"doc\":\"Metadata Index of bloom filters for all data files in the user table\",\"default\":null},{\"name\":\"ColumnStatsMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieMetadataColumnStats\",\"doc\":\"Data file column statistics\",\"fields\":[{\"name\":\"fileName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"File name for which this column statistics applies\",\"default\":null},{\"name\":\"columnName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"Column name for which this column statistics applies\",\"default\":null},{\"name\":\"minValue\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"BooleanWrapper\",\"doc\":\"A record wrapping boolean type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"boolean\"}]},{\"type\":\"record\",\"name\":\"IntWrapper\",\"doc\":\"A record wrapping int type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"int\"}]},{\"type\":\"record\",\"name\":\"LongWrapper\",\"doc\":\"A record wrapping long type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"long\"}]},{\"type\":\"record\",\"name\":\"FloatWrapper\",\"doc\":\"A record wrapping float type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"float\"}]},{\"type\":\"record\",\"name\":\"DoubleWrapper\",\"doc\":\"A record wrapping double type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"double\"}]},{\"type\":\"record\",\"name\":\"BytesWrapper\",\"doc\":\"A record wrapping bytes type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"bytes\"}]},{\"type\":\"record\",\"name\":\"StringWrapper\",\"doc\":\"A record wrapping string type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}]},{\"type\":\"record\",\"name\":\"DateWrapper\",\"doc\":\"A record wrapping Date logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"int\"}]},{\"type\":\"record\",\"name\":\"DecimalWrapper\",\"doc\":\"A record wrapping Decimal logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"bytes\",\"logicalType\":\"decimal\",\"precision\":30,\"scale\":15}}]},{\"type\":\"record\",\"name\":\"TimeMicrosWrapper\",\"doc\":\"A record wrapping Time-micros logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"long\",\"logicalType\":\"time-micros\"}}]},{\"type\":\"record\",\"name\":\"TimestampMicrosWrapper\",\"doc\":\"A record wrapping Timestamp-micros logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"long\"}]}],\"doc\":\"Minimum value in the range. Based on user data table schema, we can convert this to appropriate type\",\"default\":null},{\"name\":\"maxValue\",\"type\":[\"null\",\"BooleanWrapper\",\"IntWrapper\",\"LongWrapper\",\"FloatWrapper\",\"DoubleWrapper\",\"BytesWrapper\",\"StringWrapper\",\"DateWrapper\",\"DecimalWrapper\",\"TimeMicrosWrapper\",\"TimestampMicrosWrapper\"],\"doc\":\"Maximum value in the range. Based on user data table schema, we can convert it to appropriate type\",\"default\":null},{\"name\":\"valueCount\",\"type\":[\"null\",\"long\"],\"doc\":\"Total count of values\",\"default\":null},{\"name\":\"nullCount\",\"type\":[\"null\",\"long\"],\"doc\":\"Total count of null values\",\"default\":null},{\"name\":\"totalSize\",\"type\":[\"null\",\"long\"],\"doc\":\"Total storage size on disk\",\"default\":null},{\"name\":\"totalUncompressedSize\",\"type\":[\"null\",\"long\"],\"doc\":\"Total uncompressed storage size on disk\",\"default\":null},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"Column range entry valid/deleted flag\"}]}],\"doc\":\"Metadata Index of column statistics for all data files in the user table\",\"default\":null},{\"name\":\"recordIndexMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieRecordIndexInfo\",\"fields\":[{\"name\":\"partitionName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"Refers to the partition name the record belongs to\",\"default\":null},{\"name\":\"fileIdHighBits\",\"type\":[\"null\",\"long\"],\"doc\":\"Refers to high 64 bits if the fileId is based on UUID format. \\nA UUID based fileId is stored as 3 pieces in RLI (fileIdHighBits, fileIdLowBits and fileIndex). \\nFileID format is {UUID}-{fileIndex}.\",\"default\":null},{\"name\":\"fileIdLowBits\",\"type\":[\"null\",\"long\"],\"doc\":\"Refers to low 64 bits if the fileId is based on UUID format. \\nA UUID based fileId is stored as 3 pieces in RLI (fileIdHighBits, fileIdLowBits and fileIndex). \\nFileID format is {UUID}-{fileIndex}.\",\"default\":null},{\"name\":\"fileIndex\",\"type\":[\"null\",\"int\"],\"doc\":\"Index representing file index which is used to re-construct UUID based fileID. Applicable when the fileId is based on UUID format. \\nA UUID based fileId is stored as 3 pieces in RLI (fileIdHighBits, fileIdLowBits and fileIndex). \\nFileID format is {UUID}-{fileIndex}.\",\"default\":null},{\"name\":\"fileId\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"Represents fileId of the location where record belongs to. When the encoding is 1, fileID is stored in raw string format.\",\"default\":null},{\"name\":\"instantTime\",\"type\":[\"null\",\"long\"],\"doc\":\"Epoch time in millisecond representing the commit time at which record was added\",\"default\":null},{\"name\":\"fileIdEncoding\",\"type\":\"int\",\"doc\":\"Represents fileId encoding. Possible values are 0 and 1. O represents UUID based fileID, and 1 represents raw string format of the fileId. \\nWhen the encoding is 0, reader can deduce fileID from fileIdLowBits, fileIdLowBits and fileIndex.\",\"default\":0}]}],\"doc\":\"Metadata Index that contains information about record keys and their location in the dataset\",\"default\":null}]}" + }, + "operationType" : "UPSERT_PREPPED" +} \ No newline at end of file diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000008000.deltacommit.inflight b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000008000.deltacommit.inflight new file mode 100644 index 000000000000..73647d2bce65 --- /dev/null +++ b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000008000.deltacommit.inflight @@ -0,0 +1,105 @@ +{ + "partitionToWriteStats" : { + "record_index" : [ { + "fileId" : "", + "path" : null, + "cdcStats" : 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, + "runtimeStats" : null + }, { + "fileId" : "record-index-0000-0", + "path" : null, + "cdcStats" : null, + "prevCommit" : "00000000000000011", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "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, + "runtimeStats" : null + } ], + "files" : [ { + "fileId" : "", + "path" : null, + "cdcStats" : 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, + "runtimeStats" : null + }, { + "fileId" : "files-0000-0", + "path" : null, + "cdcStats" : null, + "prevCommit" : "00000000000000010", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 2, + "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, + "runtimeStats" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { }, + "operationType" : "UPSERT_PREPPED" +} \ No newline at end of file diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000008000.deltacommit.requested b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000008000.deltacommit.requested new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000009000.deltacommit b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000009000.deltacommit new file mode 100644 index 000000000000..3d18f3b99f06 --- /dev/null +++ b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000009000.deltacommit @@ -0,0 +1,186 @@ +{ + "partitionToWriteStats" : { + "record_index" : [ { + "fileId" : "record-index-0000-0", + "path" : "record_index/.record-index-0000-0_00000000000000011.log.5_0-360-605", + "cdcStats" : null, + "prevCommit" : "00000000000000011", + "numWrites" : 1, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "numInserts" : 0, + "totalWriteBytes" : 13282, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "record_index", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 13282, + "minEventTime" : null, + "maxEventTime" : null, + "runtimeStats" : { + "totalScanTime" : 0, + "totalUpsertTime" : 56, + "totalCreateTime" : 0 + }, + "logVersion" : 5, + "logOffset" : 0, + "baseFile" : "", + "logFiles" : [ ".record-index-0000-0_00000000000000011.log.4_0-297-521", ".record-index-0000-0_00000000000000011.log.3_0-171-352", ".record-index-0000-0_00000000000000011.log.2_0-52-201", ".record-index-0000-0_00000000000000011.log.1_0-0-0", ".record-index-0000-0_00000000000000011.log.5_0-360-605" ], + "recordsStats" : { + "val" : null + } + }, { + "fileId" : "record-index-0001-0", + "path" : "record_index/.record-index-0001-0_00000000000000011.log.4_1-360-606", + "cdcStats" : null, + "prevCommit" : "00000000000000011", + "numWrites" : 1, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "numInserts" : 0, + "totalWriteBytes" : 13282, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "record_index", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 13282, + "minEventTime" : null, + "maxEventTime" : null, + "runtimeStats" : { + "totalScanTime" : 0, + "totalUpsertTime" : 51, + "totalCreateTime" : 0 + }, + "logVersion" : 4, + "logOffset" : 0, + "baseFile" : "", + "logFiles" : [ ".record-index-0001-0_00000000000000011.log.3_1-171-353", ".record-index-0001-0_00000000000000011.log.2_1-52-202", ".record-index-0001-0_00000000000000011.log.1_0-0-0", ".record-index-0001-0_00000000000000011.log.4_1-360-606" ], + "recordsStats" : { + "val" : null + } + }, { + "fileId" : "record-index-0008-0", + "path" : "record_index/.record-index-0008-0_00000000000000011.log.5_2-360-607", + "cdcStats" : null, + "prevCommit" : "00000000000000011", + "numWrites" : 1, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "numInserts" : 0, + "totalWriteBytes" : 13281, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "record_index", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 13281, + "minEventTime" : null, + "maxEventTime" : null, + "runtimeStats" : { + "totalScanTime" : 0, + "totalUpsertTime" : 49, + "totalCreateTime" : 0 + }, + "logVersion" : 5, + "logOffset" : 0, + "baseFile" : "", + "logFiles" : [ ".record-index-0008-0_00000000000000011.log.4_0-234-440", ".record-index-0008-0_00000000000000011.log.3_2-171-354", ".record-index-0008-0_00000000000000011.log.2_2-52-203", ".record-index-0008-0_00000000000000011.log.1_0-0-0", ".record-index-0008-0_00000000000000011.log.5_2-360-607" ], + "recordsStats" : { + "val" : null + } + }, { + "fileId" : "record-index-0009-0", + "path" : "record_index/.record-index-0009-0_00000000000000011.log.5_3-360-608", + "cdcStats" : null, + "prevCommit" : "00000000000000011", + "numWrites" : 1, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "numInserts" : 0, + "totalWriteBytes" : 13281, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "record_index", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 13281, + "minEventTime" : null, + "maxEventTime" : null, + "runtimeStats" : { + "totalScanTime" : 0, + "totalUpsertTime" : 47, + "totalCreateTime" : 0 + }, + "logVersion" : 5, + "logOffset" : 0, + "baseFile" : "", + "logFiles" : [ ".record-index-0009-0_00000000000000011.log.4_1-234-441", ".record-index-0009-0_00000000000000011.log.3_3-171-355", ".record-index-0009-0_00000000000000011.log.2_3-52-204", ".record-index-0009-0_00000000000000011.log.1_0-0-0", ".record-index-0009-0_00000000000000011.log.5_3-360-608" ], + "recordsStats" : { + "val" : null + } + } ], + "files" : [ { + "fileId" : "files-0000-0", + "path" : "files/.files-0000-0_00000000000000010.log.7_4-360-609", + "cdcStats" : null, + "prevCommit" : "00000000000000010", + "numWrites" : 3, + "numDeletes" : 0, + "numUpdateWrites" : 3, + "numInserts" : 0, + "totalWriteBytes" : 13432, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "files", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 13432, + "minEventTime" : null, + "maxEventTime" : null, + "runtimeStats" : { + "totalScanTime" : 0, + "totalUpsertTime" : 54, + "totalCreateTime" : 0 + }, + "logVersion" : 7, + "logOffset" : 0, + "baseFile" : "files-0000-0_0-5-4_00000000000000010.hfile", + "logFiles" : [ ".files-0000-0_00000000000000010.log.6_1-297-522", ".files-0000-0_00000000000000010.log.5_2-234-442", ".files-0000-0_00000000000000010.log.4_4-171-356", ".files-0000-0_00000000000000010.log.3_0-108-273", ".files-0000-0_00000000000000010.log.2_4-52-205", ".files-0000-0_00000000000000010.log.1_0-0-0", ".files-0000-0_00000000000000010.log.7_4-360-609" ], + "recordsStats" : { + "val" : null + } + } ] + }, + "compacted" : false, + "extraMetadata" : { + "schema" : "{\"type\":\"record\",\"name\":\"HoodieMetadataRecord\",\"namespace\":\"org.apache.hudi.avro.model\",\"doc\":\"A record saved within the Metadata Table\",\"fields\":[{\"name\":\"key\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}},{\"name\":\"type\",\"type\":\"int\",\"doc\":\"Type of the metadata record\"},{\"name\":\"filesystemMetadata\",\"type\":[\"null\",{\"type\":\"map\",\"values\":{\"type\":\"record\",\"name\":\"HoodieMetadataFileInfo\",\"fields\":[{\"name\":\"size\",\"type\":\"long\",\"doc\":\"Size of the file\"},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"True if this file has been deleted\"}]},\"avro.java.string\":\"String\"}],\"doc\":\"Contains information about partitions and files within the dataset\"},{\"name\":\"BloomFilterMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieMetadataBloomFilter\",\"doc\":\"Data file bloom filter details\",\"fields\":[{\"name\":\"type\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"},\"doc\":\"Bloom filter type code\"},{\"name\":\"timestamp\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"},\"doc\":\"Instant timestamp when this metadata was created/updated\"},{\"name\":\"bloomFilter\",\"type\":\"bytes\",\"doc\":\"Bloom filter binary byte array\"},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"Bloom filter entry valid/deleted flag\"}]}],\"doc\":\"Metadata Index of bloom filters for all data files in the user table\",\"default\":null},{\"name\":\"ColumnStatsMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieMetadataColumnStats\",\"doc\":\"Data file column statistics\",\"fields\":[{\"name\":\"fileName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"File name for which this column statistics applies\",\"default\":null},{\"name\":\"columnName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"Column name for which this column statistics applies\",\"default\":null},{\"name\":\"minValue\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"BooleanWrapper\",\"doc\":\"A record wrapping boolean type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"boolean\"}]},{\"type\":\"record\",\"name\":\"IntWrapper\",\"doc\":\"A record wrapping int type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"int\"}]},{\"type\":\"record\",\"name\":\"LongWrapper\",\"doc\":\"A record wrapping long type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"long\"}]},{\"type\":\"record\",\"name\":\"FloatWrapper\",\"doc\":\"A record wrapping float type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"float\"}]},{\"type\":\"record\",\"name\":\"DoubleWrapper\",\"doc\":\"A record wrapping double type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"double\"}]},{\"type\":\"record\",\"name\":\"BytesWrapper\",\"doc\":\"A record wrapping bytes type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"bytes\"}]},{\"type\":\"record\",\"name\":\"StringWrapper\",\"doc\":\"A record wrapping string type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}]},{\"type\":\"record\",\"name\":\"DateWrapper\",\"doc\":\"A record wrapping Date logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"int\"}]},{\"type\":\"record\",\"name\":\"DecimalWrapper\",\"doc\":\"A record wrapping Decimal logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"bytes\",\"logicalType\":\"decimal\",\"precision\":30,\"scale\":15}}]},{\"type\":\"record\",\"name\":\"TimeMicrosWrapper\",\"doc\":\"A record wrapping Time-micros logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"long\",\"logicalType\":\"time-micros\"}}]},{\"type\":\"record\",\"name\":\"TimestampMicrosWrapper\",\"doc\":\"A record wrapping Timestamp-micros logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"long\"}]}],\"doc\":\"Minimum value in the range. Based on user data table schema, we can convert this to appropriate type\",\"default\":null},{\"name\":\"maxValue\",\"type\":[\"null\",\"BooleanWrapper\",\"IntWrapper\",\"LongWrapper\",\"FloatWrapper\",\"DoubleWrapper\",\"BytesWrapper\",\"StringWrapper\",\"DateWrapper\",\"DecimalWrapper\",\"TimeMicrosWrapper\",\"TimestampMicrosWrapper\"],\"doc\":\"Maximum value in the range. Based on user data table schema, we can convert it to appropriate type\",\"default\":null},{\"name\":\"valueCount\",\"type\":[\"null\",\"long\"],\"doc\":\"Total count of values\",\"default\":null},{\"name\":\"nullCount\",\"type\":[\"null\",\"long\"],\"doc\":\"Total count of null values\",\"default\":null},{\"name\":\"totalSize\",\"type\":[\"null\",\"long\"],\"doc\":\"Total storage size on disk\",\"default\":null},{\"name\":\"totalUncompressedSize\",\"type\":[\"null\",\"long\"],\"doc\":\"Total uncompressed storage size on disk\",\"default\":null},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"Column range entry valid/deleted flag\"}]}],\"doc\":\"Metadata Index of column statistics for all data files in the user table\",\"default\":null},{\"name\":\"recordIndexMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieRecordIndexInfo\",\"fields\":[{\"name\":\"partitionName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"Refers to the partition name the record belongs to\",\"default\":null},{\"name\":\"fileIdHighBits\",\"type\":[\"null\",\"long\"],\"doc\":\"Refers to high 64 bits if the fileId is based on UUID format. \\nA UUID based fileId is stored as 3 pieces in RLI (fileIdHighBits, fileIdLowBits and fileIndex). \\nFileID format is {UUID}-{fileIndex}.\",\"default\":null},{\"name\":\"fileIdLowBits\",\"type\":[\"null\",\"long\"],\"doc\":\"Refers to low 64 bits if the fileId is based on UUID format. \\nA UUID based fileId is stored as 3 pieces in RLI (fileIdHighBits, fileIdLowBits and fileIndex). \\nFileID format is {UUID}-{fileIndex}.\",\"default\":null},{\"name\":\"fileIndex\",\"type\":[\"null\",\"int\"],\"doc\":\"Index representing file index which is used to re-construct UUID based fileID. Applicable when the fileId is based on UUID format. \\nA UUID based fileId is stored as 3 pieces in RLI (fileIdHighBits, fileIdLowBits and fileIndex). \\nFileID format is {UUID}-{fileIndex}.\",\"default\":null},{\"name\":\"fileId\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"Represents fileId of the location where record belongs to. When the encoding is 1, fileID is stored in raw string format.\",\"default\":null},{\"name\":\"instantTime\",\"type\":[\"null\",\"long\"],\"doc\":\"Epoch time in millisecond representing the commit time at which record was added\",\"default\":null},{\"name\":\"fileIdEncoding\",\"type\":\"int\",\"doc\":\"Represents fileId encoding. Possible values are 0 and 1. O represents UUID based fileID, and 1 represents raw string format of the fileId. \\nWhen the encoding is 0, reader can deduce fileID from fileIdLowBits, fileIdLowBits and fileIndex.\",\"default\":0}]}],\"doc\":\"Metadata Index that contains information about record keys and their location in the dataset\",\"default\":null}]}" + }, + "operationType" : "UPSERT_PREPPED" +} \ No newline at end of file diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000009000.deltacommit.inflight b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000009000.deltacommit.inflight new file mode 100644 index 000000000000..29e4b7ac0002 --- /dev/null +++ b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000009000.deltacommit.inflight @@ -0,0 +1,177 @@ +{ + "partitionToWriteStats" : { + "record_index" : [ { + "fileId" : "", + "path" : null, + "cdcStats" : 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, + "runtimeStats" : null + }, { + "fileId" : "record-index-0000-0", + "path" : null, + "cdcStats" : null, + "prevCommit" : "00000000000000011", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "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, + "runtimeStats" : null + }, { + "fileId" : "record-index-0001-0", + "path" : null, + "cdcStats" : null, + "prevCommit" : "00000000000000011", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "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, + "runtimeStats" : null + }, { + "fileId" : "record-index-0008-0", + "path" : null, + "cdcStats" : null, + "prevCommit" : "00000000000000011", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "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, + "runtimeStats" : null + }, { + "fileId" : "record-index-0009-0", + "path" : null, + "cdcStats" : null, + "prevCommit" : "00000000000000011", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "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, + "runtimeStats" : null + } ], + "files" : [ { + "fileId" : "", + "path" : null, + "cdcStats" : 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, + "runtimeStats" : null + }, { + "fileId" : "files-0000-0", + "path" : null, + "cdcStats" : null, + "prevCommit" : "00000000000000010", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 3, + "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, + "runtimeStats" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { }, + "operationType" : "UPSERT_PREPPED" +} \ No newline at end of file diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000009000.deltacommit.requested b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/19700101000009000.deltacommit.requested new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/hoodie.properties b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/hoodie.properties new file mode 100644 index 000000000000..2e14f7fcbc88 --- /dev/null +++ b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/.hoodie/hoodie.properties @@ -0,0 +1,15 @@ +#Properties saved on 2024-10-31T12:17:12.354Z +#Thu Oct 31 17:47:12 IST 2024 +hoodie.datasource.write.drop.partition.columns=false +hoodie.table.type=MERGE_ON_READ +hoodie.archivelog.folder=archived +hoodie.populate.meta.fields=false +hoodie.compaction.payload.class=org.apache.hudi.metadata.HoodieMetadataPayload +hoodie.timeline.layout.version=1 +hoodie.table.version=6 +hoodie.table.base.file.format=HFILE +hoodie.table.recordkey.fields=key +hoodie.table.name=test-trip-table_metadata +hoodie.table.keygenerator.class=org.apache.hudi.metadata.HoodieTableMetadataKeyGenerator +hoodie.compaction.record.merger.strategy=eeb8d96f-b1e4-49fd-bbf8-28ac514178e5 +hoodie.table.checksum=3798427289 diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/..files-0000-0_00000000000000010.log.1_0-0-0.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/..files-0000-0_00000000000000010.log.1_0-0-0.crc new file mode 100644 index 000000000000..9f392ca9db2a Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/..files-0000-0_00000000000000010.log.1_0-0-0.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/..files-0000-0_00000000000000010.log.2_4-52-205.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/..files-0000-0_00000000000000010.log.2_4-52-205.crc new file mode 100644 index 000000000000..75bb7ba3a2c4 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/..files-0000-0_00000000000000010.log.2_4-52-205.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/..files-0000-0_00000000000000010.log.3_0-108-273.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/..files-0000-0_00000000000000010.log.3_0-108-273.crc new file mode 100644 index 000000000000..9ede2f530062 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/..files-0000-0_00000000000000010.log.3_0-108-273.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/..files-0000-0_00000000000000010.log.4_4-171-356.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/..files-0000-0_00000000000000010.log.4_4-171-356.crc new file mode 100644 index 000000000000..9969d8e78021 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/..files-0000-0_00000000000000010.log.4_4-171-356.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/..files-0000-0_00000000000000010.log.5_2-234-442.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/..files-0000-0_00000000000000010.log.5_2-234-442.crc new file mode 100644 index 000000000000..08019e799f3b Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/..files-0000-0_00000000000000010.log.5_2-234-442.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/..files-0000-0_00000000000000010.log.6_1-297-522.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/..files-0000-0_00000000000000010.log.6_1-297-522.crc new file mode 100644 index 000000000000..5ceddc289b44 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/..files-0000-0_00000000000000010.log.6_1-297-522.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/..files-0000-0_00000000000000010.log.7_4-360-609.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/..files-0000-0_00000000000000010.log.7_4-360-609.crc new file mode 100644 index 000000000000..820329478073 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/..files-0000-0_00000000000000010.log.7_4-360-609.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/..hoodie_partition_metadata.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/..hoodie_partition_metadata.crc new file mode 100644 index 000000000000..4634193be9aa Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/..hoodie_partition_metadata.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/.files-0000-0_0-5-4_00000000000000010.hfile.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/.files-0000-0_0-5-4_00000000000000010.hfile.crc new file mode 100644 index 000000000000..8b40472045a3 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/.files-0000-0_0-5-4_00000000000000010.hfile.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/.files-0000-0_00000000000000010.log.1_0-0-0 b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/.files-0000-0_00000000000000010.log.1_0-0-0 new file mode 100644 index 000000000000..7058f97b74f3 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/.files-0000-0_00000000000000010.log.1_0-0-0 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/.files-0000-0_00000000000000010.log.2_4-52-205 b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/.files-0000-0_00000000000000010.log.2_4-52-205 new file mode 100644 index 000000000000..6d833bea7be1 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/.files-0000-0_00000000000000010.log.2_4-52-205 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/.files-0000-0_00000000000000010.log.3_0-108-273 b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/.files-0000-0_00000000000000010.log.3_0-108-273 new file mode 100644 index 000000000000..29ad98b702cb Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/.files-0000-0_00000000000000010.log.3_0-108-273 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/.files-0000-0_00000000000000010.log.4_4-171-356 b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/.files-0000-0_00000000000000010.log.4_4-171-356 new file mode 100644 index 000000000000..bcf8a0ef519f Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/.files-0000-0_00000000000000010.log.4_4-171-356 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/.files-0000-0_00000000000000010.log.5_2-234-442 b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/.files-0000-0_00000000000000010.log.5_2-234-442 new file mode 100644 index 000000000000..4177df0f598e Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/.files-0000-0_00000000000000010.log.5_2-234-442 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/.files-0000-0_00000000000000010.log.6_1-297-522 b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/.files-0000-0_00000000000000010.log.6_1-297-522 new file mode 100644 index 000000000000..bc90b3dcd67f Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/.files-0000-0_00000000000000010.log.6_1-297-522 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/.files-0000-0_00000000000000010.log.7_4-360-609 b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/.files-0000-0_00000000000000010.log.7_4-360-609 new file mode 100644 index 000000000000..5583dc1b7002 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/.files-0000-0_00000000000000010.log.7_4-360-609 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/.hoodie_partition_metadata b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/.hoodie_partition_metadata new file mode 100644 index 000000000000..8d10afde0892 --- /dev/null +++ b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/.hoodie_partition_metadata @@ -0,0 +1,4 @@ +#partition metadata +#Thu Oct 31 17:47:14 IST 2024 +commitTime=00000000000000010 +partitionDepth=1 diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/files-0000-0_0-5-4_00000000000000010.hfile b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/files-0000-0_0-5-4_00000000000000010.hfile new file mode 100644 index 000000000000..15bb7c4bf6d3 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/files/files-0000-0_0-5-4_00000000000000010.hfile differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..hoodie_partition_metadata.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..hoodie_partition_metadata.crc new file mode 100644 index 000000000000..0d15d6ecd2fc Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..hoodie_partition_metadata.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0000-0_00000000000000011.log.1_0-0-0.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0000-0_00000000000000011.log.1_0-0-0.crc new file mode 100644 index 000000000000..56f7705f9b28 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0000-0_00000000000000011.log.1_0-0-0.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0000-0_00000000000000011.log.2_0-52-201.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0000-0_00000000000000011.log.2_0-52-201.crc new file mode 100644 index 000000000000..100569f194f9 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0000-0_00000000000000011.log.2_0-52-201.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0000-0_00000000000000011.log.3_0-171-352.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0000-0_00000000000000011.log.3_0-171-352.crc new file mode 100644 index 000000000000..bd38585e57b0 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0000-0_00000000000000011.log.3_0-171-352.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0000-0_00000000000000011.log.4_0-297-521.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0000-0_00000000000000011.log.4_0-297-521.crc new file mode 100644 index 000000000000..0db0140ed815 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0000-0_00000000000000011.log.4_0-297-521.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0000-0_00000000000000011.log.5_0-360-605.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0000-0_00000000000000011.log.5_0-360-605.crc new file mode 100644 index 000000000000..bcfcb69a6579 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0000-0_00000000000000011.log.5_0-360-605.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0001-0_00000000000000011.log.1_0-0-0.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0001-0_00000000000000011.log.1_0-0-0.crc new file mode 100644 index 000000000000..56f7705f9b28 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0001-0_00000000000000011.log.1_0-0-0.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0001-0_00000000000000011.log.2_1-52-202.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0001-0_00000000000000011.log.2_1-52-202.crc new file mode 100644 index 000000000000..2ba4925f08a0 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0001-0_00000000000000011.log.2_1-52-202.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0001-0_00000000000000011.log.3_1-171-353.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0001-0_00000000000000011.log.3_1-171-353.crc new file mode 100644 index 000000000000..2ee6253f91ef Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0001-0_00000000000000011.log.3_1-171-353.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0001-0_00000000000000011.log.4_1-360-606.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0001-0_00000000000000011.log.4_1-360-606.crc new file mode 100644 index 000000000000..c6247b46b9ca Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0001-0_00000000000000011.log.4_1-360-606.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0002-0_00000000000000011.log.1_0-0-0.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0002-0_00000000000000011.log.1_0-0-0.crc new file mode 100644 index 000000000000..56f7705f9b28 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0002-0_00000000000000011.log.1_0-0-0.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0003-0_00000000000000011.log.1_0-0-0.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0003-0_00000000000000011.log.1_0-0-0.crc new file mode 100644 index 000000000000..56f7705f9b28 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0003-0_00000000000000011.log.1_0-0-0.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0004-0_00000000000000011.log.1_0-0-0.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0004-0_00000000000000011.log.1_0-0-0.crc new file mode 100644 index 000000000000..56f7705f9b28 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0004-0_00000000000000011.log.1_0-0-0.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0005-0_00000000000000011.log.1_0-0-0.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0005-0_00000000000000011.log.1_0-0-0.crc new file mode 100644 index 000000000000..56f7705f9b28 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0005-0_00000000000000011.log.1_0-0-0.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0006-0_00000000000000011.log.1_0-0-0.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0006-0_00000000000000011.log.1_0-0-0.crc new file mode 100644 index 000000000000..56f7705f9b28 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0006-0_00000000000000011.log.1_0-0-0.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0007-0_00000000000000011.log.1_0-0-0.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0007-0_00000000000000011.log.1_0-0-0.crc new file mode 100644 index 000000000000..56f7705f9b28 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0007-0_00000000000000011.log.1_0-0-0.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0008-0_00000000000000011.log.1_0-0-0.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0008-0_00000000000000011.log.1_0-0-0.crc new file mode 100644 index 000000000000..56f7705f9b28 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0008-0_00000000000000011.log.1_0-0-0.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0008-0_00000000000000011.log.2_2-52-203.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0008-0_00000000000000011.log.2_2-52-203.crc new file mode 100644 index 000000000000..a6ec839a249e Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0008-0_00000000000000011.log.2_2-52-203.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0008-0_00000000000000011.log.3_2-171-354.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0008-0_00000000000000011.log.3_2-171-354.crc new file mode 100644 index 000000000000..bb828eee9493 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0008-0_00000000000000011.log.3_2-171-354.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0008-0_00000000000000011.log.4_0-234-440.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0008-0_00000000000000011.log.4_0-234-440.crc new file mode 100644 index 000000000000..c5eab79757e0 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0008-0_00000000000000011.log.4_0-234-440.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0008-0_00000000000000011.log.5_2-360-607.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0008-0_00000000000000011.log.5_2-360-607.crc new file mode 100644 index 000000000000..5870419ec5cd Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0008-0_00000000000000011.log.5_2-360-607.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0009-0_00000000000000011.log.1_0-0-0.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0009-0_00000000000000011.log.1_0-0-0.crc new file mode 100644 index 000000000000..56f7705f9b28 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0009-0_00000000000000011.log.1_0-0-0.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0009-0_00000000000000011.log.2_3-52-204.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0009-0_00000000000000011.log.2_3-52-204.crc new file mode 100644 index 000000000000..c9d49b23895c Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0009-0_00000000000000011.log.2_3-52-204.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0009-0_00000000000000011.log.3_3-171-355.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0009-0_00000000000000011.log.3_3-171-355.crc new file mode 100644 index 000000000000..f4201b91f1cf Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0009-0_00000000000000011.log.3_3-171-355.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0009-0_00000000000000011.log.4_1-234-441.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0009-0_00000000000000011.log.4_1-234-441.crc new file mode 100644 index 000000000000..317f36fc7cbe Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0009-0_00000000000000011.log.4_1-234-441.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0009-0_00000000000000011.log.5_3-360-608.crc b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0009-0_00000000000000011.log.5_3-360-608.crc new file mode 100644 index 000000000000..392e1e57a9e4 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/..record-index-0009-0_00000000000000011.log.5_3-360-608.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.hoodie_partition_metadata b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.hoodie_partition_metadata new file mode 100644 index 000000000000..599cd73023ee --- /dev/null +++ b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.hoodie_partition_metadata @@ -0,0 +1,4 @@ +#partition metadata +#Thu Oct 31 17:47:31 IST 2024 +commitTime=00000000000000011 +partitionDepth=1 diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0000-0_00000000000000011.log.1_0-0-0 b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0000-0_00000000000000011.log.1_0-0-0 new file mode 100644 index 000000000000..161cafa88e03 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0000-0_00000000000000011.log.1_0-0-0 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0000-0_00000000000000011.log.2_0-52-201 b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0000-0_00000000000000011.log.2_0-52-201 new file mode 100644 index 000000000000..04221425c171 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0000-0_00000000000000011.log.2_0-52-201 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0000-0_00000000000000011.log.3_0-171-352 b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0000-0_00000000000000011.log.3_0-171-352 new file mode 100644 index 000000000000..72309835d072 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0000-0_00000000000000011.log.3_0-171-352 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0000-0_00000000000000011.log.4_0-297-521 b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0000-0_00000000000000011.log.4_0-297-521 new file mode 100644 index 000000000000..f380cdaafebd Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0000-0_00000000000000011.log.4_0-297-521 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0000-0_00000000000000011.log.5_0-360-605 b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0000-0_00000000000000011.log.5_0-360-605 new file mode 100644 index 000000000000..94daee939be8 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0000-0_00000000000000011.log.5_0-360-605 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0001-0_00000000000000011.log.1_0-0-0 b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0001-0_00000000000000011.log.1_0-0-0 new file mode 100644 index 000000000000..161cafa88e03 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0001-0_00000000000000011.log.1_0-0-0 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0001-0_00000000000000011.log.2_1-52-202 b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0001-0_00000000000000011.log.2_1-52-202 new file mode 100644 index 000000000000..d5910b520f58 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0001-0_00000000000000011.log.2_1-52-202 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0001-0_00000000000000011.log.3_1-171-353 b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0001-0_00000000000000011.log.3_1-171-353 new file mode 100644 index 000000000000..b211c319acdd Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0001-0_00000000000000011.log.3_1-171-353 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0001-0_00000000000000011.log.4_1-360-606 b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0001-0_00000000000000011.log.4_1-360-606 new file mode 100644 index 000000000000..2294216a5d04 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0001-0_00000000000000011.log.4_1-360-606 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0002-0_00000000000000011.log.1_0-0-0 b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0002-0_00000000000000011.log.1_0-0-0 new file mode 100644 index 000000000000..161cafa88e03 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0002-0_00000000000000011.log.1_0-0-0 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0003-0_00000000000000011.log.1_0-0-0 b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0003-0_00000000000000011.log.1_0-0-0 new file mode 100644 index 000000000000..161cafa88e03 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0003-0_00000000000000011.log.1_0-0-0 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0004-0_00000000000000011.log.1_0-0-0 b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0004-0_00000000000000011.log.1_0-0-0 new file mode 100644 index 000000000000..161cafa88e03 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0004-0_00000000000000011.log.1_0-0-0 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0005-0_00000000000000011.log.1_0-0-0 b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0005-0_00000000000000011.log.1_0-0-0 new file mode 100644 index 000000000000..161cafa88e03 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0005-0_00000000000000011.log.1_0-0-0 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0006-0_00000000000000011.log.1_0-0-0 b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0006-0_00000000000000011.log.1_0-0-0 new file mode 100644 index 000000000000..161cafa88e03 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0006-0_00000000000000011.log.1_0-0-0 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0007-0_00000000000000011.log.1_0-0-0 b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0007-0_00000000000000011.log.1_0-0-0 new file mode 100644 index 000000000000..161cafa88e03 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0007-0_00000000000000011.log.1_0-0-0 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0008-0_00000000000000011.log.1_0-0-0 b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0008-0_00000000000000011.log.1_0-0-0 new file mode 100644 index 000000000000..161cafa88e03 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0008-0_00000000000000011.log.1_0-0-0 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0008-0_00000000000000011.log.2_2-52-203 b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0008-0_00000000000000011.log.2_2-52-203 new file mode 100644 index 000000000000..6ed84e2da7e8 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0008-0_00000000000000011.log.2_2-52-203 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0008-0_00000000000000011.log.3_2-171-354 b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0008-0_00000000000000011.log.3_2-171-354 new file mode 100644 index 000000000000..ed549604da35 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0008-0_00000000000000011.log.3_2-171-354 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0008-0_00000000000000011.log.4_0-234-440 b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0008-0_00000000000000011.log.4_0-234-440 new file mode 100644 index 000000000000..b8eb309288af Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0008-0_00000000000000011.log.4_0-234-440 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0008-0_00000000000000011.log.5_2-360-607 b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0008-0_00000000000000011.log.5_2-360-607 new file mode 100644 index 000000000000..b212e9424fcd Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0008-0_00000000000000011.log.5_2-360-607 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0009-0_00000000000000011.log.1_0-0-0 b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0009-0_00000000000000011.log.1_0-0-0 new file mode 100644 index 000000000000..161cafa88e03 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0009-0_00000000000000011.log.1_0-0-0 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0009-0_00000000000000011.log.2_3-52-204 b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0009-0_00000000000000011.log.2_3-52-204 new file mode 100644 index 000000000000..cf38d24e5c85 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0009-0_00000000000000011.log.2_3-52-204 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0009-0_00000000000000011.log.3_3-171-355 b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0009-0_00000000000000011.log.3_3-171-355 new file mode 100644 index 000000000000..273f0cc432be Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0009-0_00000000000000011.log.3_3-171-355 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0009-0_00000000000000011.log.4_1-234-441 b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0009-0_00000000000000011.log.4_1-234-441 new file mode 100644 index 000000000000..0479562e663e Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0009-0_00000000000000011.log.4_1-234-441 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0009-0_00000000000000011.log.5_3-360-608 b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0009-0_00000000000000011.log.5_3-360-608 new file mode 100644 index 000000000000..88abb0e83f0b Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/.hoodie/metadata/record_index/.record-index-0009-0_00000000000000011.log.5_3-360-608 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/p1/..c0bbff31-67b3-4660-99ba-d388b8bb8c3c-0_19700101000000000.log.1_0-86-264.crc b/presto-hive/src/test/resources/hudi_mor_part_update/p1/..c0bbff31-67b3-4660-99ba-d388b8bb8c3c-0_19700101000000000.log.1_0-86-264.crc new file mode 100644 index 000000000000..a188631f3ecd Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/p1/..c0bbff31-67b3-4660-99ba-d388b8bb8c3c-0_19700101000000000.log.1_0-86-264.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/p1/..c0bbff31-67b3-4660-99ba-d388b8bb8c3c-0_19700101000000000.log.2_0-149-340.crc b/presto-hive/src/test/resources/hudi_mor_part_update/p1/..c0bbff31-67b3-4660-99ba-d388b8bb8c3c-0_19700101000000000.log.2_0-149-340.crc new file mode 100644 index 000000000000..4a1c73dc556f Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/p1/..c0bbff31-67b3-4660-99ba-d388b8bb8c3c-0_19700101000000000.log.2_0-149-340.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/p1/..hoodie_partition_metadata.crc b/presto-hive/src/test/resources/hudi_mor_part_update/p1/..hoodie_partition_metadata.crc new file mode 100644 index 000000000000..f58697bc7d2d Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/p1/..hoodie_partition_metadata.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/p1/.37c2b860-eea6-4142-8bda-257b2562e4b4-0_1-338-594_19700101000009000.parquet.crc b/presto-hive/src/test/resources/hudi_mor_part_update/p1/.37c2b860-eea6-4142-8bda-257b2562e4b4-0_1-338-594_19700101000009000.parquet.crc new file mode 100644 index 000000000000..e82e4e58c6c2 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/p1/.37c2b860-eea6-4142-8bda-257b2562e4b4-0_1-338-594_19700101000009000.parquet.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/p1/.c0bbff31-67b3-4660-99ba-d388b8bb8c3c-0_0-32-192_19700101000000000.parquet.crc b/presto-hive/src/test/resources/hudi_mor_part_update/p1/.c0bbff31-67b3-4660-99ba-d388b8bb8c3c-0_0-32-192_19700101000000000.parquet.crc new file mode 100644 index 000000000000..3b8f33cfe816 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/p1/.c0bbff31-67b3-4660-99ba-d388b8bb8c3c-0_0-32-192_19700101000000000.parquet.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/p1/.c0bbff31-67b3-4660-99ba-d388b8bb8c3c-0_19700101000000000.log.1_0-86-264 b/presto-hive/src/test/resources/hudi_mor_part_update/p1/.c0bbff31-67b3-4660-99ba-d388b8bb8c3c-0_19700101000000000.log.1_0-86-264 new file mode 100644 index 000000000000..e98e4018fada Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/p1/.c0bbff31-67b3-4660-99ba-d388b8bb8c3c-0_19700101000000000.log.1_0-86-264 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/p1/.c0bbff31-67b3-4660-99ba-d388b8bb8c3c-0_19700101000000000.log.2_0-149-340 b/presto-hive/src/test/resources/hudi_mor_part_update/p1/.c0bbff31-67b3-4660-99ba-d388b8bb8c3c-0_19700101000000000.log.2_0-149-340 new file mode 100644 index 000000000000..7e36c6337658 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/p1/.c0bbff31-67b3-4660-99ba-d388b8bb8c3c-0_19700101000000000.log.2_0-149-340 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/p1/.hoodie_partition_metadata b/presto-hive/src/test/resources/hudi_mor_part_update/p1/.hoodie_partition_metadata new file mode 100644 index 000000000000..16044ac3e7d8 --- /dev/null +++ b/presto-hive/src/test/resources/hudi_mor_part_update/p1/.hoodie_partition_metadata @@ -0,0 +1,4 @@ +#partition metadata +#Thu Oct 31 17:47:29 IST 2024 +commitTime=19700101000000000 +partitionDepth=1 diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/p1/37c2b860-eea6-4142-8bda-257b2562e4b4-0_1-338-594_19700101000009000.parquet b/presto-hive/src/test/resources/hudi_mor_part_update/p1/37c2b860-eea6-4142-8bda-257b2562e4b4-0_1-338-594_19700101000009000.parquet new file mode 100644 index 000000000000..f9e7916de74b Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/p1/37c2b860-eea6-4142-8bda-257b2562e4b4-0_1-338-594_19700101000009000.parquet differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/p1/c0bbff31-67b3-4660-99ba-d388b8bb8c3c-0_0-32-192_19700101000000000.parquet b/presto-hive/src/test/resources/hudi_mor_part_update/p1/c0bbff31-67b3-4660-99ba-d388b8bb8c3c-0_0-32-192_19700101000000000.parquet new file mode 100644 index 000000000000..454fd13ff102 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/p1/c0bbff31-67b3-4660-99ba-d388b8bb8c3c-0_0-32-192_19700101000000000.parquet differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/p2/..7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_19700101000006000.log.1_0-212-431.crc b/presto-hive/src/test/resources/hudi_mor_part_update/p2/..7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_19700101000006000.log.1_0-212-431.crc new file mode 100644 index 000000000000..0ae4de1e702b Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/p2/..7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_19700101000006000.log.1_0-212-431.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/p2/..7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_19700101000006000.log.2_0-275-512.crc b/presto-hive/src/test/resources/hudi_mor_part_update/p2/..7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_19700101000006000.log.2_0-275-512.crc new file mode 100644 index 000000000000..1f4e50144e73 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/p2/..7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_19700101000006000.log.2_0-275-512.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/p2/..7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_19700101000006000.log.3_0-338-593.crc b/presto-hive/src/test/resources/hudi_mor_part_update/p2/..7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_19700101000006000.log.3_0-338-593.crc new file mode 100644 index 000000000000..c4aff995cb2e Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/p2/..7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_19700101000006000.log.3_0-338-593.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/p2/..hoodie_partition_metadata.crc b/presto-hive/src/test/resources/hudi_mor_part_update/p2/..hoodie_partition_metadata.crc new file mode 100644 index 000000000000..3d448a0c0ca9 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/p2/..hoodie_partition_metadata.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/p2/.7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_1-149-341_19700101000006000.parquet.crc b/presto-hive/src/test/resources/hudi_mor_part_update/p2/.7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_1-149-341_19700101000006000.parquet.crc new file mode 100644 index 000000000000..dcc0a7e8c462 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/p2/.7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_1-149-341_19700101000006000.parquet.crc differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/p2/.7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_19700101000006000.log.1_0-212-431 b/presto-hive/src/test/resources/hudi_mor_part_update/p2/.7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_19700101000006000.log.1_0-212-431 new file mode 100644 index 000000000000..9c42b732589d Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/p2/.7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_19700101000006000.log.1_0-212-431 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/p2/.7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_19700101000006000.log.2_0-275-512 b/presto-hive/src/test/resources/hudi_mor_part_update/p2/.7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_19700101000006000.log.2_0-275-512 new file mode 100644 index 000000000000..c5baad6c8981 Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/p2/.7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_19700101000006000.log.2_0-275-512 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/p2/.7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_19700101000006000.log.3_0-338-593 b/presto-hive/src/test/resources/hudi_mor_part_update/p2/.7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_19700101000006000.log.3_0-338-593 new file mode 100644 index 000000000000..b0abb94827df Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/p2/.7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_19700101000006000.log.3_0-338-593 differ diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/p2/.hoodie_partition_metadata b/presto-hive/src/test/resources/hudi_mor_part_update/p2/.hoodie_partition_metadata new file mode 100644 index 000000000000..0cfe218dbbc7 --- /dev/null +++ b/presto-hive/src/test/resources/hudi_mor_part_update/p2/.hoodie_partition_metadata @@ -0,0 +1,4 @@ +#partition metadata +#Thu Oct 31 17:47:43 IST 2024 +commitTime=19700101000006000 +partitionDepth=1 diff --git a/presto-hive/src/test/resources/hudi_mor_part_update/p2/7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_1-149-341_19700101000006000.parquet b/presto-hive/src/test/resources/hudi_mor_part_update/p2/7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_1-149-341_19700101000006000.parquet new file mode 100644 index 000000000000..f51cd25965cf Binary files /dev/null and b/presto-hive/src/test/resources/hudi_mor_part_update/p2/7483ef07-d1f8-4d44-b9b0-cba6df5cd1b8-0_1-149-341_19700101000006000.parquet differ