1
0

[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"
This commit is contained in:
Dongwook
2020-09-01 09:55:31 -07:00
committed by GitHub
parent 48a58c98a1
commit 8d19ebfd0f
21 changed files with 234 additions and 15 deletions

View File

@@ -437,7 +437,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
*/
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())

View File

@@ -462,7 +462,7 @@ public class TestHBaseIndex extends FunctionalTestHarness {
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()

View File

@@ -174,7 +174,7 @@ public class TestHoodieKeyLocationFetchHandle extends HoodieClientTestHarness {
*/
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())

View File

@@ -309,6 +309,7 @@ public class TestHoodieMergeHandle extends HoodieClientTestHarness {
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")

View File

@@ -207,7 +207,7 @@ public class TestCleaner extends HoodieClientTestBase {
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 @@ public class TestCleaner extends HoodieClientTestBase {
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);

View File

@@ -803,6 +803,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)
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024)
.withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1).build())
@@ -1466,6 +1467,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
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())

View File

@@ -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;
}
}

View File

@@ -125,7 +125,7 @@ public class HoodieClientTestBase extends HoodieClientTestHarness {
*/
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())

View File

@@ -165,6 +165,7 @@ public class SparkDatasetTestUtils {
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")