Skip to content
Permalink
Browse files
[HUDI-993] Let delete API use "hoodie.delete.shuffle.parallelism" (#1703
)

For Delete API, "hoodie.delete.shuffle.parallelism" isn't used as opposed to "hoodie.upsert.shuffle.parallelism" is used for upsert, this creates the performance difference between delete by upsert API with "EmptyHoodieRecordPayload" and delete API for certain cases.

This patch makes the following fixes in this regard. 
- Let deduplicateKeys method use "hoodie.delete.shuffle.parallelism"
- Repartition inputRDD as "hoodie.delete.shuffle.parallelism" in case "hoodie.combine.before.delete=false"
  • Loading branch information
kwondw committed Sep 1, 2020
1 parent 48a58c9 commit 8d19ebfd0f1edc412332045ca6ba10f50af01d9f
Show file tree
Hide file tree
Showing 21 changed files with 234 additions and 15 deletions.
@@ -17,6 +17,7 @@

hoodie.upsert.shuffle.parallelism=2
hoodie.insert.shuffle.parallelism=2
hoodie.delete.shuffle.parallelism=2
hoodie.bulkinsert.shuffle.parallelism=2
hoodie.embed.timeline.server=true
hoodie.filesystem.view.type=EMBEDDED_KV_STORE
@@ -17,6 +17,7 @@

hoodie.upsert.shuffle.parallelism=2
hoodie.insert.shuffle.parallelism=2
hoodie.delete.shuffle.parallelism=2
hoodie.bulkinsert.shuffle.parallelism=2
hoodie.datasource.write.recordkey.field=_row_key
hoodie.datasource.write.partitionpath.field=partition
@@ -17,5 +17,6 @@
#
hoodie.upsert.shuffle.parallelism=2
hoodie.insert.shuffle.parallelism=2
hoodie.delete.shuffle.parallelism=2
hoodie.bulkinsert.shuffle.parallelism=2
hoodie.datasource.write.partitionpath.field=timestamp
@@ -209,7 +209,7 @@ public int getUpsertShuffleParallelism() {
}

public int getDeleteShuffleParallelism() {
return Integer.parseInt(props.getProperty(DELETE_PARALLELISM));
return Math.max(Integer.parseInt(props.getProperty(DELETE_PARALLELISM)), 1);
}

