1
0

[HUDI-3516] Implement record iterator for HoodieDataBlock (#4909)

*  Use iterator to void eager materialization to be memory friendly
This commit is contained in:
Bo Cui
2022-03-02 10:19:36 +08:00
committed by GitHub
parent a81a6326d5
commit 3fdc9332e5
14 changed files with 452 additions and 217 deletions

View File

@@ -30,6 +30,7 @@ import org.apache.hudi.common.table.log.HoodieLogFormat;
import org.apache.hudi.common.table.log.HoodieLogFormat.Reader;
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.ClosableIterator;
import org.apache.hudi.common.util.Option;
import org.apache.avro.generic.GenericRecord;
@@ -80,8 +81,7 @@ public class ArchivedCommitsCommand implements CommandMarker {
// read the avro blocks
while (reader.hasNext()) {
HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next();
List<IndexedRecord> records = blk.getRecords();
readRecords.addAll(records);
blk.getRecordItr().forEachRemaining(readRecords::add);
}
List<Comparable[]> readCommits = readRecords.stream().map(r -> (GenericRecord) r)
.filter(r -> r.get("actionType").toString().equals(HoodieTimeline.COMMIT_ACTION)
@@ -155,8 +155,9 @@ public class ArchivedCommitsCommand implements CommandMarker {
// read the avro blocks
while (reader.hasNext()) {
HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next();
List<IndexedRecord> records = blk.getRecords();
readRecords.addAll(records);
try (ClosableIterator<IndexedRecord> recordItr = blk.getRecordItr()) {
recordItr.forEachRemaining(readRecords::add);
}
}
List<Comparable[]> readCommits = readRecords.stream().map(r -> (GenericRecord) r)
.map(r -> readCommit(r, skipMetadata)).collect(Collectors.toList());

View File

@@ -34,6 +34,7 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
import org.apache.hudi.common.util.ClosableIterator;
import org.apache.hudi.exception.HoodieException;
import org.apache.avro.generic.GenericRecord;
@@ -123,44 +124,46 @@ public class ExportCommand implements CommandMarker {
// read the avro blocks
while (reader.hasNext() && copyCount < limit) {
HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next();
for (IndexedRecord ir : blk.getRecords()) {
// Archived instants are saved as arvo encoded HoodieArchivedMetaEntry records. We need to get the
// metadata record from the entry and convert it to json.
HoodieArchivedMetaEntry archiveEntryRecord = (HoodieArchivedMetaEntry) SpecificData.get()
.deepCopy(HoodieArchivedMetaEntry.SCHEMA$, ir);
try (ClosableIterator<IndexedRecord> recordItr = blk.getRecordItr()) {
while (recordItr.hasNext()) {
IndexedRecord ir = recordItr.next();
// Archived instants are saved as arvo encoded HoodieArchivedMetaEntry records. We need to get the
// metadata record from the entry and convert it to json.
HoodieArchivedMetaEntry archiveEntryRecord = (HoodieArchivedMetaEntry) SpecificData.get()
.deepCopy(HoodieArchivedMetaEntry.SCHEMA$, ir);
final String action = archiveEntryRecord.get("actionType").toString();
if (!actionSet.contains(action)) {
continue;
}
final String action = archiveEntryRecord.get("actionType").toString();
if (!actionSet.contains(action)) {
continue;
}
GenericRecord metadata = null;
switch (action) {
case HoodieTimeline.CLEAN_ACTION:
metadata = archiveEntryRecord.getHoodieCleanMetadata();
GenericRecord metadata = null;
switch (action) {
case HoodieTimeline.CLEAN_ACTION:
metadata = archiveEntryRecord.getHoodieCleanMetadata();
break;
case HoodieTimeline.COMMIT_ACTION:
case HoodieTimeline.DELTA_COMMIT_ACTION:
metadata = archiveEntryRecord.getHoodieCommitMetadata();
break;
case HoodieTimeline.ROLLBACK_ACTION:
metadata = archiveEntryRecord.getHoodieRollbackMetadata();
break;
case HoodieTimeline.SAVEPOINT_ACTION:
metadata = archiveEntryRecord.getHoodieSavePointMetadata();
break;
case HoodieTimeline.COMPACTION_ACTION:
metadata = archiveEntryRecord.getHoodieCompactionMetadata();
break;
default:
throw new HoodieException("Unknown type of action " + action);
}
final String instantTime = archiveEntryRecord.get("commitTime").toString();
final String outPath = localFolder + Path.SEPARATOR + instantTime + "." + action;
writeToFile(outPath, HoodieAvroUtils.avroToJson(metadata, true));
if (++copyCount == limit) {
break;
case HoodieTimeline.COMMIT_ACTION:
case HoodieTimeline.DELTA_COMMIT_ACTION:
metadata = archiveEntryRecord.getHoodieCommitMetadata();
break;
case HoodieTimeline.ROLLBACK_ACTION:
metadata = archiveEntryRecord.getHoodieRollbackMetadata();
break;
case HoodieTimeline.SAVEPOINT_ACTION:
metadata = archiveEntryRecord.getHoodieSavePointMetadata();
break;
case HoodieTimeline.COMPACTION_ACTION:
metadata = archiveEntryRecord.getHoodieCompactionMetadata();
break;
default:
throw new HoodieException("Unknown type of action " + action);
}
final String instantTime = archiveEntryRecord.get("commitTime").toString();
final String outPath = localFolder + Path.SEPARATOR + instantTime + "." + action;
writeToFile(outPath, HoodieAvroUtils.avroToJson(metadata, true));
if (++copyCount == limit) {
break;
}
}
}
}

View File

@@ -37,6 +37,7 @@ import org.apache.hudi.common.table.log.block.HoodieDataBlock;
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType;
import org.apache.hudi.common.util.ClosableIterator;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieMemoryConfig;
@@ -60,6 +61,7 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Collectors;
import scala.Tuple2;
@@ -100,7 +102,7 @@ public class HoodieLogFileCommand implements CommandMarker {
while (reader.hasNext()) {
HoodieLogBlock n = reader.next();
String instantTime;
int recordCount = 0;
AtomicInteger recordCount = new AtomicInteger(0);
if (n instanceof HoodieCorruptBlock) {
try {
instantTime = n.getLogBlockHeader().get(HeaderMetadataType.INSTANT_TIME);
@@ -120,17 +122,19 @@ public class HoodieLogFileCommand implements CommandMarker {
instantTime = "dummy_instant_time_" + dummyInstantTimeCount;
}
if (n instanceof HoodieDataBlock) {
recordCount = ((HoodieDataBlock) n).getRecords().size();
try (ClosableIterator<IndexedRecord> recordItr = ((HoodieDataBlock) n).getRecordItr()) {
recordItr.forEachRemaining(r -> recordCount.incrementAndGet());
}
}
}
if (commitCountAndMetadata.containsKey(instantTime)) {
commitCountAndMetadata.get(instantTime).add(
new Tuple3<>(n.getBlockType(), new Tuple2<>(n.getLogBlockHeader(), n.getLogBlockFooter()), recordCount));
new Tuple3<>(n.getBlockType(), new Tuple2<>(n.getLogBlockHeader(), n.getLogBlockFooter()), recordCount.get()));
} else {
List<Tuple3<HoodieLogBlockType, Tuple2<Map<HeaderMetadataType, String>, Map<HeaderMetadataType, String>>, Integer>> list =
new ArrayList<>();
list.add(
new Tuple3<>(n.getBlockType(), new Tuple2<>(n.getLogBlockHeader(), n.getLogBlockFooter()), recordCount));
new Tuple3<>(n.getBlockType(), new Tuple2<>(n.getLogBlockHeader(), n.getLogBlockFooter()), recordCount.get()));
commitCountAndMetadata.put(instantTime, list);
}
}
@@ -232,11 +236,12 @@ public class HoodieLogFileCommand implements CommandMarker {
HoodieLogBlock n = reader.next();
if (n instanceof HoodieDataBlock) {
HoodieDataBlock blk = (HoodieDataBlock) n;
List<IndexedRecord> records = blk.getRecords();
for (IndexedRecord record : records) {
if (allRecords.size() < limit) {
allRecords.add(record);
}
try (ClosableIterator<IndexedRecord> recordItr = blk.getRecordItr()) {
recordItr.forEachRemaining(record -> {
if (allRecords.size() < limit) {
allRecords.add(record);
}
});
}
}
}