1
0

Adding range based pruning to bloom index

- keys compared lexicographically using String::compareTo
 - Range metadata additionally written into parquet file footers
 - Trim fat & few optimizations to speed up indexing
 - Add param to control whether input shall be cached, to speed up lookup
 - Add param to turn on/off range pruning
 - Auto compute of parallelism now simply factors in amount of comparisons done
 - More accurate parallelism computation when range pruning is on
 - tests added & hardened, docs updated
This commit is contained in:
Vinoth Chandar
2017-07-14 09:29:16 -07:00
committed by prazanna
parent 0b26b60a5c
commit 86209640f7
25 changed files with 784 additions and 473 deletions

View File

@@ -91,8 +91,7 @@ public class TestHoodieClient implements Serializable {
@Before
public void init() throws IOException {
// Initialize a local spark env
SparkConf sparkConf = new SparkConf().setAppName("TestHoodieClient").setMaster("local[4]");
jsc = new JavaSparkContext(HoodieReadClient.addHoodieSupport(sparkConf));
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieClient"));
//SQLContext stuff
sqlContext = new SQLContext(jsc);

View File

@@ -16,6 +16,7 @@
package com.uber.hoodie;
import com.uber.hoodie.common.HoodieClientTestUtils;
import com.uber.hoodie.common.HoodieMergeOnReadTestUtils;
import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.minicluster.HdfsTestService;
@@ -30,32 +31,21 @@ import com.uber.hoodie.common.table.TableFileSystemView;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.table.view.HoodieTableFileSystemView;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.config.HoodieCompactionConfig;
import com.uber.hoodie.config.HoodieIndexConfig;
import com.uber.hoodie.config.HoodieStorageConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.hadoop.realtime.HoodieRealtimeInputFormat;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.io.compact.HoodieCompactor;
import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor;
import com.uber.hoodie.table.HoodieTable;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.GenericRecordBuilder;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
import org.apache.hadoop.io.ArrayWritable;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.RecordReader;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.SQLContext;
@@ -68,8 +58,6 @@ import org.junit.rules.TemporaryFolder;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -77,7 +65,6 @@ import java.util.Optional;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static com.uber.hoodie.common.HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat;
import static com.uber.hoodie.common.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
@@ -126,10 +113,7 @@ public class TestMergeOnReadTable {
this.fs = FSUtils.getFs();
// Initialize a local spark env
SparkConf sparkConf =
new SparkConf().set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
.setAppName("TestHoodieCompactor").setMaster("local[1]");
jsc = new JavaSparkContext(HoodieReadClient.addHoodieSupport(sparkConf));
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieMergeOnReadTable"));
jsc.hadoopConfiguration().addResource(FSUtils.getFs().getConf());
// Create a temp folder as the base path

View File

@@ -16,12 +16,15 @@
package com.uber.hoodie.common;
import com.uber.hoodie.HoodieReadClient;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.util.FSUtils;
import org.apache.spark.SparkConf;
import java.io.File;
import java.io.IOException;
import java.io.RandomAccessFile;
@@ -79,4 +82,12 @@ public class HoodieClientTestUtils {
new File(path).createNewFile();
new RandomAccessFile(path, "rw").setLength(length);
}
public static SparkConf getSparkConfForTest(String appName) {
SparkConf sparkConf = new SparkConf()
.setAppName(appName)
.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
.setMaster("local[4]");
return HoodieReadClient.addHoodieSupport(sparkConf);
}
}

View File

@@ -19,6 +19,9 @@ package com.uber.hoodie.index;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.config.HoodieIndexConfig;
import com.uber.hoodie.index.bloom.HoodieBloomIndex;
import com.uber.hoodie.index.hbase.HBaseIndex;
import org.junit.Test;
import static org.junit.Assert.*;

View File

@@ -1,25 +1,29 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
* Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*
*
*/
package com.uber.hoodie.index;
package com.uber.hoodie.index.bloom;
import com.google.common.base.Optional;
import com.google.common.collect.Lists;
import com.uber.hoodie.common.HoodieClientTestUtils;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.config.HoodieIndexConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.avro.HoodieAvroWriteSupport;
import com.uber.hoodie.common.BloomFilter;
@@ -30,14 +34,22 @@ import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.index.bloom.BloomIndexFileInfo;
import com.uber.hoodie.index.bloom.HoodieBloomIndex;
import com.uber.hoodie.index.bloom.HoodieBloomIndexCheckFunction;
import com.uber.hoodie.io.storage.HoodieParquetConfig;
import com.uber.hoodie.io.storage.HoodieParquetWriter;
import com.uber.hoodie.table.HoodieTable;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.parquet.avro.AvroSchemaConverter;
import org.apache.parquet.avro.AvroWriteSupport;
import org.apache.parquet.hadoop.ParquetWriter;
import org.apache.parquet.hadoop.api.WriteSupport;
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaPairRDD;
@@ -47,6 +59,7 @@ import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import org.mockito.Mockito;
import scala.Tuple2;
@@ -54,6 +67,8 @@ import java.io.File;
import java.io.IOException;
import java.text.SimpleDateFormat;
import java.util.*;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.junit.Assert.*;
@@ -61,6 +76,8 @@ public class TestHoodieBloomIndex {
private JavaSparkContext jsc = null;
private String basePath = null;
private transient final FileSystem fs;
private String schemaStr;
private Schema schema;
public TestHoodieBloomIndex() throws Exception {
fs = FSUtils.getFs();
@@ -69,13 +86,15 @@ public class TestHoodieBloomIndex {
@Before
public void init() throws IOException {
// Initialize a local spark env
SparkConf sparkConf = new SparkConf().setAppName("TestHoodieBloomIndex").setMaster("local[4]");
jsc = new JavaSparkContext(sparkConf);
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieBloomIndex"));
// Create a temp folder as the base path
TemporaryFolder folder = new TemporaryFolder();
folder.create();
basePath = folder.getRoot().getAbsolutePath();
HoodieTestUtils.init(basePath);
// We have some records to be tagged (two different partitions)
schemaStr = IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8");
schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(schemaStr));
}
@Test
@@ -86,8 +105,6 @@ public class TestHoodieBloomIndex {
String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}";
String schemaStr =
IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8");
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
HoodieRecord record1 = new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
@@ -101,8 +118,10 @@ public class TestHoodieBloomIndex {
// Load to memory
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieBloomIndex index = new HoodieBloomIndex(config, jsc);
Map<String, Iterable<String>> map = index.getPartitionToRowKeys(recordRDD);
Map<String, Iterable<String>> map = recordRDD
.mapToPair(record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey()))
.groupByKey().collectAsMap();
assertEquals(map.size(), 2);
List<String> list1 = Lists.newArrayList(map.get("2016/01/31"));
List<String> list2 = Lists.newArrayList(map.get("2015/01/31"));
@@ -112,7 +131,9 @@ public class TestHoodieBloomIndex {
@Test
public void testLoadInvolvedFiles() throws IOException {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieWriteConfig config = HoodieWriteConfig.newBuilder()
.withPath(basePath)
.build();
HoodieBloomIndex index = new HoodieBloomIndex(config, jsc);
// Create some partitions, and put some files
@@ -122,51 +143,105 @@ public class TestHoodieBloomIndex {
new File(basePath + "/2016/01/21").mkdirs();
new File(basePath + "/2016/04/01").mkdirs();
new File(basePath + "/2015/03/12").mkdirs();
new File(basePath + "/2016/04/01/2_0_20160401010101.parquet").createNewFile();
new File(basePath + "/2015/03/12/1_0_20150312101010.parquet").createNewFile();
new File(basePath + "/2015/03/12/3_0_20150312101010.parquet").createNewFile();
new File(basePath + "/2015/03/12/4_0_20150312101010.parquet").createNewFile();
TestRawTripPayload rowChange1 = new TestRawTripPayload("{\"_row_key\":\"000\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
HoodieRecord record1 = new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
TestRawTripPayload rowChange2 = new TestRawTripPayload("{\"_row_key\":\"001\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
HoodieRecord record2 = new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
TestRawTripPayload rowChange3 = new TestRawTripPayload("{\"_row_key\":\"002\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
HoodieRecord record3 = new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3);
TestRawTripPayload rowChange4 = new TestRawTripPayload("{\"_row_key\":\"003\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
HoodieRecord record4 = new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
writeParquetFile("2016/04/01","2_0_20160401010101.parquet", Lists.newArrayList(), schema, null, false);
writeParquetFile("2015/03/12","1_0_20150312101010.parquet", Lists.newArrayList(), schema, null, false);
writeParquetFile("2015/03/12","3_0_20150312101010.parquet", Arrays.asList(record1), schema, null, false);
writeParquetFile("2015/03/12","4_0_20150312101010.parquet", Arrays.asList(record2, record3, record4), schema, null, false);
List<String> partitions = Arrays.asList("2016/01/21", "2016/04/01", "2015/03/12");
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
JavaPairRDD<String, String> rdd = index.loadInvolvedFiles(partitions, table);
List<Tuple2<String, BloomIndexFileInfo>> filesList = index.loadInvolvedFiles(partitions, table);
// Still 0, as no valid commit
assertEquals(rdd.count(), 0);
assertEquals(filesList.size(), 0);
// Add some commits
new File(basePath + "/.hoodie").mkdirs();
new File(basePath + "/.hoodie/20160401010101.commit").createNewFile();
new File(basePath + "/.hoodie/20150312101010.commit").createNewFile();
metadata = new HoodieTableMetaClient(fs, basePath);
rdd = index.loadInvolvedFiles(partitions, table);
final List<Tuple2<String, String>> filesList = rdd.collect();
filesList = index.loadInvolvedFiles(partitions, table);
assertEquals(filesList.size(), 4);
// these files will not have the key ranges
assertNull(filesList.get(0)._2().getMaxRecordKey());
assertNull(filesList.get(0)._2().getMinRecordKey());
assertFalse(filesList.get(1)._2().hasKeyRanges());
assertNotNull(filesList.get(2)._2().getMaxRecordKey());
assertNotNull(filesList.get(2)._2().getMinRecordKey());
assertTrue(filesList.get(3)._2().hasKeyRanges());
// no longer sorted, but should have same files.
Set<String> actualFiles = new HashSet<String>(){{
add(filesList.get(0)._1 + "/" + filesList.get(0)._2);
add(filesList.get(1)._1 + "/" + filesList.get(1)._2);
add(filesList.get(2)._1 + "/" + filesList.get(2)._2);
add(filesList.get(3)._1 + "/" + filesList.get(3)._2);
}};
Set<String> expected = new HashSet<String>() {{
add("2016/04/01/2_0_20160401010101.parquet");
add("2015/03/12/1_0_20150312101010.parquet");
add("2015/03/12/3_0_20150312101010.parquet");
add("2015/03/12/4_0_20150312101010.parquet");
}};
assertEquals(expected, actualFiles);
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"))
);
assertEquals(expected, filesList);
}
@Test
public void testRangePruning() {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder()
.withPath(basePath)
.build();
HoodieBloomIndex index = new HoodieBloomIndex(config, jsc);
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo = new HashMap<>();
partitionToFileIndexInfo.put("2017/10/22", Arrays.asList(
new BloomIndexFileInfo("f1"),
new BloomIndexFileInfo("f2", "000", "000"),
new BloomIndexFileInfo("f3", "001", "003"),
new BloomIndexFileInfo("f4", "002", "007"),
new BloomIndexFileInfo("f5", "009", "010")
));
JavaPairRDD<String, String> partitionRecordKeyPairRDD = jsc
.parallelize(Arrays.asList(
new Tuple2<>("2017/10/22","003"),
new Tuple2<>("2017/10/22","002"),
new Tuple2<>("2017/10/22","005"),
new Tuple2<>("2017/10/22","004")
))
.mapToPair(t -> t);
List<Tuple2<String, Tuple2<String, HoodieKey>>> comparisonKeyList = index
.explodeRecordRDDWithFileComparisons(partitionToFileIndexInfo, partitionRecordKeyPairRDD)
.collect();
assertEquals(10, comparisonKeyList.size());
Map<String, List<String>> recordKeyToFileComps = comparisonKeyList.stream()
.collect(Collectors.groupingBy(
t -> t._2()._2().getRecordKey(),
Collectors.mapping(t -> t._2()._1().split("#")[0], Collectors.toList()
)
));
assertEquals(4, recordKeyToFileComps.size());
assertEquals(Arrays.asList("f1", "f3", "f4"), recordKeyToFileComps.get("002"));
assertEquals(Arrays.asList("f1", "f3", "f4"), recordKeyToFileComps.get("003"));
assertEquals(Arrays.asList("f1", "f4"), recordKeyToFileComps.get("004"));
assertEquals(Arrays.asList("f1", "f4"), recordKeyToFileComps.get("005"));
}
@Test
public void testCheckUUIDsAgainstOneFile() throws IOException, InterruptedException, ClassNotFoundException {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieBloomIndex index = new HoodieBloomIndex(config, jsc);
String schemaStr =
IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8");
Schema schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(schemaStr));
// Create some records to use
String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
@@ -232,9 +307,6 @@ public class TestHoodieBloomIndex {
@Test
public void testTagLocation() throws Exception {
// We have some records to be tagged (two different partitions)
String schemaStr =
IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8");
Schema schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(schemaStr));
String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
@@ -292,9 +364,6 @@ public class TestHoodieBloomIndex {
@Test
public void testCheckExists() throws Exception {
// We have some records to be tagged (two different partitions)
String schemaStr =
IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8");
Schema schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(schemaStr));
String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
@@ -366,10 +435,6 @@ public class TestHoodieBloomIndex {
String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
// We write record1 to a parquet file, using a bloom filter having both records
String schemaStr =
IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8");
Schema schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(schemaStr));
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
HoodieRecord record1 = new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
@@ -407,24 +472,32 @@ public class TestHoodieBloomIndex {
String fileId = UUID.randomUUID().toString();
String filename = FSUtils.makeDataFileName(commitTime, 1, fileId);
return writeParquetFile(partitionPath, filename, records, schema, filter, createCommitTime);
}
private String writeParquetFile(String partitionPath, String filename, List<HoodieRecord> records, Schema schema,
BloomFilter filter, boolean createCommitTime) throws IOException {
if (filter == null) {
filter = new BloomFilter(10000, 0.0000001);
}
HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter);
ParquetWriter writer = new ParquetWriter(new Path(basePath + "/" + partitionPath + "/" + filename),
writeSupport, CompressionCodecName.GZIP, 120 * 1024 * 1024, ParquetWriter.DEFAULT_PAGE_SIZE);
int seqId = 1;
String commitTime = FSUtils.getCommitTime(filename);
HoodieParquetConfig config = new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP,
ParquetWriter.DEFAULT_BLOCK_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, 120 * 1024 * 1024, new Configuration());
HoodieParquetWriter writer = new HoodieParquetWriter(
commitTime,
new Path(basePath + "/" + partitionPath + "/" + filename),
config,
schema);
int seqId = 1;
for (HoodieRecord record : records) {
GenericRecord avroRecord = (GenericRecord) record.getData().getInsertValue(schema).get();
HoodieAvroUtils.addCommitMetadataToRecord(avroRecord, commitTime, "" + seqId++);
HoodieAvroUtils.addHoodieKeyToRecord(avroRecord, record.getRecordKey(), record.getPartitionPath(), filename);
writer.write(avroRecord);
writer.writeAvro(record.getRecordKey(), avroRecord);
filter.add(record.getRecordKey());
}
writer.close();

View File

@@ -19,23 +19,22 @@ package com.uber.hoodie.io;
import com.uber.hoodie.HoodieReadClient;
import com.uber.hoodie.HoodieWriteClient;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.HoodieClientTestUtils;
import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.model.FileSlice;
import com.uber.hoodie.common.model.HoodieCompactionMetadata;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieTableType;
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.model.HoodieLogFile;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.config.HoodieCompactionConfig;
import com.uber.hoodie.config.HoodieIndexConfig;
import com.uber.hoodie.config.HoodieStorageConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.index.HoodieBloomIndex;
import com.uber.hoodie.index.bloom.HoodieBloomIndex;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.io.compact.HoodieCompactor;
import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor;
@@ -52,9 +51,7 @@ import org.junit.rules.TemporaryFolder;
import java.io.File;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
@@ -68,9 +65,7 @@ public class TestHoodieCompactor {
@Before
public void init() throws IOException {
// Initialize a local spark env
SparkConf sparkConf =
new SparkConf().setAppName("TestHoodieCompactor").setMaster("local[4]");
jsc = new JavaSparkContext(HoodieReadClient.addHoodieSupport(sparkConf));
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieCompactor"));
// Create a temp folder as the base path
TemporaryFolder folder = new TemporaryFolder();

View File

@@ -69,8 +69,7 @@ public class TestCopyOnWriteTable {
public void init() throws Exception {
// Initialize a local spark env
SparkConf sparkConf = new SparkConf().setAppName("TestCopyOnWriteTable").setMaster("local[4]");
jsc = new JavaSparkContext(sparkConf);
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestCopyOnWriteTable"));
// Create a temp folder as the base path
TemporaryFolder folder = new TemporaryFolder();