1
0

[HUDI 1308] Harden RFC-15 Implementation based on production testing (#2441)

Addresses leaks, perf degradation observed during testing. These were regressions from the original rfc-15 PoC implementation.

* Pass a single instance of HoodieTableMetadata everywhere
* Fix tests and add config for enabling metrics
 - Removed special casing of assumeDatePartitioning inside FSUtils#getAllPartitionPaths()
 - Consequently, IOException is never thrown and many files had to be adjusted
- More diligent handling of open file handles in metadata table
 - Added config for controlling reuse of connections
 - Added config for turning off fallback to listing, so we can see tests fail
 - Changed all ipf listing code to cache/amortize the open/close for better performance
 - Timelineserver also reuses connections, for better performance
 - Without timelineserver, when metadata table is opened from executors, reuse is not allowed
 - HoodieMetadataConfig passed into HoodieTableMetadata#create as argument.
 -  Fix TestHoodieBackedTableMetadata#testSync
This commit is contained in:
vinoth chandar
2021-01-19 21:20:28 -08:00
committed by GitHub
parent e23967b9e9
commit 5ca0625b27
55 changed files with 767 additions and 570 deletions

View File

@@ -41,8 +41,10 @@ import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieClusteringException;
import org.apache.hudi.exception.HoodieCommitException;
import org.apache.hudi.exception.HoodieMetadataException;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.index.SparkHoodieIndex;
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
import org.apache.hudi.metrics.DistributedRegistry;
import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
import org.apache.hudi.table.BulkInsertPartitioner;
@@ -395,7 +397,11 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
@Override
public void syncTableMetadata() {
// Open up the metadata table again, for syncing
SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context);
try (HoodieTableMetadataWriter writer = SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context)) {
LOG.info("Successfully synced to metadata table");
} catch (Exception e) {
throw new HoodieMetadataException("Error syncing to metadata table.", e);
}
}
@Override

View File

