Using BufferedFsInputStream to wrap FSInputStream for FSDataInputStream
This commit is contained in:
committed by
vinoth chandar
parent
720e42f52a
commit
c3c205fc02
@@ -87,7 +87,7 @@ public class HoodieCompactedLogRecordScanner implements
|
||||
|
||||
public HoodieCompactedLogRecordScanner(FileSystem fs, String basePath, List<String> logFilePaths,
|
||||
Schema readerSchema, String latestInstantTime, Long maxMemorySizeInBytes,
|
||||
boolean readBlocksLazily, boolean reverseReader) {
|
||||
boolean readBlocksLazily, boolean reverseReader, int bufferSize) {
|
||||
this.readerSchema = readerSchema;
|
||||
this.latestInstantTime = latestInstantTime;
|
||||
this.hoodieTableMetaClient = new HoodieTableMetaClient(fs.getConf(), basePath);
|
||||
@@ -102,7 +102,7 @@ public class HoodieCompactedLogRecordScanner implements
|
||||
HoodieLogFormatReader logFormatReaderWrapper =
|
||||
new HoodieLogFormatReader(fs,
|
||||
logFilePaths.stream().map(logFile -> new HoodieLogFile(new Path(logFile)))
|
||||
.collect(Collectors.toList()), readerSchema, readBlocksLazily, reverseReader);
|
||||
.collect(Collectors.toList()), readerSchema, readBlocksLazily, reverseReader, bufferSize);
|
||||
while (logFormatReaderWrapper.hasNext()) {
|
||||
HoodieLogFile logFile = logFormatReaderWrapper.getLogFile();
|
||||
log.info("Scanning log file " + logFile);
|
||||
|
||||
@@ -35,7 +35,9 @@ import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.BufferedFSInputStream;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSInputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
@@ -47,7 +49,7 @@ import org.apache.log4j.Logger;
|
||||
*/
|
||||
class HoodieLogFileReader implements HoodieLogFormat.Reader {
|
||||
|
||||
private static final int DEFAULT_BUFFER_SIZE = 4096;
|
||||
public static final int DEFAULT_BUFFER_SIZE = 16 * 1024 * 1024; // 16 MB
|
||||
private static final Logger log = LogManager.getLogger(HoodieLogFileReader.class);
|
||||
|
||||
private final FSDataInputStream inputStream;
|
||||
@@ -63,7 +65,9 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader {
|
||||
|
||||
HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, int bufferSize,
|
||||
boolean readBlockLazily, boolean reverseReader) throws IOException {
|
||||
this.inputStream = fs.open(logFile.getPath(), bufferSize);
|
||||
this.inputStream = new FSDataInputStream(
|
||||
new BufferedFSInputStream((FSInputStream) fs.open(logFile.getPath(), bufferSize).getWrappedStream(),
|
||||
bufferSize));
|
||||
this.logFile = logFile;
|
||||
this.readerSchema = readerSchema;
|
||||
this.readBlockLazily = readBlockLazily;
|
||||
@@ -116,7 +120,6 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader {
|
||||
Map<HeaderMetadataType, String> header = null;
|
||||
|
||||
try {
|
||||
|
||||
if (isOldMagic()) {
|
||||
// 1 Read the block type for a log block
|
||||
type = inputStream.readInt();
|
||||
@@ -131,8 +134,7 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader {
|
||||
// 1 Read the total size of the block
|
||||
blocksize = (int) inputStream.readLong();
|
||||
}
|
||||
|
||||
} catch (Exception e) {
|
||||
} catch (EOFException | CorruptedLogFileException e) {
|
||||
// An exception reading any of the above indicates a corrupt block
|
||||
// Create a corrupt block by finding the next OLD_MAGIC marker or EOF
|
||||
return createCorruptBlock();
|
||||
@@ -237,6 +239,10 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader {
|
||||
inputStream.seek(currentPos + blocksize);
|
||||
} catch (EOFException e) {
|
||||
// this is corrupt
|
||||
// This seek is required because contract of seek() is different for naked DFSInputStream vs BufferedFSInputStream
|
||||
// release-3.1.0-RC1/DFSInputStream.java#L1455
|
||||
// release-3.1.0-RC1/BufferedFSInputStream.java#L73
|
||||
inputStream.seek(currentPos);
|
||||
return true;
|
||||
}
|
||||
|
||||
@@ -256,11 +262,15 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader {
|
||||
while (true) {
|
||||
long currentPos = inputStream.getPos();
|
||||
try {
|
||||
boolean isEOF = readMagic();
|
||||
return isEOF ? inputStream.getPos() : currentPos;
|
||||
} catch (CorruptedLogFileException e) {
|
||||
// No luck - advance and try again
|
||||
inputStream.seek(currentPos + 1);
|
||||
boolean hasNextMagic = hasNextMagic();
|
||||
if (hasNextMagic) {
|
||||
return currentPos;
|
||||
} else {
|
||||
// No luck - advance and try again
|
||||
inputStream.seek(currentPos + 1);
|
||||
}
|
||||
} catch (EOFException e) {
|
||||
return inputStream.getPos();
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -276,12 +286,7 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader {
|
||||
*/
|
||||
public boolean hasNext() {
|
||||
try {
|
||||
boolean isEOF = readMagic();
|
||||
if (isEOF) {
|
||||
return false;
|
||||
}
|
||||
// If not hasNext(), we either we reach EOF or throw an exception on invalid magic header
|
||||
return true;
|
||||
return readMagic();
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("IOException when reading logfile " + logFile, e);
|
||||
}
|
||||
@@ -307,27 +312,35 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader {
|
||||
|
||||
private boolean readMagic() throws IOException {
|
||||
try {
|
||||
long pos = inputStream.getPos();
|
||||
// 1. Read magic header from the start of the block
|
||||
inputStream.readFully(magicBuffer, 0, 6);
|
||||
if (!Arrays.equals(magicBuffer, HoodieLogFormat.MAGIC)) {
|
||||
inputStream.seek(pos);
|
||||
// 1. Read old magic header from the start of the block
|
||||
// (for backwards compatibility of older log files written without log version)
|
||||
inputStream.readFully(oldMagicBuffer, 0, 4);
|
||||
if (!Arrays.equals(oldMagicBuffer, HoodieLogFormat.OLD_MAGIC)) {
|
||||
throw new CorruptedLogFileException(
|
||||
logFile
|
||||
+ "could not be read. Did not find the magic bytes at the start of the block");
|
||||
}
|
||||
boolean hasMagic = hasNextMagic();
|
||||
if (!hasMagic) {
|
||||
throw new CorruptedLogFileException(
|
||||
logFile
|
||||
+ "could not be read. Did not find the magic bytes at the start of the block");
|
||||
}
|
||||
return false;
|
||||
return hasMagic;
|
||||
} catch (EOFException e) {
|
||||
// We have reached the EOF
|
||||
return true;
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
private boolean hasNextMagic() throws IOException {
|
||||
long pos = inputStream.getPos();
|
||||
// 1. Read magic header from the start of the block
|
||||
inputStream.readFully(magicBuffer, 0, 6);
|
||||
if (!Arrays.equals(magicBuffer, HoodieLogFormat.MAGIC)) {
|
||||
inputStream.seek(pos);
|
||||
// 1. Read old magic header from the start of the block
|
||||
// (for backwards compatibility of older log files written without log version)
|
||||
inputStream.readFully(oldMagicBuffer, 0, 4);
|
||||
if (!Arrays.equals(oldMagicBuffer, HoodieLogFormat.OLD_MAGIC)) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieLogBlock next() {
|
||||
try {
|
||||
|
||||
@@ -212,7 +212,7 @@ public interface HoodieLogFormat {
|
||||
|
||||
static HoodieLogFormat.Reader newReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema)
|
||||
throws IOException {
|
||||
return new HoodieLogFileReader(fs, logFile, readerSchema, false, false);
|
||||
return new HoodieLogFileReader(fs, logFile, readerSchema, HoodieLogFileReader.DEFAULT_BUFFER_SIZE, false, false);
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -34,28 +34,24 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader {
|
||||
private final Schema readerSchema;
|
||||
private final boolean readBlocksLazily;
|
||||
private final boolean reverseLogReader;
|
||||
private int bufferSize;
|
||||
|
||||
private static final Logger log = LogManager.getLogger(HoodieLogFormatReader.class);
|
||||
|
||||
HoodieLogFormatReader(FileSystem fs, List<HoodieLogFile> logFiles,
|
||||
Schema readerSchema, boolean readBlocksLazily, boolean reverseLogReader) throws IOException {
|
||||
Schema readerSchema, boolean readBlocksLazily, boolean reverseLogReader, int bufferSize) throws IOException {
|
||||
this.logFiles = logFiles;
|
||||
this.fs = fs;
|
||||
this.readerSchema = readerSchema;
|
||||
this.readBlocksLazily = readBlocksLazily;
|
||||
this.reverseLogReader = reverseLogReader;
|
||||
this.bufferSize = bufferSize;
|
||||
if (logFiles.size() > 0) {
|
||||
HoodieLogFile nextLogFile = logFiles.remove(0);
|
||||
this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, readBlocksLazily,
|
||||
false);
|
||||
this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, bufferSize, readBlocksLazily, false);
|
||||
}
|
||||
}
|
||||
|
||||
HoodieLogFormatReader(FileSystem fs, List<HoodieLogFile> logFiles,
|
||||
Schema readerSchema) throws IOException {
|
||||
this(fs, logFiles, readerSchema, false, false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (currentReader != null) {
|
||||
@@ -73,8 +69,7 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader {
|
||||
} else if (logFiles.size() > 0) {
|
||||
try {
|
||||
HoodieLogFile nextLogFile = logFiles.remove(0);
|
||||
this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema,
|
||||
readBlocksLazily,
|
||||
this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, bufferSize, readBlocksLazily,
|
||||
false);
|
||||
} catch (IOException io) {
|
||||
throw new HoodieIOException("unable to initialize read with log file ", io);
|
||||
|
||||
Reference in New Issue
Block a user