HUDI-70 : Making DeltaStreamer run in continuous mode with concurrent compaction
This commit is contained in:
committed by
Balaji Varadarajan
parent
3a210ef08e
commit
a0d7ab2384
@@ -24,11 +24,13 @@ import static org.junit.Assert.fail;
|
||||
|
||||
import com.uber.hoodie.DataSourceWriteOptions;
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieTableType;
|
||||
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.config.HoodieCompactionConfig;
|
||||
import com.uber.hoodie.exception.DatasetNotFoundException;
|
||||
import com.uber.hoodie.hive.HiveSyncConfig;
|
||||
import com.uber.hoodie.hive.HoodieHiveClient;
|
||||
@@ -36,17 +38,28 @@ import com.uber.hoodie.hive.MultiPartKeysValueExtractor;
|
||||
import com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer;
|
||||
import com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer.Operation;
|
||||
import com.uber.hoodie.utilities.schema.FilebasedSchemaProvider;
|
||||
import com.uber.hoodie.utilities.sources.DistributedTestDataSource;
|
||||
import com.uber.hoodie.utilities.sources.HoodieIncrSource;
|
||||
import com.uber.hoodie.utilities.sources.InputBatch;
|
||||
import com.uber.hoodie.utilities.sources.TestDataSource;
|
||||
import com.uber.hoodie.utilities.sources.config.TestSourceConfig;
|
||||
import com.uber.hoodie.utilities.transform.SqlQueryBasedTransformer;
|
||||
import com.uber.hoodie.utilities.transform.Transformer;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
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.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
@@ -57,6 +70,7 @@ import org.apache.spark.sql.functions;
|
||||
import org.apache.spark.sql.types.DataTypes;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
@@ -197,6 +211,22 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
assertEquals(expected, recordCount);
|
||||
}
|
||||
|
||||
static void assertAtleastNCompactionCommits(int minExpected, String datasetPath, FileSystem fs) {
|
||||
HoodieTableMetaClient meta = new HoodieTableMetaClient(fs.getConf(), datasetPath);
|
||||
HoodieTimeline timeline = meta.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
|
||||
log.info("Timeline Instants=" + meta.getActiveTimeline().getInstants().collect(Collectors.toList()));
|
||||
int numCompactionCommits = (int)timeline.getInstants().count();
|
||||
assertTrue("Got=" + numCompactionCommits + ", exp >=" + minExpected, minExpected <= numCompactionCommits);
|
||||
}
|
||||
|
||||
static void assertAtleastNDeltaCommits(int minExpected, String datasetPath, FileSystem fs) {
|
||||
HoodieTableMetaClient meta = new HoodieTableMetaClient(fs.getConf(), datasetPath);
|
||||
HoodieTimeline timeline = meta.getActiveTimeline().getDeltaCommitTimeline().filterCompletedInstants();
|
||||
log.info("Timeline Instants=" + meta.getActiveTimeline().getInstants().collect(Collectors.toList()));
|
||||
int numDeltaCommits = (int)timeline.getInstants().count();
|
||||
assertTrue("Got=" + numDeltaCommits + ", exp >=" + minExpected, minExpected <= numDeltaCommits);
|
||||
}
|
||||
|
||||
static String assertCommitMetadata(String expected, String datasetPath, FileSystem fs, int totalCommits)
|
||||
throws IOException {
|
||||
HoodieTableMetaClient meta = new HoodieTableMetaClient(fs.getConf(), datasetPath);
|
||||
@@ -208,6 +238,23 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
assertEquals(expected, commitMetadata.getMetadata(HoodieDeltaStreamer.CHECKPOINT_KEY));
|
||||
return lastInstant.getTimestamp();
|
||||
}
|
||||
|
||||
static void waitTillCondition(Function<Boolean, Boolean> condition, long timeoutInSecs) throws Exception {
|
||||
Future<Boolean> res = Executors.newSingleThreadExecutor().submit(() -> {
|
||||
boolean ret = false;
|
||||
while (!ret) {
|
||||
try {
|
||||
Thread.sleep(3000);
|
||||
ret = condition.apply(true);
|
||||
} catch (Throwable error) {
|
||||
log.warn("Got error :", error);
|
||||
ret = false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
});
|
||||
res.get(timeoutInSecs, TimeUnit.SECONDS);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -261,6 +308,51 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
assertEquals(2000, counts.get(0).getLong(1));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpsertsCOWContinuousMode() throws Exception {
|
||||
testUpsertsContinuousMode(HoodieTableType.COPY_ON_WRITE, "continuous_cow");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpsertsMORContinuousMode() throws Exception {
|
||||
testUpsertsContinuousMode(HoodieTableType.MERGE_ON_READ, "continuous_mor");
|
||||
}
|
||||
|
||||
private void testUpsertsContinuousMode(HoodieTableType tableType, String tempDir) throws Exception {
|
||||
String datasetBasePath = dfsBasePath + "/" + tempDir;
|
||||
// Keep it higher than batch-size to test continuous mode
|
||||
int totalRecords = 3000;
|
||||
|
||||
// Initial bulk insert
|
||||
HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(datasetBasePath, Operation.UPSERT);
|
||||
cfg.continuousMode = true;
|
||||
cfg.storageType = tableType.name();
|
||||
cfg.configs.add(String.format("%s=%d", TestSourceConfig.MAX_UNIQUE_RECORDS_PROP, totalRecords));
|
||||
cfg.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN_PROP));
|
||||
HoodieDeltaStreamer ds = new HoodieDeltaStreamer(cfg, jsc);
|
||||
Future dsFuture = Executors.newSingleThreadExecutor().submit(() -> {
|
||||
try {
|
||||
ds.sync();
|
||||
} catch (Exception ex) {
|
||||
throw new RuntimeException(ex.getMessage(), ex);
|
||||
}
|
||||
});
|
||||
|
||||
TestHelpers.waitTillCondition((r) -> {
|
||||
if (tableType.equals(HoodieTableType.MERGE_ON_READ)) {
|
||||
TestHelpers.assertAtleastNDeltaCommits(5, datasetBasePath, dfs);
|
||||
TestHelpers.assertAtleastNCompactionCommits(2, datasetBasePath, dfs);
|
||||
} else {
|
||||
TestHelpers.assertAtleastNCompactionCommits(5, datasetBasePath, dfs);
|
||||
}
|
||||
TestHelpers.assertRecordCount(totalRecords, datasetBasePath + "/*/*.parquet", sqlContext);
|
||||
TestHelpers.assertDistanceCount(totalRecords, datasetBasePath + "/*/*.parquet", sqlContext);
|
||||
return true;
|
||||
}, 180);
|
||||
ds.shutdownGracefully();
|
||||
dsFuture.get();
|
||||
}
|
||||
|
||||
/**
|
||||
* Test Bulk Insert and upserts with hive syncing. Tests Hudi incremental processing using a 2 step pipeline
|
||||
* The first step involves using a SQL template to transform a source
|
||||
@@ -366,6 +458,20 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
assertEquals(1000, counts.get(1).getLong(1));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDistributedTestDataSource() throws Exception {
|
||||
TypedProperties props = new TypedProperties();
|
||||
props.setProperty(TestSourceConfig.MAX_UNIQUE_RECORDS_PROP, "1000");
|
||||
props.setProperty(TestSourceConfig.NUM_SOURCE_PARTITIONS_PROP, "1");
|
||||
props.setProperty(TestSourceConfig.USE_ROCKSDB_FOR_TEST_DATAGEN_KEYS, "true");
|
||||
DistributedTestDataSource distributedTestDataSource = new DistributedTestDataSource(props,
|
||||
jsc, sparkSession, null);
|
||||
InputBatch<JavaRDD<GenericRecord>> batch = distributedTestDataSource.fetchNext(Optional.empty(), 10000000);
|
||||
batch.getBatch().get().cache();
|
||||
long c = batch.getBatch().get().count();
|
||||
Assert.assertEquals(1000, c);
|
||||
}
|
||||
|
||||
/**
|
||||
* UDF to calculate Haversine distance
|
||||
*/
|
||||
|
||||
@@ -0,0 +1,103 @@
|
||||
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.RocksDBBasedMap;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import com.uber.hoodie.utilities.schema.SchemaProvider;
|
||||
import com.uber.hoodie.utilities.sources.config.TestSourceConfig;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
|
||||
public abstract class AbstractBaseTestSource extends AvroSource {
|
||||
|
||||
// Static instance, helps with reuse across a test.
|
||||
protected static transient HoodieTestDataGenerator dataGenerator;
|
||||
|
||||
public static void initDataGen() {
|
||||
dataGenerator = new HoodieTestDataGenerator(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS);
|
||||
}
|
||||
|
||||
public static void initDataGen(TypedProperties props) {
|
||||
try {
|
||||
boolean useRocksForTestDataGenKeys = props.getBoolean(TestSourceConfig.USE_ROCKSDB_FOR_TEST_DATAGEN_KEYS,
|
||||
TestSourceConfig.DEFAULT_USE_ROCKSDB_FOR_TEST_DATAGEN_KEYS);
|
||||
String baseStoreDir = props.getString(TestSourceConfig.ROCKSDB_BASE_DIR_FOR_TEST_DATAGEN_KEYS, null);
|
||||
if (null == baseStoreDir) {
|
||||
baseStoreDir = File.createTempFile("test_data_gen", ".keys").getParent();
|
||||
}
|
||||
log.info("useRocksForTestDataGenKeys=" + useRocksForTestDataGenKeys + ", BaseStoreDir=" + baseStoreDir);
|
||||
dataGenerator = new HoodieTestDataGenerator(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS,
|
||||
useRocksForTestDataGenKeys ? new RocksDBBasedMap<>(baseStoreDir) : new HashMap<>());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
|
||||
public static void resetDataGen() {
|
||||
if (null != dataGenerator) {
|
||||
dataGenerator.close();
|
||||
}
|
||||
dataGenerator = null;
|
||||
}
|
||||
|
||||
protected AbstractBaseTestSource(TypedProperties props,
|
||||
JavaSparkContext sparkContext, SparkSession sparkSession,
|
||||
SchemaProvider schemaProvider) {
|
||||
super(props, sparkContext, sparkSession, schemaProvider);
|
||||
}
|
||||
|
||||
protected static Stream<GenericRecord> fetchNextBatch(TypedProperties props, int sourceLimit, String commitTime) {
|
||||
int maxUniqueKeys = props.getInteger(TestSourceConfig.MAX_UNIQUE_RECORDS_PROP,
|
||||
TestSourceConfig.DEFAULT_MAX_UNIQUE_RECORDS);
|
||||
|
||||
// generate `sourceLimit` number of upserts each time.
|
||||
int numExistingKeys = dataGenerator.getNumExistingKeys();
|
||||
log.info("NumExistingKeys=" + numExistingKeys);
|
||||
|
||||
int numUpdates = Math.min(numExistingKeys, sourceLimit / 2);
|
||||
int numInserts = sourceLimit - numUpdates;
|
||||
log.info("Before adjustments => numInserts=" + numInserts + ", numUpdates=" + numUpdates);
|
||||
|
||||
if (numInserts + numExistingKeys > maxUniqueKeys) {
|
||||
// Limit inserts so that maxUniqueRecords is maintained
|
||||
numInserts = Math.max(0, maxUniqueKeys - numExistingKeys);
|
||||
}
|
||||
|
||||
if ((numInserts + numUpdates) < sourceLimit) {
|
||||
// try to expand updates to safe limit
|
||||
numUpdates = Math.min(numExistingKeys, sourceLimit - numInserts);
|
||||
}
|
||||
|
||||
log.info("NumInserts=" + numInserts + ", NumUpdates=" + numUpdates + ", maxUniqueRecords=" + maxUniqueKeys);
|
||||
long memoryUsage1 = Runtime.getRuntime().totalMemory() - Runtime.getRuntime().freeMemory();
|
||||
log.info("Before DataGen. Memory Usage=" + memoryUsage1 + ", Total Memory=" + Runtime.getRuntime().totalMemory()
|
||||
+ ", Free Memory=" + Runtime.getRuntime().freeMemory());
|
||||
|
||||
List<GenericRecord> records = new ArrayList<>();
|
||||
Stream<GenericRecord> updateStream = dataGenerator.generateUniqueUpdatesStream(commitTime, numUpdates)
|
||||
.map(AbstractBaseTestSource::toGenericRecord);
|
||||
Stream<GenericRecord> insertStream = dataGenerator.generateInsertsStream(commitTime, numInserts)
|
||||
.map(AbstractBaseTestSource::toGenericRecord);
|
||||
return Stream.concat(updateStream, insertStream);
|
||||
}
|
||||
|
||||
private static GenericRecord toGenericRecord(HoodieRecord hoodieRecord) {
|
||||
try {
|
||||
Optional<IndexedRecord> recordOpt = hoodieRecord.getData().getInsertValue(dataGenerator.avroSchema);
|
||||
return (GenericRecord) recordOpt.get();
|
||||
} catch (IOException e) {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,79 @@
|
||||
/*
|
||||
* Copyright (c) 2019 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.util.TypedProperties;
|
||||
import com.uber.hoodie.utilities.schema.SchemaProvider;
|
||||
import com.uber.hoodie.utilities.sources.config.TestSourceConfig;
|
||||
import java.util.Iterator;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
|
||||
/**
|
||||
* A Test DataSource which scales test-data generation by using spark parallelism.
|
||||
*/
|
||||
public class DistributedTestDataSource extends AbstractBaseTestSource {
|
||||
|
||||
private final int numTestSourcePartitions;
|
||||
|
||||
public DistributedTestDataSource(TypedProperties props,
|
||||
JavaSparkContext sparkContext, SparkSession sparkSession,
|
||||
SchemaProvider schemaProvider) {
|
||||
super(props, sparkContext, sparkSession, schemaProvider);
|
||||
this.numTestSourcePartitions = props.getInteger(TestSourceConfig.NUM_SOURCE_PARTITIONS_PROP,
|
||||
TestSourceConfig.DEFAULT_NUM_SOURCE_PARTITIONS);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected InputBatch<JavaRDD<GenericRecord>> fetchNewData(Optional<String> lastCkptStr, long sourceLimit) {
|
||||
int nextCommitNum = lastCkptStr.map(s -> Integer.parseInt(s) + 1).orElse(0);
|
||||
String commitTime = String.format("%05d", nextCommitNum);
|
||||
log.info("Source Limit is set to " + sourceLimit);
|
||||
|
||||
// No new data.
|
||||
if (sourceLimit <= 0) {
|
||||
return new InputBatch<>(Optional.empty(), commitTime);
|
||||
}
|
||||
|
||||
TypedProperties newProps = new TypedProperties();
|
||||
newProps.putAll(props);
|
||||
|
||||
// Set the maxUniqueRecords per partition for TestDataSource
|
||||
int maxUniqueRecords = props.getInteger(TestSourceConfig.MAX_UNIQUE_RECORDS_PROP,
|
||||
TestSourceConfig.DEFAULT_MAX_UNIQUE_RECORDS);
|
||||
String maxUniqueRecordsPerPartition = String.valueOf(Math.max(1, maxUniqueRecords / numTestSourcePartitions));
|
||||
newProps.setProperty(TestSourceConfig.MAX_UNIQUE_RECORDS_PROP, maxUniqueRecordsPerPartition);
|
||||
int perPartitionSourceLimit = Math.max(1, (int) (sourceLimit / numTestSourcePartitions));
|
||||
JavaRDD<GenericRecord> avroRDD = sparkContext.parallelize(IntStream.range(0, numTestSourcePartitions).boxed()
|
||||
.collect(Collectors.toList()), numTestSourcePartitions).mapPartitions(idx -> {
|
||||
log.info("Initializing source with newProps=" + newProps);
|
||||
if (null == dataGenerator) {
|
||||
initDataGen(newProps);
|
||||
}
|
||||
Iterator<GenericRecord> itr = fetchNextBatch(newProps, perPartitionSourceLimit, commitTime).iterator();
|
||||
return itr;
|
||||
});
|
||||
return new InputBatch<>(Optional.of(avroRDD), commitTime);
|
||||
}
|
||||
}
|
||||
@@ -18,17 +18,12 @@
|
||||
|
||||
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.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;
|
||||
@@ -38,32 +33,15 @@ import org.apache.spark.sql.SparkSession;
|
||||
/**
|
||||
* An implementation of {@link Source}, that emits test upserts.
|
||||
*/
|
||||
public class TestDataSource extends AvroSource {
|
||||
public class TestDataSource extends AbstractBaseTestSource {
|
||||
|
||||
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, SparkSession sparkSession,
|
||||
SchemaProvider schemaProvider) {
|
||||
super(props, sparkContext, sparkSession, schemaProvider);
|
||||
}
|
||||
|
||||
private GenericRecord toGenericRecord(HoodieRecord hoodieRecord) {
|
||||
try {
|
||||
Optional<IndexedRecord> recordOpt = hoodieRecord.getData().getInsertValue(dataGenerator.avroSchema);
|
||||
return (GenericRecord) recordOpt.get();
|
||||
} catch (IOException e) {
|
||||
return null;
|
||||
if (null == dataGenerator) {
|
||||
initDataGen(props);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -73,26 +51,14 @@ public class TestDataSource extends AvroSource {
|
||||
|
||||
int nextCommitNum = lastCheckpointStr.map(s -> Integer.parseInt(s) + 1).orElse(0);
|
||||
String commitTime = String.format("%05d", nextCommitNum);
|
||||
log.info("Source Limit is set to " + sourceLimit);
|
||||
|
||||
// No new data.
|
||||
if (sourceLimit <= 0) {
|
||||
return new InputBatch<>(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);
|
||||
}
|
||||
|
||||
List<GenericRecord> records = fetchNextBatch(props, (int)sourceLimit, commitTime).collect(Collectors.toList());
|
||||
JavaRDD<GenericRecord> avroRDD = sparkContext.<GenericRecord>parallelize(records, 4);
|
||||
return new InputBatch<>(Optional.of(avroRDD), commitTime);
|
||||
}
|
||||
|
||||
@@ -0,0 +1,43 @@
|
||||
/*
|
||||
* Copyright (c) 2019 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.config;
|
||||
|
||||
/**
|
||||
* Configurations for Test Data Sources
|
||||
*/
|
||||
public class TestSourceConfig {
|
||||
|
||||
// Used by DistributedTestDataSource only. Number of partitions where each partitions generates test-data
|
||||
public static final String NUM_SOURCE_PARTITIONS_PROP = "hoodie.deltastreamer.source.test.num_partitions";
|
||||
public static final Integer DEFAULT_NUM_SOURCE_PARTITIONS = 10;
|
||||
|
||||
// Maximum number of unique records generated for the run
|
||||
public static final String MAX_UNIQUE_RECORDS_PROP = "hoodie.deltastreamer.source.test.max_unique_records";
|
||||
public static final Integer DEFAULT_MAX_UNIQUE_RECORDS = Integer.MAX_VALUE;
|
||||
|
||||
// Use Rocks DB for storing datagen keys
|
||||
public static final String USE_ROCKSDB_FOR_TEST_DATAGEN_KEYS =
|
||||
"hoodie.deltastreamer.source.test.datagen.use_rocksdb_for_storing_existing_keys";
|
||||
public static final Boolean DEFAULT_USE_ROCKSDB_FOR_TEST_DATAGEN_KEYS = false;
|
||||
|
||||
// Base Dir for storing datagen keys
|
||||
public static final String ROCKSDB_BASE_DIR_FOR_TEST_DATAGEN_KEYS =
|
||||
"hoodie.deltastreamer.source.test.datagen.rocksdb_base_dir";
|
||||
|
||||
}
|
||||
Reference in New Issue
Block a user