Introduce HoodieReadHandle abstraction into index
- Generalized BloomIndex to work with file ids instead of paths - Abstracted away Bloom filter checking into HoodieLookupHandle - Abstracted away range information retrieval into HoodieRangeInfoHandle
This commit is contained in:
committed by
vinoth chandar
parent
51d122b5c3
commit
b791473a6d
@@ -217,7 +217,7 @@ public class TestHoodieClientBase implements Serializable {
|
||||
for (HoodieRecord rec : taggedRecords) {
|
||||
assertTrue("Record " + rec + " found with no location.", rec.isCurrentLocationKnown());
|
||||
assertEquals("All records should have commit time " + commitTime + ", since updates were made",
|
||||
rec.getCurrentLocation().getCommitTime(), commitTime);
|
||||
rec.getCurrentLocation().getInstantTime(), commitTime);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -166,7 +166,7 @@ public class TestHbaseIndex {
|
||||
assertTrue(javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 200);
|
||||
assertTrue(javaRDD.map(record -> record.getKey().getRecordKey()).distinct().count() == 200);
|
||||
assertTrue(javaRDD.filter(
|
||||
record -> (record.getCurrentLocation() != null && record.getCurrentLocation().getCommitTime()
|
||||
record -> (record.getCurrentLocation() != null && record.getCurrentLocation().getInstantTime()
|
||||
.equals(newCommitTime))).distinct().count() == 200);
|
||||
|
||||
}
|
||||
|
||||
@@ -36,8 +36,10 @@ 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.collection.Pair;
|
||||
import com.uber.hoodie.config.HoodieIndexConfig;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.io.HoodieKeyLookupHandle;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
@@ -200,10 +202,10 @@ public class TestHoodieBloomIndex {
|
||||
// no longer sorted, but should have same files.
|
||||
|
||||
List<Tuple2<String, BloomIndexFileInfo>> expected = Arrays.asList(
|
||||
new Tuple2<>("2016/04/01", new BloomIndexFileInfo("2_0_20160401010101.parquet")),
|
||||
new Tuple2<>("2015/03/12", new BloomIndexFileInfo("1_0_20150312101010.parquet")),
|
||||
new Tuple2<>("2015/03/12", new BloomIndexFileInfo("3_0_20150312101010.parquet", "000", "000")),
|
||||
new Tuple2<>("2015/03/12", new BloomIndexFileInfo("4_0_20150312101010.parquet", "001", "003")));
|
||||
new Tuple2<>("2016/04/01", new BloomIndexFileInfo("2")),
|
||||
new Tuple2<>("2015/03/12", new BloomIndexFileInfo("1")),
|
||||
new Tuple2<>("2015/03/12", new BloomIndexFileInfo("3", "000", "000")),
|
||||
new Tuple2<>("2015/03/12", new BloomIndexFileInfo("4", "001", "003")));
|
||||
assertEquals(expected, filesList);
|
||||
}
|
||||
}
|
||||
@@ -279,7 +281,7 @@ public class TestHoodieBloomIndex {
|
||||
List<String> uuids = Arrays.asList(record1.getRecordKey(), record2.getRecordKey(), record3.getRecordKey(),
|
||||
record4.getRecordKey());
|
||||
|
||||
List<String> results = HoodieBloomIndexCheckFunction.checkCandidatesAgainstFile(jsc.hadoopConfiguration(), uuids,
|
||||
List<String> results = HoodieKeyLookupHandle.checkCandidatesAgainstFile(jsc.hadoopConfiguration(), uuids,
|
||||
new Path(basePath + "/2016/01/31/" + filename));
|
||||
assertEquals(results.size(), 2);
|
||||
assertTrue(results.get(0).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0") || results.get(1).equals(
|
||||
@@ -417,10 +419,11 @@ public class TestHoodieBloomIndex {
|
||||
|
||||
// Let's tag
|
||||
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config);
|
||||
JavaPairRDD<HoodieKey, Optional<String>> taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, jsc, table);
|
||||
JavaPairRDD<HoodieKey, Optional<Pair<String, String>>> taggedRecordRDD = bloomIndex
|
||||
.fetchRecordLocation(keysRDD, jsc, table);
|
||||
|
||||
// Should not find any files
|
||||
for (Tuple2<HoodieKey, Optional<String>> record : taggedRecordRDD.collect()) {
|
||||
for (Tuple2<HoodieKey, Optional<Pair<String, String>>> record : taggedRecordRDD.collect()) {
|
||||
assertTrue(!record._2.isPresent());
|
||||
}
|
||||
|
||||
@@ -438,18 +441,16 @@ public class TestHoodieBloomIndex {
|
||||
taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, jsc, table);
|
||||
|
||||
// Check results
|
||||
for (Tuple2<HoodieKey, Optional<String>> record : taggedRecordRDD.collect()) {
|
||||
for (Tuple2<HoodieKey, Optional<Pair<String, String>>> record : taggedRecordRDD.collect()) {
|
||||
if (record._1.getRecordKey().equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")) {
|
||||
assertTrue(record._2.isPresent());
|
||||
Path path1 = new Path(record._2.get());
|
||||
assertEquals(FSUtils.getFileId(filename1), FSUtils.getFileId(path1.getName()));
|
||||
assertEquals(FSUtils.getFileId(filename1), record._2.get().getRight());
|
||||
} else if (record._1.getRecordKey().equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")) {
|
||||
assertTrue(record._2.isPresent());
|
||||
Path path2 = new Path(record._2.get());
|
||||
if (record._1.getPartitionPath().equals("2015/01/31")) {
|
||||
assertEquals(FSUtils.getFileId(filename3), FSUtils.getFileId(path2.getName()));
|
||||
assertEquals(FSUtils.getFileId(filename3), record._2.get().getRight());
|
||||
} else {
|
||||
assertEquals(FSUtils.getFileId(filename2), FSUtils.getFileId(path2.getName()));
|
||||
assertEquals(FSUtils.getFileId(filename2), record._2.get().getRight());
|
||||
}
|
||||
} else if (record._1.getRecordKey().equals("3eb5b87c-1fej-4edd-87b4-6ec96dc405a0")) {
|
||||
assertTrue(!record._2.isPresent());
|
||||
|
||||
@@ -18,7 +18,11 @@
|
||||
|
||||
package com.uber.hoodie.index.bloom;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.uber.hoodie.common.HoodieClientTestUtils;
|
||||
@@ -32,16 +36,17 @@ import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.*;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
@@ -154,20 +159,20 @@ public class TestHoodieGlobalBloomIndex {
|
||||
|
||||
Map<String, BloomIndexFileInfo> filesMap = toFileMap(filesList);
|
||||
// key ranges checks
|
||||
assertNull(filesMap.get("2016/04/01/2_0_20160401010101.parquet").getMaxRecordKey());
|
||||
assertNull(filesMap.get("2016/04/01/2_0_20160401010101.parquet").getMinRecordKey());
|
||||
assertFalse(filesMap.get("2015/03/12/1_0_20150312101010.parquet").hasKeyRanges());
|
||||
assertNotNull(filesMap.get("2015/03/12/3_0_20150312101010.parquet").getMaxRecordKey());
|
||||
assertNotNull(filesMap.get("2015/03/12/3_0_20150312101010.parquet").getMinRecordKey());
|
||||
assertTrue(filesMap.get("2015/03/12/3_0_20150312101010.parquet").hasKeyRanges());
|
||||
assertNull(filesMap.get("2016/04/01/2").getMaxRecordKey());
|
||||
assertNull(filesMap.get("2016/04/01/2").getMinRecordKey());
|
||||
assertFalse(filesMap.get("2015/03/12/1").hasKeyRanges());
|
||||
assertNotNull(filesMap.get("2015/03/12/3").getMaxRecordKey());
|
||||
assertNotNull(filesMap.get("2015/03/12/3").getMinRecordKey());
|
||||
assertTrue(filesMap.get("2015/03/12/3").hasKeyRanges());
|
||||
|
||||
Map<String, BloomIndexFileInfo> expected = new HashMap<>();
|
||||
expected.put("2016/04/01/2_0_20160401010101.parquet", new BloomIndexFileInfo("2_0_20160401010101.parquet"));
|
||||
expected.put("2015/03/12/1_0_20150312101010.parquet", new BloomIndexFileInfo("1_0_20150312101010.parquet"));
|
||||
expected.put("2015/03/12/3_0_20150312101010.parquet",
|
||||
new BloomIndexFileInfo("3_0_20150312101010.parquet", "000", "000"));
|
||||
expected.put("2015/03/12/4_0_20150312101010.parquet",
|
||||
new BloomIndexFileInfo("4_0_20150312101010.parquet", "001", "003"));
|
||||
expected.put("2016/04/01/2", new BloomIndexFileInfo("2"));
|
||||
expected.put("2015/03/12/1", new BloomIndexFileInfo("1"));
|
||||
expected.put("2015/03/12/3",
|
||||
new BloomIndexFileInfo("3", "000", "000"));
|
||||
expected.put("2015/03/12/4",
|
||||
new BloomIndexFileInfo("4", "001", "003"));
|
||||
|
||||
assertEquals(expected, filesMap);
|
||||
}
|
||||
@@ -300,7 +305,7 @@ public class TestHoodieGlobalBloomIndex {
|
||||
private Map<String, BloomIndexFileInfo> toFileMap(List<Tuple2<String, BloomIndexFileInfo>> filesList) {
|
||||
Map<String, BloomIndexFileInfo> filesMap = new HashMap<>();
|
||||
for (Tuple2<String, BloomIndexFileInfo> t : filesList) {
|
||||
filesMap.put(t._1() + "/" + t._2().getFileName(), t._2());
|
||||
filesMap.put(t._1() + "/" + t._2().getFileId(), t._2());
|
||||
}
|
||||
return filesMap;
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user