Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Disabled;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.EnumSource;
Expand All @@ -48,15 +49,12 @@
import java.util.Collections;
import java.util.ConcurrentModificationException;
import java.util.List;
import java.util.Objects;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.Function;

import static org.apache.hudi.common.testutils.FixtureUtils.prepareFixtureTable;
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
import static org.apache.hudi.config.HoodieWriteConfig.BULKINSERT_PARALLELISM_VALUE;
import static org.apache.hudi.config.HoodieWriteConfig.BULK_INSERT_SORT_MODE;
import static org.apache.hudi.config.HoodieWriteConfig.FINALIZE_WRITE_PARALLELISM_VALUE;
Expand All @@ -68,31 +66,50 @@
import static org.apache.hudi.utilities.functional.HoodieDeltaStreamerTestBase.defaultSchemaProviderClassName;
import static org.apache.hudi.utilities.functional.HoodieDeltaStreamerTestBase.prepareInitialConfigs;
import static org.apache.hudi.utilities.functional.TestHoodieDeltaStreamer.deltaStreamerTestRunner;
import static org.apache.hudi.utilities.testutils.sources.AbstractBaseTestSource.DEFAULT_PARTITION_NUM;
import static org.apache.hudi.utilities.testutils.sources.AbstractBaseTestSource.dataGeneratorMap;
import static org.apache.hudi.utilities.testutils.sources.AbstractBaseTestSource.initDataGen;

@Tag("functional")
public class TestHoodieDeltaStreamerWithMultiWriter extends SparkClientFunctionalTestHarness {

private static final String COW_TEST_TABLE_NAME = "testtable_COPY_ON_WRITE";
private static final Logger LOG = LogManager.getLogger(TestHoodieDeltaStreamerWithMultiWriter.class);

String basePath;
String propsFilePath;
String tableBasePath;
int totalRecords;

@ParameterizedTest
@EnumSource(HoodieTableType.class)
void testUpsertsContinuousModeWithMultipleWritersForConflicts(HoodieTableType tableType) throws Exception {
// NOTE : Overriding the LockProvider to InProcessLockProvider since Zookeeper locks work in unit test but fail on Jenkins with connection timeouts
setUpTestTable(tableType);
basePath = Paths.get(URI.create(basePath().replaceAll("/$", ""))).toString();
propsFilePath = basePath + "/" + PROPS_FILENAME_TEST_MULTI_WRITER;
tableBasePath = basePath + "/testtable_" + tableType;
prepareInitialConfigs(fs(), basePath, "foo");
TypedProperties props = prepareMultiWriterProps(fs(), basePath, propsFilePath);
props.setProperty("hoodie.write.lock.provider", "org.apache.hudi.client.transaction.lock.InProcessLockProvider");
props.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY,"3000");
Comment on lines 87 to 89
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@manojpec you can set configs as needed from here. Maybe the last 2 lines can be moved into prepareMultiWriterProps() as well.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

right, quite a repetition of these props in few tests in this file. can be moved to prepareMultiWriterProps()

UtilitiesTestBase.Helpers.savePropsToDFS(props, fs(), propsFilePath);
// Keep it higher than batch-size to test continuous mode
int totalRecords = 3000;

HoodieDeltaStreamer.Config prepJobConfig = getDeltaStreamerConfig(tableBasePath, tableType.name(), WriteOperationType.UPSERT,
propsFilePath, Collections.singletonList(TestHoodieDeltaStreamer.TripsWithDistanceTransformer.class.getName()));
prepJobConfig.continuousMode = true;
prepJobConfig.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords));
prepJobConfig.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN.key()));
HoodieDeltaStreamer prepJob = new HoodieDeltaStreamer(prepJobConfig, jsc());

// Prepare base dataset with some commits
deltaStreamerTestRunner(prepJob, prepJobConfig, (r) -> {
if (tableType.equals(HoodieTableType.MERGE_ON_READ)) {
TestHoodieDeltaStreamer.TestHelpers.assertAtleastNDeltaCommits(3, tableBasePath, fs());
TestHoodieDeltaStreamer.TestHelpers.assertAtleastNCompactionCommits(1, tableBasePath, fs());
} else {
TestHoodieDeltaStreamer.TestHelpers.assertAtleastNCompactionCommits(3, tableBasePath, fs());
}
TestHoodieDeltaStreamer.TestHelpers.assertRecordCount(totalRecords, tableBasePath + "/*/*.parquet", sqlContext());
TestHoodieDeltaStreamer.TestHelpers.assertDistanceCount(totalRecords, tableBasePath + "/*/*.parquet", sqlContext());
return true;
});