@@ -28,7 +28,6 @@ import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.index.HoodieIndexUtils;
import org.apache.hudi.table.HoodieTable;
@@ -36,7 +35,6 @@ import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.Optional;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
@@ -62,13 +60,8 @@ public class SparkHoodieGlobalBloomIndex<T extends HoodieRecordPayload> extends
List<Tuple2<String, BloomIndexFileInfo>> loadInvolvedFiles(List<String> partitions, final HoodieEngineContext context,
final HoodieTable hoodieTable) {
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
try {
List<String> allPartitionPaths = FSUtils.getAllPartitionPaths(context, metaClient.getFs(), metaClient.getBasePath(),
config.useFileListingMetadata(), config.getFileListingMetadataVerify(), config.shouldAssumeDatePartitioning());
return super.loadInvolvedFiles(allPartitionPaths, context, hoodieTable);
} catch (IOException e) {
throw new HoodieIOException("Failed to load all partitions", e);
}
List<String> allPartitionPaths = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), metaClient.getBasePath());
return super.loadInvolvedFiles(allPartitionPaths, context, hoodieTable);
}
/**

View File

@@ -31,14 +31,12 @@ import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.index.HoodieIndexUtils;
import org.apache.hudi.table.HoodieTable;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
import scala.Tuple2;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
@@ -103,14 +101,9 @@ public class SparkHoodieGlobalSimpleIndex<T extends HoodieRecordPayload> extends
protected List<Pair<String, HoodieBaseFile>> getAllBaseFilesInTable(final HoodieEngineContext context,
final HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
try {
List<String> allPartitionPaths = FSUtils.getAllPartitionPaths(context, metaClient.getFs(), metaClient.getBasePath(),
config.useFileListingMetadata(), config.getFileListingMetadataVerify(), config.shouldAssumeDatePartitioning());
// Obtain the latest data files from all the partitions.
return getLatestBaseFilesForAllPartitions(allPartitionPaths, context, hoodieTable);
} catch (IOException e) {
throw new HoodieIOException("Failed to load all partitions", e);
}
List<String> allPartitionPaths = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), metaClient.getBasePath());
// Obtain the latest data files from all the partitions.
return getLatestBaseFilesForAllPartitions(allPartitionPaths, context, hoodieTable);
}
/**

View File

@@ -99,8 +99,6 @@ public class SparkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetad
@Override
protected void commit(List<HoodieRecord> records, String partitionName, String instantTime) {
ValidationUtils.checkState(enabled, "Metadata table cannot be committed to as it is not enabled");
metadata.closeReaders();
JavaRDD<HoodieRecord> recordRDD = prepRecords(records, partitionName);
try (SparkRDDWriteClient writeClient = new SparkRDDWriteClient(engineContext, metadataWriteConfig, true)) {

View File

@@ -26,13 +26,11 @@ import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieCommitException;
import org.apache.hudi.table.HoodieTable;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import scala.Tuple2;
import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -49,20 +47,13 @@ public class SparkInsertOverwriteTableCommitActionExecutor<T extends HoodieRecor
@Override
protected Map<String, List<String>> getPartitionToReplacedFileIds(JavaRDD<WriteStatus> writeStatuses) {
Map<String, List<String>> partitionToExistingFileIds = new HashMap<>();
try {
List<String> partitionPaths = FSUtils.getAllPartitionPaths(context, table.getMetaClient().getFs(),
table.getMetaClient().getBasePath(), config.useFileListingMetadata(), config.getFileListingMetadataVerify(),
false);
JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
if (partitionPaths != null && partitionPaths.size() > 0) {
context.setJobStatus(this.getClass().getSimpleName(), "Getting ExistingFileIds of all partitions");
JavaRDD<String> partitionPathRdd = jsc.parallelize(partitionPaths, partitionPaths.size());
partitionToExistingFileIds = partitionPathRdd.mapToPair(
partitionPath -> new Tuple2<>(partitionPath, getAllExistingFileIds(partitionPath))).collectAsMap();
}
} catch (IOException e) {
throw new HoodieCommitException("In InsertOverwriteTable action failed to get existing fileIds of all partition "
+ config.getBasePath() + " at time " + instantTime, e);
List<String> partitionPaths = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), table.getMetaClient().getBasePath());
JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
if (partitionPaths != null && partitionPaths.size() > 0) {
context.setJobStatus(this.getClass().getSimpleName(), "Getting ExistingFileIds of all partitions");
JavaRDD<String> partitionPathRdd = jsc.parallelize(partitionPaths, partitionPaths.size());
partitionToExistingFileIds = partitionPathRdd.mapToPair(
partitionPath -> new Tuple2<>(partitionPath, getAllExistingFileIds(partitionPath))).collectAsMap();
}
return partitionToExistingFileIds;
}

View File

@@ -195,8 +195,7 @@ public class HoodieSparkMergeOnReadTableCompactor<T extends HoodieRecordPayload>
// TODO - rollback any compactions in flight
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
LOG.info("Compacting " + metaClient.getBasePath() + " with commit " + compactionCommitTime);
List<String> partitionPaths = FSUtils.getAllPartitionPaths(context, metaClient.getFs(), metaClient.getBasePath(),
config.useFileListingMetadata(), config.getFileListingMetadataVerify(), config.shouldAssumeDatePartitioning());
List<String> partitionPaths = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), metaClient.getBasePath());
// filter the partition paths if needed to reduce list status
partitionPaths = config.getCompactionStrategy().filterPartitionPaths(config, partitionPaths);

View File

@@ -67,7 +67,7 @@ public class SparkCopyOnWriteRollbackActionExecutor<T extends HoodieRecordPayloa
@Override
protected List<HoodieRollbackStat> executeRollbackUsingFileListing(HoodieInstant instantToRollback) {
List<ListingBasedRollbackRequest> rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(context,
table.getMetaClient().getFs(), table.getMetaClient().getBasePath(), config);
table.getMetaClient().getBasePath(), config);
return new ListingBasedRollbackHelper(table.getMetaClient(), config).performRollback(context, instantToRollback, rollbackRequests);
}
}

View File

@@ -92,8 +92,7 @@ public class ZeroToOneUpgradeHandler implements UpgradeHandler {
// generate rollback stats
List<ListingBasedRollbackRequest> rollbackRequests;
if (table.getMetaClient().getTableType() == HoodieTableType.COPY_ON_WRITE) {
rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(context, table.getMetaClient().getFs(),
table.getMetaClient().getBasePath(), table.getConfig());
rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(context, table.getMetaClient().getBasePath(), table.getConfig());
} else {
rollbackRequests = RollbackUtils.generateRollbackRequestsUsingFileListingMOR(commitInstantOpt.get(), table, context);
}

View File

@@ -101,8 +101,7 @@ public class TestClientRollback extends HoodieClientTestBase {
assertNoWriteErrors(statuses);
HoodieWriteConfig config = getConfig();
List<String> partitionPaths =
FSUtils.getAllPartitionPaths(context, fs, cfg.getBasePath(), config.useFileListingMetadata(),
config.getFileListingMetadataVerify(), config.shouldAssumeDatePartitioning());
FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), cfg.getBasePath());
metaClient = HoodieTableMetaClient.reload(metaClient);
HoodieSparkTable table = HoodieSparkTable.create(getConfig(), context, metaClient);
final BaseFileOnlyView view1 = table.getBaseFileOnlyView();

View File

@@ -265,7 +265,6 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
@ParameterizedTest
@EnumSource(HoodieTableType.class)
public void testRollbackOperations(HoodieTableType tableType) throws Exception {
//FIXME(metadata): This is broken for MOR, until HUDI-1434 is fixed
init(tableType);
HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
@@ -511,10 +510,13 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
assertTrue(metadata(client).isInSync());
}
// Enable metadata table and ensure it is synced
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) {
// Restore cannot be done until the metadata table is in sync. See HUDI-1502 for details
client.syncTableMetadata();
}
// Enable metadata table and ensure it is synced
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) {
client.restoreToInstant(restoreToInstant);
assertFalse(metadata(client).isInSync());
@@ -752,8 +754,16 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
/**
* Validate the metadata tables contents to ensure it matches what is on the file system.
*/
private void validateMetadata(SparkRDDWriteClient client) throws IOException {
HoodieWriteConfig config = client.getConfig();
private void validateMetadata(SparkRDDWriteClient testClient) throws IOException {
HoodieWriteConfig config = testClient.getConfig();
SparkRDDWriteClient client;
if (config.isEmbeddedTimelineServerEnabled()) {
testClient.close();
client = new SparkRDDWriteClient(testClient.getEngineContext(), testClient.getConfig());
} else {
client = testClient;
}
HoodieTableMetadata tableMetadata = metadata(client);
assertNotNull(tableMetadata, "MetadataReader should have been initialized");
@@ -869,7 +879,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
// Metadata table has a fixed number of partitions
// Cannot use FSUtils.getAllFoldersWithPartitionMetaFile for this as that function filters all directory
// in the .hoodie folder.
List<String> metadataTablePartitions = FSUtils.getAllPartitionPaths(engineContext, fs, HoodieTableMetadata.getMetadataTableBasePath(basePath),
List<String> metadataTablePartitions = FSUtils.getAllPartitionPaths(engineContext, HoodieTableMetadata.getMetadataTableBasePath(basePath),
false, false, false);
assertEquals(MetadataPartitionType.values().length, metadataTablePartitions.size());
@@ -895,9 +905,8 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
private HoodieTableMetadata metadata(SparkRDDWriteClient client) {
HoodieWriteConfig clientConfig = client.getConfig();
return HoodieTableMetadata.create(client.getEngineContext(), clientConfig.getBasePath(),
clientConfig.getSpillableMapBasePath(), clientConfig.useFileListingMetadata(),
clientConfig.getFileListingMetadataVerify(), false, clientConfig.shouldAssumeDatePartitioning());
return HoodieTableMetadata.create(client.getEngineContext(), clientConfig.getMetadataConfig(), clientConfig.getBasePath(),
clientConfig.getSpillableMapBasePath());
}
// TODO: this can be moved to TestHarness after merge from master
@@ -915,16 +924,20 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
private HoodieWriteConfig.Builder getWriteConfigBuilder(boolean autoCommit, boolean useFileListingMetadata, boolean enableMetrics) {
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA)
.withParallelism(2, 2).withDeleteParallelism(2).withRollbackParallelism(2).withFinalizeWriteParallelism(2)
.withAutoCommit(autoCommit).withAssumeDatePartitioning(false)
.withAutoCommit(autoCommit)
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024)
.withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1)
.withAutoClean(false).retainCommits(1).retainFileVersions(1).build())
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024 * 1024).build())
.withEmbeddedTimelineServerEnabled(false).forTable("test-trip-table")
.withEmbeddedTimelineServerEnabled(true).forTable("test-trip-table")
.withFileSystemViewConfig(new FileSystemViewStorageConfig.Builder()
.withEnableBackupForRemoteFileSystemView(false).build())
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(useFileListingMetadata).build())
.withMetadataConfig(HoodieMetadataConfig.newBuilder()
.enable(useFileListingMetadata)
.enableReuse(false)
.enableMetrics(enableMetrics)
.enableFallback(false).build())
.withMetricsConfig(HoodieMetricsConfig.newBuilder().on(enableMetrics)
.withExecutorMetrics(true).usePrefix("unit-test").build());
}

