1
0

Reworking the deltastreamer tool

- Standardize version of jackson
 - DFSPropertiesConfiguration replaces usage of commons PropertiesConfiguration
 - Remove dependency on ConstructorUtils
 - Throw error if ordering value is not present, during key generation
 - Switch to shade plugin for hoodie-utilities
 - Added support for consumption for Confluent avro kafka serdes
 - Support for Confluent schema registry
 - KafkaSource now deals with skews nicely, by doing round robin allocation of source limit across partitions
 - Added support for BULK_INSERT operations as well
 - Pass in the payload class config properly into HoodieWriteClient
 - Fix documentation based on new usage
 - Adding tests on deltastreamer, sources and all new util classes.
This commit is contained in:
Vinoth Chandar
2018-08-04 03:35:30 -07:00
committed by vinoth chandar
parent fb95dbdedb
commit d58ddbd999
49 changed files with 1919 additions and 754 deletions

View File

@@ -0,0 +1,162 @@
/*
* 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
*
* 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.utilities;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
import com.uber.hoodie.common.model.HoodieCommitMetadata;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.DFSPropertiesConfiguration;
import com.uber.hoodie.common.util.TypedProperties;
import com.uber.hoodie.exception.DatasetNotFoundException;
import com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer;
import com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer.Operation;
import com.uber.hoodie.utilities.sources.TestDataSource;
import java.io.IOException;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.sql.SQLContext;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
/**
* Basic tests against {@link com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer}, by issuing bulk_inserts,
* upserts, inserts. Check counts at the end.
*/
public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
private static volatile Logger log = LogManager.getLogger(TestHoodieDeltaStreamer.class);
@BeforeClass
public static void initClass() throws Exception {
UtilitiesTestBase.initClass();
// prepare the configs.
UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/base.properties", dfs, dfsBasePath + "/base.properties");
UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/source.avsc", dfs, dfsBasePath + "/source.avsc");
TypedProperties props = new TypedProperties();
props.setProperty("include", "base.properties");
props.setProperty("hoodie.datasource.write.recordkey.field", "_row_key");
props.setProperty("hoodie.datasource.write.partitionpath.field", "not_there");
props.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", dfsBasePath + "/source.avsc");
UtilitiesTestBase.Helpers.savePropsToDFS(props, dfs, dfsBasePath + "/test-source.properties");
}
@AfterClass
public static void cleanupClass() throws Exception {
UtilitiesTestBase.cleanupClass();
}
@Before
public void setup() throws Exception {
super.setup();
TestDataSource.initDataGen();
}
@After
public void teardown() throws Exception {
super.teardown();
TestDataSource.resetDataGen();
}
static class TestHelpers {
static HoodieDeltaStreamer.Config makeConfig(String basePath, Operation op) {
HoodieDeltaStreamer.Config cfg = new HoodieDeltaStreamer.Config();
cfg.targetBasePath = basePath;
cfg.targetTableName = "hoodie_trips";
cfg.sourceClassName = TestDataSource.class.getName();
cfg.operation = op;
cfg.sourceOrderingField = "timestamp";
cfg.propsFilePath = dfsBasePath + "/test-source.properties";
cfg.sourceLimit = 1000;
return cfg;
}
static void assertRecordCount(long expected, String datasetPath, SQLContext sqlContext) {
long recordCount = sqlContext.read().format("com.uber.hoodie").load(datasetPath).count();
assertEquals(expected, recordCount);
}
static void assertCommitMetadata(String expected, String datasetPath, FileSystem fs, int totalCommits)
throws IOException {
HoodieTableMetaClient meta = new HoodieTableMetaClient(fs.getConf(), datasetPath);
HoodieTimeline timeline = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
HoodieInstant lastCommit = timeline.lastInstant().get();
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(
timeline.getInstantDetails(lastCommit).get(), HoodieCommitMetadata.class);
assertEquals(totalCommits, timeline.countInstants());
assertEquals(expected, commitMetadata.getMetadata(HoodieDeltaStreamer.CHECKPOINT_KEY));
}
}
@Test
public void testProps() throws IOException {
TypedProperties props = new DFSPropertiesConfiguration(dfs, new Path(dfsBasePath + "/test-source.properties"))
.getConfig();
assertEquals(2, props.getInteger("hoodie.upsert.shuffle.parallelism"));
assertEquals("_row_key", props.getString("hoodie.datasource.write.recordkey.field"));
}
@Test
public void testDatasetCreation() throws Exception {
try {
dfs.mkdirs(new Path(dfsBasePath + "/not_a_dataset"));
HoodieDeltaStreamer deltaStreamer = new HoodieDeltaStreamer(
TestHelpers.makeConfig(dfsBasePath + "/not_a_dataset", Operation.BULK_INSERT), jsc);
deltaStreamer.sync();
fail("Should error out when pointed out at a dir thats not a dataset");
} catch (DatasetNotFoundException e) {
//expected
log.error("Expected error during dataset creation", e);
}
}
@Test
public void testBulkInsertsAndUpserts() throws Exception {
String datasetBasePath = dfsBasePath + "/test_dataset";
// Initial bulk insert
HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(datasetBasePath, Operation.BULK_INSERT);
new HoodieDeltaStreamer(cfg, jsc).sync();
TestHelpers.assertRecordCount(1000, datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertCommitMetadata("00000", datasetBasePath, dfs, 1);
// No new data => no commits.
cfg.sourceLimit = 0;
new HoodieDeltaStreamer(cfg, jsc).sync();
TestHelpers.assertRecordCount(1000, datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertCommitMetadata("00000", datasetBasePath, dfs, 1);
// upsert() #1
cfg.sourceLimit = 2000;
cfg.operation = Operation.UPSERT;
new HoodieDeltaStreamer(cfg, jsc).sync();
TestHelpers.assertRecordCount(2000, datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertCommitMetadata("00001", datasetBasePath, dfs, 2);
}
}

View File

@@ -0,0 +1,137 @@
/*
* 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
*
* 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.utilities;
import com.uber.hoodie.common.TestRawTripPayload;
import com.uber.hoodie.common.minicluster.HdfsTestService;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.util.TypedProperties;
import com.uber.hoodie.utilities.sources.TestDataSource;
import java.io.BufferedReader;
import java.io.IOException;
import java.io.InputStreamReader;
import java.io.PrintStream;
import java.util.List;
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.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.SQLContext;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
/**
* Abstract test that provides a dfs & spark contexts.
*
* TODO(vc): this needs to be done across the board.
*/
public class UtilitiesTestBase {
protected static String dfsBasePath;
protected static HdfsTestService hdfsTestService;
protected static MiniDFSCluster dfsCluster;
protected static DistributedFileSystem dfs;
protected transient JavaSparkContext jsc = null;
protected transient SQLContext sqlContext;
@BeforeClass
public static void initClass() throws Exception {
hdfsTestService = new HdfsTestService();
dfsCluster = hdfsTestService.start(true);
dfs = dfsCluster.getFileSystem();
dfsBasePath = dfs.getWorkingDirectory().toString();
dfs.mkdirs(new Path(dfsBasePath));
}
@AfterClass
public static void cleanupClass() throws Exception {
if (hdfsTestService != null) {
hdfsTestService.stop();
}
}
@Before
public void setup() throws Exception {
TestDataSource.initDataGen();
jsc = UtilHelpers.buildSparkContext(this.getClass().getName() + "-hoodie", "local[2]");
sqlContext = new SQLContext(jsc);
}
@After
public void teardown() throws Exception {
TestDataSource.resetDataGen();
if (jsc != null) {
jsc.stop();
}
}
public static class Helpers {
// to get hold of resources bundled with jar
private static ClassLoader classLoader = Helpers.class.getClassLoader();
public static void copyToDFS(String testResourcePath, FileSystem fs, String targetPath) throws IOException {
BufferedReader reader = new BufferedReader(
new InputStreamReader(classLoader.getResourceAsStream(testResourcePath)));
PrintStream os = new PrintStream(fs.create(new Path(targetPath), true));
String line;
while ((line = reader.readLine()) != null) {
os.println(line);
}
os.flush();
os.close();
}
public static void savePropsToDFS(TypedProperties props, FileSystem fs, String targetPath) throws IOException {
String[] lines = props.keySet().stream().map(k -> String.format("%s=%s", k, props.get(k))).toArray(String[]::new);
saveStringsToDFS(lines, fs, targetPath);
}
public static void saveStringsToDFS(String[] lines, FileSystem fs, String targetPath) throws IOException {
PrintStream os = new PrintStream(fs.create(new Path(targetPath), true));
for (String l : lines) {
os.println(l);
}
os.flush();
os.close();
}
public static TypedProperties setupSchemaOnDFS() throws IOException {
UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/source.avsc", dfs, dfsBasePath + "/source.avsc");
TypedProperties props = new TypedProperties();
props.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", dfsBasePath + "/source.avsc");
return props;
}
public static String toJsonString(HoodieRecord hr) {
try {
return ((TestRawTripPayload) hr.getData()).getJsonData();
} catch (IOException ioe) {
return null;
}
}
public static String[] jsonifyRecords(List<HoodieRecord> records) throws IOException {
return records.stream().map(Helpers::toJsonString).toArray(String[]::new);
}
}
}

View File

@@ -0,0 +1,104 @@
/*
* 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
*
* 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.utilities.sources;
import static org.junit.Assert.assertEquals;
import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.util.TypedProperties;
import com.uber.hoodie.common.util.collection.Pair;
import com.uber.hoodie.utilities.UtilitiesTestBase;
import com.uber.hoodie.utilities.schema.FilebasedSchemaProvider;
import java.io.IOException;
import java.util.Optional;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.fs.Path;
import org.apache.spark.api.java.JavaRDD;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
/**
* Basic tests against all subclasses of {@link DFSSource}
*/
public class TestDFSSource extends UtilitiesTestBase {
private FilebasedSchemaProvider schemaProvider;
@BeforeClass
public static void initClass() throws Exception {
UtilitiesTestBase.initClass();
}
@AfterClass
public static void cleanupClass() throws Exception {
UtilitiesTestBase.cleanupClass();
}
@Before
public void setup() throws Exception {
super.setup();
schemaProvider = new FilebasedSchemaProvider(Helpers.setupSchemaOnDFS(), jsc);
}
@After
public void teardown() throws Exception {
super.teardown();
}
@Test
public void testJsonDFSSource() throws IOException {
dfs.mkdirs(new Path(dfsBasePath + "/jsonFiles"));
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
TypedProperties props = new TypedProperties();
props.setProperty("hoodie.deltastreamer.source.dfs.root", dfsBasePath + "/jsonFiles");
JsonDFSSource jsonSource = new JsonDFSSource(props, jsc, schemaProvider);
// 1. Extract without any checkpoint => get all the data, respecting sourceLimit
assertEquals(Optional.empty(), jsonSource.fetchNewData(Optional.empty(), Long.MAX_VALUE).getKey());
UtilitiesTestBase.Helpers.saveStringsToDFS(
Helpers.jsonifyRecords(dataGenerator.generateInserts("000", 100)), dfs,
dfsBasePath + "/jsonFiles/1.json");
assertEquals(Optional.empty(), jsonSource.fetchNewData(Optional.empty(), 10).getKey());
Pair<Optional<JavaRDD<GenericRecord>>, String> fetch1 = jsonSource.fetchNewData(Optional.empty(), 1000000);
assertEquals(100, fetch1.getKey().get().count());
// 2. Produce new data, extract new data
UtilitiesTestBase.Helpers.saveStringsToDFS(
Helpers.jsonifyRecords(dataGenerator.generateInserts("001", 10000)),
dfs, dfsBasePath + "/jsonFiles/2.json");
Pair<Optional<JavaRDD<GenericRecord>>, String> fetch2 = jsonSource.fetchNewData(
Optional.of(fetch1.getValue()), Long.MAX_VALUE);
assertEquals(10000, fetch2.getKey().get().count());
// 3. Extract with previous checkpoint => gives same data back (idempotent)
Pair<Optional<JavaRDD<GenericRecord>>, String> fetch3 = jsonSource.fetchNewData(
Optional.of(fetch1.getValue()), Long.MAX_VALUE);
assertEquals(10000, fetch3.getKey().get().count());
assertEquals(fetch2.getValue(), fetch3.getValue());
// 4. Extract with latest checkpoint => no new data returned
Pair<Optional<JavaRDD<GenericRecord>>, String> fetch4 = jsonSource.fetchNewData(
Optional.of(fetch2.getValue()), Long.MAX_VALUE);
assertEquals(Optional.empty(), fetch4.getKey());
}
}

View File

@@ -0,0 +1,99 @@
/*
* 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
*
* 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.utilities.sources;
import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.util.TypedProperties;
import com.uber.hoodie.common.util.collection.ImmutablePair;
import com.uber.hoodie.common.util.collection.Pair;
import com.uber.hoodie.utilities.schema.SchemaProvider;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
/**
* An implementation of {@link Source}, that emits test upserts.
*/
public class TestDataSource extends Source {
private static volatile Logger log = LogManager.getLogger(TestDataSource.class);
// Static instance, helps with reuse across a test.
private static HoodieTestDataGenerator dataGenerator;
public static void initDataGen() {
dataGenerator = new HoodieTestDataGenerator();
}
public static void resetDataGen() {
dataGenerator = null;
}
public TestDataSource(TypedProperties props, JavaSparkContext sparkContext, SchemaProvider schemaProvider) {
super(props, sparkContext, schemaProvider);
}
private GenericRecord toGenericRecord(HoodieRecord hoodieRecord) {
try {
Optional<IndexedRecord> recordOpt = hoodieRecord.getData().getInsertValue(dataGenerator.avroSchema);
return (GenericRecord) recordOpt.get();
} catch (IOException e) {
return null;
}
}
@Override
public Pair<Optional<JavaRDD<GenericRecord>>, String> fetchNewData(Optional<String> lastCheckpointStr,
long sourceLimit) {
int nextCommitNum = lastCheckpointStr.isPresent() ? Integer.parseInt(lastCheckpointStr.get()) + 1 : 0;
String commitTime = String.format("%05d", nextCommitNum);
// No new data.
if (sourceLimit <= 0) {
return new ImmutablePair<>(Optional.empty(), commitTime);
}
// generate `sourceLimit` number of upserts each time.
int numExistingKeys = dataGenerator.getExistingKeysList().size();
int numUpdates = Math.min(numExistingKeys, (int) sourceLimit / 2);
int numInserts = (int) sourceLimit - numUpdates;
List<GenericRecord> records = new ArrayList<>();
try {
records.addAll(dataGenerator.generateUniqueUpdates(commitTime, numUpdates).stream()
.map(this::toGenericRecord).collect(Collectors.toList()));
records.addAll(dataGenerator.generateInserts(commitTime, numInserts).stream()
.map(this::toGenericRecord).collect(Collectors.toList()));
} catch (IOException e) {
log.error("Error generating test data.", e);
}
JavaRDD<GenericRecord> avroRDD = sparkContext.<GenericRecord>parallelize(records, 4);
return new ImmutablePair<>(Optional.of(avroRDD), commitTime);
}
}

View File

@@ -0,0 +1,186 @@
/*
* 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
*
* 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.utilities.sources;
import static com.uber.hoodie.utilities.sources.KafkaSource.CheckpointUtils;
import static org.junit.Assert.assertEquals;
import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.util.TypedProperties;
import com.uber.hoodie.common.util.collection.Pair;
import com.uber.hoodie.utilities.UtilitiesTestBase;
import com.uber.hoodie.utilities.schema.FilebasedSchemaProvider;
import java.io.IOException;
import java.util.HashMap;
import java.util.Optional;
import kafka.common.TopicAndPartition;
import org.apache.avro.generic.GenericRecord;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset;
import org.apache.spark.streaming.kafka.KafkaTestUtils;
import org.apache.spark.streaming.kafka.OffsetRange;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
/**
* Tests against {@link KafkaSource}
*/
public class TestKafkaSource extends UtilitiesTestBase {
private static String TEST_TOPIC_NAME = "hoodie_test";
private FilebasedSchemaProvider schemaProvider;
private KafkaTestUtils testUtils;
@BeforeClass
public static void initClass() throws Exception {
UtilitiesTestBase.initClass();
}
@AfterClass
public static void cleanupClass() throws Exception {
UtilitiesTestBase.cleanupClass();
}
@Before
public void setup() throws Exception {
super.setup();
schemaProvider = new FilebasedSchemaProvider(Helpers.setupSchemaOnDFS(), jsc);
testUtils = new KafkaTestUtils();
testUtils.setup();
}
@After
public void teardown() throws Exception {
super.teardown();
testUtils.teardown();
}
@Test
public void testJsonKafkaSource() throws IOException {
// topic setup.
testUtils.createTopic(TEST_TOPIC_NAME, 2);
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
TypedProperties props = new TypedProperties();
props.setProperty("hoodie.deltastreamer.source.kafka.topic", TEST_TOPIC_NAME);
props.setProperty("metadata.broker.list", testUtils.brokerAddress());
props.setProperty("auto.offset.reset", "smallest");
props.setProperty("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
props.setProperty("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
Source kafkaSource = new JsonKafkaSource(props, jsc, schemaProvider);
// 1. Extract without any checkpoint => get all the data, respecting sourceLimit
assertEquals(Optional.empty(), kafkaSource.fetchNewData(Optional.empty(), Long.MAX_VALUE).getKey());
testUtils.sendMessages(TEST_TOPIC_NAME, Helpers.jsonifyRecords(dataGenerator.generateInserts("000", 1000)));
Pair<Optional<JavaRDD<GenericRecord>>, String> fetch1 = kafkaSource.fetchNewData(Optional.empty(), 900);
assertEquals(900, fetch1.getKey().get().count());
// 2. Produce new data, extract new data
testUtils.sendMessages(TEST_TOPIC_NAME, Helpers.jsonifyRecords(dataGenerator.generateInserts("001", 1000)));
Pair<Optional<JavaRDD<GenericRecord>>, String> fetch2 = kafkaSource.fetchNewData(
Optional.of(fetch1.getValue()), Long.MAX_VALUE);
assertEquals(1100, fetch2.getKey().get().count());
// 3. Extract with previous checkpoint => gives same data back (idempotent)
Pair<Optional<JavaRDD<GenericRecord>>, String> fetch3 = kafkaSource.fetchNewData(
Optional.of(fetch1.getValue()), Long.MAX_VALUE);
assertEquals(fetch2.getKey().get().count(), fetch3.getKey().get().count());
assertEquals(fetch2.getValue(), fetch3.getValue());
// 4. Extract with latest checkpoint => no new data returned
Pair<Optional<JavaRDD<GenericRecord>>, String> fetch4 = kafkaSource.fetchNewData(
Optional.of(fetch2.getValue()), Long.MAX_VALUE);
assertEquals(Optional.empty(), fetch4.getKey());
}
private static HashMap<TopicAndPartition, LeaderOffset> makeOffsetMap(int[] partitions, long[] offsets) {
HashMap<TopicAndPartition, LeaderOffset> map = new HashMap<>();
for (int i = 0; i < partitions.length; i++) {
map.put(new TopicAndPartition(TEST_TOPIC_NAME, partitions[i]), new LeaderOffset("", -1, offsets[i]));
}
return map;
}
@Test
public void testComputeOffsetRanges() {
// test totalNewMessages()
long totalMsgs = CheckpointUtils.totalNewMessages(new OffsetRange[]{
OffsetRange.apply(TEST_TOPIC_NAME, 0, 0, 100),
OffsetRange.apply(TEST_TOPIC_NAME, 0, 100, 200)
});
assertEquals(200, totalMsgs);
// should consume all the full data
OffsetRange[] ranges = CheckpointUtils.computeOffsetRanges(
makeOffsetMap(new int[]{0, 1}, new long[]{200000, 250000}),
makeOffsetMap(new int[]{0, 1}, new long[]{300000, 350000}),
1000000L
);
assertEquals(200000, CheckpointUtils.totalNewMessages(ranges));
// should only consume upto limit
ranges = CheckpointUtils.computeOffsetRanges(
makeOffsetMap(new int[]{0, 1}, new long[]{200000, 250000}),
makeOffsetMap(new int[]{0, 1}, new long[]{300000, 350000}),
10000
);
assertEquals(10000, CheckpointUtils.totalNewMessages(ranges));
assertEquals(200000, ranges[0].fromOffset());
assertEquals(205000, ranges[0].untilOffset());
assertEquals(250000, ranges[1].fromOffset());
assertEquals(255000, ranges[1].untilOffset());
// should also consume from new partitions.
ranges = CheckpointUtils.computeOffsetRanges(
makeOffsetMap(new int[]{0, 1}, new long[]{200000, 250000}),
makeOffsetMap(new int[]{0, 1, 2}, new long[]{300000, 350000, 100000}),
1000000L
);
assertEquals(300000, CheckpointUtils.totalNewMessages(ranges));
assertEquals(3, ranges.length);
// for skewed offsets, does not starve any partition & can catch up
ranges = CheckpointUtils.computeOffsetRanges(
makeOffsetMap(new int[]{0, 1}, new long[]{200000, 250000}),
makeOffsetMap(new int[]{0, 1, 2}, new long[]{200010, 350000, 10000}),
100000
);
assertEquals(100000, CheckpointUtils.totalNewMessages(ranges));
assertEquals(10, ranges[0].count());
assertEquals(89990, ranges[1].count());
assertEquals(10000, ranges[2].count());
ranges = CheckpointUtils.computeOffsetRanges(
makeOffsetMap(new int[]{0, 1}, new long[]{200000, 250000}),
makeOffsetMap(new int[]{0, 1, 2}, new long[]{200010, 350000, 10000}),
1000000
);
assertEquals(110010, CheckpointUtils.totalNewMessages(ranges));
assertEquals(10, ranges[0].count());
assertEquals(100000, ranges[1].count());
assertEquals(10000, ranges[2].count());
}
}

View File

@@ -0,0 +1,8 @@
CREATE TABLE <tempDbTable>
<storedAsClause>
LOCATION '<tempDbTablePath>'
AS
<incrementalSQL>

View File

@@ -0,0 +1,21 @@
#
# 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
#
# 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.
#
#
#
# Common hoodie client configs
hoodie.upsert.shuffle.parallelism=2
hoodie.insert.shuffle.parallelism=2
hoodie.bulkinsert.shuffle.parallelism=2

View File

@@ -0,0 +1,27 @@
#
# 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
#
# 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.
#
#
#
include=base.properties
# Key generator props
hoodie.datasource.write.recordkey.field=_row_key
hoodie.datasource.write.partitionpath.field=driver
# Schema provider props (change to absolute path based on your installation)
hoodie.deltastreamer.filebased.schemaprovider.source.schema.file=file:///path/to/hoodie/hoodie-utilities/src/main/resources/delta-streamer-props/source.avsc
hoodie.deltastreamer.filebased.schemaprovider.target.schema.file=file:///path/to/hoodie/hoodie-utilities/src/main/resources/delta-streamer-props/target.avsc
# DFS Source
hoodie.deltastreamer.source.dfs.root=file:///tmp/hoodie-dfs-input

View File

@@ -0,0 +1,30 @@
#
# 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
#
# 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.
#
#
#
include=base.properties
# Key fields, for kafka example
hoodie.datasource.write.recordkey.field=impressionid
hoodie.datasource.write.partitionpath.field=userid
# schema provider configs
hoodie.deltastreamer.schemaprovider.registry.url=http://localhost:8081/subjects/impressions-value/versions/latest
# Kafka Source
#hoodie.deltastreamer.source.kafka.topic=uber_trips
hoodie.deltastreamer.source.kafka.topic=impressions
#Kafka props
metadata.broker.list=localhost:9092
auto.offset.reset=smallest
schema.registry.url=http://localhost:8081

View File

@@ -0,0 +1,34 @@
{
"type" : "record",
"name" : "triprec",
"fields" : [
{
"name" : "timestamp",
"type" : "double"
}, {
"name" : "_row_key",
"type" : "string"
}, {
"name" : "rider",
"type" : "string"
}, {
"name" : "driver",
"type" : "string"
}, {
"name" : "begin_lat",
"type" : "double"
}, {
"name" : "begin_lon",
"type" : "double"
}, {
"name" : "end_lat",
"type" : "double"
}, {
"name" : "end_lon",
"type" : "double"
}, {
"name" : "fare",
"type" : "double"
} ]
}