1
0

[HUDI-2269] Release the disk map resource for flink streaming reader (#3384)

This commit is contained in:
Danny Chan
2021-08-03 13:55:35 +08:00
committed by GitHub
parent aa857beee0
commit bec23bda50
5 changed files with 67 additions and 43 deletions

View File

@@ -357,7 +357,6 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload, I, K, O> extends H
} }
} }
keyToNewRecords.clear();
((ExternalSpillableMap) keyToNewRecords).close(); ((ExternalSpillableMap) keyToNewRecords).close();
writtenRecordKeys.clear(); writtenRecordKeys.clear();

View File

@@ -256,7 +256,9 @@ public class ExternalSpillableMap<T extends Serializable, R extends Serializable
} }
public void close() { public void close() {
inMemoryMap.clear();
getDiskBasedMap().close(); getDiskBasedMap().close();
currentInMemoryMapSize = 0L;
} }
@Override @Override

View File

@@ -37,6 +37,7 @@ import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.sink.bootstrap.aggregate.BootstrapAggFunction; import org.apache.hudi.sink.bootstrap.aggregate.BootstrapAggFunction;
import org.apache.hudi.table.HoodieFlinkTable; import org.apache.hudi.table.HoodieFlinkTable;
import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.format.FormatUtils;
import org.apache.hudi.util.StreamerUtil; import org.apache.hudi.util.StreamerUtil;
import org.apache.avro.Schema; import org.apache.avro.Schema;
@@ -201,7 +202,8 @@ public class BootstrapFunction<I, O extends HoodieRecord>
.filter(logFile -> logFile.getFileSize() > 0) .filter(logFile -> logFile.getFileSize() > 0)
.map(logFile -> logFile.getPath().toString()) .map(logFile -> logFile.getPath().toString())
.collect(toList()); .collect(toList());
HoodieMergedLogRecordScanner scanner = scanLog(logPaths, schema, latestCommitTime.get().getTimestamp()); HoodieMergedLogRecordScanner scanner = FormatUtils.scanLog(logPaths, schema, latestCommitTime.get().getTimestamp(),
writeConfig, hadoopConf);
try { try {
for (String recordKey : scanner.getRecords().keySet()) { for (String recordKey : scanner.getRecords().keySet()) {
@@ -209,6 +211,8 @@ public class BootstrapFunction<I, O extends HoodieRecord>
} }
} catch (Exception e) { } catch (Exception e) {
throw new HoodieException(String.format("Error when loading record keys from files: %s", logPaths), e); throw new HoodieException(String.format("Error when loading record keys from files: %s", logPaths), e);
} finally {
scanner.close();
} }
} }
} }
@@ -218,27 +222,6 @@ public class BootstrapFunction<I, O extends HoodieRecord>
this.getClass().getSimpleName(), taskID, partitionPath, cost); this.getClass().getSimpleName(), taskID, partitionPath, cost);
} }
private HoodieMergedLogRecordScanner scanLog(
List<String> logPaths,
Schema logSchema,
String latestInstantTime) {
String basePath = this.hoodieTable.getMetaClient().getBasePath();
return HoodieMergedLogRecordScanner.newBuilder()
.withFileSystem(FSUtils.getFs(basePath, this.hadoopConf))
.withBasePath(basePath)
.withLogFilePaths(logPaths)
.withReaderSchema(logSchema)
.withLatestInstantTime(latestInstantTime)
.withReadBlocksLazily(this.writeConfig.getCompactionLazyBlockReadEnabled())
.withReverseReader(false)
.withBufferSize(this.writeConfig.getMaxDFSStreamBufferSize())
.withMaxMemorySizeInBytes(this.writeConfig.getMaxMemoryPerPartitionMerge())
.withSpillableMapBasePath(this.writeConfig.getSpillableMapBasePath())
.withDiskMapType(this.writeConfig.getCommonConfig().getSpillableDiskMapType())
.withBitCaskDiskMapCompressionEnabled(this.writeConfig.getCommonConfig().isBitCaskDiskMapCompressionEnabled())
.build();
}
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
public static HoodieRecord generateHoodieRecord(HoodieKey hoodieKey, FileSlice fileSlice) { public static HoodieRecord generateHoodieRecord(HoodieKey hoodieKey, FileSlice fileSlice) {
HoodieRecord hoodieRecord = new HoodieRecord(hoodieKey, null); HoodieRecord hoodieRecord = new HoodieRecord(hoodieKey, null);

View File

@@ -20,6 +20,7 @@ package org.apache.hudi.table.format;
import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.hadoop.config.HoodieRealtimeConfig; import org.apache.hudi.hadoop.config.HoodieRealtimeConfig;
import org.apache.hudi.table.format.mor.MergeOnReadInputSplit; import org.apache.hudi.table.format.mor.MergeOnReadInputSplit;
@@ -83,6 +84,29 @@ public class FormatUtils {
.build(); .build();
} }
public static HoodieMergedLogRecordScanner scanLog(
List<String> logPaths,
Schema logSchema,
String latestInstantTime,
HoodieWriteConfig writeConfig,
Configuration hadoopConf) {
String basePath = writeConfig.getBasePath();
return HoodieMergedLogRecordScanner.newBuilder()
.withFileSystem(FSUtils.getFs(basePath, hadoopConf))
.withBasePath(basePath)
.withLogFilePaths(logPaths)
.withReaderSchema(logSchema)
.withLatestInstantTime(latestInstantTime)
.withReadBlocksLazily(writeConfig.getCompactionLazyBlockReadEnabled())
.withReverseReader(false)
.withBufferSize(writeConfig.getMaxDFSStreamBufferSize())
.withMaxMemorySizeInBytes(writeConfig.getMaxMemoryPerPartitionMerge())
.withSpillableMapBasePath(writeConfig.getSpillableMapBasePath())
.withDiskMapType(writeConfig.getCommonConfig().getSpillableDiskMapType())
.withBitCaskDiskMapCompressionEnabled(writeConfig.getCommonConfig().isBitCaskDiskMapCompressionEnabled())
.build();
}
private static Boolean string2Boolean(String s) { private static Boolean string2Boolean(String s) {
return "true".equals(s.toLowerCase(Locale.ROOT)); return "true".equals(s.toLowerCase(Locale.ROOT));
} }

View File

@@ -19,7 +19,7 @@
package org.apache.hudi.table.format.mor; package org.apache.hudi.table.format.mor;
import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
import org.apache.hudi.common.table.log.InstantRange; import org.apache.hudi.common.table.log.InstantRange;
import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.Option;
import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.configuration.FlinkOptions;
@@ -59,7 +59,6 @@ import java.util.HashSet;
import java.util.Iterator; import java.util.Iterator;
import java.util.LinkedHashMap; import java.util.LinkedHashMap;
import java.util.List; import java.util.List;
import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.stream.IntStream; import java.util.stream.IntStream;
@@ -293,15 +292,14 @@ public class MergeOnReadInputFormat
Long.MAX_VALUE); // read the whole file Long.MAX_VALUE); // read the whole file
} }
private Iterator<RowData> getLogFileIterator(MergeOnReadInputSplit split) { private ClosableIterator<RowData> getLogFileIterator(MergeOnReadInputSplit split) {
final Schema tableSchema = new Schema.Parser().parse(tableState.getAvroSchema()); final Schema tableSchema = new Schema.Parser().parse(tableState.getAvroSchema());
final Schema requiredSchema = new Schema.Parser().parse(tableState.getRequiredAvroSchema()); final Schema requiredSchema = new Schema.Parser().parse(tableState.getRequiredAvroSchema());
final GenericRecordBuilder recordBuilder = new GenericRecordBuilder(requiredSchema); final GenericRecordBuilder recordBuilder = new GenericRecordBuilder(requiredSchema);
final AvroToRowDataConverters.AvroToRowDataConverter avroToRowDataConverter = final AvroToRowDataConverters.AvroToRowDataConverter avroToRowDataConverter =
AvroToRowDataConverters.createRowConverter(tableState.getRequiredRowType()); AvroToRowDataConverters.createRowConverter(tableState.getRequiredRowType());
final Map<String, HoodieRecord<? extends HoodieRecordPayload>> logRecords = final HoodieMergedLogRecordScanner scanner = FormatUtils.scanLog(split, tableSchema, hadoopConf);
FormatUtils.scanLog(split, tableSchema, hadoopConf).getRecords(); final Iterator<String> logRecordsKeyIterator = scanner.getRecords().keySet().iterator();
final Iterator<String> logRecordsKeyIterator = logRecords.keySet().iterator();
final int[] pkOffset = tableState.getPkOffsetsInRequired(); final int[] pkOffset = tableState.getPkOffsetsInRequired();
// flag saying whether the pk semantics has been dropped by user specified // flag saying whether the pk semantics has been dropped by user specified
// projections. For e.g, if the pk fields are [a, b] but user only select a, // projections. For e.g, if the pk fields are [a, b] but user only select a,
@@ -310,7 +308,7 @@ public class MergeOnReadInputFormat
final LogicalType[] pkTypes = pkSemanticLost ? null : tableState.getPkTypes(pkOffset); final LogicalType[] pkTypes = pkSemanticLost ? null : tableState.getPkTypes(pkOffset);
final StringToRowDataConverter converter = pkSemanticLost ? null : new StringToRowDataConverter(pkTypes); final StringToRowDataConverter converter = pkSemanticLost ? null : new StringToRowDataConverter(pkTypes);
return new Iterator<RowData>() { return new ClosableIterator<RowData>() {
private RowData currentRecord; private RowData currentRecord;
@Override @Override
@@ -318,7 +316,7 @@ public class MergeOnReadInputFormat
while (logRecordsKeyIterator.hasNext()) { while (logRecordsKeyIterator.hasNext()) {
String curAvroKey = logRecordsKeyIterator.next(); String curAvroKey = logRecordsKeyIterator.next();
Option<IndexedRecord> curAvroRecord = null; Option<IndexedRecord> curAvroRecord = null;
final HoodieRecord<?> hoodieRecord = logRecords.get(curAvroKey); final HoodieRecord<?> hoodieRecord = scanner.getRecords().get(curAvroKey);
try { try {
curAvroRecord = hoodieRecord.getData().getInsertValue(tableSchema); curAvroRecord = hoodieRecord.getData().getInsertValue(tableSchema);
} catch (IOException e) { } catch (IOException e) {
@@ -359,6 +357,11 @@ public class MergeOnReadInputFormat
public RowData next() { public RowData next() {
return currentRecord; return currentRecord;
} }
@Override
public void close() {
scanner.close();
}
}; };
} }
@@ -366,6 +369,11 @@ public class MergeOnReadInputFormat
// Inner Class // Inner Class
// ------------------------------------------------------------------------- // -------------------------------------------------------------------------
private interface ClosableIterator<E> extends Iterator<E>, AutoCloseable {
@Override
void close(); // override to not throw exception
}
private interface RecordIterator { private interface RecordIterator {
boolean reachedEnd() throws IOException; boolean reachedEnd() throws IOException;
@@ -453,9 +461,9 @@ public class MergeOnReadInputFormat
static class LogFileOnlyIterator implements RecordIterator { static class LogFileOnlyIterator implements RecordIterator {
// iterator for log files // iterator for log files
private final Iterator<RowData> iterator; private final ClosableIterator<RowData> iterator;
LogFileOnlyIterator(Iterator<RowData> iterator) { LogFileOnlyIterator(ClosableIterator<RowData> iterator) {
this.iterator = iterator; this.iterator = iterator;
} }
@@ -471,7 +479,9 @@ public class MergeOnReadInputFormat
@Override @Override
public void close() { public void close() {
// no operation if (this.iterator != null) {
this.iterator.close();
}
} }
} }
@@ -479,7 +489,7 @@ public class MergeOnReadInputFormat
// base file reader // base file reader
private final ParquetColumnarRowSplitReader reader; private final ParquetColumnarRowSplitReader reader;
// iterator for log files // iterator for log files
private final Iterator<RowData> iterator; private final ClosableIterator<RowData> iterator;
// add the flag because the flink ParquetColumnarRowSplitReader is buggy: // add the flag because the flink ParquetColumnarRowSplitReader is buggy:
// method #reachedEnd() returns false after it returns true. // method #reachedEnd() returns false after it returns true.
@@ -488,7 +498,7 @@ public class MergeOnReadInputFormat
private RowData currentRecord; private RowData currentRecord;
SkipMergeIterator(ParquetColumnarRowSplitReader reader, Iterator<RowData> iterator) { SkipMergeIterator(ParquetColumnarRowSplitReader reader, ClosableIterator<RowData> iterator) {
this.reader = reader; this.reader = reader;
this.iterator = iterator; this.iterator = iterator;
} }
@@ -517,6 +527,9 @@ public class MergeOnReadInputFormat
if (this.reader != null) { if (this.reader != null) {
this.reader.close(); this.reader.close();
} }
if (this.iterator != null) {
this.iterator.close();
}
} }
} }
@@ -525,8 +538,8 @@ public class MergeOnReadInputFormat
private final ParquetColumnarRowSplitReader reader; private final ParquetColumnarRowSplitReader reader;
// log keys used for merging // log keys used for merging
private final Iterator<String> logKeysIterator; private final Iterator<String> logKeysIterator;
// log records // scanner
private final Map<String, HoodieRecord<? extends HoodieRecordPayload>> logRecords; private final HoodieMergedLogRecordScanner scanner;
private final Schema tableSchema; private final Schema tableSchema;
private final Schema requiredSchema; private final Schema requiredSchema;
@@ -559,8 +572,8 @@ public class MergeOnReadInputFormat
ParquetColumnarRowSplitReader reader) { // the reader should be with full schema ParquetColumnarRowSplitReader reader) { // the reader should be with full schema
this.tableSchema = tableSchema; this.tableSchema = tableSchema;
this.reader = reader; this.reader = reader;
this.logRecords = FormatUtils.scanLog(split, tableSchema, hadoopConf).getRecords(); this.scanner = FormatUtils.scanLog(split, tableSchema, hadoopConf);
this.logKeysIterator = this.logRecords.keySet().iterator(); this.logKeysIterator = scanner.getRecords().keySet().iterator();
this.requiredSchema = requiredSchema; this.requiredSchema = requiredSchema;
this.requiredPos = requiredPos; this.requiredPos = requiredPos;
this.recordBuilder = new GenericRecordBuilder(requiredSchema); this.recordBuilder = new GenericRecordBuilder(requiredSchema);
@@ -582,7 +595,7 @@ public class MergeOnReadInputFormat
} }
} }
final String curKey = currentRecord.getString(HOODIE_RECORD_KEY_COL_POS).toString(); final String curKey = currentRecord.getString(HOODIE_RECORD_KEY_COL_POS).toString();
if (logRecords.containsKey(curKey)) { if (scanner.getRecords().containsKey(curKey)) {
keyToSkip.add(curKey); keyToSkip.add(curKey);
Option<IndexedRecord> mergedAvroRecord = mergeRowWithLog(currentRecord, curKey); Option<IndexedRecord> mergedAvroRecord = mergeRowWithLog(currentRecord, curKey);
if (!mergedAvroRecord.isPresent()) { if (!mergedAvroRecord.isPresent()) {
@@ -608,7 +621,7 @@ public class MergeOnReadInputFormat
final String curKey = logKeysIterator.next(); final String curKey = logKeysIterator.next();
if (!keyToSkip.contains(curKey)) { if (!keyToSkip.contains(curKey)) {
Option<IndexedRecord> insertAvroRecord = Option<IndexedRecord> insertAvroRecord =
logRecords.get(curKey).getData().getInsertValue(tableSchema); scanner.getRecords().get(curKey).getData().getInsertValue(tableSchema);
if (insertAvroRecord.isPresent()) { if (insertAvroRecord.isPresent()) {
// the record is a DELETE if insertAvroRecord not present, skipping // the record is a DELETE if insertAvroRecord not present, skipping
GenericRecord requiredAvroRecord = buildAvroRecordBySchema( GenericRecord requiredAvroRecord = buildAvroRecordBySchema(
@@ -634,13 +647,16 @@ public class MergeOnReadInputFormat
if (this.reader != null) { if (this.reader != null) {
this.reader.close(); this.reader.close();
} }
if (this.scanner != null) {
this.scanner.close();
}
} }
private Option<IndexedRecord> mergeRowWithLog( private Option<IndexedRecord> mergeRowWithLog(
RowData curRow, RowData curRow,
String curKey) throws IOException { String curKey) throws IOException {
GenericRecord historyAvroRecord = (GenericRecord) rowDataToAvroConverter.convert(tableSchema, curRow); GenericRecord historyAvroRecord = (GenericRecord) rowDataToAvroConverter.convert(tableSchema, curRow);
return logRecords.get(curKey).getData().combineAndGetUpdateValue(historyAvroRecord, tableSchema); return scanner.getRecords().get(curKey).getData().combineAndGetUpdateValue(historyAvroRecord, tableSchema);
} }
} }