/** * Get the archived commits as a timeline. This is costly operation, as all data from the archived * files are read. This should not be used, unless for historical debugging purposes * * @return Active commit timeline */ public synchronized HoodieArchivedTimeline getArchivedTimeline() { if (archivedTimeline == null) { archivedTimeline = new HoodieArchivedTimeline(this); } return archivedTimeline; }
@Test public void checkArchiveCommitTimeline() throws IOException { Path archiveLogPath = HoodieArchivedTimeline.getArchiveLogPath(metaClient.getArchivePath()); SequenceFile.Writer writer = SequenceFile .createWriter(metaClient.getHadoopConf(), SequenceFile.Writer.file(archiveLogPath), SequenceFile.Writer.keyClass(Text.class), SequenceFile.Writer.valueClass(Text.class)); writer.append(new Text("1"), new Text("data1")); writer.append(new Text("2"), new Text("data2")); writer.append(new Text("3"), new Text("data3")); IOUtils.closeStream(writer); HoodieArchivedTimeline archivedTimeline = metaClient.getArchivedTimeline(); HoodieInstant instant1 = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "1"); HoodieInstant instant2 = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "2"); HoodieInstant instant3 = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "3"); assertEquals(Lists.newArrayList(instant1, instant2, instant3), archivedTimeline.getInstants().collect(Collectors.toList())); assertArrayEquals(new Text("data1").getBytes(), archivedTimeline.getInstantDetails(instant1).get()); assertArrayEquals(new Text("data2").getBytes(), archivedTimeline.getInstantDetails(instant2).get()); assertArrayEquals(new Text("data3").getBytes(), archivedTimeline.getInstantDetails(instant3).get()); }
public HoodieCommitArchiveLog(HoodieWriteConfig config, HoodieTableMetaClient metaClient) { this.config = config; this.metaClient = metaClient; this.archiveFilePath = HoodieArchivedTimeline.getArchiveLogPath(metaClient.getArchivePath()); }
public HoodieCommitArchiveLog(HoodieWriteConfig config, HoodieTableMetaClient metaClient) { this.config = config; this.metaClient = metaClient; this.archiveFilePath = HoodieArchivedTimeline.getArchiveLogPath(metaClient.getArchivePath()); }
public HoodieArchivedTimeline(HoodieTableMetaClient metaClient) { // Read back the commits to make sure Path archiveLogPath = HoodieArchivedTimeline.getArchiveLogPath(metaClient.getArchivePath()); try (SequenceFile.Reader reader = new SequenceFile.Reader(metaClient.getHadoopConf(), SequenceFile.Reader.file(archiveLogPath))) { Text key = new Text(); Text val = new Text(); while (reader.next(key, val)) { // TODO - limit the number of commits loaded in memory. this could get very large. // This is okay because only tooling will load the archived commit timeline today readCommits.put(key.toString(), Arrays.copyOf(val.getBytes(), val.getLength())); } this.instants = readCommits.keySet().stream().map( s -> new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, s)).collect( Collectors.toList()); } catch (IOException e) { throw new HoodieIOException( "Could not load archived commit timeline from path " + archiveLogPath, e); } // multiple casts will make this lambda serializable - // http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16 this.details = (Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails; this.metaClient = metaClient; }
public HoodieArchivedTimeline reload() { return new HoodieArchivedTimeline(metaClient); }