1
0

[HUDI-2767] Enabling timeline-server-based marker as default (#4112)

- Changes the default config of marker type (HoodieWriteConfig.MARKERS_TYPE or hoodie.write.markers.type) from DIRECT to TIMELINE_SERVER_BASED for Spark Engine.
- Adds engine-specific marker type configs: Spark -> TIMELINE_SERVER_BASED, Flink -> DIRECT, Java -> DIRECT.
- Uses DIRECT markers as well for Spark structured streaming due to timeline server only available for the first mini-batch.
- Fixes the marker creation method for non-partitioned table in TimelineServerBasedWriteMarkers.
- Adds the fallback to direct markers even when TIMELINE_SERVER_BASED is configured, in WriteMarkersFactory: when HDFS is used, or embedded timeline server is disabled, the fallback to direct markers happens.
- Fixes the closing of timeline service.
- Fixes tests that depend on markers, mainly by starting the timeline service for each test.
This commit is contained in:
Y Ethan Guo
2021-11-26 13:41:05 -08:00
committed by GitHub
parent f8e0176eb0
commit d1e83e4ba0
35 changed files with 529 additions and 134 deletions

View File

@@ -25,6 +25,7 @@ import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.TableServiceType;
import org.apache.hudi.common.model.WriteConcurrencyMode;
import org.apache.hudi.common.table.marker.MarkerType;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.common.table.view.FileSystemViewStorageType;
@@ -91,6 +92,8 @@ public class TestHoodieClientMultiWriter extends HoodieClientTestBase {
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).withAutoClean(false).build())
.withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL)
// Timeline-server-based markers are not used for multi-writer tests
.withMarkersType(MarkerType.DIRECT.name())
.withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(FileSystemBasedLockProviderTestClass.class)
.build()).withAutoCommit(false).withProperties(properties).build();
// Create the first commit
@@ -162,6 +165,8 @@ public class TestHoodieClientMultiWriter extends HoodieClientTestBase {
.withLockProvider(FileSystemBasedLockProviderTestClass.class)
.build())
.withAutoCommit(false)
// Timeline-server-based markers are not used for multi-writer tests
.withMarkersType(MarkerType.DIRECT.name())
.withProperties(properties)
.build();
@@ -208,11 +213,13 @@ public class TestHoodieClientMultiWriter extends HoodieClientTestBase {
properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY, "5000");
// Disabling embedded timeline server, it doesn't work with multiwriter
HoodieWriteConfig.Builder writeConfigBuilder = getConfigBuilder()
.withCompactionConfig(HoodieCompactionConfig.newBuilder().withAutoClean(false)
.withInlineCompaction(false).withAsyncClean(true)
.withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY)
.withMaxNumDeltaCommitsBeforeCompaction(2).build())
.withCompactionConfig(HoodieCompactionConfig.newBuilder().withAutoClean(false)
.withInlineCompaction(false).withAsyncClean(true)
.withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY)
.withMaxNumDeltaCommitsBeforeCompaction(2).build())
.withEmbeddedTimelineServerEnabled(false)
// Timeline-server-based markers are not used for multi-writer tests
.withMarkersType(MarkerType.DIRECT.name())
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder().withStorageType(
FileSystemViewStorageType.MEMORY).build())
.withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL)
@@ -326,6 +333,8 @@ public class TestHoodieClientMultiWriter extends HoodieClientTestBase {
.withCompactionConfig(HoodieCompactionConfig.newBuilder().withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY)
.withAutoClean(false).build())
.withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL)
// Timeline-server-based markers are not used for multi-writer tests
.withMarkersType(MarkerType.DIRECT.name())
.withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(FileSystemBasedLockProviderTestClass.class)
.build()).withAutoCommit(false).withProperties(properties);
HoodieWriteConfig cfg = writeConfigBuilder.build();

View File

@@ -23,6 +23,7 @@ import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordLocation;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.common.testutils.HoodieTestUtils;
import org.apache.hudi.common.testutils.RawTripTestPayload;
import org.apache.hudi.common.util.BaseFileUtils;
@@ -64,6 +65,7 @@ public class TestUpdateSchemaEvolution extends HoodieClientTestHarness {
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath);
initSparkContexts("TestUpdateSchemaEvolution");
initFileSystem();
initTimelineService();
}
@AfterEach
@@ -228,6 +230,9 @@ public class TestUpdateSchemaEvolution extends HoodieClientTestHarness {
private HoodieWriteConfig makeHoodieClientConfig(String name) {
Schema schema = getSchemaFromResource(getClass(), name);
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schema.toString()).build();
return HoodieWriteConfig.newBuilder().withPath(basePath)
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
.withRemoteServerPort(timelineServicePort).build())
.withSchema(schema.toString()).build();
}
}