HoodieDeltaStreamer.Config cfgIngestionJob = getDeltaStreamerConfig(tableBasePath, tableType.name(), WriteOperationType.UPSERT,
propsFilePath, Collections.singletonList(TestHoodieDeltaStreamer.TripsWithDistanceTransformer.class.getName()));
Expand Down Expand Up @@ -125,12 +142,36 @@ void testUpsertsContinuousModeWithMultipleWritersForConflicts(HoodieTableType ta
@EnumSource(HoodieTableType.class)
void testUpsertsContinuousModeWithMultipleWritersWithoutConflicts(HoodieTableType tableType) throws Exception {
// NOTE : Overriding the LockProvider to InProcessLockProvider since Zookeeper locks work in unit test but fail on Jenkins with connection timeouts
setUpTestTable(tableType);
basePath = Paths.get(URI.create(basePath().replaceAll("/$", ""))).toString();
propsFilePath = basePath + "/" + PROPS_FILENAME_TEST_MULTI_WRITER;
tableBasePath = basePath + "/testtable_" + tableType;
prepareInitialConfigs(fs(), basePath, "foo");
TypedProperties props = prepareMultiWriterProps(fs(), basePath, propsFilePath);
props.setProperty("hoodie.write.lock.provider", "org.apache.hudi.client.transaction.lock.InProcessLockProvider");
props.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY,"3000");
UtilitiesTestBase.Helpers.savePropsToDFS(props, fs(), propsFilePath);
// Keep it higher than batch-size to test continuous mode
int totalRecords = 3000;

HoodieDeltaStreamer.Config prepJobConfig = getDeltaStreamerConfig(tableBasePath, tableType.name(), WriteOperationType.UPSERT,
propsFilePath, Collections.singletonList(TestHoodieDeltaStreamer.TripsWithDistanceTransformer.class.getName()));
prepJobConfig.continuousMode = true;
prepJobConfig.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords));
prepJobConfig.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN.key()));
HoodieDeltaStreamer prepJob = new HoodieDeltaStreamer(prepJobConfig, jsc());

// Prepare base dataset with some commits
deltaStreamerTestRunner(prepJob, prepJobConfig, (r) -> {
if (tableType.equals(HoodieTableType.MERGE_ON_READ)) {
TestHoodieDeltaStreamer.TestHelpers.assertAtleastNDeltaCommits(3, tableBasePath, fs());
TestHoodieDeltaStreamer.TestHelpers.assertAtleastNCompactionCommits(1, tableBasePath, fs());
} else {
TestHoodieDeltaStreamer.TestHelpers.assertAtleastNCompactionCommits(3, tableBasePath, fs());
}
TestHoodieDeltaStreamer.TestHelpers.assertRecordCount(totalRecords, tableBasePath + "/*/*.parquet", sqlContext());
TestHoodieDeltaStreamer.TestHelpers.assertDistanceCount(totalRecords, tableBasePath + "/*/*.parquet", sqlContext());
return true;
});

// create new ingestion & backfill job config to generate only INSERTS to avoid conflict
props = prepareMultiWriterProps(fs(), basePath, propsFilePath);
Expand Down Expand Up @@ -164,26 +205,41 @@ void testUpsertsContinuousModeWithMultipleWritersWithoutConflicts(HoodieTableTyp
cfgIngestionJob2, backfillJob2, cfgBackfillJob2, false, "batch2");
}

@Disabled
@ParameterizedTest
@EnumSource(value = HoodieTableType.class, names = {"COPY_ON_WRITE"})
public void testLatestCheckpointCarryOverWithMultipleWriters(HoodieTableType tableType) throws Exception {
testCheckpointCarryOver(tableType);
}

