1
0

[HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment (#1150)

* [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
This commit is contained in:
Pratyaksh Sharma
2020-04-08 04:40:26 +05:30
committed by GitHub
parent b5d093a21b
commit d610252d6b
26 changed files with 1184 additions and 78 deletions

View File

@@ -69,6 +69,7 @@ import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.api.java.UDF4;
import org.apache.spark.sql.functions;
import org.apache.spark.sql.types.DataTypes;
import org.apache.spark.streaming.kafka010.KafkaTestUtils;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
@@ -100,6 +101,10 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
private static final Random RANDOM = new Random();
private static final String PROPS_FILENAME_TEST_SOURCE = "test-source.properties";
public static final String PROPS_FILENAME_TEST_SOURCE1 = "test-source1.properties";
public static final String PROPS_INVALID_HIVE_SYNC_TEST_SOURCE1 = "test-invalid-hive-sync-source1.properties";
public static final String PROPS_INVALID_FILE = "test-invalid-props.properties";
public static final String PROPS_INVALID_TABLE_CONFIG_FILE = "test-invalid-table-config.properties";
private static final String PROPS_FILENAME_TEST_INVALID = "test-invalid.properties";
private static final String PROPS_FILENAME_TEST_CSV = "test-csv-dfs-source.properties";
private static final String PROPS_FILENAME_TEST_PARQUET = "test-parquet-dfs-source.properties";
@@ -107,6 +112,7 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
private static final int PARQUET_NUM_RECORDS = 5;
private static final int CSV_NUM_RECORDS = 3;
private static final Logger LOG = LogManager.getLogger(TestHoodieDeltaStreamer.class);
public static KafkaTestUtils testUtils;
private static int testNum = 1;
@@ -114,9 +120,12 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
public static void initClass() throws Exception {
UtilitiesTestBase.initClass(true);
PARQUET_SOURCE_ROOT = dfsBasePath + "/parquetFiles";
testUtils = new KafkaTestUtils();
testUtils.setup();
// prepare the configs.
UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/base.properties", dfs, dfsBasePath + "/base.properties");
UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/base.properties", dfs, dfsBasePath + "/config/base.properties");
UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/sql-transformer.properties", dfs,
dfsBasePath + "/sql-transformer.properties");
UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/source.avsc", dfs, dfsBasePath + "/source.avsc");
@@ -124,6 +133,14 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/target.avsc", dfs, dfsBasePath + "/target.avsc");
UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/target-flattened.avsc", dfs, dfsBasePath + "/target-flattened.avsc");
UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/source_short_trip_uber.avsc", dfs, dfsBasePath + "/source_short_trip_uber.avsc");
UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/source_uber.avsc", dfs, dfsBasePath + "/source_uber.avsc");
UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/target_short_trip_uber.avsc", dfs, dfsBasePath + "/target_short_trip_uber.avsc");
UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/target_uber.avsc", dfs, dfsBasePath + "/target_uber.avsc");
UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/invalid_hive_sync_uber_config.properties", dfs, dfsBasePath + "/config/invalid_hive_sync_uber_config.properties");
UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/uber_config.properties", dfs, dfsBasePath + "/config/uber_config.properties");
UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/short_trip_uber_config.properties", dfs, dfsBasePath + "/config/short_trip_uber_config.properties");
TypedProperties props = new TypedProperties();
props.setProperty("include", "sql-transformer.properties");
props.setProperty("hoodie.datasource.write.keygenerator.class", TestGenerator.class.getName());
@@ -163,11 +180,54 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
invalidProps.setProperty("hoodie.deltastreamer.schemaprovider.target.schema.file", dfsBasePath + "/target.avsc");
UtilitiesTestBase.Helpers.savePropsToDFS(invalidProps, dfs, dfsBasePath + "/" + PROPS_FILENAME_TEST_INVALID);
TypedProperties props1 = new TypedProperties();
populateCommonProps(props1);
UtilitiesTestBase.Helpers.savePropsToDFS(props1, dfs, dfsBasePath + "/" + PROPS_FILENAME_TEST_SOURCE1);
TypedProperties properties = new TypedProperties();
populateInvalidTableConfigFilePathProps(properties);
UtilitiesTestBase.Helpers.savePropsToDFS(properties, dfs, dfsBasePath + "/" + PROPS_INVALID_TABLE_CONFIG_FILE);
TypedProperties invalidHiveSyncProps = new TypedProperties();
invalidHiveSyncProps.setProperty("hoodie.deltastreamer.ingestion.tablesToBeIngested", "uber_db.dummy_table_uber");
invalidHiveSyncProps.setProperty("hoodie.deltastreamer.ingestion.uber_db.dummy_table_uber.configFile", dfsBasePath + "/config/invalid_hive_sync_uber_config.properties");
UtilitiesTestBase.Helpers.savePropsToDFS(invalidHiveSyncProps, dfs, dfsBasePath + "/" + PROPS_INVALID_HIVE_SYNC_TEST_SOURCE1);
prepareParquetDFSFiles(PARQUET_NUM_RECORDS);
}
private static void populateInvalidTableConfigFilePathProps(TypedProperties props) {
props.setProperty("hoodie.datasource.write.keygenerator.class", TestHoodieDeltaStreamer.TestGenerator.class.getName());
props.setProperty("hoodie.deltastreamer.keygen.timebased.output.dateformat", "yyyyMMdd");
props.setProperty("hoodie.deltastreamer.ingestion.tablesToBeIngested", "uber_db.dummy_table_uber");
props.setProperty("hoodie.deltastreamer.ingestion.uber_db.dummy_table_uber.configFile", dfsBasePath + "/config/invalid_uber_config.properties");
}
private static void populateCommonProps(TypedProperties props) {
props.setProperty("hoodie.datasource.write.keygenerator.class", TestHoodieDeltaStreamer.TestGenerator.class.getName());
props.setProperty("hoodie.deltastreamer.keygen.timebased.output.dateformat", "yyyyMMdd");
props.setProperty("hoodie.deltastreamer.ingestion.tablesToBeIngested", "short_trip_db.dummy_table_short_trip,uber_db.dummy_table_uber");
props.setProperty("hoodie.deltastreamer.ingestion.uber_db.dummy_table_uber.configFile", dfsBasePath + "/config/uber_config.properties");
props.setProperty("hoodie.deltastreamer.ingestion.short_trip_db.dummy_table_short_trip.configFile", dfsBasePath + "/config/short_trip_uber_config.properties");
//Kafka source properties
props.setProperty("bootstrap.servers", testUtils.brokerAddress());
props.setProperty("auto.offset.reset", "earliest");
props.setProperty("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
props.setProperty("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
props.setProperty("hoodie.deltastreamer.kafka.source.maxEvents", String.valueOf(5000));
// Hive Configs
props.setProperty(DataSourceWriteOptions.HIVE_URL_OPT_KEY(), "jdbc:hive2://127.0.0.1:9999/");
props.setProperty(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY(), "testdb2");
props.setProperty(DataSourceWriteOptions.HIVE_ASSUME_DATE_PARTITION_OPT_KEY(), "false");
props.setProperty(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY(), "datestr");
props.setProperty(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY(),
MultiPartKeysValueExtractor.class.getName());
}
@AfterClass
public static void cleanupClass() throws Exception {
public static void cleanupClass() {
UtilitiesTestBase.cleanupClass();
}
@@ -649,7 +709,7 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
String path = PARQUET_SOURCE_ROOT + "/1.parquet";
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
Helpers.saveParquetToDFS(Helpers.toGenericRecords(
dataGenerator.generateInserts("000", numRecords), dataGenerator), new Path(path));
dataGenerator.generateInserts("000", numRecords)), new Path(path));
}
private void prepareParquetDFSSource(boolean useSchemaProvider, boolean hasTransformer) throws IOException {

View File

@@ -0,0 +1,166 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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 org.apache.hudi.utilities;
import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.common.HoodieTestDataGenerator;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.utilities.deltastreamer.HoodieMultiTableDeltaStreamer;
import org.apache.hudi.utilities.deltastreamer.TableExecutionContext;
import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
import org.apache.hudi.utilities.sources.JsonKafkaSource;
import org.apache.hudi.utilities.sources.TestDataSource;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.junit.Ignore;
import org.junit.Test;
import java.io.IOException;
import java.util.List;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
public class TestHoodieMultiTableDeltaStreamer extends TestHoodieDeltaStreamer {
private static volatile Logger log = LogManager.getLogger(TestHoodieMultiTableDeltaStreamer.class);
static class TestHelpers {
static HoodieMultiTableDeltaStreamer.Config getConfig(String fileName, String configFolder, String sourceClassName, boolean enableHiveSync) {
HoodieMultiTableDeltaStreamer.Config config = new HoodieMultiTableDeltaStreamer.Config();
config.configFolder = configFolder;
config.targetTableName = "dummy_table";
config.basePathPrefix = dfsBasePath + "/multi_table_dataset";
config.propsFilePath = dfsBasePath + "/" + fileName;
config.tableType = "COPY_ON_WRITE";
config.sourceClassName = sourceClassName;
config.sourceOrderingField = "timestamp";
config.schemaProviderClassName = FilebasedSchemaProvider.class.getName();
config.enableHiveSync = enableHiveSync;
return config;
}
}
@Test
public void testInvalidHiveSyncProps() throws IOException {
HoodieMultiTableDeltaStreamer.Config cfg = TestHelpers.getConfig(PROPS_INVALID_HIVE_SYNC_TEST_SOURCE1,dfsBasePath + "/config", TestDataSource.class.getName(), true);
try {
new HoodieMultiTableDeltaStreamer(cfg, jsc);
fail("Should fail when hive sync table not provided with enableHiveSync flag");
} catch (HoodieException he) {
log.error("Expected error when creating table execution objects", he);
assertTrue(he.getMessage().contains("Hive sync table field not provided!"));
}
}
@Test
public void testInvalidPropsFilePath() throws IOException {
HoodieMultiTableDeltaStreamer.Config cfg = TestHelpers.getConfig(PROPS_INVALID_FILE,dfsBasePath + "/config", TestDataSource.class.getName(), true);
try {
new HoodieMultiTableDeltaStreamer(cfg, jsc);
fail("Should fail when invalid props file is provided");
} catch (IllegalArgumentException iae) {
log.error("Expected error when creating table execution objects", iae);
assertTrue(iae.getMessage().contains("Please provide valid common config file path!"));
}
}
@Test
public void testInvalidTableConfigFilePath() throws IOException {
HoodieMultiTableDeltaStreamer.Config cfg = TestHelpers.getConfig(PROPS_INVALID_TABLE_CONFIG_FILE,dfsBasePath + "/config", TestDataSource.class.getName(), true);
try {
new HoodieMultiTableDeltaStreamer(cfg, jsc);
fail("Should fail when invalid table config props file path is provided");
} catch (IllegalArgumentException iae) {
log.error("Expected error when creating table execution objects", iae);
assertTrue(iae.getMessage().contains("Please provide valid table config file path!"));
}
}
@Test
public void testCustomConfigProps() throws IOException {
HoodieMultiTableDeltaStreamer.Config cfg = TestHelpers.getConfig(PROPS_FILENAME_TEST_SOURCE1,dfsBasePath + "/config", TestDataSource.class.getName(), false);
HoodieMultiTableDeltaStreamer streamer = new HoodieMultiTableDeltaStreamer(cfg, jsc);
TableExecutionContext executionContext = streamer.getTableExecutionContexts().get(1);
assertEquals(streamer.getTableExecutionContexts().size(), 2);
assertEquals(executionContext.getConfig().targetBasePath, dfsBasePath + "/multi_table_dataset/uber_db/dummy_table_uber");
assertEquals(executionContext.getConfig().targetTableName, "uber_db.dummy_table_uber");
assertEquals(executionContext.getProperties().getString(HoodieMultiTableDeltaStreamer.Constants.KAFKA_TOPIC_PROP), "topic1");
assertEquals(executionContext.getProperties().getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()), "_row_key");
assertEquals(executionContext.getProperties().getString(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY()), TestHoodieDeltaStreamer.TestGenerator.class.getName());
assertEquals(executionContext.getProperties().getString(HoodieMultiTableDeltaStreamer.Constants.HIVE_SYNC_TABLE_PROP), "uber_hive_dummy_table");
}
@Test
@Ignore
public void testInvalidIngestionProps() {
try {
HoodieMultiTableDeltaStreamer.Config cfg = TestHelpers.getConfig(PROPS_FILENAME_TEST_SOURCE1,dfsBasePath + "/config", TestDataSource.class.getName(), true);
new HoodieMultiTableDeltaStreamer(cfg, jsc);
fail("Creation of execution object should fail without kafka topic");
} catch (Exception e) {
log.error("Creation of execution object failed with error: " + e.getMessage(), e);
assertTrue(e.getMessage().contains("Please provide valid table config arguments!"));
}
}
@Test //0 corresponds to fg
public void testMultiTableExecution() throws IOException {
//create topics for each table
testUtils.createTopic("topic1", 2);
testUtils.createTopic("topic2", 2);
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
testUtils.sendMessages("topic1", Helpers.jsonifyRecords(dataGenerator.generateInsertsAsPerSchema("000", 5, HoodieTestDataGenerator.TRIP_SCHEMA)));
testUtils.sendMessages("topic2", Helpers.jsonifyRecords(dataGenerator.generateInsertsAsPerSchema("000", 10, HoodieTestDataGenerator.SHORT_TRIP_SCHEMA)));
HoodieMultiTableDeltaStreamer.Config cfg = TestHelpers.getConfig(PROPS_FILENAME_TEST_SOURCE1,dfsBasePath + "/config", JsonKafkaSource.class.getName(), false);
HoodieMultiTableDeltaStreamer streamer = new HoodieMultiTableDeltaStreamer(cfg, jsc);
List<TableExecutionContext> executionContexts = streamer.getTableExecutionContexts();
TypedProperties properties = executionContexts.get(1).getProperties();
properties.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", dfsBasePath + "/source_uber.avsc");
properties.setProperty("hoodie.deltastreamer.schemaprovider.target.schema.file", dfsBasePath + "/target_uber.avsc");
executionContexts.get(1).setProperties(properties);
TypedProperties properties1 = executionContexts.get(0).getProperties();
properties1.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", dfsBasePath + "/source_short_trip_uber.avsc");
properties1.setProperty("hoodie.deltastreamer.schemaprovider.target.schema.file", dfsBasePath + "/target_short_trip_uber.avsc");
executionContexts.get(0).setProperties(properties1);
String targetBasePath1 = executionContexts.get(1).getConfig().targetBasePath;
String targetBasePath2 = executionContexts.get(0).getConfig().targetBasePath;
streamer.sync();
TestHoodieDeltaStreamer.TestHelpers.assertRecordCount(5, targetBasePath1 + "/*/*.parquet", sqlContext);
TestHoodieDeltaStreamer.TestHelpers.assertRecordCount(10, targetBasePath2 + "/*/*.parquet", sqlContext);
//insert updates for already existing records in kafka topics
testUtils.sendMessages("topic1", Helpers.jsonifyRecords(dataGenerator.generateUpdatesAsPerSchema("001", 5, HoodieTestDataGenerator.TRIP_SCHEMA)));
testUtils.sendMessages("topic2", Helpers.jsonifyRecords(dataGenerator.generateUpdatesAsPerSchema("001", 10, HoodieTestDataGenerator.SHORT_TRIP_SCHEMA)));
streamer.sync();
assertEquals(streamer.getSuccessTables().size(), 2);
assertTrue(streamer.getFailedTables().isEmpty());
//assert the record count matches now
TestHoodieDeltaStreamer.TestHelpers.assertRecordCount(5, targetBasePath1 + "/*/*.parquet", sqlContext);
TestHoodieDeltaStreamer.TestHelpers.assertRecordCount(10, targetBasePath2 + "/*/*.parquet", sqlContext);
}
}

View File

@@ -83,6 +83,7 @@ public class UtilitiesTestBase {
protected transient SparkSession sparkSession = null;
protected transient SQLContext sqlContext;
protected static HiveServer2 hiveServer;
protected static HiveTestService hiveTestService;
private static ObjectMapper mapper = new ObjectMapper();
@BeforeClass
@@ -97,20 +98,23 @@ public class UtilitiesTestBase {
dfsBasePath = dfs.getWorkingDirectory().toString();
dfs.mkdirs(new Path(dfsBasePath));
if (startHiveService) {
HiveTestService hiveService = new HiveTestService(hdfsTestService.getHadoopConf());
hiveServer = hiveService.start();
hiveTestService = new HiveTestService(hdfsTestService.getHadoopConf());
hiveServer = hiveTestService.start();
clearHiveDb();
}
}
@AfterClass
public static void cleanupClass() throws Exception {
public static void cleanupClass() {
if (hdfsTestService != null) {
hdfsTestService.stop();
}
if (hiveServer != null) {
hiveServer.stop();
}
if (hiveTestService != null) {
hiveTestService.stop();
}
}
@Before
@@ -263,20 +267,19 @@ public class UtilitiesTestBase {
return props;
}
public static GenericRecord toGenericRecord(HoodieRecord hoodieRecord, HoodieTestDataGenerator dataGenerator) {
public static GenericRecord toGenericRecord(HoodieRecord hoodieRecord) {
try {
Option<IndexedRecord> recordOpt = hoodieRecord.getData().getInsertValue(dataGenerator.AVRO_SCHEMA);
Option<IndexedRecord> recordOpt = hoodieRecord.getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA);
return (GenericRecord) recordOpt.get();
} catch (IOException e) {
return null;
}
}
public static List<GenericRecord> toGenericRecords(List<HoodieRecord> hoodieRecords,
HoodieTestDataGenerator dataGenerator) {
public static List<GenericRecord> toGenericRecords(List<HoodieRecord> hoodieRecords) {
List<GenericRecord> records = new ArrayList<GenericRecord>();
for (HoodieRecord hoodieRecord : hoodieRecords) {
records.add(toGenericRecord(hoodieRecord, dataGenerator));
records.add(toGenericRecord(hoodieRecord));
}
return records;
}

View File

@@ -88,7 +88,7 @@ public abstract class AbstractBaseTestSource extends AvroSource {
HoodieTestDataGenerator dataGenerator = dataGeneratorMap.get(partition);
// generate `sourceLimit` number of upserts each time.
int numExistingKeys = dataGenerator.getNumExistingKeys();
int numExistingKeys = dataGenerator.getNumExistingKeys(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA);
LOG.info("NumExistingKeys=" + numExistingKeys);
int numUpdates = Math.min(numExistingKeys, sourceLimit / 2);
@@ -116,21 +116,22 @@ public abstract class AbstractBaseTestSource extends AvroSource {
LOG.info("After adjustments => NumInserts=" + numInserts + ", NumUpdates=" + (numUpdates - 50) + ", NumDeletes=50, maxUniqueRecords="
+ maxUniqueKeys);
// if we generate update followed by deletes -> some keys in update batch might be picked up for deletes. Hence generating delete batch followed by updates
deleteStream = dataGenerator.generateUniqueDeleteRecordStream(instantTime, 50).map(hr -> AbstractBaseTestSource.toGenericRecord(hr, dataGenerator));
updateStream = dataGenerator.generateUniqueUpdatesStream(instantTime, numUpdates - 50).map(hr -> AbstractBaseTestSource.toGenericRecord(hr, dataGenerator));
deleteStream = dataGenerator.generateUniqueDeleteRecordStream(instantTime, 50).map(AbstractBaseTestSource::toGenericRecord);
updateStream = dataGenerator.generateUniqueUpdatesStream(instantTime, numUpdates - 50, HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
.map(AbstractBaseTestSource::toGenericRecord);
} else {
LOG.info("After adjustments => NumInserts=" + numInserts + ", NumUpdates=" + numUpdates + ", maxUniqueRecords=" + maxUniqueKeys);
updateStream = dataGenerator.generateUniqueUpdatesStream(instantTime, numUpdates)
.map(hr -> AbstractBaseTestSource.toGenericRecord(hr, dataGenerator));
updateStream = dataGenerator.generateUniqueUpdatesStream(instantTime, numUpdates, HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
.map(AbstractBaseTestSource::toGenericRecord);
}
Stream<GenericRecord> insertStream = dataGenerator.generateInsertsStream(instantTime, numInserts, false)
.map(hr -> AbstractBaseTestSource.toGenericRecord(hr, dataGenerator));
Stream<GenericRecord> insertStream = dataGenerator.generateInsertsStream(instantTime, numInserts, false, HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
.map(AbstractBaseTestSource::toGenericRecord);
return Stream.concat(deleteStream, Stream.concat(updateStream, insertStream));
}
private static GenericRecord toGenericRecord(HoodieRecord hoodieRecord, HoodieTestDataGenerator dataGenerator) {
private static GenericRecord toGenericRecord(HoodieRecord hoodieRecord) {
try {
Option<IndexedRecord> recordOpt = hoodieRecord.getData().getInsertValue(dataGenerator.AVRO_SCHEMA);
Option<IndexedRecord> recordOpt = hoodieRecord.getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA);
return (GenericRecord) recordOpt.get();
} catch (IOException e) {
return null;

View File

@@ -64,7 +64,7 @@ public abstract class AbstractDFSSourceTestBase extends UtilitiesTestBase {
}
@AfterClass
public static void cleanupClass() throws Exception {
public static void cleanupClass() {
UtilitiesTestBase.cleanupClass();
}

View File

@@ -42,7 +42,6 @@ import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import java.io.IOException;
import java.util.HashMap;
import java.util.UUID;
@@ -64,7 +63,7 @@ public class TestKafkaSource extends UtilitiesTestBase {
}
@AfterClass
public static void cleanupClass() throws Exception {
public static void cleanupClass() {
UtilitiesTestBase.cleanupClass();
}
@@ -95,7 +94,7 @@ public class TestKafkaSource extends UtilitiesTestBase {
}
@Test
public void testJsonKafkaSource() throws IOException {
public void testJsonKafkaSource() {
// topic setup.
testUtils.createTopic(TEST_TOPIC_NAME, 2);
@@ -143,7 +142,7 @@ public class TestKafkaSource extends UtilitiesTestBase {
}
@Test
public void testJsonKafkaSourceWithDefaultUpperCap() throws IOException {
public void testJsonKafkaSourceWithDefaultUpperCap() {
// topic setup.
testUtils.createTopic(TEST_TOPIC_NAME, 2);
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
@@ -172,7 +171,7 @@ public class TestKafkaSource extends UtilitiesTestBase {
}
@Test
public void testJsonKafkaSourceWithConfigurableUpperCap() throws IOException {
public void testJsonKafkaSourceWithConfigurableUpperCap() {
// topic setup.
testUtils.createTopic(TEST_TOPIC_NAME, 2);
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();

View File

@@ -48,6 +48,6 @@ public class TestParquetDFSSource extends AbstractDFSSourceTestBase {
@Override
void writeNewDataToFile(List<HoodieRecord> records, Path path) throws IOException {
Helpers.saveParquetToDFS(Helpers.toGenericRecords(records, dataGenerator), path);
Helpers.saveParquetToDFS(Helpers.toGenericRecords(records), path);
}
}

View File

@@ -0,0 +1,23 @@
###
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you 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
hoodie.datasource.write.recordkey.field=_row_key
hoodie.datasource.write.partitionpath.field=created_at
hoodie.deltastreamer.source.kafka.topic=test_topic
hoodie.deltastreamer.keygen.timebased.timestamp.type=UNIX_TIMESTAMP
hoodie.deltastreamer.keygen.timebased.input.dateformat=yyyy-MM-dd

View File

@@ -0,0 +1,24 @@
###
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you 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
hoodie.datasource.write.recordkey.field=_row_key
hoodie.datasource.write.partitionpath.field=created_at
hoodie.deltastreamer.source.kafka.topic=topic2
hoodie.deltastreamer.keygen.timebased.timestamp.type=UNIX_TIMESTAMP
hoodie.deltastreamer.keygen.timebased.input.dateformat=yyyy-MM-dd HH:mm:ss.S
hoodie.datasource.hive_sync.table=short_trip_uber_hive_dummy_table

View File

@@ -0,0 +1,44 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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.
*/
{
"type" : "record",
"name" : "shortTripRec",
"fields" : [
{
"name" : "timestamp",
"type" : "double"
}, {
"name" : "_row_key",
"type" : "string"
}, {
"name" : "rider",
"type" : "string"
}, {
"name" : "driver",
"type" : "string"
}, {
"name" : "fare",
"type" : "double"
},
{
"name" : "_hoodie_is_deleted",
"type" : "boolean",
"default" : false
} ]
}

View File

@@ -0,0 +1,44 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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.
*/
{
"type" : "record",
"name" : "tripUberRec",
"fields" : [
{
"name" : "timestamp",
"type" : "double"
}, {
"name" : "_row_key",
"type" : "string"
}, {
"name" : "rider",
"type" : "string"
}, {
"name" : "driver",
"type" : "string"
}, {
"name" : "fare",
"type" : "double"
},
{
"name" : "_hoodie_is_deleted",
"type" : "boolean",
"default" : false
} ]
}

View File

@@ -0,0 +1,44 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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.
*/
{
"type" : "record",
"name" : "shortTripRec",
"fields" : [
{
"name" : "timestamp",
"type" : "double"
}, {
"name" : "_row_key",
"type" : "string"
}, {
"name" : "rider",
"type" : "string"
}, {
"name" : "driver",
"type" : "string"
}, {
"name" : "fare",
"type" : "double"
},
{
"name" : "_hoodie_is_deleted",
"type" : "boolean",
"default" : false
} ]
}

View File

@@ -0,0 +1,44 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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.
*/
{
"type" : "record",
"name" : "tripUberRec",
"fields" : [
{
"name" : "timestamp",
"type" : "double"
}, {
"name" : "_row_key",
"type" : "string"
}, {
"name" : "rider",
"type" : "string"
}, {
"name" : "driver",
"type" : "string"
}, {
"name" : "fare",
"type" : "double"
},
{
"name" : "_hoodie_is_deleted",
"type" : "boolean",
"default" : false
} ]
}

View File

@@ -0,0 +1,25 @@
###
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you 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
hoodie.datasource.write.recordkey.field=_row_key
hoodie.datasource.write.partitionpath.field=created_at
hoodie.deltastreamer.source.kafka.topic=topic1
hoodie.deltastreamer.keygen.timebased.timestamp.type=UNIX_TIMESTAMP
hoodie.deltastreamer.keygen.timebased.input.dateformat=yyyy-MM-dd HH:mm:ss.S
hoodie.datasource.hive_sync.database=uber_hive_db
hoodie.datasource.hive_sync.table=uber_hive_dummy_table