View File

@@ -1316,6 +1316,7 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build())
.withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
.withEnableBackupForRemoteFileSystemView(false) // Fail test if problem connecting to timeline-server
.withRemoteServerPort(timelineServicePort)
.withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE).build());
}

View File

@@ -47,11 +47,13 @@ import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.model.IOType;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.marker.MarkerType;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.common.testutils.HoodieTestTable;
@@ -61,6 +63,7 @@ import org.apache.hudi.common.util.BaseFileUtils;
import org.apache.hudi.common.util.ClusteringUtils;
import org.apache.hudi.common.util.CollectionUtils;
import org.apache.hudi.common.util.FileIOUtils;
import org.apache.hudi.common.util.MarkerUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.common.util.collection.Pair;
@@ -2336,13 +2339,22 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
// Create a dummy marker file to simulate the case that a marker file was created without data file.
// This should fail the commit
String partitionPath = Arrays
.stream(fs.globStatus(new Path(String.format("%s/*/*/*/*", metaClient.getMarkerFolderPath(instantTime))),
path -> path.toString().contains(HoodieTableMetaClient.MARKER_EXTN)))
.limit(1).map(status -> status.getPath().getParent().toString()).collect(Collectors.toList()).get(0);
String partitionPath;
String markerFolderPath = metaClient.getMarkerFolderPath(instantTime);
if (cfg.getMarkersType() == MarkerType.TIMELINE_SERVER_BASED) {
String markerName = MarkerUtils.readTimelineServerBasedMarkersFromFileSystem(
markerFolderPath, fs, context, 1).values().stream()
.flatMap(Collection::stream).findFirst().get();
partitionPath = new Path(markerFolderPath, markerName).getParent().toString();
} else {
partitionPath = Arrays
.stream(fs.globStatus(new Path(String.format("%s/*/*/*/*", markerFolderPath)),
path -> path.toString().contains(HoodieTableMetaClient.MARKER_EXTN)))
.limit(1).map(status -> status.getPath().getParent().toString()).collect(Collectors.toList()).get(0);
}
Option<Path> markerFilePath = WriteMarkersFactory.get(
cfg.getMarkersType(), getHoodieTable(metaClient, cfg), instantTime)
cfg.getMarkersType(), getHoodieTable(metaClient, cfg), instantTime)
.create(partitionPath,
FSUtils.makeDataFileName(instantTime, "1-0-1", UUID.randomUUID().toString()),
IOType.MERGE);
@@ -2489,6 +2501,8 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
.withAutoClean(false).build())
.withTimelineLayoutVersion(1)
.withHeartbeatIntervalInMs(3 * 1000)
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
.withRemoteServerPort(timelineServicePort).build())
.withAutoCommit(false)
.withProperties(populateMetaFields ? new Properties() : getPropertiesForKeyGen()).build();
}

View File

@@ -85,6 +85,7 @@ public class TestHoodieMetadataBase extends HoodieClientTestHarness {
initSparkContexts("TestHoodieMetadata");
initFileSystem();
fs.mkdirs(new Path(basePath));
initTimelineService();
initMetaClient(tableType);
initTestDataGenerator();
metadataTableBasePath = HoodieTableMetadata.getMetadataTableBasePath(basePath);

View File

@@ -21,6 +21,7 @@ package org.apache.hudi.client.functional;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.common.testutils.HoodieMetadataTestTable;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.common.testutils.HoodieTestTable;
@@ -241,6 +242,8 @@ public class TestHoodieMetadataBootstrap extends TestHoodieMetadataBase {
return HoodieWriteConfig.newBuilder().withPath(basePath)
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(minArchivalCommits, maxArchivalCommits).build())
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
.withRemoteServerPort(timelineServicePort).build())
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
.forTable("test-trip-table").build();
}

View File

