[HUDI-583] Code Cleanup, remove redundant code, and other changes (#1237)
This commit is contained in:
@@ -70,6 +70,7 @@ import java.text.SimpleDateFormat;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Calendar;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
@@ -125,7 +126,7 @@ public class HoodieTestUtils {
|
||||
return new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
|
||||
}
|
||||
|
||||
public static final void createCommitFiles(String basePath, String... commitTimes) throws IOException {
|
||||
public static void createCommitFiles(String basePath, String... commitTimes) throws IOException {
|
||||
for (String commitTime : commitTimes) {
|
||||
new File(
|
||||
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"
|
||||
@@ -139,11 +140,11 @@ public class HoodieTestUtils {
|
||||
}
|
||||
}
|
||||
|
||||
public static final void createMetadataFolder(String basePath) {
|
||||
public static void createMetadataFolder(String basePath) {
|
||||
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME).mkdirs();
|
||||
}
|
||||
|
||||
public static final void createInflightCommitFiles(String basePath, String... commitTimes) throws IOException {
|
||||
public static void createInflightCommitFiles(String basePath, String... commitTimes) throws IOException {
|
||||
|
||||
for (String commitTime : commitTimes) {
|
||||
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"
|
||||
@@ -153,7 +154,7 @@ public class HoodieTestUtils {
|
||||
}
|
||||
}
|
||||
|
||||
public static final void createPendingCleanFiles(HoodieTableMetaClient metaClient, String... commitTimes) {
|
||||
public static void createPendingCleanFiles(HoodieTableMetaClient metaClient, String... commitTimes) {
|
||||
for (String commitTime : commitTimes) {
|
||||
Arrays.asList(HoodieTimeline.makeRequestedCleanerFileName(commitTime),
|
||||
HoodieTimeline.makeInflightCleanerFileName(commitTime)).forEach(f -> {
|
||||
@@ -180,19 +181,19 @@ public class HoodieTestUtils {
|
||||
}
|
||||
}
|
||||
|
||||
public static final String createNewDataFile(String basePath, String partitionPath, String commitTime)
|
||||
public static String createNewDataFile(String basePath, String partitionPath, String commitTime)
|
||||
throws IOException {
|
||||
String fileID = UUID.randomUUID().toString();
|
||||
return createDataFile(basePath, partitionPath, commitTime, fileID);
|
||||
}
|
||||
|
||||
public static final String createNewMarkerFile(String basePath, String partitionPath, String commitTime)
|
||||
public static String createNewMarkerFile(String basePath, String partitionPath, String commitTime)
|
||||
throws IOException {
|
||||
String fileID = UUID.randomUUID().toString();
|
||||
return createMarkerFile(basePath, partitionPath, commitTime, fileID);
|
||||
}
|
||||
|
||||
public static final String createDataFile(String basePath, String partitionPath, String commitTime, String fileID)
|
||||
public static String createDataFile(String basePath, String partitionPath, String commitTime, String fileID)
|
||||
throws IOException {
|
||||
String folderPath = basePath + "/" + partitionPath + "/";
|
||||
new File(folderPath).mkdirs();
|
||||
@@ -200,7 +201,7 @@ public class HoodieTestUtils {
|
||||
return fileID;
|
||||
}
|
||||
|
||||
public static final String createMarkerFile(String basePath, String partitionPath, String commitTime, String fileID)
|
||||
public static String createMarkerFile(String basePath, String partitionPath, String commitTime, String fileID)
|
||||
throws IOException {
|
||||
String folderPath =
|
||||
basePath + "/" + HoodieTableMetaClient.TEMPFOLDER_NAME + "/" + commitTime + "/" + partitionPath + "/";
|
||||
@@ -210,7 +211,7 @@ public class HoodieTestUtils {
|
||||
return f.getAbsolutePath();
|
||||
}
|
||||
|
||||
public static final String createNewLogFile(FileSystem fs, String basePath, String partitionPath, String commitTime,
|
||||
public static String createNewLogFile(FileSystem fs, String basePath, String partitionPath, String commitTime,
|
||||
String fileID, Option<Integer> version) throws IOException {
|
||||
String folderPath = basePath + "/" + partitionPath + "/";
|
||||
boolean makeDir = fs.mkdirs(new Path(folderPath));
|
||||
@@ -226,7 +227,7 @@ public class HoodieTestUtils {
|
||||
return fileID;
|
||||
}
|
||||
|
||||
public static final void createCompactionCommitFiles(FileSystem fs, String basePath, String... commitTimes)
|
||||
public static void createCompactionCommitFiles(FileSystem fs, String basePath, String... commitTimes)
|
||||
throws IOException {
|
||||
for (String commitTime : commitTimes) {
|
||||
boolean createFile = fs.createNewFile(new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"
|
||||
@@ -237,7 +238,7 @@ public class HoodieTestUtils {
|
||||
}
|
||||
}
|
||||
|
||||
public static final void createCompactionRequest(HoodieTableMetaClient metaClient, String instant,
|
||||
public static void createCompactionRequest(HoodieTableMetaClient metaClient, String instant,
|
||||
List<Pair<String, FileSlice>> fileSliceList) throws IOException {
|
||||
HoodieCompactionPlan plan = CompactionUtils.buildFromFileSlices(fileSliceList, Option.empty(), Option.empty());
|
||||
HoodieInstant compactionInstant = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, instant);
|
||||
@@ -245,47 +246,47 @@ public class HoodieTestUtils {
|
||||
AvroUtils.serializeCompactionPlan(plan));
|
||||
}
|
||||
|
||||
public static final String getDataFilePath(String basePath, String partitionPath, String commitTime, String fileID) {
|
||||
public static String getDataFilePath(String basePath, String partitionPath, String commitTime, String fileID) {
|
||||
return basePath + "/" + partitionPath + "/" + FSUtils.makeDataFileName(commitTime, DEFAULT_WRITE_TOKEN, fileID);
|
||||
}
|
||||
|
||||
public static final String getLogFilePath(String basePath, String partitionPath, String commitTime, String fileID,
|
||||
public static String getLogFilePath(String basePath, String partitionPath, String commitTime, String fileID,
|
||||
Option<Integer> version) {
|
||||
return basePath + "/" + partitionPath + "/" + FSUtils.makeLogFileName(fileID, ".log", commitTime,
|
||||
version.orElse(DEFAULT_LOG_VERSION), HoodieLogFormat.UNKNOWN_WRITE_TOKEN);
|
||||
}
|
||||
|
||||
public static final String getCommitFilePath(String basePath, String commitTime) {
|
||||
public static String getCommitFilePath(String basePath, String commitTime) {
|
||||
return basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime + HoodieTimeline.COMMIT_EXTENSION;
|
||||
}
|
||||
|
||||
public static final String getInflightCommitFilePath(String basePath, String commitTime) {
|
||||
public static String getInflightCommitFilePath(String basePath, String commitTime) {
|
||||
return basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime
|
||||
+ HoodieTimeline.INFLIGHT_COMMIT_EXTENSION;
|
||||
}
|
||||
|
||||
public static final String getRequestedCompactionFilePath(String basePath, String commitTime) {
|
||||
public static String getRequestedCompactionFilePath(String basePath, String commitTime) {
|
||||
return basePath + "/" + HoodieTableMetaClient.AUXILIARYFOLDER_NAME + "/" + commitTime
|
||||
+ HoodieTimeline.INFLIGHT_COMMIT_EXTENSION;
|
||||
}
|
||||
|
||||
public static final boolean doesDataFileExist(String basePath, String partitionPath, String commitTime,
|
||||
public static boolean doesDataFileExist(String basePath, String partitionPath, String commitTime,
|
||||
String fileID) {
|
||||
return new File(getDataFilePath(basePath, partitionPath, commitTime, fileID)).exists();
|
||||
}
|
||||
|
||||
public static final boolean doesLogFileExist(String basePath, String partitionPath, String commitTime, String fileID,
|
||||
public static boolean doesLogFileExist(String basePath, String partitionPath, String commitTime, String fileID,
|
||||
Option<Integer> version) {
|
||||
return new File(getLogFilePath(basePath, partitionPath, commitTime, fileID, version)).exists();
|
||||
}
|
||||
|
||||
public static final boolean doesCommitExist(String basePath, String commitTime) {
|
||||
public static boolean doesCommitExist(String basePath, String commitTime) {
|
||||
return new File(
|
||||
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime + HoodieTimeline.COMMIT_EXTENSION)
|
||||
.exists();
|
||||
}
|
||||
|
||||
public static final boolean doesInflightExist(String basePath, String commitTime) {
|
||||
public static boolean doesInflightExist(String basePath, String commitTime) {
|
||||
return new File(
|
||||
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime + HoodieTimeline.INFLIGHT_EXTENSION)
|
||||
.exists();
|
||||
@@ -298,19 +299,16 @@ public class HoodieTestUtils {
|
||||
Path commitFile = new Path(
|
||||
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeCleanerFileName(commitTime));
|
||||
FileSystem fs = FSUtils.getFs(basePath, configuration);
|
||||
FSDataOutputStream os = fs.create(commitFile, true);
|
||||
try {
|
||||
try (FSDataOutputStream os = fs.create(commitFile, true)) {
|
||||
HoodieCleanStat cleanStats = new HoodieCleanStat(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS,
|
||||
DEFAULT_PARTITION_PATHS[rand.nextInt(DEFAULT_PARTITION_PATHS.length)], new ArrayList<>(), new ArrayList<>(),
|
||||
new ArrayList<>(), commitTime);
|
||||
// Create the clean metadata
|
||||
|
||||
HoodieCleanMetadata cleanMetadata =
|
||||
CleanerUtils.convertCleanMetadata(metaClient, commitTime, Option.of(0L), Arrays.asList(cleanStats));
|
||||
CleanerUtils.convertCleanMetadata(metaClient, commitTime, Option.of(0L), Collections.singletonList(cleanStats));
|
||||
// Write empty clean metadata
|
||||
os.write(AvroUtils.serializeCleanMetadata(cleanMetadata).get());
|
||||
} finally {
|
||||
os.close();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -25,7 +25,6 @@ import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.io.SequenceFile;
|
||||
@@ -34,6 +33,7 @@ import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.junit.Assert.assertArrayEquals;
|
||||
@@ -120,7 +120,7 @@ public class TestHoodieTableMetaClient extends HoodieCommonTestHarness {
|
||||
HoodieInstant instant2 = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "2");
|
||||
HoodieInstant instant3 = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "3");
|
||||
|
||||
assertEquals(Lists.newArrayList(instant1, instant2, instant3),
|
||||
assertEquals(Arrays.asList(instant1, instant2, instant3),
|
||||
archivedTimeline.getInstants().collect(Collectors.toList()));
|
||||
|
||||
assertArrayEquals(new Text("data1").getBytes(), archivedTimeline.getInstantDetails(instant1).get());
|
||||
|
||||
@@ -86,7 +86,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
|
||||
private FileSystem fs;
|
||||
private Path partitionPath;
|
||||
private int bufferSize = 4096;
|
||||
private Boolean readBlocksLazily = true;
|
||||
private Boolean readBlocksLazily;
|
||||
|
||||
public TestHoodieLogFormat(Boolean readBlocksLazily) {
|
||||
this.readBlocksLazily = readBlocksLazily;
|
||||
|
||||
@@ -209,7 +209,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
|
||||
* @return
|
||||
*/
|
||||
private Stream<FileSlice> getAllRawFileSlices(String partitionPath) {
|
||||
return fsView.getAllFileGroups(partitionPath).map(group -> group.getAllFileSlicesIncludingInflight())
|
||||
return fsView.getAllFileGroups(partitionPath).map(HoodieFileGroup::getAllFileSlicesIncludingInflight)
|
||||
.flatMap(sliceList -> sliceList);
|
||||
}
|
||||
|
||||
@@ -220,7 +220,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
|
||||
* @return
|
||||
*/
|
||||
public Stream<FileSlice> getLatestRawFileSlices(String partitionPath) {
|
||||
return fsView.getAllFileGroups(partitionPath).map(fileGroup -> fileGroup.getLatestFileSlicesIncludingInflight())
|
||||
return fsView.getAllFileGroups(partitionPath).map(HoodieFileGroup::getLatestFileSlicesIncludingInflight)
|
||||
.filter(fileSliceOpt -> fileSliceOpt.isPresent()).map(Option::get);
|
||||
}
|
||||
|
||||
@@ -275,7 +275,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
|
||||
partitionFileSlicesPairs.add(Pair.of(partitionPath, fileSlices.get(0)));
|
||||
HoodieCompactionPlan compactionPlan =
|
||||
CompactionUtils.buildFromFileSlices(partitionFileSlicesPairs, Option.empty(), Option.empty());
|
||||
HoodieInstant compactionInstant = null;
|
||||
HoodieInstant compactionInstant;
|
||||
if (isCompactionInFlight) {
|
||||
// Create a Data-file but this should be skipped by view
|
||||
new File(basePath + "/" + partitionPath + "/" + compactDataFileName).createNewFile();
|
||||
|
||||
@@ -62,6 +62,7 @@ import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
@@ -115,13 +116,13 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
|
||||
unscheduleCompaction(view, "14", "13", "11");
|
||||
|
||||
// Add one more delta instant
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view, Arrays.asList("15"), true, "11"));
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view, Collections.singletonList("15"), true, "11"));
|
||||
|
||||
// Schedule Compaction again
|
||||
scheduleCompaction(view, "16");
|
||||
|
||||
// Run Compaction - This will be the second file-slice
|
||||
testMultipleWriteSteps(view, Arrays.asList("16"), false, "16", 2);
|
||||
testMultipleWriteSteps(view, Collections.singletonList("16"), false, "16", 2);
|
||||
|
||||
// Run 2 more ingest
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view, Arrays.asList("17", "18"), true, "16", 2));
|
||||
@@ -130,25 +131,25 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
|
||||
scheduleCompaction(view, "19");
|
||||
|
||||
// Run one more ingestion after pending compaction. THis will be 3rd slice
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view, Arrays.asList("20"), true, "19", 3));
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view, Collections.singletonList("20"), true, "19", 3));
|
||||
|
||||
// Clean first slice
|
||||
testCleans(view, Arrays.asList("21"),
|
||||
testCleans(view, Collections.singletonList("21"),
|
||||
new ImmutableMap.Builder<String, List<String>>().put("11", Arrays.asList("12", "13", "15")).build(),
|
||||
instantsToFiles, Arrays.asList("11"));
|
||||
instantsToFiles, Collections.singletonList("11"));
|
||||
|
||||
// Add one more ingestion instant. This should be 2nd slice now
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view, Arrays.asList("22"), true, "19", 2));
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view, Collections.singletonList("22"), true, "19", 2));
|
||||
|
||||
// Restore last ingestion
|
||||
testRestore(view, Arrays.asList("23"), true, new HashMap<>(), Arrays.asList("22"), "24", false);
|
||||
testRestore(view, Collections.singletonList("23"), true, new HashMap<>(), Collections.singletonList("22"), "24", false);
|
||||
|
||||
// Run one more ingestion. THis is still 2nd slice
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view, Arrays.asList("24"), true, "19", 2));
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view, Collections.singletonList("24"), true, "19", 2));
|
||||
|
||||
// Finish Compaction
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view, Arrays.asList("19"), false, "19", 2,
|
||||
Arrays.asList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "24"))));
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view, Collections.singletonList("19"), false, "19", 2,
|
||||
Collections.singletonList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "24"))));
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -198,13 +199,13 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
|
||||
|
||||
SyncableFileSystemView view1 = getFileSystemView(metaClient);
|
||||
view1.sync();
|
||||
Map<String, List<String>> instantsToFiles = null;
|
||||
Map<String, List<String>> instantsToFiles;
|
||||
|
||||
/**
|
||||
* Case where incremental syncing is catching up on more than one ingestion at a time
|
||||
*/
|
||||
// Run 1 ingestion on MOR table (1 delta commits). View1 is now sync up to this point
|
||||
instantsToFiles = testMultipleWriteSteps(view1, Arrays.asList("11"), true, "11");
|
||||
instantsToFiles = testMultipleWriteSteps(view1, Collections.singletonList("11"), true, "11");
|
||||
|
||||
SyncableFileSystemView view2 =
|
||||
getFileSystemView(new HoodieTableMetaClient(metaClient.getHadoopConf(), metaClient.getBasePath()));
|
||||
@@ -213,7 +214,7 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("12", "13"), true, "11"));
|
||||
|
||||
// Now Sync view1 and add 1 more ingestion. Check if view1 is able to catchup correctly
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view1, Arrays.asList("14"), true, "11"));
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view1, Collections.singletonList("14"), true, "11"));
|
||||
|
||||
view2.sync();
|
||||
SyncableFileSystemView view3 =
|
||||
@@ -238,8 +239,8 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
|
||||
scheduleCompaction(view2, "16");
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("17", "18"), true, "16", 2));
|
||||
// Compaction
|
||||
testMultipleWriteSteps(view2, Arrays.asList("16"), false, "16", 2,
|
||||
Arrays.asList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "18")));
|
||||
testMultipleWriteSteps(view2, Collections.singletonList("16"), false, "16", 2,
|
||||
Collections.singletonList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "18")));
|
||||
view1.sync();
|
||||
areViewsConsistent(view1, view2, partitions.size() * fileIdsPerPartition.size() * 2);
|
||||
SyncableFileSystemView view5 =
|
||||
@@ -249,14 +250,14 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
|
||||
/**
|
||||
* Case where a clean happened and then rounds of ingestion and compaction happened
|
||||
*/
|
||||
testCleans(view2, Arrays.asList("19"),
|
||||
testCleans(view2, Collections.singletonList("19"),
|
||||
new ImmutableMap.Builder<String, List<String>>().put("11", Arrays.asList("12", "13", "14")).build(),
|
||||
instantsToFiles, Arrays.asList("11"));
|
||||
instantsToFiles, Collections.singletonList("11"));
|
||||
scheduleCompaction(view2, "20");
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("21", "22"), true, "20", 2));
|
||||
// Compaction
|
||||
testMultipleWriteSteps(view2, Arrays.asList("20"), false, "20", 2,
|
||||
Arrays.asList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "22")));
|
||||
testMultipleWriteSteps(view2, Collections.singletonList("20"), false, "20", 2,
|
||||
Collections.singletonList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "22")));
|
||||
// Run one more round of ingestion
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("23", "24"), true, "20", 2));
|
||||
view1.sync();
|
||||
@@ -268,14 +269,14 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
|
||||
/**
|
||||
* Case where multiple restores and ingestions happened
|
||||
*/
|
||||
testRestore(view2, Arrays.asList("25"), true, new HashMap<>(), Arrays.asList("24"), "29", true);
|
||||
testRestore(view2, Arrays.asList("26"), true, new HashMap<>(), Arrays.asList("23"), "29", false);
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("27"), true, "20", 2));
|
||||
testRestore(view2, Collections.singletonList("25"), true, new HashMap<>(), Collections.singletonList("24"), "29", true);
|
||||
testRestore(view2, Collections.singletonList("26"), true, new HashMap<>(), Collections.singletonList("23"), "29", false);
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view2, Collections.singletonList("27"), true, "20", 2));
|
||||
scheduleCompaction(view2, "28");
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("29"), true, "28", 3));
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view2, Collections.singletonList("29"), true, "28", 3));
|
||||
// Compaction
|
||||
testMultipleWriteSteps(view2, Arrays.asList("28"), false, "28", 3,
|
||||
Arrays.asList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "29")));
|
||||
testMultipleWriteSteps(view2, Collections.singletonList("28"), false, "28", 3,
|
||||
Collections.singletonList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "29")));
|
||||
|
||||
Arrays.asList(view1, view2, view3, view4, view5, view6).forEach(v -> {
|
||||
v.sync();
|
||||
@@ -371,8 +372,7 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
|
||||
LOG.info("Last Instant is :" + view.getLastInstant().get());
|
||||
if (isRestore) {
|
||||
Assert.assertEquals(newRestoreInstants.get(idx), view.getLastInstant().get().getTimestamp());
|
||||
Assert.assertEquals(isRestore ? HoodieTimeline.RESTORE_ACTION : HoodieTimeline.ROLLBACK_ACTION,
|
||||
view.getLastInstant().get().getAction());
|
||||
Assert.assertEquals(HoodieTimeline.RESTORE_ACTION, view.getLastInstant().get().getAction());
|
||||
}
|
||||
Assert.assertEquals(State.COMPLETED, view.getLastInstant().get().getState());
|
||||
|
||||
@@ -532,9 +532,7 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
|
||||
|
||||
view.sync();
|
||||
Assert.assertEquals(newLastInstant, view.getLastInstant().get().getTimestamp());
|
||||
partitions.forEach(p -> view.getLatestFileSlices(p).forEach(fs -> {
|
||||
Assert.assertEquals(newBaseInstant, fs.getBaseInstantTime());
|
||||
}));
|
||||
partitions.forEach(p -> view.getLatestFileSlices(p).forEach(fs -> Assert.assertEquals(newBaseInstant, fs.getBaseInstantTime())));
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -618,17 +616,11 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
|
||||
final long expTotalFileSlicesPerPartition = fileIdsPerPartition.size() * multiple;
|
||||
partitions.forEach(p -> Assert.assertEquals(expTotalFileSlicesPerPartition, view.getAllFileSlices(p).count()));
|
||||
if (deltaCommit) {
|
||||
partitions.forEach(p -> {
|
||||
view.getLatestFileSlices(p).forEach(f -> {
|
||||
Assert.assertEquals(baseInstantForDeltaCommit, f.getBaseInstantTime());
|
||||
});
|
||||
});
|
||||
partitions.forEach(p ->
|
||||
view.getLatestFileSlices(p).forEach(f -> Assert.assertEquals(baseInstantForDeltaCommit, f.getBaseInstantTime()))
|
||||
);
|
||||
} else {
|
||||
partitions.forEach(p -> {
|
||||
view.getLatestBaseFiles(p).forEach(f -> {
|
||||
Assert.assertEquals(instant, f.getCommitTime());
|
||||
});
|
||||
});
|
||||
partitions.forEach(p -> view.getLatestBaseFiles(p).forEach(f -> Assert.assertEquals(instant, f.getCommitTime())));
|
||||
}
|
||||
|
||||
metaClient.reloadActiveTimeline();
|
||||
@@ -704,23 +696,21 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
|
||||
|
||||
private List<String> addInstant(HoodieTableMetaClient metaClient, String instant, boolean deltaCommit,
|
||||
String baseInstant) throws IOException {
|
||||
List<Pair<String, HoodieWriteStat>> writeStats = partitions.stream().flatMap(p -> {
|
||||
return fileIdsPerPartition.stream().map(f -> {
|
||||
try {
|
||||
File file = new File(basePath + "/" + p + "/"
|
||||
+ (deltaCommit
|
||||
? FSUtils.makeLogFileName(f, ".log", baseInstant, Integer.parseInt(instant), TEST_WRITE_TOKEN)
|
||||
: FSUtils.makeDataFileName(instant, TEST_WRITE_TOKEN, f)));
|
||||
file.createNewFile();
|
||||
HoodieWriteStat w = new HoodieWriteStat();
|
||||
w.setFileId(f);
|
||||
w.setPath(String.format("%s/%s", p, file.getName()));
|
||||
return Pair.of(p, w);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
});
|
||||
}).collect(Collectors.toList());
|
||||
List<Pair<String, HoodieWriteStat>> writeStats = partitions.stream().flatMap(p -> fileIdsPerPartition.stream().map(f -> {
|
||||
try {
|
||||
File file = new File(basePath + "/" + p + "/"
|
||||
+ (deltaCommit
|
||||
? FSUtils.makeLogFileName(f, ".log", baseInstant, Integer.parseInt(instant), TEST_WRITE_TOKEN)
|
||||
: FSUtils.makeDataFileName(instant, TEST_WRITE_TOKEN, f)));
|
||||
file.createNewFile();
|
||||
HoodieWriteStat w = new HoodieWriteStat();
|
||||
w.setFileId(f);
|
||||
w.setPath(String.format("%s/%s", p, file.getName()));
|
||||
return Pair.of(p, w);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
})).collect(Collectors.toList());
|
||||
|
||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
|
||||
writeStats.forEach(e -> metadata.addWriteStat(e.getKey(), e.getValue()));
|
||||
|
||||
@@ -229,8 +229,7 @@ public class TestFSUtils extends HoodieCommonTestHarness {
|
||||
String log1Ver1 = makeOldLogFileName("file1", ".log", "1", 1);
|
||||
String log1base2 = makeOldLogFileName("file1", ".log", "2", 0);
|
||||
List<HoodieLogFile> logFiles = Stream.of(log1base2, log1Ver1, log1Ver0).map(HoodieLogFile::new)
|
||||
.collect(Collectors.toList());
|
||||
logFiles.sort(HoodieLogFile.getLogFileComparator());
|
||||
.sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList());
|
||||
assertEquals(log1Ver0, logFiles.get(0).getFileName());
|
||||
assertEquals(log1Ver1, logFiles.get(1).getFileName());
|
||||
assertEquals(log1base2, logFiles.get(2).getFileName());
|
||||
@@ -250,8 +249,7 @@ public class TestFSUtils extends HoodieCommonTestHarness {
|
||||
|
||||
List<HoodieLogFile> logFiles =
|
||||
Stream.of(log1Ver1W1, log1base2W0, log1base2W1, log1Ver1W0, log1Ver0W1, log1Ver0W0)
|
||||
.map(HoodieLogFile::new).collect(Collectors.toList());
|
||||
logFiles.sort(HoodieLogFile.getLogFileComparator());
|
||||
.map(HoodieLogFile::new).sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList());
|
||||
assertEquals(log1Ver0W0, logFiles.get(0).getFileName());
|
||||
assertEquals(log1Ver0W1, logFiles.get(1).getFileName());
|
||||
assertEquals(log1Ver1W0, logFiles.get(2).getFileName());
|
||||
|
||||
Reference in New Issue
Block a user