[HUDI-153] Use com.uber.hoodie.common.util.Option instead of Java and Guava Optional
This commit is contained in:
committed by
Balaji Varadarajan
parent
d288e32833
commit
722b6be04a
@@ -42,6 +42,7 @@ import com.uber.hoodie.common.table.view.FileSystemViewStorageType;
|
||||
import com.uber.hoodie.common.table.view.HoodieTableFileSystemView;
|
||||
import com.uber.hoodie.common.util.AvroUtils;
|
||||
import com.uber.hoodie.common.util.CompactionUtils;
|
||||
import com.uber.hoodie.common.util.Option;
|
||||
import com.uber.hoodie.common.util.collection.Pair;
|
||||
import com.uber.hoodie.config.HoodieCompactionConfig;
|
||||
import com.uber.hoodie.config.HoodieIndexConfig;
|
||||
@@ -54,7 +55,6 @@ import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
@@ -485,7 +485,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
|
||||
private void scheduleCompaction(String compactionInstantTime, HoodieWriteClient client, HoodieWriteConfig cfg)
|
||||
throws IOException {
|
||||
client.scheduleCompactionAtInstant(compactionInstantTime, Optional.empty());
|
||||
client.scheduleCompactionAtInstant(compactionInstantTime, Option.empty());
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieInstant instant = metaClient.getActiveTimeline().filterPendingCompactionTimeline().lastInstant().get();
|
||||
assertEquals("Last compaction instant must be the one set",
|
||||
@@ -545,7 +545,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
client.commit(instantTime, statuses);
|
||||
}
|
||||
|
||||
Optional<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().reload().getDeltaCommitTimeline()
|
||||
Option<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().reload().getDeltaCommitTimeline()
|
||||
.filterCompletedInstants().lastInstant();
|
||||
if (skipCommit && !cfg.shouldAutoCommit()) {
|
||||
assertTrue("Delta commit should not be latest instant",
|
||||
|
||||
@@ -50,6 +50,7 @@ import com.uber.hoodie.common.util.AvroUtils;
|
||||
import com.uber.hoodie.common.util.CompactionUtils;
|
||||
import com.uber.hoodie.common.util.ConsistencyGuardConfig;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.Option;
|
||||
import com.uber.hoodie.common.util.collection.Pair;
|
||||
import com.uber.hoodie.config.HoodieCompactionConfig;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
@@ -62,7 +63,6 @@ import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
import java.util.function.Predicate;
|
||||
@@ -78,7 +78,6 @@ import org.apache.spark.scheduler.SparkListenerTaskEnd;
|
||||
import org.apache.spark.util.AccumulatorV2;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import scala.Option;
|
||||
import scala.collection.Iterator;
|
||||
|
||||
/**
|
||||
@@ -214,11 +213,11 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metadata, getConfig(), jsc);
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
TableFileSystemView fsView = table.getFileSystemView();
|
||||
Optional<Boolean> added = fsView.getAllFileGroups(partitionPath).findFirst()
|
||||
Option<Boolean> added = Option.fromJavaOptional(fsView.getAllFileGroups(partitionPath).findFirst()
|
||||
.map(fg -> {
|
||||
fg.getLatestFileSlice().map(fs -> compactionFileIdToLatestFileSlice.put(fg.getFileGroupId(), fs));
|
||||
return true;
|
||||
});
|
||||
}));
|
||||
if (added.isPresent()) {
|
||||
// Select only one file-group for compaction
|
||||
break;
|
||||
@@ -229,7 +228,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
List<Pair<String, FileSlice>> partitionFileSlicePairs = compactionFileIdToLatestFileSlice.entrySet().stream()
|
||||
.map(e -> Pair.of(e.getKey().getPartitionPath(), e.getValue())).collect(Collectors.toList());
|
||||
HoodieCompactionPlan compactionPlan =
|
||||
CompactionUtils.buildFromFileSlices(partitionFileSlicePairs, Optional.empty(), Optional.empty());
|
||||
CompactionUtils.buildFromFileSlices(partitionFileSlicePairs, Option.empty(), Option.empty());
|
||||
List<String> instantTimes = HoodieTestUtils.monotonicIncreasingCommitTimestamps(9, 1);
|
||||
String compactionTime = instantTimes.get(0);
|
||||
table.getActiveTimeline().saveToCompactionRequested(
|
||||
@@ -275,11 +274,11 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
for (HoodieFileGroup fileGroup : fileGroups) {
|
||||
if (compactionFileIdToLatestFileSlice.containsKey(fileGroup.getFileGroupId())) {
|
||||
// Ensure latest file-slice selected for compaction is retained
|
||||
Optional<HoodieDataFile> dataFileForCompactionPresent =
|
||||
fileGroup.getAllDataFiles().filter(df -> {
|
||||
Option<HoodieDataFile> dataFileForCompactionPresent =
|
||||
Option.fromJavaOptional(fileGroup.getAllDataFiles().filter(df -> {
|
||||
return compactionFileIdToLatestFileSlice.get(fileGroup.getFileGroupId())
|
||||
.getBaseInstantTime().equals(df.getCommitTime());
|
||||
}).findAny();
|
||||
}).findAny());
|
||||
Assert.assertTrue("Data File selected for compaction is retained",
|
||||
dataFileForCompactionPresent.isPresent());
|
||||
} else {
|
||||
@@ -386,7 +385,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTable table1 = HoodieTable.getHoodieTable(metadata, cfg, jsc);
|
||||
HoodieTimeline activeTimeline = table1.getCompletedCommitsTimeline();
|
||||
Optional<HoodieInstant> earliestRetainedCommit = activeTimeline.nthFromLastInstant(maxCommits - 1);
|
||||
Option<HoodieInstant> earliestRetainedCommit = activeTimeline.nthFromLastInstant(maxCommits - 1);
|
||||
Set<HoodieInstant> acceptableCommits = activeTimeline.getInstants().collect(Collectors.toSet());
|
||||
if (earliestRetainedCommit.isPresent()) {
|
||||
acceptableCommits.removeAll(
|
||||
@@ -503,20 +502,20 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
// Make 3 files, one base file and 2 log files associated with base file
|
||||
String file1P0 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "000");
|
||||
String file2P0L0 = HoodieTestUtils
|
||||
.createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, "000", file1P0, Optional.empty());
|
||||
.createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, "000", file1P0, Option.empty());
|
||||
String file2P0L1 = HoodieTestUtils
|
||||
.createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, "000", file1P0, Optional.of(2));
|
||||
.createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, "000", file1P0, Option.of(2));
|
||||
// make 1 compaction commit
|
||||
HoodieTestUtils.createCompactionCommitFiles(fs, basePath, "000");
|
||||
|
||||
// Make 4 files, one base file and 3 log files associated with base file
|
||||
HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "001", file1P0);
|
||||
file2P0L0 = HoodieTestUtils
|
||||
.createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, "001", file1P0, Optional.empty());
|
||||
.createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, "001", file1P0, Option.empty());
|
||||
file2P0L0 = HoodieTestUtils
|
||||
.createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, "001", file1P0, Optional.of(2));
|
||||
.createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, "001", file1P0, Option.of(2));
|
||||
file2P0L0 = HoodieTestUtils
|
||||
.createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, "001", file1P0, Optional.of(3));
|
||||
.createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, "001", file1P0, Option.of(3));
|
||||
// make 1 compaction commit
|
||||
HoodieTestUtils.createCompactionCommitFiles(fs, basePath, "001");
|
||||
|
||||
@@ -526,9 +525,9 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
getCleanStat(hoodieCleanStats, DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles().size());
|
||||
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, DEFAULT_FIRST_PARTITION_PATH, "000", file1P0));
|
||||
assertFalse(
|
||||
HoodieTestUtils.doesLogFileExist(basePath, DEFAULT_FIRST_PARTITION_PATH, "000", file2P0L0, Optional.empty()));
|
||||
HoodieTestUtils.doesLogFileExist(basePath, DEFAULT_FIRST_PARTITION_PATH, "000", file2P0L0, Option.empty()));
|
||||
assertFalse(
|
||||
HoodieTestUtils.doesLogFileExist(basePath, DEFAULT_FIRST_PARTITION_PATH, "000", file2P0L0, Optional.of(2)));
|
||||
HoodieTestUtils.doesLogFileExist(basePath, DEFAULT_FIRST_PARTITION_PATH, "000", file2P0L0, Option.of(2)));
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -810,9 +809,9 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
final String fileId = HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, instants[0],
|
||||
fileIds[i]);
|
||||
HoodieTestUtils.createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, instants[0],
|
||||
fileId, Optional.empty());
|
||||
fileId, Option.empty());
|
||||
HoodieTestUtils.createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, instants[0],
|
||||
fileId, Optional.of(2));
|
||||
fileId, Option.of(2));
|
||||
fileIdToLatestInstantBeforeCompaction.put(fileId, instants[0]);
|
||||
for (int j = 1; j <= i; j++) {
|
||||
if (j == i && j <= maxNumFileIdsForCompaction) {
|
||||
@@ -830,15 +829,15 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
compactionInstantsToFileSlices.put(compactionInstants[j], slices);
|
||||
// Add log-files to simulate delta-commits after pending compaction
|
||||
HoodieTestUtils.createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, compactionInstants[j],
|
||||
fileId, Optional.empty());
|
||||
fileId, Option.empty());
|
||||
HoodieTestUtils.createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, compactionInstants[j],
|
||||
fileId, Optional.of(2));
|
||||
fileId, Option.of(2));
|
||||
} else {
|
||||
HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, instants[j], fileId);
|
||||
HoodieTestUtils.createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, instants[j], fileId,
|
||||
Optional.empty());
|
||||
Option.empty());
|
||||
HoodieTestUtils.createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, instants[j], fileId,
|
||||
Optional.of(2));
|
||||
Option.of(2));
|
||||
fileIdToLatestInstantBeforeCompaction.put(fileId, instants[j]);
|
||||
}
|
||||
}
|
||||
@@ -867,9 +866,10 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
expFileIdToPendingCompaction.entrySet().stream().forEach(entry -> {
|
||||
String fileId = entry.getKey();
|
||||
String baseInstantForCompaction = fileIdToLatestInstantBeforeCompaction.get(fileId);
|
||||
Optional<FileSlice> fileSliceForCompaction =
|
||||
hoodieTable.getRTFileSystemView().getLatestFileSlicesBeforeOrOn(DEFAULT_FIRST_PARTITION_PATH,
|
||||
baseInstantForCompaction, true).filter(fs -> fs.getFileId().equals(fileId)).findFirst();
|
||||
Option<FileSlice> fileSliceForCompaction =
|
||||
Option.fromJavaOptional(
|
||||
hoodieTable.getRTFileSystemView().getLatestFileSlicesBeforeOrOn(DEFAULT_FIRST_PARTITION_PATH,
|
||||
baseInstantForCompaction, true).filter(fs -> fs.getFileId().equals(fileId)).findFirst());
|
||||
Assert.assertTrue("Base Instant for Compaction must be preserved", fileSliceForCompaction.isPresent());
|
||||
Assert.assertTrue("FileSlice has data-file", fileSliceForCompaction.get().getDataFile().isPresent());
|
||||
Assert.assertEquals("FileSlice has log-files", 2,
|
||||
|
||||
@@ -20,7 +20,6 @@ package com.uber.hoodie;
|
||||
|
||||
import static com.uber.hoodie.common.model.HoodieTestUtils.getDefaultHadoopConf;
|
||||
|
||||
import com.google.common.base.Optional;
|
||||
import com.uber.hoodie.CompactionAdminClient.ValidationOpResult;
|
||||
import com.uber.hoodie.common.model.CompactionOperation;
|
||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
@@ -30,6 +29,7 @@ import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.view.HoodieTableFileSystemView;
|
||||
import com.uber.hoodie.common.util.CompactionTestUtils;
|
||||
import com.uber.hoodie.common.util.CompactionUtils;
|
||||
import com.uber.hoodie.common.util.Option;
|
||||
import com.uber.hoodie.common.util.collection.Pair;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
@@ -139,7 +139,7 @@ public class TestCompactionAdminClient extends TestHoodieClientBase {
|
||||
// Now repair
|
||||
List<Pair<HoodieLogFile, HoodieLogFile>> undoFiles = result.stream().flatMap(r ->
|
||||
client.getRenamingActionsToAlignWithCompactionOperation(metaClient,
|
||||
compactionInstant, r.getOperation(), Optional.absent()).stream())
|
||||
compactionInstant, r.getOperation(), Option.empty()).stream())
|
||||
.map(rn -> {
|
||||
try {
|
||||
client.renameLogFile(metaClient, rn.getKey(), rn.getValue());
|
||||
@@ -238,7 +238,7 @@ public class TestCompactionAdminClient extends TestHoodieClientBase {
|
||||
// Check suggested rename operations
|
||||
List<Pair<HoodieLogFile, HoodieLogFile>> renameFiles =
|
||||
client.getRenamingActionsForUnschedulingCompactionPlan(metaClient, compactionInstant, 1,
|
||||
Optional.absent(), false);
|
||||
Option.empty(), false);
|
||||
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
|
||||
|
||||
// Log files belonging to file-slices created because of compaction request must be renamed
|
||||
@@ -311,7 +311,7 @@ public class TestCompactionAdminClient extends TestHoodieClientBase {
|
||||
// Check suggested rename operations
|
||||
List<Pair<HoodieLogFile, HoodieLogFile>> renameFiles =
|
||||
client.getRenamingActionsForUnschedulingCompactionOperation(metaClient, compactionInstant, op,
|
||||
Optional.absent(), false);
|
||||
Option.empty(), false);
|
||||
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
|
||||
|
||||
// Log files belonging to file-slices created because of compaction request must be renamed
|
||||
|
||||
@@ -39,6 +39,7 @@ import com.uber.hoodie.common.table.view.FileSystemViewStorageConfig;
|
||||
import com.uber.hoodie.common.table.view.FileSystemViewStorageType;
|
||||
import com.uber.hoodie.common.util.ConsistencyGuardConfig;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.Option;
|
||||
import com.uber.hoodie.config.HoodieCompactionConfig;
|
||||
import com.uber.hoodie.config.HoodieIndexConfig;
|
||||
import com.uber.hoodie.config.HoodieStorageConfig;
|
||||
@@ -54,7 +55,6 @@ import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.LocalFileSystem;
|
||||
@@ -341,7 +341,7 @@ public class TestHoodieClientBase implements Serializable {
|
||||
final Function2<List<HoodieRecord>, String, Integer> recordGenFunction =
|
||||
generateWrapRecordsFn(isPreppedAPI, writeConfig, dataGen::generateInserts);
|
||||
|
||||
return writeBatch(client, newCommitTime, initCommitTime, Optional.empty(), initCommitTime,
|
||||
return writeBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime,
|
||||
numRecordsInThisCommit, recordGenFunction, writeFn, assertForCommit,
|
||||
expRecordsInThisCommit, expRecordsInThisCommit, 1);
|
||||
}
|
||||
@@ -370,7 +370,7 @@ public class TestHoodieClientBase implements Serializable {
|
||||
HoodieWriteClient client,
|
||||
String newCommitTime,
|
||||
String prevCommitTime,
|
||||
Optional<List<String>> commitTimesBetweenPrevAndNew,
|
||||
Option<List<String>> commitTimesBetweenPrevAndNew,
|
||||
String initCommitTime,
|
||||
int numRecordsInThisCommit,
|
||||
Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> writeFn,
|
||||
@@ -409,7 +409,7 @@ public class TestHoodieClientBase implements Serializable {
|
||||
HoodieWriteClient client,
|
||||
String newCommitTime,
|
||||
String prevCommitTime,
|
||||
Optional<List<String>> commitTimesBetweenPrevAndNew,
|
||||
Option<List<String>> commitTimesBetweenPrevAndNew,
|
||||
String initCommitTime,
|
||||
int numRecordsInThisCommit,
|
||||
Function2<List<HoodieRecord>, String, Integer> recordGenFunction,
|
||||
|
||||
@@ -41,6 +41,7 @@ import com.uber.hoodie.common.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.util.ConsistencyGuardConfig;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.Option;
|
||||
import com.uber.hoodie.common.util.ParquetUtils;
|
||||
import com.uber.hoodie.common.util.collection.Pair;
|
||||
import com.uber.hoodie.config.HoodieCompactionConfig;
|
||||
@@ -58,7 +59,6 @@ import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
@@ -68,7 +68,6 @@ import org.apache.hadoop.fs.Path;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import scala.Option;
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
@@ -283,7 +282,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
numRecords = 100;
|
||||
String commitTimeBetweenPrevAndNew = "002";
|
||||
updateBatch(hoodieWriteConfig, client, newCommitTime, prevCommitTime,
|
||||
Optional.of(Arrays.asList(commitTimeBetweenPrevAndNew)),
|
||||
Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)),
|
||||
initCommitTime, numRecords, writeFn, isPrepped, true, numRecords, 200, 2);
|
||||
}
|
||||
|
||||
@@ -311,7 +310,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
recordsInFirstBatch.addAll(fewRecordsForDelete);
|
||||
return recordsInFirstBatch;
|
||||
};
|
||||
writeBatch(client, newCommitTime, initCommitTime, Optional.empty(), initCommitTime,
|
||||
writeBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime,
|
||||
//unused as genFn uses hard-coded number of inserts/updates/deletes
|
||||
-1,
|
||||
recordGenFunction, HoodieWriteClient::upsert, true,
|
||||
@@ -332,7 +331,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
recordsInSecondBatch.addAll(fewRecordsForUpdate);
|
||||
return recordsInSecondBatch;
|
||||
};
|
||||
writeBatch(client, newCommitTime, prevCommitTime, Optional.empty(), initCommitTime,
|
||||
writeBatch(client, newCommitTime, prevCommitTime, Option.empty(), initCommitTime,
|
||||
100, recordGenFunction, HoodieWriteClient::upsert, true,
|
||||
50, 150, 2);
|
||||
}
|
||||
|
||||
@@ -21,17 +21,16 @@ package com.uber.hoodie;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.util.Option;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import scala.Option;
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
/**
|
||||
@@ -192,7 +191,7 @@ public class TestHoodieReadClient extends TestHoodieClientBase {
|
||||
numRecords = 100;
|
||||
String commitTimeBetweenPrevAndNew = "002";
|
||||
result = updateBatch(hoodieWriteConfig, client, newCommitTime, prevCommitTime,
|
||||
Optional.of(Arrays.asList(commitTimeBetweenPrevAndNew)),
|
||||
Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)),
|
||||
initCommitTime, numRecords, updateFn, isPrepped,
|
||||
true, numRecords, 200, 2);
|
||||
recordRDD =
|
||||
|
||||
@@ -31,6 +31,7 @@ import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.util.AvroUtils;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||
import com.uber.hoodie.common.util.Option;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
@@ -41,7 +42,6 @@ import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Random;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
@@ -126,7 +126,7 @@ public class HoodieTestDataGenerator {
|
||||
*/
|
||||
public static HoodieAvroPayload generateAvroPayload(HoodieKey key, String commitTime) throws IOException {
|
||||
GenericRecord rec = generateGenericRecord(key.getRecordKey(), "rider-" + commitTime, "driver-" + commitTime, 0.0);
|
||||
return new HoodieAvroPayload(Optional.of(rec));
|
||||
return new HoodieAvroPayload(Option.of(rec));
|
||||
}
|
||||
|
||||
public static GenericRecord generateGenericRecord(String rowKey, String riderName, String driverName,
|
||||
@@ -290,7 +290,7 @@ public class HoodieTestDataGenerator {
|
||||
}
|
||||
|
||||
public HoodieRecord generateDeleteRecord(HoodieKey key) throws IOException {
|
||||
TestRawTripPayload payload = new TestRawTripPayload(Optional.empty(), key.getRecordKey(), key.getPartitionPath(),
|
||||
TestRawTripPayload payload = new TestRawTripPayload(Option.empty(), key.getRecordKey(), key.getPartitionPath(),
|
||||
null, true);
|
||||
return new HoodieRecord(key, payload);
|
||||
}
|
||||
|
||||
@@ -23,6 +23,7 @@ import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.avro.MercifulJsonConverter;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.util.Option;
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.IOException;
|
||||
@@ -31,7 +32,6 @@ import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Optional;
|
||||
import java.util.zip.Deflater;
|
||||
import java.util.zip.DeflaterOutputStream;
|
||||
import java.util.zip.InflaterInputStream;
|
||||
@@ -52,7 +52,7 @@ public class TestRawTripPayload implements HoodieRecordPayload<TestRawTripPayloa
|
||||
private int dataSize;
|
||||
private boolean isDeleted;
|
||||
|
||||
public TestRawTripPayload(Optional<String> jsonData, String rowKey, String partitionPath, String schemaStr,
|
||||
public TestRawTripPayload(Option<String> jsonData, String rowKey, String partitionPath, String schemaStr,
|
||||
Boolean isDeleted) throws IOException {
|
||||
if (jsonData.isPresent()) {
|
||||
this.jsonDataCompressed = compressData(jsonData.get());
|
||||
@@ -64,7 +64,7 @@ public class TestRawTripPayload implements HoodieRecordPayload<TestRawTripPayloa
|
||||
}
|
||||
|
||||
public TestRawTripPayload(String jsonData, String rowKey, String partitionPath, String schemaStr) throws IOException {
|
||||
this(Optional.of(jsonData), rowKey, partitionPath, schemaStr, false);
|
||||
this(Option.of(jsonData), rowKey, partitionPath, schemaStr, false);
|
||||
}
|
||||
|
||||
public TestRawTripPayload(String jsonData) throws IOException {
|
||||
@@ -87,27 +87,27 @@ public class TestRawTripPayload implements HoodieRecordPayload<TestRawTripPayloa
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<IndexedRecord> combineAndGetUpdateValue(IndexedRecord oldRec, Schema schema) throws IOException {
|
||||
public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord oldRec, Schema schema) throws IOException {
|
||||
return this.getInsertValue(schema);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<IndexedRecord> getInsertValue(Schema schema) throws IOException {
|
||||
public Option<IndexedRecord> getInsertValue(Schema schema) throws IOException {
|
||||
if (isDeleted) {
|
||||
return Optional.empty();
|
||||
return Option.empty();
|
||||
} else {
|
||||
MercifulJsonConverter jsonConverter = new MercifulJsonConverter(schema);
|
||||
return Optional.of(jsonConverter.convert(getJsonData()));
|
||||
return Option.of(jsonConverter.convert(getJsonData()));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<Map<String, String>> getMetadata() {
|
||||
public Option<Map<String, String>> getMetadata() {
|
||||
// Let's assume we want to count the number of input row change events
|
||||
// that are processed. Let the time-bucket for this row change event be 1506582000.
|
||||
Map<String, String> metadataMap = new HashMap<>();
|
||||
metadataMap.put("InputRecordCount_1506582000", "2");
|
||||
return Optional.of(metadataMap);
|
||||
return Option.of(metadataMap);
|
||||
}
|
||||
|
||||
public String getRowKey() {
|
||||
@@ -174,7 +174,7 @@ public class TestRawTripPayload implements HoodieRecordPayload<TestRawTripPayloa
|
||||
}
|
||||
|
||||
@Override
|
||||
public void markSuccess(HoodieRecord record, Optional<Map<String, String>> recordMetadata) {
|
||||
public void markSuccess(HoodieRecord record, Option<Map<String, String>> recordMetadata) {
|
||||
super.markSuccess(record, recordMetadata);
|
||||
if (recordMetadata.isPresent()) {
|
||||
mergeMetadataMaps(recordMetadata.get(), mergedMetadataMap);
|
||||
@@ -182,7 +182,7 @@ public class TestRawTripPayload implements HoodieRecordPayload<TestRawTripPayloa
|
||||
}
|
||||
|
||||
@Override
|
||||
public void markFailure(HoodieRecord record, Throwable t, Optional<Map<String, String>> recordMetadata) {
|
||||
public void markFailure(HoodieRecord record, Throwable t, Option<Map<String, String>> recordMetadata) {
|
||||
super.markFailure(record, t, recordMetadata);
|
||||
if (recordMetadata.isPresent()) {
|
||||
mergeMetadataMaps(recordMetadata.get(), mergedMetadataMap);
|
||||
|
||||
@@ -25,11 +25,11 @@ import static org.mockito.Mockito.when;
|
||||
import com.uber.hoodie.common.HoodieTestDataGenerator;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
|
||||
import com.uber.hoodie.common.util.Option;
|
||||
import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueConsumer;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.func.CopyOnWriteLazyInsertIterable.HoodieInsertValueGenResult;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
@@ -41,7 +41,7 @@ public class TestBoundedInMemoryExecutor {
|
||||
private final HoodieTestDataGenerator hoodieTestDataGenerator = new HoodieTestDataGenerator();
|
||||
private final String commitTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
private SparkBoundedInMemoryExecutor<HoodieRecord,
|
||||
Tuple2<HoodieRecord, Optional<IndexedRecord>>, Integer> executor = null;
|
||||
Tuple2<HoodieRecord, Option<IndexedRecord>>, Integer> executor = null;
|
||||
|
||||
@After
|
||||
public void afterTest() {
|
||||
|
||||
@@ -26,6 +26,7 @@ import com.uber.hoodie.common.HoodieTestDataGenerator;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
|
||||
import com.uber.hoodie.common.util.DefaultSizeEstimator;
|
||||
import com.uber.hoodie.common.util.Option;
|
||||
import com.uber.hoodie.common.util.SizeEstimator;
|
||||
import com.uber.hoodie.common.util.queue.BoundedInMemoryQueue;
|
||||
import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueProducer;
|
||||
@@ -38,7 +39,6 @@ import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
@@ -94,7 +94,7 @@ public class TestBoundedInMemoryQueue {
|
||||
int recordsRead = 0;
|
||||
while (queue.iterator().hasNext()) {
|
||||
final HoodieRecord originalRecord = originalRecordIterator.next();
|
||||
final Optional<IndexedRecord> originalInsertValue = originalRecord.getData()
|
||||
final Option<IndexedRecord> originalInsertValue = originalRecord.getData()
|
||||
.getInsertValue(HoodieTestDataGenerator.avroSchema);
|
||||
final HoodieInsertValueGenResult<HoodieRecord> payload = queue.iterator().next();
|
||||
// Ensure that record ordering is guaranteed.
|
||||
@@ -263,7 +263,7 @@ public class TestBoundedInMemoryQueue {
|
||||
public void testException() throws Exception {
|
||||
final int numRecords = 256;
|
||||
final List<HoodieRecord> hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords);
|
||||
final SizeEstimator<Tuple2<HoodieRecord, Optional<IndexedRecord>>> sizeEstimator =
|
||||
final SizeEstimator<Tuple2<HoodieRecord, Option<IndexedRecord>>> sizeEstimator =
|
||||
new DefaultSizeEstimator<>();
|
||||
// queue memory limit
|
||||
HoodieInsertValueGenResult<HoodieRecord> payload = getTransformFunction(HoodieTestDataGenerator.avroSchema)
|
||||
@@ -274,7 +274,7 @@ public class TestBoundedInMemoryQueue {
|
||||
// first let us throw exception from queueIterator reader and test that queueing thread
|
||||
// stops and throws
|
||||
// correct exception back.
|
||||
BoundedInMemoryQueue<HoodieRecord, Tuple2<HoodieRecord, Optional<IndexedRecord>>> queue1 =
|
||||
BoundedInMemoryQueue<HoodieRecord, Tuple2<HoodieRecord, Option<IndexedRecord>>> queue1 =
|
||||
new BoundedInMemoryQueue(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.avroSchema));
|
||||
|
||||
// Produce
|
||||
@@ -305,7 +305,7 @@ public class TestBoundedInMemoryQueue {
|
||||
final Iterator<HoodieRecord> mockHoodieRecordsIterator = mock(Iterator.class);
|
||||
when(mockHoodieRecordsIterator.hasNext()).thenReturn(true);
|
||||
when(mockHoodieRecordsIterator.next()).thenThrow(expectedException);
|
||||
BoundedInMemoryQueue<HoodieRecord, Tuple2<HoodieRecord, Optional<IndexedRecord>>> queue2 =
|
||||
BoundedInMemoryQueue<HoodieRecord, Tuple2<HoodieRecord, Option<IndexedRecord>>> queue2 =
|
||||
new BoundedInMemoryQueue(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.avroSchema));
|
||||
|
||||
// Produce
|
||||
|
||||
@@ -21,6 +21,7 @@ package com.uber.hoodie.index;
|
||||
import com.uber.hoodie.common.HoodieClientTestUtils;
|
||||
import com.uber.hoodie.common.HoodieTestDataGenerator;
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
import com.uber.hoodie.common.util.Option;
|
||||
import com.uber.hoodie.config.HoodieCompactionConfig;
|
||||
import com.uber.hoodie.config.HoodieHBaseIndexConfig;
|
||||
import com.uber.hoodie.config.HoodieIndexConfig;
|
||||
@@ -30,7 +31,6 @@ import com.uber.hoodie.index.hbase.DefaultHBaseQPSResourceAllocator;
|
||||
import com.uber.hoodie.index.hbase.HBaseIndex;
|
||||
import com.uber.hoodie.index.hbase.HBaseIndexQPSResourceAllocator;
|
||||
import java.io.File;
|
||||
import java.util.Optional;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
@@ -84,7 +84,7 @@ public class TestHBaseQPSResourceAllocator {
|
||||
|
||||
@Test
|
||||
public void testsDefaultQPSResourceAllocator() {
|
||||
HoodieWriteConfig config = getConfig(Optional.empty());
|
||||
HoodieWriteConfig config = getConfig(Option.empty());
|
||||
HBaseIndex index = new HBaseIndex(config);
|
||||
HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator = index.createQPSResourceAllocator(config);
|
||||
Assert.assertEquals(hBaseIndexQPSResourceAllocator.getClass().getName(),
|
||||
@@ -95,7 +95,7 @@ public class TestHBaseQPSResourceAllocator {
|
||||
|
||||
@Test
|
||||
public void testsExplicitDefaultQPSResourceAllocator() {
|
||||
HoodieWriteConfig config = getConfig(Optional.of(HoodieHBaseIndexConfig.DEFAULT_HBASE_INDEX_QPS_ALLOCATOR_CLASS));
|
||||
HoodieWriteConfig config = getConfig(Option.of(HoodieHBaseIndexConfig.DEFAULT_HBASE_INDEX_QPS_ALLOCATOR_CLASS));
|
||||
HBaseIndex index = new HBaseIndex(config);
|
||||
HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator = index.createQPSResourceAllocator(config);
|
||||
Assert.assertEquals(hBaseIndexQPSResourceAllocator.getClass().getName(),
|
||||
@@ -106,7 +106,7 @@ public class TestHBaseQPSResourceAllocator {
|
||||
|
||||
@Test
|
||||
public void testsInvalidQPSResourceAllocator() {
|
||||
HoodieWriteConfig config = getConfig(Optional.of("InvalidResourceAllocatorClassName"));
|
||||
HoodieWriteConfig config = getConfig(Option.of("InvalidResourceAllocatorClassName"));
|
||||
HBaseIndex index = new HBaseIndex(config);
|
||||
HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator = index.createQPSResourceAllocator(config);
|
||||
Assert.assertEquals(hBaseIndexQPSResourceAllocator.getClass().getName(),
|
||||
@@ -115,7 +115,7 @@ public class TestHBaseQPSResourceAllocator {
|
||||
hBaseIndexQPSResourceAllocator.acquireQPSResources(config.getHbaseIndexQPSFraction(), 100), 0.0f);
|
||||
}
|
||||
|
||||
private HoodieWriteConfig getConfig(Optional<String> resourceAllocatorClass) {
|
||||
private HoodieWriteConfig getConfig(Option<String> resourceAllocatorClass) {
|
||||
HoodieHBaseIndexConfig hoodieHBaseIndexConfig = getConfigWithResourceAllocator(resourceAllocatorClass);
|
||||
return getConfigBuilder(hoodieHBaseIndexConfig).build();
|
||||
}
|
||||
@@ -132,7 +132,7 @@ public class TestHBaseQPSResourceAllocator {
|
||||
.build());
|
||||
}
|
||||
|
||||
private HoodieHBaseIndexConfig getConfigWithResourceAllocator(Optional<String> resourceAllocatorClass) {
|
||||
private HoodieHBaseIndexConfig getConfigWithResourceAllocator(Option<String> resourceAllocatorClass) {
|
||||
HoodieHBaseIndexConfig.Builder builder =
|
||||
new HoodieHBaseIndexConfig.Builder()
|
||||
.hbaseZkPort(Integer.valueOf(hbaseConfig.get("hbase.zookeeper.property.clientPort")))
|
||||
|
||||
@@ -25,7 +25,6 @@ import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import com.google.common.base.Optional;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.uber.hoodie.common.BloomFilter;
|
||||
import com.uber.hoodie.common.HoodieClientTestUtils;
|
||||
@@ -36,6 +35,7 @@ import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||
import com.uber.hoodie.common.util.Option;
|
||||
import com.uber.hoodie.common.util.collection.Pair;
|
||||
import com.uber.hoodie.config.HoodieIndexConfig;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
@@ -419,11 +419,11 @@ public class TestHoodieBloomIndex {
|
||||
|
||||
// Let's tag
|
||||
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config);
|
||||
JavaPairRDD<HoodieKey, Optional<Pair<String, String>>> taggedRecordRDD = bloomIndex
|
||||
JavaPairRDD<HoodieKey, Option<Pair<String, String>>> taggedRecordRDD = bloomIndex
|
||||
.fetchRecordLocation(keysRDD, jsc, table);
|
||||
|
||||
// Should not find any files
|
||||
for (Tuple2<HoodieKey, Optional<Pair<String, String>>> record : taggedRecordRDD.collect()) {
|
||||
for (Tuple2<HoodieKey, Option<Pair<String, String>>> record : taggedRecordRDD.collect()) {
|
||||
assertTrue(!record._2.isPresent());
|
||||
}
|
||||
|
||||
@@ -441,7 +441,7 @@ public class TestHoodieBloomIndex {
|
||||
taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, jsc, table);
|
||||
|
||||
// Check results
|
||||
for (Tuple2<HoodieKey, Optional<Pair<String, String>>> record : taggedRecordRDD.collect()) {
|
||||
for (Tuple2<HoodieKey, Option<Pair<String, String>>> record : taggedRecordRDD.collect()) {
|
||||
if (record._1.getRecordKey().equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")) {
|
||||
assertTrue(record._2.isPresent());
|
||||
assertEquals(FSUtils.getFileId(filename1), record._2.get().getRight());
|
||||
|
||||
@@ -27,6 +27,7 @@ import com.google.common.collect.Maps;
|
||||
import com.uber.hoodie.avro.model.HoodieCompactionOperation;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
import com.uber.hoodie.common.util.Option;
|
||||
import com.uber.hoodie.common.util.collection.Pair;
|
||||
import com.uber.hoodie.config.HoodieCompactionConfig;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
@@ -41,7 +42,6 @@ import java.util.ArrayList;
|
||||
import java.util.Date;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Random;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.commons.lang3.time.DateUtils;
|
||||
@@ -259,7 +259,7 @@ public class TestHoodieCompactionStrategy {
|
||||
df.getPath(),
|
||||
df.getFileId(),
|
||||
partitionPath,
|
||||
config.getCompactionStrategy().captureMetrics(config, Optional.of(df), partitionPath, logFiles)));
|
||||
config.getCompactionStrategy().captureMetrics(config, Option.of(df), partitionPath, logFiles)));
|
||||
});
|
||||
return operations;
|
||||
}
|
||||
|
||||
@@ -36,6 +36,7 @@ import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.Option;
|
||||
import com.uber.hoodie.common.util.ParquetUtils;
|
||||
import com.uber.hoodie.common.util.collection.Pair;
|
||||
import com.uber.hoodie.config.HoodieCompactionConfig;
|
||||
@@ -63,7 +64,6 @@ import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
import scala.Option;
|
||||
import scala.Tuple2;
|
||||
|
||||
public class TestCopyOnWriteTable {
|
||||
@@ -418,7 +418,7 @@ public class TestCopyOnWriteTable {
|
||||
HoodieCopyOnWriteTable.UpsertPartitioner partitioner =
|
||||
(HoodieCopyOnWriteTable.UpsertPartitioner) table.getUpsertPartitioner(profile);
|
||||
assertEquals("Update record should have gone to the 1 update partiton", 0, partitioner.getPartition(
|
||||
new Tuple2<>(updateRecords.get(0).getKey(), Option.apply(updateRecords.get(0).getCurrentLocation()))));
|
||||
new Tuple2<>(updateRecords.get(0).getKey(), Option.ofNullable(updateRecords.get(0).getCurrentLocation()))));
|
||||
return partitioner;
|
||||
}
|
||||
|
||||
|
||||
@@ -49,6 +49,7 @@ import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant.State;
|
||||
import com.uber.hoodie.common.table.view.HoodieTableFileSystemView;
|
||||
import com.uber.hoodie.common.util.Option;
|
||||
import com.uber.hoodie.config.HoodieCompactionConfig;
|
||||
import com.uber.hoodie.config.HoodieIndexConfig;
|
||||
import com.uber.hoodie.config.HoodieStorageConfig;
|
||||
@@ -62,7 +63,6 @@ import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
@@ -182,11 +182,11 @@ public class TestMergeOnReadTable {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
|
||||
Optional<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
|
||||
Option<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
|
||||
assertTrue(deltaCommit.isPresent());
|
||||
assertEquals("Delta commit should be 001", "001", deltaCommit.get().getTimestamp());
|
||||
|
||||
Optional<HoodieInstant> commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
|
||||
Option<HoodieInstant> commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
|
||||
assertFalse(commit.isPresent());
|
||||
|
||||
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
|
||||
@@ -225,7 +225,7 @@ public class TestMergeOnReadTable {
|
||||
commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
|
||||
assertFalse(commit.isPresent());
|
||||
|
||||
String compactionCommitTime = client.scheduleCompaction(Optional.empty()).get().toString();
|
||||
String compactionCommitTime = client.scheduleCompaction(Option.empty()).get().toString();
|
||||
client.compact(compactionCommitTime);
|
||||
|
||||
allFiles = HoodieTestUtils.listAllDataFilesInPath(dfs, cfg.getBasePath());
|
||||
@@ -289,11 +289,11 @@ public class TestMergeOnReadTable {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
|
||||
Optional<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
|
||||
Option<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
|
||||
assertTrue(deltaCommit.isPresent());
|
||||
assertEquals("Delta commit should be 001", "001", deltaCommit.get().getTimestamp());
|
||||
|
||||
Optional<HoodieInstant> commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
|
||||
Option<HoodieInstant> commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
|
||||
assertFalse(commit.isPresent());
|
||||
|
||||
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
|
||||
@@ -373,7 +373,7 @@ public class TestMergeOnReadTable {
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
Optional<HoodieInstant> commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
|
||||
Option<HoodieInstant> commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
|
||||
assertTrue(commit.isPresent());
|
||||
assertEquals("commit should be 001", "001", commit.get().getTimestamp());
|
||||
|
||||
@@ -436,11 +436,11 @@ public class TestMergeOnReadTable {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
|
||||
Optional<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
|
||||
Option<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
|
||||
assertTrue(deltaCommit.isPresent());
|
||||
assertEquals("Delta commit should be 001", "001", deltaCommit.get().getTimestamp());
|
||||
|
||||
Optional<HoodieInstant> commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
|
||||
Option<HoodieInstant> commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
|
||||
assertFalse(commit.isPresent());
|
||||
|
||||
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
|
||||
@@ -539,9 +539,9 @@ public class TestMergeOnReadTable {
|
||||
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
|
||||
String compactionInstantTime = client.scheduleCompaction(Optional.empty()).get().toString();
|
||||
String compactionInstantTime = client.scheduleCompaction(Option.empty()).get().toString();
|
||||
JavaRDD<WriteStatus> ws = client.compact(compactionInstantTime);
|
||||
client.commitCompaction(compactionInstantTime, ws, Optional.empty());
|
||||
client.commitCompaction(compactionInstantTime, ws, Option.empty());
|
||||
|
||||
allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
@@ -601,11 +601,11 @@ public class TestMergeOnReadTable {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
|
||||
Optional<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
|
||||
Option<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
|
||||
assertTrue(deltaCommit.isPresent());
|
||||
assertEquals("Delta commit should be 001", "001", deltaCommit.get().getTimestamp());
|
||||
|
||||
Optional<HoodieInstant> commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
|
||||
Option<HoodieInstant> commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
|
||||
assertFalse(commit.isPresent());
|
||||
|
||||
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
|
||||
@@ -666,7 +666,7 @@ public class TestMergeOnReadTable {
|
||||
|
||||
String compactionInstantTime = "004";
|
||||
allCommits.add(compactionInstantTime);
|
||||
client.scheduleCompactionAtInstant(compactionInstantTime, Optional.empty());
|
||||
client.scheduleCompactionAtInstant(compactionInstantTime, Option.empty());
|
||||
|
||||
// Compaction commit
|
||||
/**
|
||||
@@ -689,9 +689,9 @@ public class TestMergeOnReadTable {
|
||||
|
||||
compactionInstantTime = "006";
|
||||
allCommits.add(compactionInstantTime);
|
||||
client.scheduleCompactionAtInstant(compactionInstantTime, Optional.empty());
|
||||
client.scheduleCompactionAtInstant(compactionInstantTime, Option.empty());
|
||||
JavaRDD<WriteStatus> ws = client.compact(compactionInstantTime);
|
||||
client.commitCompaction(compactionInstantTime, ws, Optional.empty());
|
||||
client.commitCompaction(compactionInstantTime, ws, Option.empty());
|
||||
|
||||
allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
@@ -773,11 +773,11 @@ public class TestMergeOnReadTable {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
|
||||
Optional<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
|
||||
Option<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
|
||||
assertTrue(deltaCommit.isPresent());
|
||||
assertEquals("Delta commit should be 001", "001", deltaCommit.get().getTimestamp());
|
||||
|
||||
Optional<HoodieInstant> commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
|
||||
Option<HoodieInstant> commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
|
||||
assertFalse(commit.isPresent());
|
||||
|
||||
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
|
||||
@@ -879,10 +879,10 @@ public class TestMergeOnReadTable {
|
||||
|
||||
// Mark 2nd delta-instant as completed
|
||||
metaClient.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, newCommitTime), Optional.empty());
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, newCommitTime), Option.empty());
|
||||
|
||||
// Do a compaction
|
||||
String compactionInstantTime = writeClient.scheduleCompaction(Optional.empty()).get().toString();
|
||||
String compactionInstantTime = writeClient.scheduleCompaction(Option.empty()).get().toString();
|
||||
JavaRDD<WriteStatus> result = writeClient.compact(compactionInstantTime);
|
||||
|
||||
// Verify that recently written compacted data file has no log file
|
||||
@@ -941,9 +941,9 @@ public class TestMergeOnReadTable {
|
||||
Assert.assertTrue(totalUpsertTime > 0);
|
||||
|
||||
// Do a compaction
|
||||
String compactionInstantTime = writeClient.scheduleCompaction(Optional.empty()).get().toString();
|
||||
String compactionInstantTime = writeClient.scheduleCompaction(Option.empty()).get().toString();
|
||||
statuses = writeClient.compact(compactionInstantTime);
|
||||
writeClient.commitCompaction(compactionInstantTime, statuses, Optional.empty());
|
||||
writeClient.commitCompaction(compactionInstantTime, statuses, Option.empty());
|
||||
// total time taken for scanning log files should be greater than 0
|
||||
long timeTakenForScanner = statuses.map(writeStatus -> writeStatus.getStat().getRuntimeStats().getTotalScanTime())
|
||||
.reduce((a, b) -> a + b).longValue();
|
||||
@@ -982,11 +982,11 @@ public class TestMergeOnReadTable {
|
||||
|
||||
Assert.assertTrue(numLogFiles > 0);
|
||||
// Do a compaction
|
||||
String commitTime = writeClient.scheduleCompaction(Optional.empty()).get().toString();
|
||||
String commitTime = writeClient.scheduleCompaction(Option.empty()).get().toString();
|
||||
statuses = writeClient.compact(commitTime);
|
||||
Assert.assertTrue(statuses.map(status -> status.getStat().getPath().contains("parquet")).count() == numLogFiles);
|
||||
Assert.assertEquals(statuses.count(), numLogFiles);
|
||||
writeClient.commitCompaction(commitTime, statuses, Optional.empty());
|
||||
writeClient.commitCompaction(commitTime, statuses, Option.empty());
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -1077,12 +1077,12 @@ public class TestMergeOnReadTable {
|
||||
|
||||
Assert.assertTrue(numLogFiles > 0);
|
||||
// Do a compaction
|
||||
newCommitTime = writeClient.scheduleCompaction(Optional.empty()).get().toString();
|
||||
newCommitTime = writeClient.scheduleCompaction(Option.empty()).get().toString();
|
||||
statuses = writeClient.compact(newCommitTime);
|
||||
// Ensure all log files have been compacted into parquet files
|
||||
Assert.assertTrue(statuses.map(status -> status.getStat().getPath().contains("parquet")).count() == numLogFiles);
|
||||
Assert.assertEquals(statuses.count(), numLogFiles);
|
||||
writeClient.commitCompaction(newCommitTime, statuses, Optional.empty());
|
||||
writeClient.commitCompaction(newCommitTime, statuses, Option.empty());
|
||||
// Trigger a rollback of compaction
|
||||
writeClient.rollback(newCommitTime);
|
||||
table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath), config, jsc);
|
||||
@@ -1112,7 +1112,7 @@ public class TestMergeOnReadTable {
|
||||
String commitActionType = table.getMetaClient().getCommitActionType();
|
||||
HoodieInstant instant = new HoodieInstant(true, commitActionType, "000");
|
||||
activeTimeline.createInflight(instant);
|
||||
activeTimeline.saveAsComplete(instant, Optional.empty());
|
||||
activeTimeline.saveAsComplete(instant, Option.empty());
|
||||
|
||||
String commitTime = "001";
|
||||
client.startCommitWithTime(commitTime);
|
||||
@@ -1259,9 +1259,9 @@ public class TestMergeOnReadTable {
|
||||
|
||||
// Test small file handling after compaction
|
||||
commitTime = "002";
|
||||
client.scheduleCompactionAtInstant(commitTime, Optional.of(metadata.getExtraMetadata()));
|
||||
client.scheduleCompactionAtInstant(commitTime, Option.of(metadata.getExtraMetadata()));
|
||||
statuses = client.compact(commitTime);
|
||||
client.commitCompaction(commitTime, statuses, Optional.empty());
|
||||
client.commitCompaction(commitTime, statuses, Option.empty());
|
||||
|
||||
// Read from commit file
|
||||
table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
|
||||
Reference in New Issue
Block a user