Skip to content

Commit

Permalink
[HUDI-4924] Auto-tune dedup parallelism (apache#6802)
Browse files Browse the repository at this point in the history
  • Loading branch information
yihua authored and voonhous committed Oct 7, 2022
1 parent 6de8b09 commit c9edd94
Show file tree
Hide file tree
Showing 7 changed files with 80 additions and 6 deletions.
Expand Up @@ -54,6 +54,9 @@ protected HoodieData<HoodieRecord<T>> tag(HoodieData<HoodieRecord<T>> dedupedRec
public HoodieData<HoodieRecord<T>> deduplicateRecords(
HoodieData<HoodieRecord<T>> records, HoodieIndex<?, ?> index, int parallelism) {
boolean isIndexingGlobal = index.isGlobal();
// Auto-tunes the parallelism for reduce transformation based on the number of data partitions
// in engine-specific representation
int reduceParallelism = Math.max(1, Math.min(records.getNumPartitions(), parallelism));
return records.mapToPair(record -> {
HoodieKey hoodieKey = record.getKey();
// If index used is global, then records are expected to differ in their partitionPath
Expand All @@ -65,7 +68,7 @@ public HoodieData<HoodieRecord<T>> deduplicateRecords(
HoodieKey reducedKey = rec1.getData().equals(reducedData) ? rec1.getKey() : rec2.getKey();

return new HoodieAvroRecord<>(reducedKey, reducedData);
}, parallelism).map(Pair::getRight);
}, reduceParallelism).map(Pair::getRight);
}

}
Expand Up @@ -102,6 +102,11 @@ public long count() {
return rddData.count();
}

@Override
public int getNumPartitions() {
return rddData.getNumPartitions();
}

@Override
public <O> HoodieData<O> map(SerializableFunction<T, O> func) {
return HoodieJavaRDD.of(rddData.map(func::apply));
Expand Down
Expand Up @@ -460,25 +460,33 @@ private void testDeduplication(

HoodieData<HoodieRecord<RawTripTestPayload>> records = HoodieJavaRDD.of(
jsc.parallelize(Arrays.asList(recordOne, recordTwo, recordThree), 1));
HoodieWriteConfig.Builder configBuilder = getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY)
.combineInput(true, true);
addConfigsForPopulateMetaFields(configBuilder, populateMetaFields);

// Global dedup should be done based on recordKey only
HoodieIndex index = mock(HoodieIndex.class);
when(index.isGlobal()).thenReturn(true);
List<HoodieRecord<RawTripTestPayload>> dedupedRecs = HoodieWriteHelper.newInstance().deduplicateRecords(records, index, 1).collectAsList();
int dedupParallelism = records.getNumPartitions() + 100;
HoodieData<HoodieRecord<RawTripTestPayload>> dedupedRecsRdd = HoodieWriteHelper.newInstance().deduplicateRecords(records, index, dedupParallelism);
List<HoodieRecord<RawTripTestPayload>> dedupedRecs = dedupedRecsRdd.collectAsList();
assertEquals(records.getNumPartitions(), dedupedRecsRdd.getNumPartitions());
assertEquals(1, dedupedRecs.size());
assertEquals(dedupedRecs.get(0).getPartitionPath(), recordThree.getPartitionPath());
assertNodupesWithinPartition(dedupedRecs);

// non-Global dedup should be done based on both recordKey and partitionPath
index = mock(HoodieIndex.class);
when(index.isGlobal()).thenReturn(false);
dedupedRecs = HoodieWriteHelper.newInstance().deduplicateRecords(records, index, 1).collectAsList();
dedupedRecsRdd = HoodieWriteHelper.newInstance().deduplicateRecords(records, index, dedupParallelism);
dedupedRecs = dedupedRecsRdd.collectAsList();
assertEquals(records.getNumPartitions(), dedupedRecsRdd.getNumPartitions());
assertEquals(2, dedupedRecs.size());
assertNodupesWithinPartition(dedupedRecs);

// Perform write-action and check
JavaRDD<HoodieRecord> recordList = jsc.parallelize(Arrays.asList(recordOne, recordTwo, recordThree), 1);
HoodieWriteConfig.Builder configBuilder = getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY)
configBuilder = getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY)
.combineInput(true, true);
addConfigsForPopulateMetaFields(configBuilder, populateMetaFields);

Expand Down
@@ -0,0 +1,40 @@
/*
* 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.data;

import org.apache.hudi.common.data.HoodieData;
import org.apache.hudi.testutils.HoodieClientTestBase;

import org.junit.jupiter.api.Test;

import java.util.stream.Collectors;
import java.util.stream.IntStream;

import static org.junit.jupiter.api.Assertions.assertEquals;

public class TestHoodieJavaRDD extends HoodieClientTestBase {
@Test
public void testGetNumPartitions() {
int numPartitions = 6;
HoodieData<Integer> rddData = HoodieJavaRDD.of(jsc.parallelize(
IntStream.rangeClosed(0, 100).boxed().collect(Collectors.toList()), numPartitions));
assertEquals(numPartitions, rddData.getNumPartitions());
}
}
Expand Up @@ -67,14 +67,19 @@ public interface HoodieData<T> extends Serializable {

/**
* Returns number of objects held in the collection
*
* <p>
* NOTE: This is a terminal operation
*/
long count();

/**
* @return the number of data partitions in the engine-specific representation.
*/
int getNumPartitions();

/**
* Maps every element in the collection using provided mapping {@code func}.
*
* <p>
* This is an intermediate operation
*
* @param func serializable map function
Expand Down
Expand Up @@ -175,6 +175,11 @@ public long count() {
return super.count();
}

@Override
public int getNumPartitions() {
return 1;
}

@Override
public List<T> collectAsList() {
return super.collectAsList();
Expand Down
Expand Up @@ -29,6 +29,7 @@
import java.util.Arrays;
import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import java.util.stream.Stream;

import static org.junit.jupiter.api.Assertions.assertEquals;
Expand Down Expand Up @@ -64,4 +65,11 @@ void testEagerSemantic() {
assertEquals(3, originalListData.count());
assertEquals(sourceList, originalListData.collectAsList());
}

@Test
public void testGetNumPartitions() {
HoodieData<Integer> listData = HoodieListData.eager(
IntStream.rangeClosed(0, 100).boxed().collect(Collectors.toList()));
assertEquals(1, listData.getNumPartitions());
}
}

0 comments on commit c9edd94

Please sign in to comment.