@@ -36,6 +36,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.timeline.versioning.clean.CleanPlanV2MigrationHandler;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.common.testutils.HoodieMetadataTestTable;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.common.testutils.HoodieTestTable;
@@ -94,6 +95,7 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
public void init(HoodieTableType tableType) throws Exception {
initPath();
initSparkContexts();
initTimelineService();
initMetaClient();
hadoopConf = context.getHadoopConf().get();
metaClient.getFs().mkdirs(new Path(basePath));
@@ -126,6 +128,8 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath(basePath)
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(minArchivalCommits, maxArchivalCommits).build())
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
.withRemoteServerPort(timelineServicePort).build())
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadata)
.withMaxNumDeltaCommitsBeforeCompaction(maxDeltaCommitsMetadataTable).build())
.forTable("test-trip-table").build();
@@ -210,6 +214,8 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2).forTable("test-trip-table")
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build())
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
.withRemoteServerPort(timelineServicePort).build())
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
.build();
@@ -328,6 +334,8 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
.withParallelism(2, 2).forTable("test-trip-table")
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build())
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
.withRemoteServerPort(timelineServicePort).build())
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
.build();
metaClient = HoodieTableMetaClient.reload(metaClient);
@@ -484,6 +492,8 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
.withParallelism(2, 2).forTable("test-trip-table")
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(minInstantsToKeep, maxInstantsToKeep).build())
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
.withRemoteServerPort(timelineServicePort).build())
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
.build();
metaClient = HoodieTableMetaClient.reload(metaClient);
@@ -519,6 +529,8 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
.withParallelism(2, 2).forTable("test-trip-table")
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build())
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
.withRemoteServerPort(timelineServicePort).build())
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
.build();
metaClient = HoodieTableMetaClient.reload(metaClient);

View File