View File

@@ -30,6 +30,7 @@ import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.HoodieCleanStat;
import org.apache.hudi.common.bootstrap.TestBootstrapIndex;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.fs.ConsistencyGuardConfig;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.BootstrapFileMapping;
@@ -524,7 +525,8 @@ public class TestCleaner extends HoodieClientTestBase {
@ValueSource(booleans = {false, true})
public void testKeepLatestFileVersions(Boolean enableBootstrapSourceClean) throws Exception {
HoodieWriteConfig config =
HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true)
HoodieWriteConfig.newBuilder().withPath(basePath)
.withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build())
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withCleanBootstrapBaseFileEnabled(enableBootstrapSourceClean)
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1).build())
@@ -615,7 +617,8 @@ public class TestCleaner extends HoodieClientTestBase {
public void testKeepLatestFileVersionsMOR() throws Exception {
HoodieWriteConfig config =
HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true)
HoodieWriteConfig.newBuilder().withPath(basePath)
.withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build())
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1).build())
.build();
@@ -652,7 +655,8 @@ public class TestCleaner extends HoodieClientTestBase {
public void testKeepLatestCommitsMOR() throws Exception {
HoodieWriteConfig config =
HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true)
HoodieWriteConfig.newBuilder().withPath(basePath)
.withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build())
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1).build())
.build();
@@ -691,7 +695,8 @@ public class TestCleaner extends HoodieClientTestBase {
@Test
public void testCleanWithReplaceCommits() throws Exception {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true)
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
.withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build())
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build())
.build();
@@ -971,7 +976,8 @@ public class TestCleaner extends HoodieClientTestBase {
@ParameterizedTest
@MethodSource("argumentsForTestKeepLatestCommits")
public void testKeepLatestCommits(boolean simulateFailureRetry, boolean enableIncrementalClean, boolean enableBootstrapSourceClean) throws Exception {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true)
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
.withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build())
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withIncrementalCleaningMode(enableIncrementalClean)
.withCleanBootstrapBaseFileEnabled(enableBootstrapSourceClean)
@@ -1146,7 +1152,8 @@ public class TestCleaner extends HoodieClientTestBase {
*/
@Test
public void testCleaningWithZeroPartitionPaths() throws Exception {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true)
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
.withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build())
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build())
.build();
@@ -1167,7 +1174,8 @@ public class TestCleaner extends HoodieClientTestBase {
*/
@Test
public void testKeepLatestCommitsWithPendingCompactions() throws Exception {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true)
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
.withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build())
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build())
.build();
@@ -1190,7 +1198,8 @@ public class TestCleaner extends HoodieClientTestBase {
@ValueSource(booleans = {false, true})
public void testKeepLatestVersionsWithPendingCompactions(boolean retryFailure) throws Exception {
HoodieWriteConfig config =
HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true)
HoodieWriteConfig.newBuilder().withPath(basePath)
.withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build())
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(2).build())
.build();
@@ -1213,7 +1222,8 @@ public class TestCleaner extends HoodieClientTestBase {
public void testCleanPreviousCorruptedCleanFiles() throws IOException {
HoodieWriteConfig config =
HoodieWriteConfig.newBuilder()
.withPath(basePath).withAssumeDatePartitioning(true)
.withPath(basePath)
.withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build())
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1).build())
.build();

