[HUDI-296] Explore use of spotless to auto fix formatting errors (#945)
- Add spotless format fixing to project - One time reformatting for conformity - Build fails for formatting changes and mvn spotless:apply autofixes them
This commit is contained in:
@@ -43,8 +43,8 @@ public class SchedulerConfGeneratorTest {
|
||||
cfg.continuousMode = true;
|
||||
cfg.storageType = HoodieTableType.COPY_ON_WRITE.name();
|
||||
configs = SchedulerConfGenerator.getSparkSchedulingConfigs(cfg);
|
||||
assertNull("storageType is not MERGE_ON_READ", configs.get(
|
||||
SchedulerConfGenerator.SPARK_SCHEDULER_ALLOCATION_FILE_KEY));
|
||||
assertNull("storageType is not MERGE_ON_READ",
|
||||
configs.get(SchedulerConfGenerator.SPARK_SCHEDULER_ALLOCATION_FILE_KEY));
|
||||
|
||||
cfg.storageType = HoodieTableType.MERGE_ON_READ.name();
|
||||
configs = SchedulerConfGenerator.getSparkSchedulingConfigs(cfg);
|
||||
|
||||
@@ -34,23 +34,20 @@ public class TestFlatteningTransformer {
|
||||
FlatteningTransformer transformer = new FlatteningTransformer();
|
||||
|
||||
// Init
|
||||
StructField[] nestedStructFields = new StructField[]{
|
||||
new StructField("nestedIntColumn", DataTypes.IntegerType, true, Metadata.empty()),
|
||||
new StructField("nestedStringColumn", DataTypes.StringType, true, Metadata.empty()),
|
||||
};
|
||||
StructField[] nestedStructFields =
|
||||
new StructField[] {new StructField("nestedIntColumn", DataTypes.IntegerType, true, Metadata.empty()),
|
||||
new StructField("nestedStringColumn", DataTypes.StringType, true, Metadata.empty()),};
|
||||
|
||||
StructField[] structFields = new StructField[]{
|
||||
new StructField("intColumn", DataTypes.IntegerType, true, Metadata.empty()),
|
||||
new StructField("stringColumn", DataTypes.StringType, true, Metadata.empty()),
|
||||
new StructField("nestedStruct", DataTypes.createStructType(nestedStructFields), true, Metadata.empty())
|
||||
};
|
||||
StructField[] structFields =
|
||||
new StructField[] {new StructField("intColumn", DataTypes.IntegerType, true, Metadata.empty()),
|
||||
new StructField("stringColumn", DataTypes.StringType, true, Metadata.empty()),
|
||||
new StructField("nestedStruct", DataTypes.createStructType(nestedStructFields), true, Metadata.empty())};
|
||||
|
||||
StructType schema = new StructType(structFields);
|
||||
String flattenedSql = transformer.flattenSchema(schema, null);
|
||||
|
||||
assertEquals("intColumn as intColumn,stringColumn as stringColumn,"
|
||||
+ "nestedStruct.nestedIntColumn as nestedStruct_nestedIntColumn,"
|
||||
+ "nestedStruct.nestedStringColumn as nestedStruct_nestedStringColumn",
|
||||
flattenedSql);
|
||||
+ "nestedStruct.nestedIntColumn as nestedStruct_nestedIntColumn,"
|
||||
+ "nestedStruct.nestedStringColumn as nestedStruct_nestedStringColumn", flattenedSql);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -90,8 +90,7 @@ public class TestHDFSParquetImporter implements Serializable {
|
||||
jsc = getJavaSparkContext();
|
||||
|
||||
// Test root folder.
|
||||
String basePath = (new Path(dfsBasePath,
|
||||
Thread.currentThread().getStackTrace()[1].getMethodName())).toString();
|
||||
String basePath = (new Path(dfsBasePath, Thread.currentThread().getStackTrace()[1].getMethodName())).toString();
|
||||
|
||||
// Hoodie root folder
|
||||
Path hoodieFolder = new Path(basePath, "testTarget");
|
||||
@@ -99,13 +98,12 @@ public class TestHDFSParquetImporter implements Serializable {
|
||||
// Create schema file.
|
||||
String schemaFile = new Path(basePath, "file.schema").toString();
|
||||
|
||||
//Create generic records.
|
||||
// Create generic records.
|
||||
Path srcFolder = new Path(basePath, "testSrc");
|
||||
createRecords(srcFolder);
|
||||
|
||||
HDFSParquetImporter.Config cfg = getHDFSParquetImporterConfig(srcFolder.toString(),
|
||||
hoodieFolder.toString(), "testTable", "COPY_ON_WRITE", "_row_key", "timestamp", 1,
|
||||
schemaFile);
|
||||
HDFSParquetImporter.Config cfg = getHDFSParquetImporterConfig(srcFolder.toString(), hoodieFolder.toString(),
|
||||
"testTable", "COPY_ON_WRITE", "_row_key", "timestamp", 1, schemaFile);
|
||||
AtomicInteger retry = new AtomicInteger(3);
|
||||
AtomicInteger fileCreated = new AtomicInteger(0);
|
||||
HDFSParquetImporter dataImporter = new HDFSParquetImporter(cfg) {
|
||||
@@ -134,8 +132,7 @@ public class TestHDFSParquetImporter implements Serializable {
|
||||
RemoteIterator<LocatedFileStatus> hoodieFiles = dfs.listFiles(hoodieFolder, true);
|
||||
while (hoodieFiles.hasNext()) {
|
||||
LocatedFileStatus f = hoodieFiles.next();
|
||||
isCommitFilePresent =
|
||||
isCommitFilePresent || f.getPath().toString().endsWith(HoodieTimeline.COMMIT_EXTENSION);
|
||||
isCommitFilePresent = isCommitFilePresent || f.getPath().toString().endsWith(HoodieTimeline.COMMIT_EXTENSION);
|
||||
|
||||
if (f.getPath().toString().endsWith("parquet")) {
|
||||
SQLContext sc = new SQLContext(jsc);
|
||||
@@ -164,14 +161,11 @@ public class TestHDFSParquetImporter implements Serializable {
|
||||
long startTime = HoodieActiveTimeline.COMMIT_FORMATTER.parse("20170203000000").getTime() / 1000;
|
||||
List<GenericRecord> records = new ArrayList<GenericRecord>();
|
||||
for (long recordNum = 0; recordNum < 96; recordNum++) {
|
||||
records.add(HoodieTestDataGenerator
|
||||
.generateGenericRecord(Long.toString(recordNum), "rider-" + recordNum,
|
||||
"driver-" + recordNum, startTime + TimeUnit.HOURS.toSeconds(recordNum)));
|
||||
records.add(HoodieTestDataGenerator.generateGenericRecord(Long.toString(recordNum), "rider-" + recordNum,
|
||||
"driver-" + recordNum, startTime + TimeUnit.HOURS.toSeconds(recordNum)));
|
||||
}
|
||||
ParquetWriter<GenericRecord> writer = AvroParquetWriter.<GenericRecord>builder(srcFile)
|
||||
.withSchema(HoodieTestDataGenerator.avroSchema)
|
||||
.withConf(HoodieTestUtils.getDefaultHadoopConf())
|
||||
.build();
|
||||
.withSchema(HoodieTestDataGenerator.avroSchema).withConf(HoodieTestUtils.getDefaultHadoopConf()).build();
|
||||
for (GenericRecord record : records) {
|
||||
writer.write(record);
|
||||
}
|
||||
@@ -194,15 +188,13 @@ public class TestHDFSParquetImporter implements Serializable {
|
||||
jsc = getJavaSparkContext();
|
||||
|
||||
// Test root folder.
|
||||
String basePath = (new Path(dfsBasePath,
|
||||
Thread.currentThread().getStackTrace()[1].getMethodName())).toString();
|
||||
String basePath = (new Path(dfsBasePath, Thread.currentThread().getStackTrace()[1].getMethodName())).toString();
|
||||
// Hoodie root folder
|
||||
Path hoodieFolder = new Path(basePath, "testTarget");
|
||||
Path srcFolder = new Path(basePath.toString(), "srcTest");
|
||||
Path schemaFile = new Path(basePath.toString(), "missingFile.schema");
|
||||
HDFSParquetImporter.Config cfg = getHDFSParquetImporterConfig(srcFolder.toString(),
|
||||
hoodieFolder.toString(), "testTable", "COPY_ON_WRITE", "_row_key", "timestamp", 1,
|
||||
schemaFile.toString());
|
||||
HDFSParquetImporter.Config cfg = getHDFSParquetImporterConfig(srcFolder.toString(), hoodieFolder.toString(),
|
||||
"testTable", "COPY_ON_WRITE", "_row_key", "timestamp", 1, schemaFile.toString());
|
||||
HDFSParquetImporter dataImporter = new HDFSParquetImporter(cfg);
|
||||
// Should fail - return : -1.
|
||||
assertEquals(-1, dataImporter.dataImport(jsc, 0));
|
||||
@@ -228,12 +220,11 @@ public class TestHDFSParquetImporter implements Serializable {
|
||||
jsc = getJavaSparkContext();
|
||||
|
||||
// Test root folder.
|
||||
String basePath = (new Path(dfsBasePath,
|
||||
Thread.currentThread().getStackTrace()[1].getMethodName())).toString();
|
||||
String basePath = (new Path(dfsBasePath, Thread.currentThread().getStackTrace()[1].getMethodName())).toString();
|
||||
// Hoodie root folder
|
||||
Path hoodieFolder = new Path(basePath, "testTarget");
|
||||
|
||||
//Create generic records.
|
||||
// Create generic records.
|
||||
Path srcFolder = new Path(basePath, "testSrc");
|
||||
createRecords(srcFolder);
|
||||
|
||||
@@ -245,14 +236,14 @@ public class TestHDFSParquetImporter implements Serializable {
|
||||
HDFSParquetImporter.Config cfg;
|
||||
|
||||
// Check for invalid row key.
|
||||
cfg = getHDFSParquetImporterConfig(srcFolder.toString(), hoodieFolder.toString(), "testTable",
|
||||
"COPY_ON_WRITE", "invalidRowKey", "timestamp", 1, schemaFile.toString());
|
||||
cfg = getHDFSParquetImporterConfig(srcFolder.toString(), hoodieFolder.toString(), "testTable", "COPY_ON_WRITE",
|
||||
"invalidRowKey", "timestamp", 1, schemaFile.toString());
|
||||
dataImporter = new HDFSParquetImporter(cfg);
|
||||
assertEquals(-1, dataImporter.dataImport(jsc, 0));
|
||||
|
||||
// Check for invalid partition key.
|
||||
cfg = getHDFSParquetImporterConfig(srcFolder.toString(), hoodieFolder.toString(), "testTable",
|
||||
"COPY_ON_WRITE", "_row_key", "invalidTimeStamp", 1, schemaFile.toString());
|
||||
cfg = getHDFSParquetImporterConfig(srcFolder.toString(), hoodieFolder.toString(), "testTable", "COPY_ON_WRITE",
|
||||
"_row_key", "invalidTimeStamp", 1, schemaFile.toString());
|
||||
dataImporter = new HDFSParquetImporter(cfg);
|
||||
assertEquals(-1, dataImporter.dataImport(jsc, 0));
|
||||
|
||||
@@ -263,9 +254,8 @@ public class TestHDFSParquetImporter implements Serializable {
|
||||
}
|
||||
}
|
||||
|
||||
private HDFSParquetImporter.Config getHDFSParquetImporterConfig(String srcPath, String targetPath,
|
||||
String tableName, String tableType, String rowKey, String partitionKey, int parallelism,
|
||||
String schemaFile) {
|
||||
private HDFSParquetImporter.Config getHDFSParquetImporterConfig(String srcPath, String targetPath, String tableName,
|
||||
String tableType, String rowKey, String partitionKey, int parallelism, String schemaFile) {
|
||||
HDFSParquetImporter.Config cfg = new HDFSParquetImporter.Config();
|
||||
cfg.srcPath = srcPath;
|
||||
cfg.targetPath = targetPath;
|
||||
|
||||
@@ -77,8 +77,7 @@ import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
/**
|
||||
* Basic tests against {@link HoodieDeltaStreamer}, by issuing bulk_inserts,
|
||||
* upserts, inserts. Check counts at the end.
|
||||
* Basic tests against {@link HoodieDeltaStreamer}, by issuing bulk_inserts, upserts, inserts. Check counts at the end.
|
||||
*/
|
||||
public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
|
||||
@@ -124,8 +123,7 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
// Source schema is the target schema of upstream table
|
||||
downstreamProps.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", dfsBasePath + "/target.avsc");
|
||||
downstreamProps.setProperty("hoodie.deltastreamer.schemaprovider.target.schema.file", dfsBasePath + "/target.avsc");
|
||||
UtilitiesTestBase.Helpers.savePropsToDFS(downstreamProps, dfs,
|
||||
dfsBasePath + "/test-downstream-source.properties");
|
||||
UtilitiesTestBase.Helpers.savePropsToDFS(downstreamProps, dfs, dfsBasePath + "/test-downstream-source.properties");
|
||||
|
||||
// Properties used for testing invalid key generator
|
||||
TypedProperties invalidProps = new TypedProperties();
|
||||
@@ -135,8 +133,7 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
invalidProps.setProperty("hoodie.datasource.write.partitionpath.field", "not_there");
|
||||
invalidProps.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", dfsBasePath + "/source.avsc");
|
||||
invalidProps.setProperty("hoodie.deltastreamer.schemaprovider.target.schema.file", dfsBasePath + "/target.avsc");
|
||||
UtilitiesTestBase.Helpers.savePropsToDFS(invalidProps, dfs,
|
||||
dfsBasePath + "/" + PROPS_FILENAME_TEST_INVALID);
|
||||
UtilitiesTestBase.Helpers.savePropsToDFS(invalidProps, dfs, dfsBasePath + "/" + PROPS_FILENAME_TEST_INVALID);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
@@ -235,7 +232,7 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
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();
|
||||
int numCompactionCommits = (int) timeline.getInstants().count();
|
||||
assertTrue("Got=" + numCompactionCommits + ", exp >=" + minExpected, minExpected <= numCompactionCommits);
|
||||
}
|
||||
|
||||
@@ -243,7 +240,7 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
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();
|
||||
int numDeltaCommits = (int) timeline.getInstants().count();
|
||||
assertTrue("Got=" + numDeltaCommits + ", exp >=" + minExpected, minExpected <= numDeltaCommits);
|
||||
}
|
||||
|
||||
@@ -252,8 +249,8 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
HoodieTableMetaClient meta = new HoodieTableMetaClient(fs.getConf(), datasetPath);
|
||||
HoodieTimeline timeline = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
|
||||
HoodieInstant lastInstant = timeline.lastInstant().get();
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(
|
||||
timeline.getInstantDetails(lastInstant).get(), HoodieCommitMetadata.class);
|
||||
HoodieCommitMetadata commitMetadata =
|
||||
HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(lastInstant).get(), HoodieCommitMetadata.class);
|
||||
assertEquals(totalCommits, timeline.countInstants());
|
||||
assertEquals(expected, commitMetadata.getMetadata(HoodieDeltaStreamer.CHECKPOINT_KEY));
|
||||
return lastInstant.getTimestamp();
|
||||
@@ -279,28 +276,25 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
|
||||
@Test
|
||||
public void testProps() throws IOException {
|
||||
TypedProperties props = new DFSPropertiesConfiguration(
|
||||
dfs, new Path(dfsBasePath + "/" + PROPS_FILENAME_TEST_SOURCE)).getConfig();
|
||||
TypedProperties props =
|
||||
new DFSPropertiesConfiguration(dfs, new Path(dfsBasePath + "/" + PROPS_FILENAME_TEST_SOURCE)).getConfig();
|
||||
assertEquals(2, props.getInteger("hoodie.upsert.shuffle.parallelism"));
|
||||
assertEquals("_row_key", props.getString("hoodie.datasource.write.recordkey.field"));
|
||||
assertEquals(
|
||||
"org.apache.hudi.utilities.TestHoodieDeltaStreamer$TestGenerator",
|
||||
props.getString("hoodie.datasource.write.keygenerator.class")
|
||||
);
|
||||
assertEquals("org.apache.hudi.utilities.TestHoodieDeltaStreamer$TestGenerator",
|
||||
props.getString("hoodie.datasource.write.keygenerator.class"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPropsWithInvalidKeyGenerator() throws Exception {
|
||||
try {
|
||||
String datasetBasePath = dfsBasePath + "/test_dataset";
|
||||
HoodieDeltaStreamer deltaStreamer = new HoodieDeltaStreamer(
|
||||
TestHelpers.makeConfig(
|
||||
datasetBasePath, Operation.BULK_INSERT, TripsWithDistanceTransformer.class.getName(),
|
||||
PROPS_FILENAME_TEST_INVALID, false), jsc);
|
||||
HoodieDeltaStreamer deltaStreamer =
|
||||
new HoodieDeltaStreamer(TestHelpers.makeConfig(datasetBasePath, Operation.BULK_INSERT,
|
||||
TripsWithDistanceTransformer.class.getName(), PROPS_FILENAME_TEST_INVALID, false), jsc);
|
||||
deltaStreamer.sync();
|
||||
fail("Should error out when setting the key generator class property to an invalid value");
|
||||
} catch (IOException e) {
|
||||
//expected
|
||||
// expected
|
||||
log.error("Expected error during getting the key generator", e);
|
||||
assertTrue(e.getMessage().contains("Could not load key generator class"));
|
||||
}
|
||||
@@ -310,12 +304,12 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
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);
|
||||
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
|
||||
// expected
|
||||
log.error("Expected error during dataset creation", e);
|
||||
}
|
||||
}
|
||||
@@ -395,11 +389,9 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
}
|
||||
|
||||
/**
|
||||
* 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
|
||||
* TEST-DATA-SOURCE ============================> HUDI TABLE 1 ===============> HUDI TABLE 2
|
||||
* (incr-pull with transform) (incr-pull)
|
||||
* Hudi Table 1 is synced with Hive.
|
||||
* 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 TEST-DATA-SOURCE ============================> HUDI TABLE
|
||||
* 1 ===============> HUDI TABLE 2 (incr-pull with transform) (incr-pull) Hudi Table 1 is synced with Hive.
|
||||
*/
|
||||
@Test
|
||||
public void testBulkInsertsAndUpsertsWithSQLBasedTransformerFor2StepPipeline() throws Exception {
|
||||
@@ -467,12 +459,11 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
|
||||
// Test Hive integration
|
||||
HoodieHiveClient hiveClient = new HoodieHiveClient(hiveSyncConfig, hiveServer.getHiveConf(), dfs);
|
||||
assertTrue("Table " + hiveSyncConfig.tableName + " should exist",
|
||||
hiveClient.doesTableExist());
|
||||
assertTrue("Table " + hiveSyncConfig.tableName + " should exist", hiveClient.doesTableExist());
|
||||
assertEquals("Table partitions should match the number of partitions we wrote", 1,
|
||||
hiveClient.scanTablePartitions().size());
|
||||
assertEquals("The last commit that was sycned should be updated in the TBLPROPERTIES",
|
||||
lastInstantForUpstreamTable, hiveClient.getLastCommitTimeSynced().get());
|
||||
assertEquals("The last commit that was sycned should be updated in the TBLPROPERTIES", lastInstantForUpstreamTable,
|
||||
hiveClient.getLastCommitTimeSynced().get());
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -499,8 +490,7 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
|
||||
// Test with empty commits
|
||||
HoodieTableMetaClient mClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), datasetBasePath, true);
|
||||
HoodieInstant lastFinished =
|
||||
mClient.getCommitsTimeline().filterCompletedInstants().lastInstant().get();
|
||||
HoodieInstant lastFinished = mClient.getCommitsTimeline().filterCompletedInstants().lastInstant().get();
|
||||
HoodieDeltaStreamer.Config cfg2 = TestHelpers.makeDropAllConfig(datasetBasePath, Operation.UPSERT);
|
||||
cfg2.filterDupes = true;
|
||||
cfg2.sourceLimit = 2000;
|
||||
@@ -509,14 +499,13 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
HoodieDeltaStreamer ds2 = new HoodieDeltaStreamer(cfg2, jsc);
|
||||
ds2.sync();
|
||||
mClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), datasetBasePath, true);
|
||||
HoodieInstant newLastFinished =
|
||||
mClient.getCommitsTimeline().filterCompletedInstants().lastInstant().get();
|
||||
HoodieInstant newLastFinished = mClient.getCommitsTimeline().filterCompletedInstants().lastInstant().get();
|
||||
Assert.assertTrue(HoodieTimeline.compareTimestamps(newLastFinished.getTimestamp(), lastFinished.getTimestamp(),
|
||||
HoodieTimeline.GREATER));
|
||||
|
||||
// Ensure it is empty
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(
|
||||
mClient.getActiveTimeline().getInstantDetails(newLastFinished).get(), HoodieCommitMetadata.class);
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(mClient.getActiveTimeline().getInstantDetails(newLastFinished).get(), HoodieCommitMetadata.class);
|
||||
System.out.println("New Commit Metadata=" + commitMetadata);
|
||||
Assert.assertTrue(commitMetadata.getPartitionToWriteStats().isEmpty());
|
||||
}
|
||||
@@ -527,8 +516,7 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
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);
|
||||
DistributedTestDataSource distributedTestDataSource = new DistributedTestDataSource(props, jsc, sparkSession, null);
|
||||
InputBatch<JavaRDD<GenericRecord>> batch = distributedTestDataSource.fetchNext(Option.empty(), 10000000);
|
||||
batch.getBatch().get().cache();
|
||||
long c = batch.getBatch().get().count();
|
||||
@@ -542,13 +530,10 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
|
||||
/**
|
||||
* Taken from https://stackoverflow.com/questions/3694380/calculating-distance-between-two-points-using-latitude-
|
||||
* longitude-what-am-i-doi
|
||||
* Calculate distance between two points in latitude and longitude taking
|
||||
* into account height difference. If you are not interested in height
|
||||
* difference pass 0.0. Uses Haversine method as its base.
|
||||
* longitude-what-am-i-doi Calculate distance between two points in latitude and longitude taking into account
|
||||
* height difference. If you are not interested in height difference pass 0.0. Uses Haversine method as its base.
|
||||
*
|
||||
* lat1, lon1 Start point lat2, lon2 End point el1 Start altitude in meters
|
||||
* el2 End altitude in meters
|
||||
* lat1, lon1 Start point lat2, lon2 End point el1 Start altitude in meters el2 End altitude in meters
|
||||
*
|
||||
* @returns Distance in Meters
|
||||
*/
|
||||
@@ -559,9 +544,8 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
|
||||
double latDistance = Math.toRadians(lat2 - lat1);
|
||||
double lonDistance = Math.toRadians(lon2 - lon1);
|
||||
double a = Math.sin(latDistance / 2) * Math.sin(latDistance / 2)
|
||||
+ Math.cos(Math.toRadians(lat1)) * Math.cos(Math.toRadians(lat2))
|
||||
* Math.sin(lonDistance / 2) * Math.sin(lonDistance / 2);
|
||||
double a = Math.sin(latDistance / 2) * Math.sin(latDistance / 2) + Math.cos(Math.toRadians(lat1))
|
||||
* Math.cos(Math.toRadians(lat2)) * Math.sin(lonDistance / 2) * Math.sin(lonDistance / 2);
|
||||
double c = 2 * Math.atan2(Math.sqrt(a), Math.sqrt(1 - a));
|
||||
double distance = R * c * 1000; // convert to meters
|
||||
|
||||
@@ -579,12 +563,11 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
public static class TripsWithDistanceTransformer implements Transformer {
|
||||
|
||||
@Override
|
||||
public Dataset<Row> apply(JavaSparkContext jsc, SparkSession sparkSession,
|
||||
Dataset<Row> rowDataset, TypedProperties properties) {
|
||||
public Dataset<Row> apply(JavaSparkContext jsc, SparkSession sparkSession, Dataset<Row> rowDataset,
|
||||
TypedProperties properties) {
|
||||
rowDataset.sqlContext().udf().register("distance_udf", new DistanceUDF(), DataTypes.DoubleType);
|
||||
return rowDataset.withColumn("haversine_distance",
|
||||
functions.callUDF("distance_udf", functions.col("begin_lat"),
|
||||
functions.col("end_lat"), functions.col("begin_lon"), functions.col("end_lat")));
|
||||
return rowDataset.withColumn("haversine_distance", functions.callUDF("distance_udf", functions.col("begin_lat"),
|
||||
functions.col("end_lat"), functions.col("begin_lon"), functions.col("end_lat")));
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -77,8 +77,8 @@ public class TestHoodieSnapshotCopier extends HoodieCommonTestHarness {
|
||||
assertFalse(fs.exists(new Path(outputPath + "/_SUCCESS")));
|
||||
}
|
||||
|
||||
//TODO - uncomment this after fixing test failures
|
||||
//@Test
|
||||
// TODO - uncomment this after fixing test failures
|
||||
// @Test
|
||||
public void testSnapshotCopy() throws Exception {
|
||||
// Generate some commits and corresponding parquets
|
||||
String commitTime1 = "20160501010101";
|
||||
@@ -95,40 +95,30 @@ public class TestHoodieSnapshotCopier extends HoodieCommonTestHarness {
|
||||
new File(basePath + "/2016/05/01/").mkdirs();
|
||||
new File(basePath + "/2016/05/02/").mkdirs();
|
||||
new File(basePath + "/2016/05/06/").mkdirs();
|
||||
HoodieTestDataGenerator
|
||||
.writePartitionMetadata(fs, new String[] {"2016/05/01", "2016/05/02", "2016/05/06"},
|
||||
basePath);
|
||||
HoodieTestDataGenerator.writePartitionMetadata(fs, new String[] {"2016/05/01", "2016/05/02", "2016/05/06"},
|
||||
basePath);
|
||||
// Make commit1
|
||||
File file11 = new File(
|
||||
basePath + "/2016/05/01/" + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, "id11"));
|
||||
File file11 = new File(basePath + "/2016/05/01/" + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, "id11"));
|
||||
file11.createNewFile();
|
||||
File file12 = new File(
|
||||
basePath + "/2016/05/02/" + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, "id12"));
|
||||
File file12 = new File(basePath + "/2016/05/02/" + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, "id12"));
|
||||
file12.createNewFile();
|
||||
File file13 = new File(
|
||||
basePath + "/2016/05/06/" + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, "id13"));
|
||||
File file13 = new File(basePath + "/2016/05/06/" + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, "id13"));
|
||||
file13.createNewFile();
|
||||
|
||||
// Make commit2
|
||||
File file21 = new File(
|
||||
basePath + "/2016/05/01/" + FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, "id21"));
|
||||
File file21 = new File(basePath + "/2016/05/01/" + FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, "id21"));
|
||||
file21.createNewFile();
|
||||
File file22 = new File(
|
||||
basePath + "/2016/05/02/" + FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, "id22"));
|
||||
File file22 = new File(basePath + "/2016/05/02/" + FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, "id22"));
|
||||
file22.createNewFile();
|
||||
File file23 = new File(
|
||||
basePath + "/2016/05/06/" + FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, "id23"));
|
||||
File file23 = new File(basePath + "/2016/05/06/" + FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, "id23"));
|
||||
file23.createNewFile();
|
||||
|
||||
// Make commit3
|
||||
File file31 = new File(
|
||||
basePath + "/2016/05/01/" + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, "id31"));
|
||||
File file31 = new File(basePath + "/2016/05/01/" + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, "id31"));
|
||||
file31.createNewFile();
|
||||
File file32 = new File(
|
||||
basePath + "/2016/05/02/" + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, "id32"));
|
||||
File file32 = new File(basePath + "/2016/05/02/" + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, "id32"));
|
||||
file32.createNewFile();
|
||||
File file33 = new File(
|
||||
basePath + "/2016/05/06/" + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, "id33"));
|
||||
File file33 = new File(basePath + "/2016/05/06/" + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, "id33"));
|
||||
file33.createNewFile();
|
||||
|
||||
// Do a snapshot copy
|
||||
|
||||
@@ -110,6 +110,7 @@ public class UtilitiesTestBase {
|
||||
|
||||
/**
|
||||
* Helper to get hive sync config
|
||||
*
|
||||
* @param basePath
|
||||
* @param tableName
|
||||
* @return
|
||||
@@ -130,6 +131,7 @@ public class UtilitiesTestBase {
|
||||
|
||||
/**
|
||||
* Initialize Hive DB
|
||||
*
|
||||
* @throws IOException
|
||||
*/
|
||||
private static void clearHiveDb() throws IOException {
|
||||
@@ -151,8 +153,8 @@ public class UtilitiesTestBase {
|
||||
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)));
|
||||
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) {
|
||||
|
||||
@@ -69,16 +69,15 @@ public abstract class AbstractBaseTestSource extends AvroSource {
|
||||
dataGeneratorMap.clear();
|
||||
}
|
||||
|
||||
protected AbstractBaseTestSource(TypedProperties props,
|
||||
JavaSparkContext sparkContext, SparkSession sparkSession,
|
||||
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 partition) {
|
||||
int maxUniqueKeys = props.getInteger(TestSourceConfig.MAX_UNIQUE_RECORDS_PROP,
|
||||
TestSourceConfig.DEFAULT_MAX_UNIQUE_RECORDS);
|
||||
int maxUniqueKeys =
|
||||
props.getInteger(TestSourceConfig.MAX_UNIQUE_RECORDS_PROP, TestSourceConfig.DEFAULT_MAX_UNIQUE_RECORDS);
|
||||
|
||||
HoodieTestDataGenerator dataGenerator = dataGeneratorMap.get(partition);
|
||||
|
||||
|
||||
@@ -37,12 +37,11 @@ public class DistributedTestDataSource extends AbstractBaseTestSource {
|
||||
|
||||
private final int numTestSourcePartitions;
|
||||
|
||||
public DistributedTestDataSource(TypedProperties props,
|
||||
JavaSparkContext sparkContext, SparkSession sparkSession,
|
||||
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);
|
||||
this.numTestSourcePartitions =
|
||||
props.getInteger(TestSourceConfig.NUM_SOURCE_PARTITIONS_PROP, TestSourceConfig.DEFAULT_NUM_SOURCE_PARTITIONS);
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -60,20 +59,21 @@ public class DistributedTestDataSource extends AbstractBaseTestSource {
|
||||
newProps.putAll(props);
|
||||
|
||||
// Set the maxUniqueRecords per partition for TestDataSource
|
||||
int maxUniqueRecords = props.getInteger(TestSourceConfig.MAX_UNIQUE_RECORDS_PROP,
|
||||
TestSourceConfig.DEFAULT_MAX_UNIQUE_RECORDS);
|
||||
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).mapPartitionsWithIndex((p, idx) -> {
|
||||
log.info("Initializing source with newProps=" + newProps);
|
||||
if (!dataGeneratorMap.containsKey(p)) {
|
||||
initDataGen(newProps, p);
|
||||
}
|
||||
Iterator<GenericRecord> itr = fetchNextBatch(newProps, perPartitionSourceLimit, commitTime, p).iterator();
|
||||
return itr;
|
||||
}, true);
|
||||
JavaRDD<GenericRecord> avroRDD =
|
||||
sparkContext.parallelize(IntStream.range(0, numTestSourcePartitions).boxed().collect(Collectors.toList()),
|
||||
numTestSourcePartitions).mapPartitionsWithIndex((p, idx) -> {
|
||||
log.info("Initializing source with newProps=" + newProps);
|
||||
if (!dataGeneratorMap.containsKey(p)) {
|
||||
initDataGen(newProps, p);
|
||||
}
|
||||
Iterator<GenericRecord> itr = fetchNextBatch(newProps, perPartitionSourceLimit, commitTime, p).iterator();
|
||||
return itr;
|
||||
}, true);
|
||||
return new InputBatch<>(Option.of(avroRDD), commitTime);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -80,16 +80,13 @@ public class TestDFSSource extends UtilitiesTestBase {
|
||||
|
||||
// 1. Extract without any checkpoint => get all the data, respecting sourceLimit
|
||||
assertEquals(Option.empty(), jsonSource.fetchNewDataInAvroFormat(Option.empty(), Long.MAX_VALUE).getBatch());
|
||||
UtilitiesTestBase.Helpers.saveStringsToDFS(
|
||||
Helpers.jsonifyRecords(dataGenerator.generateInserts("000", 100)), dfs,
|
||||
UtilitiesTestBase.Helpers.saveStringsToDFS(Helpers.jsonifyRecords(dataGenerator.generateInserts("000", 100)), dfs,
|
||||
dfsBasePath + "/jsonFiles/1.json");
|
||||
assertEquals(Option.empty(), jsonSource.fetchNewDataInAvroFormat(Option.empty(), 10).getBatch());
|
||||
InputBatch<JavaRDD<GenericRecord>> fetch1 =
|
||||
jsonSource.fetchNewDataInAvroFormat(Option.empty(), 1000000);
|
||||
InputBatch<JavaRDD<GenericRecord>> fetch1 = jsonSource.fetchNewDataInAvroFormat(Option.empty(), 1000000);
|
||||
assertEquals(100, fetch1.getBatch().get().count());
|
||||
// Test json -> Row format
|
||||
InputBatch<Dataset<Row>> fetch1AsRows =
|
||||
jsonSource.fetchNewDataInRowFormat(Option.empty(), 1000000);
|
||||
InputBatch<Dataset<Row>> fetch1AsRows = jsonSource.fetchNewDataInRowFormat(Option.empty(), 1000000);
|
||||
assertEquals(100, fetch1AsRows.getBatch().get().count());
|
||||
// Test Avro -> Row format
|
||||
Dataset<Row> fetch1Rows = AvroConversionUtils.createDataFrame(JavaRDD.toRDD(fetch1.getBatch().get()),
|
||||
@@ -97,16 +94,15 @@ public class TestDFSSource extends UtilitiesTestBase {
|
||||
assertEquals(100, fetch1Rows.count());
|
||||
|
||||
// 2. Produce new data, extract new data
|
||||
UtilitiesTestBase.Helpers.saveStringsToDFS(
|
||||
Helpers.jsonifyRecords(dataGenerator.generateInserts("001", 10000)),
|
||||
dfs, dfsBasePath + "/jsonFiles/2.json");
|
||||
InputBatch<Dataset<Row>> fetch2 = jsonSource.fetchNewDataInRowFormat(
|
||||
Option.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE);
|
||||
UtilitiesTestBase.Helpers.saveStringsToDFS(Helpers.jsonifyRecords(dataGenerator.generateInserts("001", 10000)), dfs,
|
||||
dfsBasePath + "/jsonFiles/2.json");
|
||||
InputBatch<Dataset<Row>> fetch2 =
|
||||
jsonSource.fetchNewDataInRowFormat(Option.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE);
|
||||
assertEquals(10000, fetch2.getBatch().get().count());
|
||||
|
||||
// 3. Extract with previous checkpoint => gives same data back (idempotent)
|
||||
InputBatch<Dataset<Row>> fetch3 = jsonSource.fetchNewDataInRowFormat(
|
||||
Option.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE);
|
||||
InputBatch<Dataset<Row>> fetch3 =
|
||||
jsonSource.fetchNewDataInRowFormat(Option.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE);
|
||||
assertEquals(10000, fetch3.getBatch().get().count());
|
||||
assertEquals(fetch2.getCheckpointForNextBatch(), fetch3.getCheckpointForNextBatch());
|
||||
fetch3.getBatch().get().registerTempTable("test_dfs_table");
|
||||
@@ -114,8 +110,8 @@ public class TestDFSSource extends UtilitiesTestBase {
|
||||
assertEquals(10000, rowDataset.count());
|
||||
|
||||
// 4. Extract with latest checkpoint => no new data returned
|
||||
InputBatch<JavaRDD<GenericRecord>> fetch4 = jsonSource.fetchNewDataInAvroFormat(
|
||||
Option.of(fetch2.getCheckpointForNextBatch()), Long.MAX_VALUE);
|
||||
InputBatch<JavaRDD<GenericRecord>> fetch4 =
|
||||
jsonSource.fetchNewDataInAvroFormat(Option.of(fetch2.getCheckpointForNextBatch()), Long.MAX_VALUE);
|
||||
assertEquals(Option.empty(), fetch4.getBatch());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -44,8 +44,7 @@ public class TestDataSource extends AbstractBaseTestSource {
|
||||
}
|
||||
|
||||
@Override
|
||||
protected InputBatch<JavaRDD<GenericRecord>> fetchNewData(Option<String> lastCheckpointStr,
|
||||
long sourceLimit) {
|
||||
protected InputBatch<JavaRDD<GenericRecord>> fetchNewData(Option<String> lastCheckpointStr, long sourceLimit) {
|
||||
|
||||
int nextCommitNum = lastCheckpointStr.map(s -> Integer.parseInt(s) + 1).orElse(0);
|
||||
String commitTime = String.format("%05d", nextCommitNum);
|
||||
@@ -56,8 +55,8 @@ public class TestDataSource extends AbstractBaseTestSource {
|
||||
return new InputBatch<>(Option.empty(), lastCheckpointStr.orElse(null));
|
||||
}
|
||||
|
||||
List<GenericRecord> records = fetchNextBatch(props, (int)sourceLimit, commitTime, DEFAULT_PARTITION_NUM)
|
||||
.collect(Collectors.toList());
|
||||
List<GenericRecord> records =
|
||||
fetchNextBatch(props, (int) sourceLimit, commitTime, DEFAULT_PARTITION_NUM).collect(Collectors.toList());
|
||||
JavaRDD<GenericRecord> avroRDD = sparkContext.<GenericRecord>parallelize(records, 4);
|
||||
return new InputBatch<>(Option.of(avroRDD), commitTime);
|
||||
}
|
||||
|
||||
@@ -107,13 +107,13 @@ public class TestKafkaSource extends UtilitiesTestBase {
|
||||
|
||||
// 2. Produce new data, extract new data
|
||||
testUtils.sendMessages(TEST_TOPIC_NAME, Helpers.jsonifyRecords(dataGenerator.generateInserts("001", 1000)));
|
||||
InputBatch<Dataset<Row>> fetch2 = kafkaSource.fetchNewDataInRowFormat(
|
||||
Option.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE);
|
||||
InputBatch<Dataset<Row>> fetch2 =
|
||||
kafkaSource.fetchNewDataInRowFormat(Option.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE);
|
||||
assertEquals(1100, fetch2.getBatch().get().count());
|
||||
|
||||
// 3. Extract with previous checkpoint => gives same data back (idempotent)
|
||||
InputBatch<JavaRDD<GenericRecord>> fetch3 = kafkaSource.fetchNewDataInAvroFormat(
|
||||
Option.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE);
|
||||
InputBatch<JavaRDD<GenericRecord>> fetch3 =
|
||||
kafkaSource.fetchNewDataInAvroFormat(Option.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE);
|
||||
assertEquals(fetch2.getBatch().get().count(), fetch3.getBatch().get().count());
|
||||
assertEquals(fetch2.getCheckpointForNextBatch(), fetch3.getCheckpointForNextBatch());
|
||||
// Same using Row API
|
||||
@@ -123,8 +123,8 @@ public class TestKafkaSource extends UtilitiesTestBase {
|
||||
assertEquals(fetch2.getCheckpointForNextBatch(), fetch3AsRows.getCheckpointForNextBatch());
|
||||
|
||||
// 4. Extract with latest checkpoint => no new data returned
|
||||
InputBatch<JavaRDD<GenericRecord>> fetch4 = kafkaSource.fetchNewDataInAvroFormat(
|
||||
Option.of(fetch2.getCheckpointForNextBatch()), Long.MAX_VALUE);
|
||||
InputBatch<JavaRDD<GenericRecord>> fetch4 =
|
||||
kafkaSource.fetchNewDataInAvroFormat(Option.of(fetch2.getCheckpointForNextBatch()), Long.MAX_VALUE);
|
||||
assertEquals(Option.empty(), fetch4.getBatch());
|
||||
// Same using Row API
|
||||
InputBatch<Dataset<Row>> fetch4AsRows =
|
||||
@@ -144,26 +144,19 @@ public class TestKafkaSource extends UtilitiesTestBase {
|
||||
@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)
|
||||
});
|
||||
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
|
||||
);
|
||||
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
|
||||
);
|
||||
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());
|
||||
@@ -171,30 +164,21 @@ public class TestKafkaSource extends UtilitiesTestBase {
|
||||
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
|
||||
);
|
||||
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
|
||||
);
|
||||
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
|
||||
);
|
||||
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());
|
||||
|
||||
@@ -21,7 +21,7 @@ package org.apache.hudi.utilities.sources.config;
|
||||
/**
|
||||
* Configurations for Test Data Sources
|
||||
*/
|
||||
public class TestSourceConfig {
|
||||
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";
|
||||
|
||||
Reference in New Issue
Block a user