public int getRollbackParallelism() {
@@ -824,6 +824,11 @@ public Builder withUserDefinedBulkInsertPartitionerClass(String className) {
return this;
}

public Builder withDeleteParallelism(int parallelism) {
props.setProperty(DELETE_PARALLELISM, String.valueOf(parallelism));
return this;
}

public Builder withParallelism(int insertShuffleParallelism, int upsertShuffleParallelism) {
props.setProperty(INSERT_PARALLELISM, String.valueOf(insertShuffleParallelism));
props.setProperty(UPSERT_PARALLELISM, String.valueOf(upsertShuffleParallelism));
@@ -851,6 +856,11 @@ public Builder combineInput(boolean onInsert, boolean onUpsert) {
return this;
}

public Builder combineDeleteInput(boolean onDelete) {
props.setProperty(COMBINE_BEFORE_DELETE_PROP, String.valueOf(onDelete));
return this;
}

public Builder withWriteStatusStorageLevel(String level) {
props.setProperty(WRITE_STATUS_STORAGE_LEVEL, level);
return this;
@@ -44,27 +44,36 @@
* Deduplicate Hoodie records, using the given deduplication function.
*
* @param keys RDD of HoodieKey to deduplicate
* @param table target Hoodie table for deduplicating
* @param parallelism parallelism or partitions to be used while reducing/deduplicating
* @return RDD of HoodieKey already be deduplicated
*/
private static <T extends HoodieRecordPayload<T>> JavaRDD<HoodieKey> deduplicateKeys(JavaRDD<HoodieKey> keys,
HoodieTable<T> table) {
HoodieTable<T> table, int parallelism) {
boolean isIndexingGlobal = table.getIndex().isGlobal();
if (isIndexingGlobal) {
return keys.keyBy(HoodieKey::getRecordKey)
.reduceByKey((key1, key2) -> key1)
.reduceByKey((key1, key2) -> key1, parallelism)
.values();
} else {
return keys.distinct();
return keys.distinct(parallelism);
}
}

public static <T extends HoodieRecordPayload<T>> HoodieWriteMetadata execute(String instantTime,
JavaRDD<HoodieKey> keys, JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable<T> table,
JavaRDD<HoodieKey> keys, JavaSparkContext jsc,
HoodieWriteConfig config, HoodieTable<T> table,
CommitActionExecutor<T> deleteExecutor) {
try {
HoodieWriteMetadata result = null;
// De-dupe/merge if needed
JavaRDD<HoodieKey> dedupedKeys = config.shouldCombineBeforeDelete() ? deduplicateKeys(keys, table) : keys;
JavaRDD<HoodieKey> dedupedKeys = keys;
final int parallelism = config.getDeleteShuffleParallelism();
if (config.shouldCombineBeforeDelete()) {
// De-dupe/merge if needed
dedupedKeys = deduplicateKeys(keys, table, parallelism);
} else if (!keys.partitions().isEmpty()) {
dedupedKeys = keys.repartition(parallelism);
}

JavaRDD<HoodieRecord<T>> dedupedRecords =
dedupedKeys.map(key -> new HoodieRecord(key, new EmptyHoodieRecordPayload()));
@@ -74,7 +83,7 @@ public static <T extends HoodieRecordPayload<T>> HoodieWriteMetadata execute(Str
((HoodieTable<T>)table).getIndex().tagLocation(dedupedRecords, jsc, (HoodieTable<T>)table);
Duration tagLocationDuration = Duration.between(beginTag, Instant.now());

// filter out non existant keys/records
// filter out non existent keys/records
JavaRDD<HoodieRecord<T>> taggedValidRecords = taggedRecords.filter(HoodieRecord::isCurrentLocationKnown);
if (!taggedValidRecords.isEmpty()) {
result = deleteExecutor.execute(taggedValidRecords);
@@ -437,7 +437,7 @@ HoodieWriteConfig.Builder getConfigBuilder(String schemaStr) {
*/
private HoodieWriteConfig.Builder getConfigBuilder(String schemaStr, IndexType indexType) {
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schemaStr)
.withParallelism(2, 2).withBulkInsertParallelism(2).withFinalizeWriteParallelism(2)
.withParallelism(2, 2).withBulkInsertParallelism(2).withFinalizeWriteParallelism(2).withDeleteParallelism(2)
.withWriteStatusClass(MetadataMergeWriteStatus.class)
.withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build())
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build())
@@ -462,7 +462,7 @@ private HoodieWriteConfig getConfig(int hbaseIndexBatchSize) {

private HoodieWriteConfig.Builder getConfigBuilder(int hbaseIndexBatchSize) {
return HoodieWriteConfig.newBuilder().withPath(basePath()).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
.withParallelism(1, 1)
.withParallelism(1, 1).withDeleteParallelism(1)
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024)
.withInlineCompaction(false).build())
.withAutoCommit(false).withStorageConfig(HoodieStorageConfig.newBuilder()
@@ -174,7 +174,7 @@ public HoodieWriteConfig.Builder getConfigBuilder() {
*/
private HoodieWriteConfig.Builder getConfigBuilder(String schemaStr) {
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schemaStr)
.withParallelism(2, 2).withBulkInsertParallelism(2).withFinalizeWriteParallelism(2)
.withParallelism(2, 2).withBulkInsertParallelism(2).withFinalizeWriteParallelism(2).withDeleteParallelism(2)
.withWriteStatusClass(MetadataMergeWriteStatus.class)
.withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build())
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build())
@@ -309,6 +309,7 @@ private Dataset<Row> getRecords() {
HoodieWriteConfig.Builder getConfigBuilder() {
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())
.forTable("test-trip-table")
@@ -207,7 +207,7 @@ private void testInsertAndCleanByVersions(
HoodieWriteConfig cfg = getConfigBuilder()
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(maxVersions).build())
.withParallelism(1, 1).withBulkInsertParallelism(1).withFinalizeWriteParallelism(1)
.withParallelism(1, 1).withBulkInsertParallelism(1).withFinalizeWriteParallelism(1).withDeleteParallelism(1)
.withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build())
.build();
try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
@@ -368,7 +368,7 @@ private void testInsertAndCleanByCommits(
HoodieWriteConfig cfg = getConfigBuilder()
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(maxCommits).build())
.withParallelism(1, 1).withBulkInsertParallelism(1).withFinalizeWriteParallelism(1)
.withParallelism(1, 1).withBulkInsertParallelism(1).withFinalizeWriteParallelism(1).withDeleteParallelism(1)
.withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build())
.build();
HoodieWriteClient client = getHoodieWriteClient(cfg);
@@ -803,6 +803,7 @@ public void testMultiRollbackWithDeltaAndCompactionCommit() throws Exception {

protected HoodieWriteConfig getHoodieWriteConfigWithSmallFileHandlingOff() {
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.withDeleteParallelism(2)
.withAutoCommit(false).withAssumeDatePartitioning(true)
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024)
.withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1).build())
@@ -1466,6 +1467,7 @@ protected HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit, HoodieI

protected HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit, Boolean rollbackUsingMarkers, HoodieIndex.IndexType indexType) {
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.withDeleteParallelism(2)
.withAutoCommit(autoCommit).withAssumeDatePartitioning(true)
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024)
.withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1).build())
@@ -0,0 +1,186 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.hudi.table.action.commit;

import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.index.bloom.HoodieBloomIndex;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.action.HoodieWriteMetadata;
import org.apache.spark.Partition;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;

import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.Mock;
import org.mockito.junit.jupiter.MockitoExtension;
import scala.Tuple2;

import java.util.Collections;
import java.util.List;

import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyInt;
import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.doNothing;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.never;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;

@ExtendWith(MockitoExtension.class)
public class TestDeleteHelper {

private enum CombineTestMode {
None, GlobalIndex, NoneGlobalIndex;
}

private static final String BASE_PATH = "/tmp/";
private static final boolean WITH_COMBINE = true;
private static final boolean WITHOUT_COMBINE = false;
private static final int DELETE_PARALLELISM = 200;

@Mock private HoodieBloomIndex index;
@Mock private HoodieTable<EmptyHoodieRecordPayload> table;
@Mock private CommitActionExecutor<EmptyHoodieRecordPayload> executor;
@Mock private HoodieWriteMetadata metadata;
@Mock private JavaPairRDD keyPairs;
@Mock private JavaSparkContext jsc;

private JavaRDD<HoodieKey> rddToDelete;
private HoodieWriteConfig config;

@BeforeEach
public void setUp() {
when(table.getIndex()).thenReturn(index);
}

@Test
public void deleteWithEmptyRDDShouldNotExecute() {
rddToDelete = mockEmptyHoodieKeyRdd();
config = newWriteConfig(WITHOUT_COMBINE);

DeleteHelper.execute("test-time", rddToDelete, jsc, config, table, executor);

verify(rddToDelete, never()).repartition(DELETE_PARALLELISM);
verifyNoDeleteExecution();
}

@Test
public void deleteWithoutCombineShouldRepartitionForNonEmptyRdd() {
rddToDelete = newHoodieKeysRddMock(2, CombineTestMode.None);
config = newWriteConfig(WITHOUT_COMBINE);

DeleteHelper.execute("test-time", rddToDelete, jsc, config, table, executor);

verify(rddToDelete, times(1)).repartition(DELETE_PARALLELISM);
verifyDeleteExecution();
}

@Test
public void deleteWithCombineShouldRepartitionForNonEmptyRddAndNonGlobalIndex() {
rddToDelete = newHoodieKeysRddMock(2, CombineTestMode.NoneGlobalIndex);
config = newWriteConfig(WITH_COMBINE);

DeleteHelper.execute("test-time", rddToDelete, jsc, config, table, executor);

verify(rddToDelete, times(1)).distinct(DELETE_PARALLELISM);
verifyDeleteExecution();
}

@Test
public void deleteWithCombineShouldRepartitionForNonEmptyRddAndGlobalIndex() {
rddToDelete = newHoodieKeysRddMock(2, CombineTestMode.GlobalIndex);
config = newWriteConfig(WITH_COMBINE);
when(index.isGlobal()).thenReturn(true);

DeleteHelper.execute("test-time", rddToDelete, jsc, config, table, executor);

verify(keyPairs, times(1)).reduceByKey(any(), eq(DELETE_PARALLELISM));
verifyDeleteExecution();
}

private void verifyDeleteExecution() {
verify(executor, times(1)).execute(any());
verify(metadata, times(1)).setIndexLookupDuration(any());
}

private void verifyNoDeleteExecution() {
verify(executor, never()).execute(any());
}

private HoodieWriteConfig newWriteConfig(boolean combine) {
return HoodieWriteConfig.newBuilder()
.combineDeleteInput(combine)
.withPath(BASE_PATH)
.withDeleteParallelism(DELETE_PARALLELISM)
.build();
}

private JavaRDD<HoodieKey> newHoodieKeysRddMock(int howMany, CombineTestMode combineMode) {
JavaRDD<HoodieKey> keysToDelete = mock(JavaRDD.class);

JavaRDD recordsRdd = mock(JavaRDD.class);
when(recordsRdd.filter(any())).thenReturn(recordsRdd);
when(recordsRdd.isEmpty()).thenReturn(howMany <= 0);
when(index.tagLocation(any(), any(), any())).thenReturn(recordsRdd);

if (combineMode == CombineTestMode.GlobalIndex) {
when(keyPairs.reduceByKey(any(), anyInt())).thenReturn(keyPairs);
when(keyPairs.values()).thenReturn(keysToDelete);
when(keysToDelete.keyBy(any())).thenReturn(keyPairs);
} else if (combineMode == CombineTestMode.NoneGlobalIndex) {
when(keysToDelete.distinct(anyInt())).thenReturn(keysToDelete);
} else if (combineMode == CombineTestMode.None) {
List<Partition> parts = mock(List.class);
when(parts.isEmpty()).thenReturn(howMany <= 0);
when(keysToDelete.repartition(anyInt())).thenReturn(keysToDelete);
when(keysToDelete.partitions()).thenReturn(parts);
}

when(keysToDelete.map(any())).thenReturn(recordsRdd);
when(executor.execute(any())).thenReturn(metadata);
return keysToDelete;
}

private JavaRDD<HoodieKey> mockEmptyHoodieKeyRdd() {
JavaRDD<HoodieKey> emptyRdd = mock(JavaRDD.class);
doReturn(true).when(emptyRdd).isEmpty();
doReturn(Collections.emptyList()).when(emptyRdd).partitions();
doReturn(emptyRdd).when(emptyRdd).map(any());

JavaPairRDD<Tuple2, Long> emptyPairRdd = mock(JavaPairRDD.class);
doReturn(Collections.emptyMap()).when(emptyPairRdd).countByKey();
doReturn(emptyPairRdd).when(emptyRdd).mapToPair(any());

doReturn(emptyRdd).when(index).tagLocation(any(), any(), any());
doReturn(emptyRdd).when(emptyRdd).filter(any());

doNothing().when(executor).saveWorkloadProfileMetadataToInflight(any(), anyString());
doReturn(emptyRdd).when(jsc).emptyRDD();
return emptyRdd;
}

}
@@ -125,7 +125,7 @@ public HoodieWriteConfig.Builder getConfigBuilder(String schemaStr) {
*/
public HoodieWriteConfig.Builder getConfigBuilder(String schemaStr, IndexType indexType) {
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schemaStr)
.withParallelism(2, 2).withBulkInsertParallelism(2).withFinalizeWriteParallelism(2)
.withParallelism(2, 2).withBulkInsertParallelism(2).withFinalizeWriteParallelism(2).withDeleteParallelism(2)
.withTimelineLayoutVersion(TimelineLayoutVersion.CURR_VERSION)
.withWriteStatusClass(MetadataMergeWriteStatus.class)
.withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build())
@@ -165,6 +165,7 @@ public static InternalRow getInternalRowWithError(String partitionPath) {
public static HoodieWriteConfig.Builder getConfigBuilder(String basePath) {
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())
.forTable("test-trip-table")
@@ -88,7 +88,8 @@ public static void main(String[] args) throws Exception {

// Create the write client to write some records in
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath)
.withSchema(HoodieExampleDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2).forTable(tableName)
.withSchema(HoodieExampleDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.withDeleteParallelism(2).forTable(tableName)
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
.withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(20, 30).build()).build();
HoodieWriteClient<HoodieAvroPayload> client = new HoodieWriteClient<>(jsc, cfg);
@@ -113,6 +113,7 @@ private HoodieWriteConfig.Builder makeHoodieClientConfigBuilder() throws Excepti
// Prepare the AvroParquetIO
return HoodieWriteConfig.newBuilder().withPath(dfsBasePath)
.withParallelism(2, 2)
.withDeleteParallelism(2)
.withSchema(HoodieTestDataGenerator
.TRIP_EXAMPLE_SCHEMA);
}

0 comments on commit 8d19ebf

Please sign in to comment.