@@ -18,8 +18,6 @@
package org.apache.hudi.io.storage.row;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.bloom.BloomFilter;
import org.apache.hudi.common.bloom.BloomFilterFactory;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
@@ -27,6 +25,9 @@ import org.apache.hudi.config.HoodieStorageConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.testutils.HoodieClientTestHarness;
import org.apache.hudi.testutils.SparkDatasetTestUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
@@ -66,7 +67,8 @@ public class TestHoodieInternalRowParquetWriter extends HoodieClientTestHarness
@ParameterizedTest
@ValueSource(booleans = {true, false})
public void endToEndTest(boolean parquetWriteLegacyFormatEnabled) throws Exception {
HoodieWriteConfig.Builder writeConfigBuilder = SparkDatasetTestUtils.getConfigBuilder(basePath);
HoodieWriteConfig.Builder writeConfigBuilder =
SparkDatasetTestUtils.getConfigBuilder(basePath, timelineServicePort);
for (int i = 0; i < 5; i++) {
// init write support and parquet config
HoodieRowParquetWriteSupport writeSupport = getWriteSupport(writeConfigBuilder, hadoopConf, parquetWriteLegacyFormatEnabled);

View File

@@ -28,8 +28,8 @@ import org.apache.hudi.exception.HoodieInsertException;
import org.apache.hudi.table.HoodieSparkTable;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.testutils.HoodieClientTestHarness;
import org.apache.hudi.testutils.SparkDatasetTestUtils;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.catalyst.InternalRow;
@@ -65,6 +65,7 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness {
initFileSystem();
initTestDataGenerator();
initMetaClient();
initTimelineService();
}
@AfterEach
@@ -75,7 +76,8 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness {
@Test
public void testRowCreateHandle() throws Exception {
// init config and table
HoodieWriteConfig cfg = SparkDatasetTestUtils.getConfigBuilder(basePath).build();
HoodieWriteConfig cfg =
SparkDatasetTestUtils.getConfigBuilder(basePath, timelineServicePort).build();
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
List<String> fileNames = new ArrayList<>();
List<String> fileAbsPaths = new ArrayList<>();
@@ -116,7 +118,8 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness {
@Test
public void testGlobalFailure() throws Exception {
// init config and table
HoodieWriteConfig cfg = SparkDatasetTestUtils.getConfigBuilder(basePath).build();
HoodieWriteConfig cfg =
SparkDatasetTestUtils.getConfigBuilder(basePath, timelineServicePort).build();
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0];
@@ -124,7 +127,8 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness {
String fileId = UUID.randomUUID().toString();
String instantTime = "000";
HoodieRowCreateHandle handle = new HoodieRowCreateHandle(table, cfg, partitionPath, fileId, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), SparkDatasetTestUtils.STRUCT_TYPE);
HoodieRowCreateHandle handle =
new HoodieRowCreateHandle(table, cfg, partitionPath, fileId, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), SparkDatasetTestUtils.STRUCT_TYPE);
int size = 10 + RANDOM.nextInt(1000);
int totalFailures = 5;
// Generate first batch of valid rows
@@ -169,7 +173,8 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness {
@Test
public void testInstantiationFailure() throws IOException {
// init config and table
HoodieWriteConfig cfg = SparkDatasetTestUtils.getConfigBuilder(basePath).withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
HoodieWriteConfig cfg = SparkDatasetTestUtils.getConfigBuilder(basePath, timelineServicePort)
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
.withPath("/dummypath/abc/").build();
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);

View File

@@ -18,8 +18,6 @@
package org.apache.hudi.table;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.model.HoodieActionInstant;
import org.apache.hudi.avro.model.HoodieCleanMetadata;
import org.apache.hudi.avro.model.HoodieCleanPartitionMetadata;
@@ -62,6 +60,7 @@ import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
import org.apache.hudi.common.table.timeline.versioning.clean.CleanMetadataMigrator;
import org.apache.hudi.common.table.timeline.versioning.clean.CleanPlanMigrator;
import org.apache.hudi.common.table.timeline.versioning.clean.CleanPlanV1MigrationHandler;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.common.table.view.TableFileSystemView;
import org.apache.hudi.common.testutils.HoodieMetadataTestTable;
import org.apache.hudi.common.testutils.HoodieTestTable;
@@ -81,6 +80,9 @@ import org.apache.hudi.metadata.HoodieTableMetadataWriter;
import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
import org.apache.hudi.table.action.clean.CleanPlanner;
import org.apache.hudi.testutils.HoodieClientTestBase;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaRDD;
@@ -89,7 +91,6 @@ import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.Arguments;
import org.junit.jupiter.params.provider.MethodSource;
import org.junit.jupiter.params.provider.ValueSource;
import scala.Tuple3;
import java.io.File;
import java.io.IOException;
@@ -111,6 +112,8 @@ import java.util.function.Predicate;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import scala.Tuple3;
import static org.apache.hudi.common.testutils.HoodieTestTable.makeIncrementalCommitTimes;
import static org.apache.hudi.common.testutils.HoodieTestTable.makeNewCommitTime;
import static org.apache.hudi.common.testutils.HoodieTestUtils.DEFAULT_PARTITION_PATHS;
@@ -1300,7 +1303,10 @@ public class TestCleaner extends HoodieClientTestBase {
final int numTempFilesBefore = testTable.listAllFilesInTempFolder().length;
assertEquals(10, numTempFilesBefore, "Some marker files are created.");
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieWriteConfig config = HoodieWriteConfig.newBuilder()
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
.withRemoteServerPort(timelineServicePort).build())
.withPath(basePath).build();
metaClient = HoodieTableMetaClient.reload(metaClient);
HoodieTable table = HoodieSparkTable.create(config, context, metaClient);
table.getActiveTimeline().transitionRequestedToInflight(

View File

@@ -26,6 +26,7 @@ import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.common.testutils.HoodieTestUtils;
import org.apache.hudi.common.testutils.RawTripTestPayload;
import org.apache.hudi.common.testutils.Transformations;
@@ -112,7 +113,9 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
private HoodieWriteConfig.Builder makeHoodieClientConfigBuilder() {
// Prepare the AvroParquetIO
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(SCHEMA.toString());
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(SCHEMA.toString())
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
.withRemoteServerPort(timelineServicePort).build());
}
// TODO (weiy): Add testcases for crossing file writing.
@@ -405,8 +408,10 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
public void testInsertUpsertWithHoodieAvroPayload() throws Exception {
Schema schema = getSchemaFromResource(TestCopyOnWriteActionExecutor.class, "/testDataGeneratorSchema.txt");
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schema.toString())
.withStorageConfig(HoodieStorageConfig.newBuilder()
.parquetMaxFileSize(1000 * 1024).hfileMaxFileSize(1000 * 1024).build()).build();
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
.withRemoteServerPort(timelineServicePort).build())
.withStorageConfig(HoodieStorageConfig.newBuilder()
.parquetMaxFileSize(1000 * 1024).hfileMaxFileSize(1000 * 1024).build()).build();
metaClient = HoodieTableMetaClient.reload(metaClient);
HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, metaClient);
String instantTime = "000";

View File

@@ -30,8 +30,10 @@ import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.marker.MarkerType;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
import org.apache.hudi.common.table.view.SyncableFileSystemView;
import org.apache.hudi.common.table.view.TableFileSystemView;
@@ -297,6 +299,8 @@ public class TestHoodieSparkMergeOnReadTableRollback extends SparkClientFunction
void testMultiRollbackWithDeltaAndCompactionCommit() throws Exception {
boolean populateMetaFields = true;
HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(false)
// Timeline-server-based markers are not used for multi-rollback tests
.withMarkersType(MarkerType.DIRECT.name())
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build());
addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields);
HoodieWriteConfig cfg = cfgBuilder.build();
@@ -347,6 +351,8 @@ public class TestHoodieSparkMergeOnReadTableRollback extends SparkClientFunction
newCommitTime = "002";
// WriteClient with custom config (disable small file handling)
HoodieWriteConfig smallFileWriteConfig = getHoodieWriteConfigWithSmallFileHandlingOffBuilder(populateMetaFields)
// Timeline-server-based markers are not used for multi-rollback tests
.withMarkersType(MarkerType.DIRECT.name())
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build()).build();
try (SparkRDDWriteClient nClient = getHoodieWriteClient(smallFileWriteConfig)) {
nClient.startCommitWithTime(newCommitTime);
@@ -468,6 +474,8 @@ public class TestHoodieSparkMergeOnReadTableRollback extends SparkClientFunction
.withAutoCommit(false)
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024)
.withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1).build())
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
.withRemoteServerPort(timelineServicePort).build())
.withEmbeddedTimelineServerEnabled(true)
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024).parquetMaxFileSize(1024).build()).forTable("test-trip-table");
@@ -562,7 +570,7 @@ public class TestHoodieSparkMergeOnReadTableRollback extends SparkClientFunction
}
}
if (rollbackUsingMarkers) {
metaClient.getFs().copyFromLocalFile(markerDir,
metaClient.getFs().copyFromLocalFile(new Path(markerDir, lastCommitTime),
new Path(metaClient.getMarkerFolderPath(lastCommitTime)));
}
Thread.sleep(1000);

View File

@@ -59,15 +59,18 @@ public class TestDirectWriteMarkers extends TestWriteMarkersBase {
}
@Override
void verifyMarkersInFileSystem() throws IOException {
void verifyMarkersInFileSystem(boolean isTablePartitioned) throws IOException {
List<FileStatus> markerFiles = FileSystemTestUtils.listRecursive(fs, markerFolderPath)
.stream().filter(status -> status.getPath().getName().contains(".marker"))
.sorted().collect(Collectors.toList());
assertEquals(3, markerFiles.size());
assertIterableEquals(CollectionUtils.createImmutableList(
"file:" + markerFolderPath.toString() + "/2020/06/01/file1.marker.MERGE",
"file:" + markerFolderPath.toString() + "/2020/06/02/file2.marker.APPEND",
"file:" + markerFolderPath.toString() + "/2020/06/03/file3.marker.CREATE"),
"file:" + markerFolderPath.toString()
+ (isTablePartitioned ? "/2020/06/01" : "") + "/file1.marker.MERGE",
"file:" + markerFolderPath.toString()
+ (isTablePartitioned ? "/2020/06/02" : "") + "/file2.marker.APPEND",
"file:" + markerFolderPath.toString()
+ (isTablePartitioned ? "/2020/06/03" : "") + "/file3.marker.CREATE"),
markerFiles.stream().map(m -> m.getPath().toString()).collect(Collectors.toList())
);
}

View File

@@ -93,18 +93,20 @@ public class TestTimelineServerBasedWriteMarkers extends TestWriteMarkersBase {
}
@Override
void verifyMarkersInFileSystem() throws IOException {
void verifyMarkersInFileSystem(boolean isTablePartitioned) throws IOException {
// Verifies the markers
List<String> allMarkers = MarkerUtils.readTimelineServerBasedMarkersFromFileSystem(
markerFolderPath.toString(), fs, context, 1)
markerFolderPath.toString(), fs, context, 1)
.values().stream().flatMap(Collection::stream).sorted()
.collect(Collectors.toList());
assertEquals(3, allMarkers.size());
assertIterableEquals(CollectionUtils.createImmutableList(
"2020/06/01/file1.marker.MERGE",
"2020/06/02/file2.marker.APPEND",
"2020/06/03/file3.marker.CREATE"),
allMarkers);
List<String> expectedMarkers = isTablePartitioned
? CollectionUtils.createImmutableList(
"2020/06/01/file1.marker.MERGE", "2020/06/02/file2.marker.APPEND",
"2020/06/03/file3.marker.CREATE")
: CollectionUtils.createImmutableList(
"file1.marker.MERGE", "file2.marker.APPEND", "file3.marker.CREATE");
assertIterableEquals(expectedMarkers, allMarkers);
// Verifies the marker type file
Path markerTypeFilePath = new Path(markerFolderPath, MarkerUtils.MARKER_TYPE_FILENAME);
assertTrue(MarkerUtils.doesMarkerTypeFileExist(fs, markerFolderPath.toString()));

View File

@@ -31,8 +31,11 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.spark.api.java.JavaSparkContext;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.ValueSource;
import java.io.IOException;
import java.util.List;
import java.util.stream.Collectors;
import static org.junit.jupiter.api.Assertions.assertEquals;
@@ -48,10 +51,10 @@ public abstract class TestWriteMarkersBase extends HoodieCommonTestHarness {
protected JavaSparkContext jsc;
protected HoodieSparkEngineContext context;
private void createSomeMarkers() {
writeMarkers.create("2020/06/01", "file1", IOType.MERGE);
writeMarkers.create("2020/06/02", "file2", IOType.APPEND);
writeMarkers.create("2020/06/03", "file3", IOType.CREATE);
private void createSomeMarkers(boolean isTablePartitioned) {
writeMarkers.create(isTablePartitioned ? "2020/06/01" : "", "file1", IOType.MERGE);
writeMarkers.create(isTablePartitioned ? "2020/06/02" : "", "file2", IOType.APPEND);
writeMarkers.create(isTablePartitioned ? "2020/06/03" : "", "file3", IOType.CREATE);
}
private void createInvalidFile(String partitionPath, String invalidFileName) {
@@ -64,22 +67,24 @@ public abstract class TestWriteMarkersBase extends HoodieCommonTestHarness {
}
}
abstract void verifyMarkersInFileSystem() throws IOException;
abstract void verifyMarkersInFileSystem(boolean isTablePartitioned) throws IOException;
@Test
public void testCreation() throws Exception {
@ParameterizedTest
@ValueSource(booleans = {true, false})
public void testCreation(boolean isTablePartitioned) throws Exception {
// when
createSomeMarkers();
createSomeMarkers(isTablePartitioned);
// then
assertTrue(fs.exists(markerFolderPath));
verifyMarkersInFileSystem();
verifyMarkersInFileSystem(isTablePartitioned);
}
@Test
public void testDeletionWhenMarkerDirExists() throws IOException {
@ParameterizedTest
@ValueSource(booleans = {true, false})
public void testDeletionWhenMarkerDirExists(boolean isTablePartitioned) throws IOException {
//when
writeMarkers.create("2020/06/01", "file1", IOType.MERGE);
writeMarkers.create(isTablePartitioned ? "2020/06/01" : "", "file1", IOType.MERGE);
// then
assertTrue(writeMarkers.doesMarkerDirExist());
@@ -95,32 +100,40 @@ public abstract class TestWriteMarkersBase extends HoodieCommonTestHarness {
assertFalse(writeMarkers.deleteMarkerDir(context, 2));
}
@Test
public void testDataPathsWhenCreatingOrMerging() throws IOException {
@ParameterizedTest
@ValueSource(booleans = {true, false})
public void testDataPathsWhenCreatingOrMerging(boolean isTablePartitioned) throws IOException {
// add markfiles
createSomeMarkers();
createSomeMarkers(isTablePartitioned);
// add invalid file
createInvalidFile("2020/06/01", "invalid_file3");
createInvalidFile(isTablePartitioned ? "2020/06/01" : "", "invalid_file3");
long fileSize = FileSystemTestUtils.listRecursive(fs, markerFolderPath).stream()
.filter(fileStatus -> !fileStatus.getPath().getName().contains(MarkerUtils.MARKER_TYPE_FILENAME))
.count();
assertEquals(fileSize, 4);
List<String> expectedPaths = isTablePartitioned
? CollectionUtils.createImmutableList("2020/06/01/file1", "2020/06/03/file3")
: CollectionUtils.createImmutableList("file1", "file3");
// then
assertIterableEquals(CollectionUtils.createImmutableList(
"2020/06/01/file1", "2020/06/03/file3"),
assertIterableEquals(expectedPaths,
writeMarkers.createdAndMergedDataPaths(context, 2).stream().sorted().collect(Collectors.toList())
);
}
@Test
public void testAllMarkerPaths() throws IOException {
@ParameterizedTest
@ValueSource(booleans = {true, false})
public void testAllMarkerPaths(boolean isTablePartitioned) throws IOException {
// given
createSomeMarkers();
createSomeMarkers(isTablePartitioned);
List<String> expectedPaths = isTablePartitioned
? CollectionUtils.createImmutableList("2020/06/01/file1.marker.MERGE",
"2020/06/02/file2.marker.APPEND", "2020/06/03/file3.marker.CREATE")
: CollectionUtils.createImmutableList(
"file1.marker.MERGE", "file2.marker.APPEND", "file3.marker.CREATE");
// then
assertIterableEquals(CollectionUtils.createImmutableList("2020/06/01/file1.marker.MERGE",
"2020/06/02/file2.marker.APPEND", "2020/06/03/file3.marker.CREATE"),
assertIterableEquals(expectedPaths,
writeMarkers.allMarkerFilePaths().stream()
.filter(path -> !path.contains(MarkerUtils.MARKER_TYPE_FILENAME))
.sorted().collect(Collectors.toList())

View File

@@ -326,9 +326,11 @@ public class TestUpgradeDowngrade extends HoodieClientTestBase {
new UpgradeDowngrade(metaClient, cfg, context, SparkUpgradeDowngradeHelper.getInstance())
.run(toVersion, null);
// assert marker files
assertMarkerFilesForDowngrade(table, commitInstant, toVersion == HoodieTableVersion.ONE);
if (fromVersion == HoodieTableVersion.TWO) {
// assert marker files
assertMarkerFilesForDowngrade(table, commitInstant, toVersion == HoodieTableVersion.ONE);
}
// verify hoodie.table.version got downgraded
metaClient = HoodieTableMetaClient.builder().setConf(context.getHadoopConf().get()).setBasePath(cfg.getBasePath())
.setLayoutVersion(Option.of(new TimelineLayoutVersion(cfg.getTimelineLayoutVersion()))).build();

View File

@@ -152,6 +152,7 @@ public class HoodieClientTestBase extends HoodieClientTestHarness {
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build())
.withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
.withEnableBackupForRemoteFileSystemView(false) // Fail test if problem connecting to timeline-server
.withRemoteServerPort(timelineServicePort)
.withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE).build());
}

View File

@@ -32,6 +32,7 @@ import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
import org.apache.hudi.common.table.view.TableFileSystemView;
import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
@@ -56,6 +57,7 @@ import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
import org.apache.hudi.table.HoodieSparkTable;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.WorkloadStat;
import org.apache.hudi.timeline.service.TimelineService;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
@@ -102,6 +104,8 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
private static final Logger LOG = LogManager.getLogger(HoodieClientTestHarness.class);
protected static int timelineServicePort =
FileSystemViewStorageConfig.REMOTE_PORT_NUM.defaultValue();
private String testMethodName;
protected transient JavaSparkContext jsc = null;
protected transient HoodieSparkEngineContext context = null;
@@ -113,6 +117,7 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
protected transient SparkRDDWriteClient writeClient;
protected transient HoodieReadClient readClient;
protected transient HoodieTableFileSystemView tableView;
protected transient TimelineService timelineService;
protected final SparkTaskContextSupplier supplier = new SparkTaskContextSupplier();
@@ -145,12 +150,14 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
initTestDataGenerator();
initFileSystem();
initMetaClient();
initTimelineService();
}
/**
* Cleanups resource group for the subclasses of {@link HoodieClientTestBase}.
*/
public void cleanupResources() throws IOException {
cleanupTimelineService();
cleanupClients();
cleanupSparkContexts();
cleanupTestDataGenerator();
@@ -245,6 +252,7 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
*
* @throws IOException
*/
@Override
protected void initMetaClient() throws IOException {
initMetaClient(getTableType());
}
@@ -272,6 +280,28 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
metaClient = HoodieTestUtils.init(hadoopConf, basePath, tableType, properties);
}
/**
* Initializes timeline service based on the write config.
*/
protected void initTimelineService() {
timelineService = HoodieClientTestUtils.initTimelineService(
context, basePath, incrementTimelineServicePortToUse());
timelineServicePort = timelineService.getServerPort();
}
protected void cleanupTimelineService() {
if (timelineService != null) {
timelineService.close();
}
}
protected int incrementTimelineServicePortToUse() {
// Increment the timeline service port for each individual test
// to avoid port reuse causing failures
timelineServicePort = (timelineServicePort + 1 - 1024) % (65536 - 1024) + 1024;
return timelineServicePort;
}
protected Properties getPropertiesForKeyGen() {
Properties properties = new Properties();
properties.put(HoodieTableConfig.POPULATE_META_FIELDS.key(), "false");

View File

@@ -20,6 +20,7 @@ package org.apache.hudi.testutils;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.client.HoodieReadClient;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieCommitMetadata;
@@ -30,13 +31,18 @@ import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.view.FileSystemViewManager;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.timeline.service.TimelineService;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
@@ -257,6 +263,36 @@ public class HoodieClientTestUtils {
return valuesAsList.stream();
}
/**
* Initializes timeline service based on the write config.
*
* @param context {@link HoodieEngineContext} instance to use.
* @param basePath Base path of the table.
* @param timelineServicePort Port number to use for timeline service.
* @return started {@link TimelineService} instance.
*/
public static TimelineService initTimelineService(
HoodieEngineContext context, String basePath, int timelineServicePort) {
try {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder()
.withPath(basePath)
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
.withRemoteServerPort(timelineServicePort).build())
.build();
TimelineService timelineService = new TimelineService(context, new Configuration(),
TimelineService.Config.builder().enableMarkerRequests(true)
.serverPort(config.getViewStorageConfig().getRemoteViewServerPort()).build(),
FileSystem.get(new Configuration()),
FileSystemViewManager.createViewManager(context, config.getMetadataConfig(),
config.getViewStorageConfig(), config.getCommonConfig()));
timelineService.startService();
LOG.info("Timeline service server port: " + timelineServicePort);
return timelineService;
} catch (Exception ex) {
throw new RuntimeException(ex);
}
}
public static Option<HoodieCommitMetadata> getCommitMetadataForLatestInstant(HoodieTableMetaClient metaClient) {
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
if (timeline.lastInstant().isPresent()) {

View File

@@ -49,6 +49,7 @@ import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.testutils.providers.HoodieMetaClientProvider;
import org.apache.hudi.testutils.providers.HoodieWriteClientProvider;
import org.apache.hudi.testutils.providers.SparkProvider;
import org.apache.hudi.timeline.service.TimelineService;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
@@ -81,10 +82,13 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
public class SparkClientFunctionalTestHarness implements SparkProvider, HoodieMetaClientProvider, HoodieWriteClientProvider {
protected static int timelineServicePort =
FileSystemViewStorageConfig.REMOTE_PORT_NUM.defaultValue();
private static transient SparkSession spark;
private static transient SQLContext sqlContext;
private static transient JavaSparkContext jsc;
private static transient HoodieSparkEngineContext context;
private static transient TimelineService timelineService;
/**
* An indicator of the initialization status.
@@ -174,6 +178,9 @@ public class SparkClientFunctionalTestHarness implements SparkProvider, HoodieMe
sqlContext = spark.sqlContext();
jsc = new JavaSparkContext(spark.sparkContext());
context = new HoodieSparkEngineContext(jsc);
timelineService = HoodieClientTestUtils.initTimelineService(
context, basePath(), incrementTimelineServicePortToUse());
timelineServicePort = timelineService.getServerPort();
}
}
@@ -189,6 +196,9 @@ public class SparkClientFunctionalTestHarness implements SparkProvider, HoodieMe
spark.close();
spark = null;
}
if (timelineService != null) {
timelineService.close();
}
}
protected JavaRDD<HoodieRecord> tagLocation(
@@ -312,9 +322,17 @@ public class SparkClientFunctionalTestHarness implements SparkProvider, HoodieMe
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024 * 1024).parquetMaxFileSize(1024 * 1024 * 1024).build())
.withEmbeddedTimelineServerEnabled(true).forTable("test-trip-table")
.withFileSystemViewConfig(new FileSystemViewStorageConfig.Builder()
.withRemoteServerPort(timelineServicePort)
.withEnableBackupForRemoteFileSystemView(false).build())
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build())
.withClusteringConfig(clusteringConfig)
.withRollbackUsingMarkers(rollbackUsingMarkers);
}
protected int incrementTimelineServicePortToUse() {
// Increment the timeline service port for each individual test
// to avoid port reuse causing failures
timelineServicePort = (timelineServicePort + 1 - 1024) % (65536 - 1024) + 1024;
return timelineServicePort;
}
}

View File

@@ -19,6 +19,7 @@
package org.apache.hudi.testutils;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieIndexConfig;
@@ -176,12 +177,14 @@ public class SparkDatasetTestUtils {
return new GenericInternalRow(values);
}
public static HoodieWriteConfig.Builder getConfigBuilder(String basePath) {
public static HoodieWriteConfig.Builder getConfigBuilder(String basePath, int timelineServicePort) {
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
.withParallelism(2, 2)
.withDeleteParallelism(2)
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build())
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).build())
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
.withRemoteServerPort(timelineServicePort).build())
.forTable("test-trip-table")
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
.withBulkInsertParallelism(2);