1
0

Using BufferedFsInputStream to wrap FSInputStream for FSDataInputStream

This commit is contained in:
Nishith Agarwal
2018-04-02 22:53:28 -07:00
committed by vinoth chandar
parent 720e42f52a
commit c3c205fc02
15 changed files with 221 additions and 116 deletions

View File

@@ -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);

View File

@@ -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 {

View File

@@ -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);
}
/**

View File

@@ -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);

View File

@@ -122,14 +122,21 @@ public class HoodieTestUtils {
}
}
public static final void createInflightCleanFiles(String basePath, String... commitTimes) throws IOException {
public static final void createInflightCleanFiles(String basePath, Configuration configuration, String... commitTimes)
throws IOException {
for (String commitTime : commitTimes) {
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline
Path commitFile = new Path((basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline
.makeInflightCleanerFileName(
commitTime)).createNewFile();
commitTime)));
FileSystem fs = FSUtils.getFs(basePath, configuration);
FSDataOutputStream os = fs.create(commitFile, true);
}
}
public static final void createInflightCleanFiles(String basePath, String... commitTimes) throws IOException {
createInflightCleanFiles(basePath, HoodieTestUtils.getDefaultHadoopConf(), commitTimes);
}
public static final String createNewDataFile(String basePath, String partitionPath, String commitTime)
throws IOException {
String fileID = UUID.randomUUID().toString();
@@ -214,10 +221,11 @@ public class HoodieTestUtils {
return instant + TEST_EXTENSION + HoodieTimeline.INFLIGHT_EXTENSION;
}
public static void createCleanFiles(String basePath, String commitTime) throws IOException {
public static void createCleanFiles(String basePath, String commitTime, Configuration configuration)
throws IOException {
Path commitFile = new Path(
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeCleanerFileName(commitTime));
FileSystem fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf());
FileSystem fs = FSUtils.getFs(basePath, configuration);
FSDataOutputStream os = fs.create(commitFile, true);
try {
HoodieCleanStat cleanStats = new HoodieCleanStat(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS,
@@ -233,6 +241,10 @@ public class HoodieTestUtils {
}
}
public static void createCleanFiles(String basePath, String commitTime) throws IOException {
createCleanFiles(basePath, commitTime, HoodieTestUtils.getDefaultHadoopConf());
}
public static String makeTestFileName(String instant) {
return instant + TEST_EXTENSION;
}

View File

@@ -36,6 +36,7 @@ import com.uber.hoodie.common.table.log.block.HoodieCommandBlock;
import com.uber.hoodie.common.table.log.block.HoodieCorruptBlock;
import com.uber.hoodie.common.table.log.block.HoodieDeleteBlock;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
@@ -75,6 +76,7 @@ public class HoodieLogFormatTest {
private FileSystem fs;
private Path partitionPath;
private static String basePath;
private int bufferSize = 4096;
private Boolean readBlocksLazily = true;
@@ -399,7 +401,7 @@ public class HoodieLogFormatTest {
// scan all log blocks (across multiple log files)
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath,
logFiles.stream().map(logFile -> logFile.getPath().toString()).collect(Collectors.toList()), schema, "100",
10240L, readBlocksLazily, false);
10240L, readBlocksLazily, false, bufferSize);
List<IndexedRecord> scannedRecords = new ArrayList<>();
for (HoodieRecord record : scanner) {
@@ -525,7 +527,7 @@ public class HoodieLogFormatTest {
"100").map(s -> s.getPath().toString()).collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema,
"100", 10240L, readBlocksLazily, false);
"100", 10240L, readBlocksLazily, false, bufferSize);
assertEquals("", 200, scanner.getTotalLogRecords());
Set<String> readKeys = new HashSet<>(200);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
@@ -585,7 +587,7 @@ public class HoodieLogFormatTest {
"100").map(s -> s.getPath().toString()).collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema,
"102", 10240L, readBlocksLazily, false);
"102", 10240L, readBlocksLazily, false, bufferSize);
assertEquals("We read 200 records from 2 write batches", 200, scanner.getTotalLogRecords());
Set<String> readKeys = new HashSet<>(200);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
@@ -663,7 +665,7 @@ public class HoodieLogFormatTest {
"100").map(s -> s.getPath().toString()).collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema,
"103", 10240L, true, false);
"103", 10240L, true, false, bufferSize);
assertEquals("We would read 200 records", 200, scanner.getTotalLogRecords());
Set<String> readKeys = new HashSet<>(200);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
@@ -717,7 +719,7 @@ public class HoodieLogFormatTest {
"100").map(s -> s.getPath().toString()).collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema,
"102", 10240L, readBlocksLazily, false);
"102", 10240L, readBlocksLazily, false, bufferSize);
assertEquals("We still would read 200 records", 200, scanner.getTotalLogRecords());
final List<String> readKeys = new ArrayList<>(200);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
@@ -737,7 +739,7 @@ public class HoodieLogFormatTest {
readKeys.clear();
scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, "101", 10240L, readBlocksLazily,
false);
false, bufferSize);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
assertEquals("Stream collect should return all 200 records after rollback of delete", 200, readKeys.size());
}
@@ -798,7 +800,7 @@ public class HoodieLogFormatTest {
// all data must be rolled back before merge
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema,
"100", 10240L, readBlocksLazily, false);
"100", 10240L, readBlocksLazily, false, bufferSize);
assertEquals("We would have scanned 0 records because of rollback", 0, scanner.getTotalLogRecords());
final List<String> readKeys = new ArrayList<>();
@@ -847,7 +849,7 @@ public class HoodieLogFormatTest {
"100").map(s -> s.getPath().toString()).collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema,
"100", 10240L, readBlocksLazily, false);
"100", 10240L, readBlocksLazily, false, bufferSize);
assertEquals("We would read 0 records", 0, scanner.getTotalLogRecords());
}
@@ -879,7 +881,7 @@ public class HoodieLogFormatTest {
"100").map(s -> s.getPath().toString()).collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema,
"100", 10240L, readBlocksLazily, false);
"100", 10240L, readBlocksLazily, false, bufferSize);
assertEquals("We still would read 100 records", 100, scanner.getTotalLogRecords());
final List<String> readKeys = new ArrayList<>(100);
scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
@@ -929,7 +931,7 @@ public class HoodieLogFormatTest {
"100").map(s -> s.getPath().toString()).collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema,
"101", 10240L, readBlocksLazily, false);
"101", 10240L, readBlocksLazily, false, bufferSize);
assertEquals("We would read 0 records", 0, scanner.getTotalLogRecords());
}
@@ -948,7 +950,6 @@ public class HoodieLogFormatTest {
List<IndexedRecord> records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
writer = writer.appendBlock(dataBlock);
@@ -956,7 +957,7 @@ public class HoodieLogFormatTest {
writer = writer.appendBlock(dataBlock);
writer.close();
// Append some arbit byte[] to thee end of the log (mimics a partially written commit)
// Append some arbit byte[] to the end of the log (mimics a partially written commit)
fs = FSUtils.getFs(fs.getUri().toString(), fs.getConf());
FSDataOutputStream outputStream = fs.append(writer.getLogFile().getPath());
// create a block with
@@ -969,7 +970,7 @@ public class HoodieLogFormatTest {
outputStream.flush();
outputStream.close();
// Append some arbit byte[] to thee end of the log (mimics a partially written commit)
// Append some arbit byte[] to the end of the log (mimics a partially written commit)
fs = FSUtils.getFs(fs.getUri().toString(), fs.getConf());
outputStream = fs.append(writer.getLogFile().getPath());
// create a block with
@@ -989,7 +990,7 @@ public class HoodieLogFormatTest {
writer = writer.appendBlock(dataBlock);
writer.close();
// Append some arbit byte[] to thee end of the log (mimics a partially written commit)
// Append some arbit byte[] to the end of the log (mimics a partially written commit)
fs = FSUtils.getFs(fs.getUri().toString(), fs.getConf());
outputStream = fs.append(writer.getLogFile().getPath());
// create a block with
@@ -1005,18 +1006,20 @@ public class HoodieLogFormatTest {
writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100")
.withFs(fs).build();
// Write 1 rollback block for a failed write
// Write 1 rollback block for the last commit instant
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "101");
header.put(HeaderMetadataType.TARGET_INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE,
String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal()));
HoodieCommandBlock commandBlock = new HoodieCommandBlock(header);
writer = writer.appendBlock(commandBlock);
writer.close();
List<String> allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION,
"100").map(s -> s.getPath().toString()).collect(Collectors.toList());
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema,
"101", 10240L, readBlocksLazily, false);
"101", 10240L, readBlocksLazily, false, bufferSize);
assertEquals("We would read 0 records", 0, scanner.getTotalLogRecords());
}
@@ -1136,7 +1139,7 @@ public class HoodieLogFormatTest {
writer.close();
HoodieLogFileReader reader = new HoodieLogFileReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(),
readBlocksLazily, true);
bufferSize, readBlocksLazily, true);
assertTrue("Last block should be available", reader.hasPrev());
HoodieLogBlock prevBlock = reader.prev();
@@ -1207,7 +1210,8 @@ public class HoodieLogFormatTest {
writer.close();
// First round of reads - we should be able to read the first block and then EOF
HoodieLogFileReader reader = new HoodieLogFileReader(fs, writer.getLogFile(), schema, readBlocksLazily, true);
HoodieLogFileReader reader = new HoodieLogFileReader(fs, writer.getLogFile(), schema, bufferSize,
readBlocksLazily, true);
assertTrue("Last block should be available", reader.hasPrev());
HoodieLogBlock block = reader.prev();
@@ -1261,7 +1265,7 @@ public class HoodieLogFormatTest {
writer.close();
HoodieLogFileReader reader = new HoodieLogFileReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(),
readBlocksLazily, true);
bufferSize, readBlocksLazily, true);
assertTrue("Third block should be available", reader.hasPrev());
reader.moveToPrev();