public static long countNewRecords(HoodieTableMetaClient target, List<String> commitsToCatchup) throws IOException { long totalNew = 0; HoodieTimeline timeline = target.getActiveTimeline().reload().getCommitTimeline().filterCompletedInstants(); for (String commit : commitsToCatchup) { HoodieCommitMetadata c = HoodieCommitMetadata.fromBytes( timeline.getInstantDetails(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commit)).get(), HoodieCommitMetadata.class); totalNew += c.fetchTotalRecordsWritten() - c.fetchTotalUpdateRecordsWritten(); } return totalNew; } }
private List<FileSlice> getCurrentLatestFileSlices(HoodieTable table, HoodieWriteConfig cfg) throws IOException { HoodieTableFileSystemView view = new HoodieTableFileSystemView(table.getMetaClient(), table.getMetaClient().getActiveTimeline().reload().getCommitsAndCompactionTimeline()); List<FileSlice> fileSliceList = Arrays.asList(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS).stream().flatMap(partition -> view.getLatestFileSlices(partition)).collect(Collectors.toList()); return fileSliceList; }
Optional<HoodieInstant> lastInstant = activeTimeline.reload().getCommitsAndCompactionTimeline() .filterCompletedAndCompactionInstants().lastInstant(); Preconditions.checkArgument(lastInstant.isPresent());
Optional<HoodieInstant> lastInstant = activeTimeline.reload().getCommitsAndCompactionTimeline() .filterCompletedAndCompactionInstants().lastInstant(); Preconditions.checkArgument(lastInstant.isPresent());
private void verifyInflightInstants(HoodieTableMetaClient metaClient, int expectedTotalInstants) { HoodieTimeline timeline = metaClient.getActiveTimeline().reload() .getTimelineOfActions(Sets.newHashSet(HoodieTimeline.CLEAN_ACTION)).filterInflights(); assertEquals("Loaded inflight clean actions and the count should match", expectedTotalInstants, timeline.countInstants()); } }
@Test public void checkSerDe() throws IOException, ClassNotFoundException { // check if this object is serialized and de-serialized, we are able to read from the file system HoodieTableMetaClient deseralizedMetaClient = HoodieTestUtils .serializeDeserialize(metaClient, HoodieTableMetaClient.class); assertNotNull(deseralizedMetaClient); HoodieActiveTimeline commitTimeline = deseralizedMetaClient.getActiveTimeline(); HoodieInstant instant = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1"); commitTimeline.createInflight(instant); commitTimeline.saveAsComplete(instant, Optional.of("test-detail".getBytes())); commitTimeline = commitTimeline.reload(); HoodieInstant completedInstant = HoodieTimeline.getCompletedInstant(instant); assertEquals("Commit should be 1 and completed", completedInstant, commitTimeline.getInstants().findFirst().get()); assertArrayEquals("Commit value should be \"test-detail\"", "test-detail".getBytes(), commitTimeline.getInstantDetails(completedInstant).get()); }
private void moveCompactionFromRequestedToInflight(String compactionInstantTime, HoodieWriteClient client, HoodieWriteConfig cfg) throws IOException { HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); HoodieInstant compactionInstant = HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime); HoodieCompactionPlan workload = AvroUtils.deserializeCompactionPlan( metaClient.getActiveTimeline().getInstantAuxiliaryDetails(compactionInstant).get()); metaClient.getActiveTimeline().transitionCompactionRequestedToInflight(compactionInstant); HoodieInstant instant = metaClient.getActiveTimeline().reload().filterPendingCompactionTimeline().getInstants() .filter(in -> in.getTimestamp().equals(compactionInstantTime)).findAny().get(); assertTrue("Instant must be marked inflight", instant.isInflight()); }
@Test public void checkCommitTimeline() throws IOException { HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); HoodieTimeline activeCommitTimeline = activeTimeline.getCommitTimeline(); assertTrue("Should be empty commit timeline", activeCommitTimeline.empty()); HoodieInstant instant = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1"); activeTimeline.createInflight(instant); activeTimeline.saveAsComplete(instant, Optional.of("test-detail".getBytes())); // Commit timeline should not auto-reload every time getActiveCommitTimeline(), it should be cached activeTimeline = metaClient.getActiveTimeline(); activeCommitTimeline = activeTimeline.getCommitTimeline(); assertTrue("Should be empty commit timeline", activeCommitTimeline.empty()); HoodieInstant completedInstant = HoodieTimeline.getCompletedInstant(instant); activeTimeline = activeTimeline.reload(); activeCommitTimeline = activeTimeline.getCommitTimeline(); assertFalse("Should be the 1 commit we made", activeCommitTimeline.empty()); assertEquals("Commit should be 1", completedInstant, activeCommitTimeline.getInstants().findFirst().get()); assertArrayEquals("Commit value should be \"test-detail\"", "test-detail".getBytes(), activeCommitTimeline.getInstantDetails(completedInstant).get()); }
private List<WriteStatus> createNextDeltaCommit(String instantTime, List<HoodieRecord> records, HoodieWriteClient client, HoodieTableMetaClient metaClient, HoodieWriteConfig cfg, boolean skipCommit) { JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1); client.startCommitWithTime(instantTime); JavaRDD<WriteStatus> statuses = client.upsert(writeRecords, instantTime); List<WriteStatus> statusList = statuses.collect(); TestHoodieClientBase.assertNoWriteErrors(statusList); if (!cfg.shouldAutoCommit() && !skipCommit) { client.commit(instantTime, statuses); } Optional<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().reload().getDeltaCommitTimeline() .filterCompletedInstants().lastInstant(); if (skipCommit && !cfg.shouldAutoCommit()) { assertTrue("Delta commit should not be latest instant", deltaCommit.get().getTimestamp().compareTo(instantTime) < 0); } else { assertTrue(deltaCommit.isPresent()); assertEquals("Delta commit should be latest instant", instantTime, deltaCommit.get().getTimestamp()); } return statusList; }
timeline.saveAsComplete(instant4, Optional.empty()); timeline.createInflight(instant5); timeline = timeline.reload();
hoodieTable.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants(), allFiles); dataFilesToRead = roView.getLatestDataFiles(); List<HoodieDataFile> newDataFilesList = dataFilesToRead.collect(Collectors.toList());
List<HoodieDataFile> dataFiles2 = roView.getLatestDataFiles().collect(Collectors.toList()); final String compactedCommitTime = metaClient.getActiveTimeline().reload().getCommitsTimeline().lastInstant().get() .getTimestamp();
timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants(); List<HoodieInstant> originalCommits = timeline.getInstants().collect(Collectors.toList()); timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants(); originalCommits.removeAll(timeline.getInstants().collect(Collectors.toList()));
boolean result = archiveLog.archiveIfRequired(jsc); assertTrue(result); timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants(); assertEquals("Should not archive commits when maxCommitsToKeep is 5", 4, timeline.countInstants());
@Test public void testArchiveCommitSafety() throws IOException { HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath) .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .forTable("test-trip-table").withCompactionConfig( HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build(); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(dfs.getConf(), basePath); HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient); HoodieTestDataGenerator.createCommitFile(basePath, "100", dfs.getConf()); HoodieTestDataGenerator.createCommitFile(basePath, "101", dfs.getConf()); HoodieTestDataGenerator.createCommitFile(basePath, "102", dfs.getConf()); HoodieTestDataGenerator.createCommitFile(basePath, "103", dfs.getConf()); HoodieTestDataGenerator.createCommitFile(basePath, "104", dfs.getConf()); HoodieTestDataGenerator.createCommitFile(basePath, "105", dfs.getConf()); HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants()); boolean result = archiveLog.archiveIfRequired(jsc); assertTrue(result); timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants(); assertTrue("Archived commits should always be safe", timeline.containsOrBeforeTimelineStarts("100")); assertTrue("Archived commits should always be safe", timeline.containsOrBeforeTimelineStarts("101")); assertTrue("Archived commits should always be safe", timeline.containsOrBeforeTimelineStarts("102")); assertTrue("Archived commits should always be safe", timeline.containsOrBeforeTimelineStarts("103")); }
@Test public void testArchiveCommitSavepointNoHole() throws IOException { HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath) .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .forTable("test-trip-table").withCompactionConfig( HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build(); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(dfs.getConf(), basePath); HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient); HoodieTestDataGenerator.createCommitFile(basePath, "100", dfs.getConf()); HoodieTestDataGenerator.createCommitFile(basePath, "101", dfs.getConf()); HoodieTestDataGenerator.createSavepointFile(basePath, "101", dfs.getConf()); HoodieTestDataGenerator.createCommitFile(basePath, "102", dfs.getConf()); HoodieTestDataGenerator.createCommitFile(basePath, "103", dfs.getConf()); HoodieTestDataGenerator.createCommitFile(basePath, "104", dfs.getConf()); HoodieTestDataGenerator.createCommitFile(basePath, "105", dfs.getConf()); HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants()); boolean result = archiveLog.archiveIfRequired(jsc); assertTrue(result); timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants(); assertEquals( "Since we have a savepoint at 101, we should never archive any commit after 101 (we only " + "archive 100)", 5, timeline.countInstants()); assertTrue("Archived commits should always be safe", timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "101"))); assertTrue("Archived commits should always be safe", timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "102"))); assertTrue("Archived commits should always be safe", timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "103"))); }
boolean result = archiveLog.archiveIfRequired(jsc); assertTrue(result); timeline = metaClient.getActiveTimeline().reload().getCommitsAndCompactionTimeline(); assertFalse("Instants before oldest pending compaction can be removed", timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "100")));