private void testCheckpointCarryOver(HoodieTableType tableType) throws Exception {
void testLatestCheckpointCarryOverWithMultipleWriters(HoodieTableType tableType) throws Exception {
// NOTE : Overriding the LockProvider to InProcessLockProvider since Zookeeper locks work in unit test but fail on Jenkins with connection timeouts
setUpTestTable(tableType);
basePath = Paths.get(URI.create(basePath().replaceAll("/$", ""))).toString();
propsFilePath = basePath + "/" + PROPS_FILENAME_TEST_MULTI_WRITER;
tableBasePath = basePath + "/testtable_" + tableType;
prepareInitialConfigs(fs(), basePath, "foo");
TypedProperties props = prepareMultiWriterProps(fs(), basePath, propsFilePath);
props.setProperty("hoodie.write.lock.provider", "org.apache.hudi.client.transaction.lock.InProcessLockProvider");
props.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY,"3000");
UtilitiesTestBase.Helpers.savePropsToDFS(props, fs(), propsFilePath);
// Keep it higher than batch-size to test continuous mode
int totalRecords = 3000;

HoodieDeltaStreamer.Config cfgIngestionJob = getDeltaStreamerConfig(tableBasePath, tableType.name(), WriteOperationType.UPSERT,
HoodieDeltaStreamer.Config prepJobConfig = getDeltaStreamerConfig(tableBasePath, tableType.name(), WriteOperationType.UPSERT,
propsFilePath, Collections.singletonList(TestHoodieDeltaStreamer.TripsWithDistanceTransformer.class.getName()));
cfgIngestionJob.continuousMode = true;
cfgIngestionJob.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords));
cfgIngestionJob.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN.key()));
prepJobConfig.continuousMode = true;
prepJobConfig.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords));
prepJobConfig.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN.key()));
HoodieDeltaStreamer prepJob = new HoodieDeltaStreamer(prepJobConfig, jsc());

// Prepare base dataset with some commits
deltaStreamerTestRunner(prepJob, prepJobConfig, (r) -> {
if (tableType.equals(HoodieTableType.MERGE_ON_READ)) {
TestHoodieDeltaStreamer.TestHelpers.assertAtleastNDeltaCommits(3, tableBasePath, fs());
TestHoodieDeltaStreamer.TestHelpers.assertAtleastNCompactionCommits(1, tableBasePath, fs());
} else {
TestHoodieDeltaStreamer.TestHelpers.assertAtleastNCompactionCommits(3, tableBasePath, fs());
}
TestHoodieDeltaStreamer.TestHelpers.assertRecordCount(totalRecords, tableBasePath + "/*/*.parquet", sqlContext());
TestHoodieDeltaStreamer.TestHelpers.assertDistanceCount(totalRecords, tableBasePath + "/*/*.parquet", sqlContext());
return true;
});

// create a backfill job with checkpoint from the first instant
HoodieDeltaStreamer.Config cfgBackfillJob = getDeltaStreamerConfig(tableBasePath, tableType.name(), WriteOperationType.UPSERT,
Expand Down Expand Up @@ -292,26 +348,6 @@ private static HoodieDeltaStreamer.Config getDeltaStreamerConfig(String basePath
return cfg;
}

/**
* Specifically used for {@link TestHoodieDeltaStreamerWithMultiWriter}.
*
* The fixture test tables have random records generated by
* {@link org.apache.hudi.common.testutils.HoodieTestDataGenerator} using
* {@link org.apache.hudi.common.testutils.HoodieTestDataGenerator#TRIP_EXAMPLE_SCHEMA}.
*
* The COW fixture test table has 3000 unique records in 7 commits.
* The MOR fixture test table has 3000 unique records in 9 deltacommits and 1 compaction commit.
*/
private void setUpTestTable(HoodieTableType tableType) throws IOException {
basePath = Paths.get(URI.create(basePath().replaceAll("/$", ""))).toString();
propsFilePath = basePath + "/" + PROPS_FILENAME_TEST_MULTI_WRITER;
String fixtureName = String.format("fixtures/testUpsertsContinuousModeWithMultipleWriters.%s.zip", tableType.name());
tableBasePath = prepareFixtureTable(Objects.requireNonNull(getClass()
.getClassLoader().getResource(fixtureName)), Paths.get(basePath)).toString();
initDataGen(sqlContext(), tableBasePath + "/*/*.parquet", DEFAULT_PARTITION_NUM);
totalRecords = dataGeneratorMap.get(DEFAULT_PARTITION_NUM).getNumExistingKeys(TRIP_EXAMPLE_SCHEMA);
}

private void runJobsInParallel(String tableBasePath, HoodieTableType tableType, int totalRecords,
HoodieDeltaStreamer ingestionJob, HoodieDeltaStreamer.Config cfgIngestionJob, HoodieDeltaStreamer backfillJob,
HoodieDeltaStreamer.Config cfgBackfillJob, boolean expectConflict, String jobId) throws Exception {
Expand Down
Binary file not shown.
Binary file not shown.