View File

@@ -887,7 +887,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
protected HoodieWriteConfig getHoodieWriteConfigWithSmallFileHandlingOff() {
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.withDeleteParallelism(2)
.withAutoCommit(false).withAssumeDatePartitioning(true)
.withAutoCommit(false)
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024)
.withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1).build())
.withEmbeddedTimelineServerEnabled(true)
@@ -1564,7 +1564,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
long compactionSmallFileSize, HoodieClusteringConfig clusteringConfig) {
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.withDeleteParallelism(2)
.withAutoCommit(autoCommit).withAssumeDatePartitioning(true)
.withAutoCommit(autoCommit)
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(compactionSmallFileSize)
.withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1).build())
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024 * 1024).parquetMaxFileSize(1024 * 1024 * 1024).build())

View File

@@ -23,6 +23,7 @@ import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.client.HoodieReadClient;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieFileGroupId;
@@ -70,7 +71,8 @@ public class CompactionTestBase extends HoodieClientTestBase {
return HoodieWriteConfig.newBuilder().withPath(basePath)
.withSchema(TRIP_EXAMPLE_SCHEMA)
.withParallelism(2, 2)
.withAutoCommit(autoCommit).withAssumeDatePartitioning(true)
.withAutoCommit(autoCommit)
.withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build())
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024)
.withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1).build())
.withStorageConfig(HoodieStorageConfig.newBuilder()