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

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