/** * Check if commits need to be archived. If yes, archive commits. */ public boolean archiveIfRequired(final JavaSparkContext jsc) throws IOException { try { List<HoodieInstant> instantsToArchive = getInstantsToArchive(jsc).collect(Collectors.toList()); boolean success = true; if (instantsToArchive.iterator().hasNext()) { this.writer = openWriter(); log.info("Archiving instants " + instantsToArchive); archive(instantsToArchive); success = deleteArchivedInstants(instantsToArchive); } else { log.info("No Instants to archive"); } return success; } finally { close(); } }
public void archive(List<HoodieInstant> instants) throws HoodieCommitException { try { HoodieTimeline commitTimeline = metaClient.getActiveTimeline().getAllCommitsTimeline() .filterCompletedInstants(); Schema wrapperSchema = HoodieArchivedMetaEntry.getClassSchema(); log.info("Wrapper schema " + wrapperSchema.toString()); List<IndexedRecord> records = new ArrayList<>(); for (HoodieInstant hoodieInstant : instants) { records.add(convertToAvroRecord(commitTimeline, hoodieInstant)); } Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, wrapperSchema.toString()); HoodieAvroDataBlock block = new HoodieAvroDataBlock(records, header); this.writer = writer.appendBlock(block); } catch (Exception e) { throw new HoodieCommitException("Failed to archive commits", e); } }
private boolean deleteArchivedInstants(List<HoodieInstant> archivedInstants) throws IOException { log.info("Deleting instants " + archivedInstants); boolean success = true; for (HoodieInstant archivedInstant : archivedInstants) { Path commitFile = new Path(metaClient.getMetaPath(), archivedInstant.getFileName()); try { if (metaClient.getFs().exists(commitFile)) { success &= metaClient.getFs().delete(commitFile, false); log.info("Archived and deleted instant file " + commitFile); } } catch (IOException e) { throw new HoodieIOException("Failed to delete archived instant " + archivedInstant, e); } } // Remove older meta-data from auxiliary path too Optional<HoodieInstant> latestCommitted = archivedInstants.stream() .filter(i -> { return i.isCompleted() && (i.getAction().equals(COMMIT_ACTION) || (i.getAction().equals(DELTA_COMMIT_ACTION))); }) .sorted(Comparator.comparing(HoodieInstant::getTimestamp).reversed()).findFirst(); if (latestCommitted.isPresent()) { success &= deleteAllInstantsOlderorEqualsInAuxMetaFolder(latestCommitted.get()); } return success; }
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(config, new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true)); archiveLog.archiveIfRequired(jsc); if (config.isAutoClean()) {
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata .fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieCommitMetadata.class); archivedMetaWrapper.setHoodieCommitMetadata(commitMetadataConverter(commitMetadata)); archivedMetaWrapper.setActionType(ActionType.commit.name()); break; HoodieCommitMetadata commitMetadata = HoodieCommitMetadata .fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieCommitMetadata.class); archivedMetaWrapper.setHoodieCommitMetadata(commitMetadataConverter(commitMetadata)); archivedMetaWrapper.setActionType(ActionType.commit.name()); break;
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(config, new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true)); archiveLog.archiveIfRequired(jsc); if (config.isAutoClean()) {
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata .fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieCommitMetadata.class); archivedMetaWrapper.setHoodieCommitMetadata(commitMetadataConverter(commitMetadata)); archivedMetaWrapper.setActionType(ActionType.commit.name()); break; HoodieCommitMetadata commitMetadata = HoodieCommitMetadata .fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieCommitMetadata.class); archivedMetaWrapper.setHoodieCommitMetadata(commitMetadataConverter(commitMetadata)); archivedMetaWrapper.setActionType(ActionType.commit.name()); break;
@Test public void testArchiveEmptyDataset() throws IOException { HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath) .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .forTable("test-trip-table").build(); HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, new HoodieTableMetaClient(dfs.getConf(), cfg.getBasePath(), true)); boolean result = archiveLog.archiveIfRequired(jsc); assertTrue(result); }
/** * Check if commits need to be archived. If yes, archive commits. */ public boolean archiveIfRequired(final JavaSparkContext jsc) throws IOException { try { List<HoodieInstant> instantsToArchive = getInstantsToArchive(jsc).collect(Collectors.toList()); boolean success = true; if (instantsToArchive.iterator().hasNext()) { this.writer = openWriter(); log.info("Archiving instants " + instantsToArchive); archive(instantsToArchive); success = deleteArchivedInstants(instantsToArchive); } else { log.info("No Instants to archive"); } return success; } finally { close(); } }
public void archive(List<HoodieInstant> instants) throws HoodieCommitException { try { HoodieTimeline commitTimeline = metaClient.getActiveTimeline().getAllCommitsTimeline() .filterCompletedInstants(); Schema wrapperSchema = HoodieArchivedMetaEntry.getClassSchema(); log.info("Wrapper schema " + wrapperSchema.toString()); List<IndexedRecord> records = new ArrayList<>(); for (HoodieInstant hoodieInstant : instants) { records.add(convertToAvroRecord(commitTimeline, hoodieInstant)); } Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, wrapperSchema.toString()); HoodieAvroDataBlock block = new HoodieAvroDataBlock(records, header); this.writer = writer.appendBlock(block); } catch (Exception e) { throw new HoodieCommitException("Failed to archive commits", e); } }
private boolean deleteArchivedInstants(List<HoodieInstant> archivedInstants) throws IOException { log.info("Deleting instants " + archivedInstants); boolean success = true; for (HoodieInstant archivedInstant : archivedInstants) { Path commitFile = new Path(metaClient.getMetaPath(), archivedInstant.getFileName()); try { if (metaClient.getFs().exists(commitFile)) { success &= metaClient.getFs().delete(commitFile, false); log.info("Archived and deleted instant file " + commitFile); } } catch (IOException e) { throw new HoodieIOException("Failed to delete archived instant " + archivedInstant, e); } } // Remove older meta-data from auxiliary path too Optional<HoodieInstant> latestCommitted = archivedInstants.stream() .filter(i -> { return i.isCompleted() && (i.getAction().equals(COMMIT_ACTION) || (i.getAction().equals(DELTA_COMMIT_ACTION))); }).max(Comparator.comparing(HoodieInstant::getTimestamp)); if (latestCommitted.isPresent()) { success &= deleteAllInstantsOlderorEqualsInAuxMetaFolder(latestCommitted.get()); } return success; }
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, new HoodieTableMetaClient(dfs.getConf(), basePath, true)); assertTrue(archiveLog.archiveIfRequired(jsc));
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build(); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(dfs.getConf(), basePath); HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient); boolean result = archiveLog.archiveIfRequired(jsc); assertTrue(result); timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants();
@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"))); }
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.createCompactionRequestedFile(basePath, "101", dfs.getConf()); boolean result = archiveLog.archiveIfRequired(jsc); assertTrue(result); timeline = metaClient.getActiveTimeline().reload().getCommitsAndCompactionTimeline();