[HUDI-595] code cleanup, refactoring code out of PR# 1159 (#1302)
This commit is contained in:
@@ -86,11 +86,8 @@ public class HoodieJsonPayload implements HoodieRecordPayload<HoodieJsonPayload>
|
||||
}
|
||||
|
||||
private String unCompressData(byte[] data) throws IOException {
|
||||
InflaterInputStream iis = new InflaterInputStream(new ByteArrayInputStream(data));
|
||||
try {
|
||||
try (InflaterInputStream iis = new InflaterInputStream(new ByteArrayInputStream(data))) {
|
||||
return FileIOUtils.readAsUTFString(iis, dataSize);
|
||||
} finally {
|
||||
iis.close();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -46,6 +46,7 @@ import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* Scans a log file and provides block level iterator on the log file Loads the entire block contents in memory Can emit
|
||||
@@ -107,25 +108,22 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader {
|
||||
* Close the inputstream if not closed when the JVM exits.
|
||||
*/
|
||||
private void addShutDownHook() {
|
||||
Runtime.getRuntime().addShutdownHook(new Thread() {
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
close();
|
||||
} catch (Exception e) {
|
||||
LOG.warn("unable to close input stream for log file " + logFile, e);
|
||||
// fail silently for any sort of exception
|
||||
}
|
||||
Runtime.getRuntime().addShutdownHook(new Thread(() -> {
|
||||
try {
|
||||
close();
|
||||
} catch (Exception e) {
|
||||
LOG.warn("unable to close input stream for log file " + logFile, e);
|
||||
// fail silently for any sort of exception
|
||||
}
|
||||
});
|
||||
}));
|
||||
}
|
||||
|
||||
// TODO : convert content and block length to long by using ByteBuffer, raw byte [] allows
|
||||
// for max of Integer size
|
||||
private HoodieLogBlock readBlock() throws IOException {
|
||||
|
||||
int blocksize = -1;
|
||||
int type = -1;
|
||||
int blocksize;
|
||||
int type;
|
||||
HoodieLogBlockType blockType = null;
|
||||
Map<HeaderMetadataType, String> header = null;
|
||||
|
||||
@@ -190,7 +188,7 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader {
|
||||
// 9. Read the log block end position in the log file
|
||||
long blockEndPos = inputStream.getPos();
|
||||
|
||||
switch (blockType) {
|
||||
switch (Objects.requireNonNull(blockType)) {
|
||||
// based on type read the block
|
||||
case AVRO_DATA_BLOCK:
|
||||
if (nextBlockVersion.getVersion() == HoodieLogFormatVersion.DEFAULT_VERSION) {
|
||||
@@ -278,10 +276,10 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader {
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
/**
|
||||
/*
|
||||
* hasNext is not idempotent. TODO - Fix this. It is okay for now - PR
|
||||
*/
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
try {
|
||||
return readMagic();
|
||||
@@ -315,10 +313,7 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader {
|
||||
long pos = inputStream.getPos();
|
||||
// 1. Read magic header from the start of the block
|
||||
inputStream.readFully(MAGIC_BUFFER, 0, 6);
|
||||
if (!Arrays.equals(MAGIC_BUFFER, HoodieLogFormat.MAGIC)) {
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
return Arrays.equals(MAGIC_BUFFER, HoodieLogFormat.MAGIC);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -193,7 +193,7 @@ public class FSUtils {
|
||||
return partitions;
|
||||
}
|
||||
|
||||
public static final List<String> getAllDataFilesForMarkers(FileSystem fs, String basePath, String instantTs,
|
||||
public static List<String> getAllDataFilesForMarkers(FileSystem fs, String basePath, String instantTs,
|
||||
String markerDir) throws IOException {
|
||||
List<String> dataFiles = new LinkedList<>();
|
||||
processFiles(fs, markerDir, (status) -> {
|
||||
|
||||
@@ -79,7 +79,7 @@ public class HdfsTestService {
|
||||
|
||||
// Configure and start the HDFS cluster
|
||||
// boolean format = shouldFormatDFSCluster(localDFSLocation, clean);
|
||||
hadoopConf = configureDFSCluster(hadoopConf, localDFSLocation, bindIP, namenodeRpcPort,
|
||||
configureDFSCluster(hadoopConf, localDFSLocation, bindIP, namenodeRpcPort,
|
||||
datanodePort, datanodeIpcPort, datanodeHttpPort);
|
||||
miniDfsCluster = new MiniDFSCluster.Builder(hadoopConf).numDataNodes(1).format(format).checkDataNodeAddrConfig(true)
|
||||
.checkDataNodeHostConfig(true).build();
|
||||
|
||||
@@ -285,7 +285,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
/*
|
||||
* This is actually a test on concurrent append and not recovery lease. Commenting this out.
|
||||
* https://issues.apache.org/jira/browse/HUDI-117
|
||||
*/
|
||||
@@ -337,7 +337,6 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
assertEquals(2, statuses.length);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Test
|
||||
public void testBasicWriteAndScan() throws IOException, URISyntaxException, InterruptedException {
|
||||
Writer writer =
|
||||
@@ -366,7 +365,6 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
reader.close();
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Test
|
||||
public void testBasicAppendAndRead() throws IOException, URISyntaxException, InterruptedException {
|
||||
Writer writer =
|
||||
@@ -434,7 +432,6 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
reader.close();
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Test
|
||||
public void testBasicAppendAndScanMultipleFiles() throws IOException, URISyntaxException, InterruptedException {
|
||||
Writer writer =
|
||||
@@ -911,11 +908,6 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
|
||||
writer = writer.appendBlock(dataBlock);
|
||||
|
||||
List<String> originalKeys =
|
||||
copyOfRecords1.stream().map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString())
|
||||
.collect(Collectors.toList());
|
||||
|
||||
// Delete 50 keys
|
||||
// Delete 50 keys
|
||||
List<HoodieKey> deletedKeys = copyOfRecords1.stream()
|
||||
.map(s -> (new HoodieKey(((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(),
|
||||
@@ -1127,8 +1119,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
* duplicate data.
|
||||
*
|
||||
*/
|
||||
private void testAvroLogRecordReaderMergingMultipleLogFiles(int numRecordsInLog1, int numRecordsInLog2)
|
||||
throws IOException, URISyntaxException, InterruptedException {
|
||||
private void testAvroLogRecordReaderMergingMultipleLogFiles(int numRecordsInLog1, int numRecordsInLog2) {
|
||||
try {
|
||||
// Write one Data block with same InstantTime (written in same batch)
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
|
||||
@@ -1178,8 +1169,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAvroLogRecordReaderWithFailedTaskInFirstStageAttempt()
|
||||
throws IOException, URISyntaxException, InterruptedException {
|
||||
public void testAvroLogRecordReaderWithFailedTaskInFirstStageAttempt() {
|
||||
/*
|
||||
* FIRST_ATTEMPT_FAILED:
|
||||
* Original task from the stage attempt failed, but subsequent stage retry succeeded.
|
||||
@@ -1188,8 +1178,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAvroLogRecordReaderWithFailedTaskInSecondStageAttempt()
|
||||
throws IOException, URISyntaxException, InterruptedException {
|
||||
public void testAvroLogRecordReaderWithFailedTaskInSecondStageAttempt() {
|
||||
/*
|
||||
* SECOND_ATTEMPT_FAILED:
|
||||
* Original task from stage attempt succeeded, but subsequent retry attempt failed.
|
||||
@@ -1198,8 +1187,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAvroLogRecordReaderTasksSucceededInBothStageAttempts()
|
||||
throws IOException, URISyntaxException, InterruptedException {
|
||||
public void testAvroLogRecordReaderTasksSucceededInBothStageAttempts() {
|
||||
/*
|
||||
* BOTH_ATTEMPTS_SUCCEEDED:
|
||||
* Original task from the stage attempt and duplicate task from the stage retry succeeded.
|
||||
@@ -1207,7 +1195,6 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
testAvroLogRecordReaderMergingMultipleLogFiles(100, 100);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Test
|
||||
public void testBasicAppendAndReadInReverse() throws IOException, URISyntaxException, InterruptedException {
|
||||
Writer writer =
|
||||
@@ -1335,7 +1322,6 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
reader.close();
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Test
|
||||
public void testBasicAppendAndTraverseInReverse() throws IOException, URISyntaxException, InterruptedException {
|
||||
Writer writer =
|
||||
@@ -1392,7 +1378,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testV0Format() throws IOException, InterruptedException, URISyntaxException {
|
||||
public void testV0Format() throws IOException, URISyntaxException {
|
||||
// HoodieLogFormatVersion.DEFAULT_VERSION has been deprecated so we cannot
|
||||
// create a writer for it. So these tests are only for the HoodieAvroDataBlock
|
||||
// of older version.
|
||||
|
||||
@@ -221,7 +221,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
|
||||
*/
|
||||
public Stream<FileSlice> getLatestRawFileSlices(String partitionPath) {
|
||||
return fsView.getAllFileGroups(partitionPath).map(HoodieFileGroup::getLatestFileSlicesIncludingInflight)
|
||||
.filter(fileSliceOpt -> fileSliceOpt.isPresent()).map(Option::get);
|
||||
.filter(Option::isPresent).map(Option::get);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -322,7 +322,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
|
||||
assertEquals("Expect only valid data-file", dataFileName, dataFiles.get(0).getFileName());
|
||||
}
|
||||
|
||||
/** Merge API Tests **/
|
||||
// Merge API Tests
|
||||
List<FileSlice> fileSliceList =
|
||||
rtView.getLatestMergedFileSlicesBeforeOrOn(partitionPath, deltaInstantTime5).collect(Collectors.toList());
|
||||
assertEquals("Expect file-slice to be merged", 1, fileSliceList.size());
|
||||
@@ -355,7 +355,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
|
||||
assertEquals("Log File Order check", fileName4, logFiles.get(0).getFileName());
|
||||
assertEquals("Log File Order check", fileName3, logFiles.get(1).getFileName());
|
||||
|
||||
/** Data Files API tests */
|
||||
// Data Files API tests
|
||||
dataFiles = roView.getLatestBaseFiles().collect(Collectors.toList());
|
||||
if (skipCreatingDataFile) {
|
||||
assertEquals("Expect no data file to be returned", 0, dataFiles.size());
|
||||
@@ -385,7 +385,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
|
||||
dataFiles.forEach(df -> assertEquals("Expect data-file for instant 1 be returned", df.getCommitTime(), instantTime1));
|
||||
}
|
||||
|
||||
/** Inflight/Orphan File-groups needs to be in the view **/
|
||||
// Inflight/Orphan File-groups needs to be in the view
|
||||
|
||||
// There is a data-file with this inflight file-id
|
||||
final String inflightFileId1 = UUID.randomUUID().toString();
|
||||
@@ -507,7 +507,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
|
||||
assertEquals("Log File Order check", fileName4, logFiles.get(0).getFileName());
|
||||
assertEquals("Log File Order check", fileName3, logFiles.get(1).getFileName());
|
||||
|
||||
/** Data Files API tests */
|
||||
// Data Files API tests
|
||||
dataFiles = roView.getLatestBaseFiles().collect(Collectors.toList());
|
||||
assertEquals("Expect only one data-file to be sent", 1, dataFiles.size());
|
||||
dataFiles.forEach(df -> assertEquals("Expect data-file created by compaction be returned", df.getCommitTime(), compactionRequestedTime));
|
||||
|
||||
@@ -167,7 +167,7 @@ public class TestDiskBasedMap extends HoodieCommonTestHarness {
|
||||
schema = SchemaTestUtil.getSimpleSchema();
|
||||
List<IndexedRecord> indexedRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1);
|
||||
hoodieRecords =
|
||||
indexedRecords.stream().map(r -> new HoodieRecord(new HoodieKey(UUID.randomUUID().toString(), "0000/00/00"),
|
||||
indexedRecords.stream().map(r -> new HoodieRecord<>(new HoodieKey(UUID.randomUUID().toString(), "0000/00/00"),
|
||||
new AvroBinaryTestPayload(Option.of((GenericRecord) r)))).collect(Collectors.toList());
|
||||
payloadSize = SpillableMapUtils.computePayloadSize(hoodieRecords.remove(0), new HoodieRecordSizeEstimator(schema));
|
||||
assertTrue(payloadSize > 0);
|
||||
@@ -176,7 +176,7 @@ public class TestDiskBasedMap extends HoodieCommonTestHarness {
|
||||
final Schema simpleSchemaWithMetadata = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
|
||||
indexedRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1);
|
||||
hoodieRecords = indexedRecords.stream()
|
||||
.map(r -> new HoodieRecord(new HoodieKey(UUID.randomUUID().toString(), "0000/00/00"),
|
||||
.map(r -> new HoodieRecord<>(new HoodieKey(UUID.randomUUID().toString(), "0000/00/00"),
|
||||
new AvroBinaryTestPayload(
|
||||
Option.of(HoodieAvroUtils.rewriteRecord((GenericRecord) r, simpleSchemaWithMetadata)))))
|
||||
.collect(Collectors.toList());
|
||||
@@ -193,7 +193,7 @@ public class TestDiskBasedMap extends HoodieCommonTestHarness {
|
||||
// Test sizeEstimatorPerformance with simpleSchema
|
||||
Schema schema = SchemaTestUtil.getSimpleSchema();
|
||||
List<HoodieRecord> hoodieRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1, schema);
|
||||
HoodieRecordSizeEstimator sizeEstimator = new HoodieRecordSizeEstimator(schema);
|
||||
HoodieRecordSizeEstimator sizeEstimator = new HoodieRecordSizeEstimator<>(schema);
|
||||
HoodieRecord record = hoodieRecords.remove(0);
|
||||
long startTime = System.currentTimeMillis();
|
||||
SpillableMapUtils.computePayloadSize(record, sizeEstimator);
|
||||
|
||||
Reference in New Issue
Block a user