[HUDI-2439] Replace RDD with HoodieData in HoodieSparkTable and commit executors (#4856)
- Adopt HoodieData in Spark action commit executors - Make Spark independent DeleteHelper, WriteHelper, MergeHelper in hudi-client-common - Make HoodieTable in WriteClient APIs have raw type to decouple with Client's generic types
This commit is contained in:
@@ -32,6 +32,7 @@ import org.apache.hudi.client.validator.SparkPreCommitValidator;
|
||||
import org.apache.hudi.client.validator.SqlQueryEqualityPreCommitValidator;
|
||||
import org.apache.hudi.client.validator.SqlQuerySingleResultPreCommitValidator;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.data.HoodieData;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.fs.ConsistencyGuardConfig;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
@@ -76,6 +77,7 @@ import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodiePreCommitValidatorConfig;
|
||||
import org.apache.hudi.config.HoodieStorageConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.data.HoodieJavaRDD;
|
||||
import org.apache.hudi.exception.HoodieCommitException;
|
||||
import org.apache.hudi.exception.HoodieCorruptedDataException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
@@ -83,17 +85,19 @@ import org.apache.hudi.exception.HoodieInsertException;
|
||||
import org.apache.hudi.exception.HoodieRollbackException;
|
||||
import org.apache.hudi.exception.HoodieUpsertException;
|
||||
import org.apache.hudi.exception.HoodieValidationException;
|
||||
import org.apache.hudi.execution.bulkinsert.RDDCustomColumnsSortPartitioner;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.index.HoodieIndex.IndexType;
|
||||
import org.apache.hudi.io.HoodieMergeHandle;
|
||||
import org.apache.hudi.keygen.BaseKeyGenerator;
|
||||
import org.apache.hudi.keygen.KeyGenerator;
|
||||
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
|
||||
import org.apache.hudi.table.BulkInsertPartitioner;
|
||||
import org.apache.hudi.table.HoodieSparkCopyOnWriteTable;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.hudi.table.action.commit.SparkWriteHelper;
|
||||
import org.apache.hudi.table.action.commit.HoodieWriteHelper;
|
||||
import org.apache.hudi.table.marker.WriteMarkersFactory;
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase;
|
||||
import org.apache.hudi.testutils.HoodieClientTestUtils;
|
||||
@@ -451,13 +455,13 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
HoodieRecord recordThree =
|
||||
new HoodieAvroRecord(keyTwo, dataGen.generateRandomValue(keyTwo, newCommitTime));
|
||||
|
||||
JavaRDD<HoodieRecord<RawTripTestPayload>> records =
|
||||
jsc.parallelize(Arrays.asList(recordOne, recordTwo, recordThree), 1);
|
||||
HoodieData<HoodieRecord<RawTripTestPayload>> records = HoodieJavaRDD.of(
|
||||
jsc.parallelize(Arrays.asList(recordOne, recordTwo, recordThree), 1));
|
||||
|
||||
// Global dedup should be done based on recordKey only
|
||||
HoodieIndex index = mock(HoodieIndex.class);
|
||||
when(index.isGlobal()).thenReturn(true);
|
||||
List<HoodieRecord<RawTripTestPayload>> dedupedRecs = SparkWriteHelper.newInstance().deduplicateRecords(records, index, 1).collect();
|
||||
List<HoodieRecord<RawTripTestPayload>> dedupedRecs = HoodieWriteHelper.newInstance().deduplicateRecords(records, index, 1).collectAsList();
|
||||
assertEquals(1, dedupedRecs.size());
|
||||
assertEquals(dedupedRecs.get(0).getPartitionPath(), recordThree.getPartitionPath());
|
||||
assertNodupesWithinPartition(dedupedRecs);
|
||||
@@ -465,7 +469,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
// non-Global dedup should be done based on both recordKey and partitionPath
|
||||
index = mock(HoodieIndex.class);
|
||||
when(index.isGlobal()).thenReturn(false);
|
||||
dedupedRecs = SparkWriteHelper.newInstance().deduplicateRecords(records, index, 1).collect();
|
||||
dedupedRecs = HoodieWriteHelper.newInstance().deduplicateRecords(records, index, 1).collectAsList();
|
||||
assertEquals(2, dedupedRecs.size());
|
||||
assertNodupesWithinPartition(dedupedRecs);
|
||||
|
||||
@@ -779,6 +783,20 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
firstInsertRecords + secondInsertRecords, 2, false, config.populateMetaFields());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBulkInsertWithCustomPartitioner() {
|
||||
HoodieWriteConfig config = getConfigBuilder().withRollbackUsingMarkers(true).build();
|
||||
try (SparkRDDWriteClient client = getHoodieWriteClient(config)) {
|
||||
final String commitTime1 = "001";
|
||||
client.startCommitWithTime(commitTime1);
|
||||
List<HoodieRecord> inserts1 = dataGen.generateInserts(commitTime1, 100);
|
||||
JavaRDD<HoodieRecord> insertRecordsRDD1 = jsc.parallelize(inserts1, 10);
|
||||
BulkInsertPartitioner<JavaRDD<HoodieRecord>> partitioner = new RDDCustomColumnsSortPartitioner(new String[]{"rider"}, HoodieTestDataGenerator.AVRO_SCHEMA, false);
|
||||
List<WriteStatus> statuses = client.bulkInsert(insertRecordsRDD1, commitTime1, Option.of(partitioner)).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests deletion of records.
|
||||
*/
|
||||
@@ -2594,7 +2612,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
.withProperties(populateMetaFields ? new Properties() : getPropertiesForKeyGen()).build();
|
||||
}
|
||||
|
||||
public static class FailingPreCommitValidator<T extends HoodieRecordPayload, I, K, O extends JavaRDD<WriteStatus>> extends SparkPreCommitValidator<T, I, K, O> {
|
||||
public static class FailingPreCommitValidator<T extends HoodieRecordPayload, I, K, O extends HoodieData<WriteStatus>> extends SparkPreCommitValidator<T, I, K, O> {
|
||||
|
||||
public FailingPreCommitValidator(HoodieSparkTable table, HoodieEngineContext context, HoodieWriteConfig config) {
|
||||
super(table, context, config);
|
||||
|
||||
@@ -39,6 +39,7 @@ import org.apache.hudi.common.testutils.Transformations;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieClusteringConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.data.HoodieJavaRDD;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.index.HoodieIndex.IndexType;
|
||||
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
|
||||
@@ -556,7 +557,7 @@ public class TestHoodieMergeOnReadTable extends SparkClientFunctionalTestHarness
|
||||
// initialize partitioner
|
||||
hoodieTable.getHoodieView().sync();
|
||||
BaseSparkDeltaCommitActionExecutor actionExecutor = new SparkDeleteDeltaCommitActionExecutor(context(), cfg, hoodieTable,
|
||||
newDeleteTime, deleteRDD);
|
||||
newDeleteTime, HoodieJavaRDD.of(deleteRDD));
|
||||
actionExecutor.getUpsertPartitioner(new WorkloadProfile(buildProfile(deleteRDD)));
|
||||
final List<List<WriteStatus>> deleteStatus = jsc().parallelize(Arrays.asList(1)).map(x -> {
|
||||
return actionExecutor.handleUpdate(partitionPath, fileId, fewRecordsForDelete.iterator());
|
||||
|
||||
@@ -21,6 +21,7 @@ package org.apache.hudi.table.action.commit;
|
||||
import org.apache.hudi.client.SparkRDDWriteClient;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.bloom.BloomFilter;
|
||||
import org.apache.hudi.common.data.HoodieData;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
@@ -38,6 +39,7 @@ import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieLayoutConfig;
|
||||
import org.apache.hudi.config.HoodieStorageConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.data.HoodieJavaRDD;
|
||||
import org.apache.hudi.hadoop.HoodieParquetInputFormat;
|
||||
import org.apache.hudi.hadoop.utils.HoodieHiveUtils;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
@@ -325,7 +327,7 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
|
||||
|
||||
// Insert new records
|
||||
BaseSparkCommitActionExecutor actionExecutor = new SparkInsertCommitActionExecutor(context, config, table,
|
||||
firstCommitTime, jsc.parallelize(records));
|
||||
firstCommitTime, context.parallelize(records));
|
||||
List<WriteStatus> writeStatuses = jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
return actionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), records.iterator());
|
||||
}).flatMap(Transformations::flattenAsIterator).collect();
|
||||
@@ -368,7 +370,7 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
|
||||
// Insert new records
|
||||
final List<HoodieRecord> recs2 = records;
|
||||
BaseSparkCommitActionExecutor actionExecutor = new SparkInsertPreppedCommitActionExecutor(context, config, table,
|
||||
instantTime, jsc.parallelize(recs2));
|
||||
instantTime, context.parallelize(recs2));
|
||||
List<WriteStatus> returnedStatuses = jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
return actionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), recs2.iterator());
|
||||
}).flatMap(Transformations::flattenAsIterator).collect();
|
||||
@@ -389,7 +391,7 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
|
||||
// Insert new records
|
||||
final List<HoodieRecord> recs3 = records;
|
||||
BaseSparkCommitActionExecutor newActionExecutor = new SparkUpsertPreppedCommitActionExecutor(context, config, table,
|
||||
instantTime, jsc.parallelize(recs3));
|
||||
instantTime, context.parallelize(recs3));
|
||||
returnedStatuses = jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
return newActionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), recs3.iterator());
|
||||
}).flatMap(Transformations::flattenAsIterator).collect();
|
||||
@@ -422,7 +424,7 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
|
||||
|
||||
// Insert new records
|
||||
BaseSparkCommitActionExecutor actionExecutor = new SparkUpsertCommitActionExecutor(context, config, table,
|
||||
instantTime, jsc.parallelize(records));
|
||||
instantTime, context.parallelize(records));
|
||||
jsc.parallelize(Arrays.asList(1))
|
||||
.map(i -> actionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), records.iterator()))
|
||||
.map(Transformations::flatten).collect();
|
||||
@@ -452,7 +454,7 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
|
||||
// Perform inserts of 100 records to test CreateHandle and BufferedExecutor
|
||||
final List<HoodieRecord> inserts = dataGen.generateInsertsWithHoodieAvroPayload(instantTime, 100);
|
||||
BaseSparkCommitActionExecutor actionExecutor = new SparkInsertCommitActionExecutor(context, config, table,
|
||||
instantTime, jsc.parallelize(inserts));
|
||||
instantTime, context.parallelize(inserts));
|
||||
final List<List<WriteStatus>> ws = jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
return actionExecutor.handleInsert(UUID.randomUUID().toString(), inserts.iterator());
|
||||
}).map(Transformations::flatten).collect();
|
||||
@@ -466,7 +468,7 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
|
||||
long numRecordsInPartition = updates.stream().filter(u -> u.getPartitionPath().equals(partitionPath)).count();
|
||||
table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, HoodieTableMetaClient.reload(metaClient));
|
||||
BaseSparkCommitActionExecutor newActionExecutor = new SparkUpsertCommitActionExecutor(context, config, table,
|
||||
instantTime, jsc.parallelize(updates));
|
||||
instantTime, context.parallelize(updates));
|
||||
final List<List<WriteStatus>> updateStatus = jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
return newActionExecutor.handleUpdate(partitionPath, fileId, updates.iterator());
|
||||
}).map(Transformations::flatten).collect();
|
||||
@@ -486,8 +488,8 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
|
||||
// Insert new records
|
||||
final JavaRDD<HoodieRecord> inputRecords = generateTestRecordsForBulkInsert(jsc);
|
||||
SparkBulkInsertCommitActionExecutor bulkInsertExecutor = new SparkBulkInsertCommitActionExecutor(
|
||||
context, config, table, instantTime, inputRecords, Option.empty());
|
||||
List<WriteStatus> returnedStatuses = ((JavaRDD<WriteStatus>)bulkInsertExecutor.execute().getWriteStatuses()).collect();
|
||||
context, config, table, instantTime, HoodieJavaRDD.of(inputRecords), Option.empty());
|
||||
List<WriteStatus> returnedStatuses = ((HoodieData<WriteStatus>) bulkInsertExecutor.execute().getWriteStatuses()).collectAsList();
|
||||
verifyStatusResult(returnedStatuses, generateExpectedPartitionNumRecords(inputRecords));
|
||||
}
|
||||
|
||||
|
||||
@@ -1,194 +0,0 @@
|
||||
/*
|
||||
* 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.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.data.HoodieJavaRDD;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
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 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 HoodieIndex index;
|
||||
@Mock
|
||||
private HoodieTable<EmptyHoodieRecordPayload, JavaRDD<HoodieRecord>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table;
|
||||
@Mock
|
||||
private BaseSparkCommitActionExecutor<EmptyHoodieRecordPayload> executor;
|
||||
@Mock
|
||||
private HoodieWriteMetadata metadata;
|
||||
@Mock
|
||||
private JavaPairRDD keyPairs;
|
||||
@Mock
|
||||
private JavaSparkContext jsc;
|
||||
@Mock
|
||||
private HoodieSparkEngineContext context;
|
||||
|
||||
private JavaRDD<HoodieKey> rddToDelete;
|
||||
private HoodieWriteConfig config;
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() {
|
||||
when(table.getIndex()).thenReturn(index);
|
||||
when(context.getJavaSparkContext()).thenReturn(jsc);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void deleteWithEmptyRDDShouldNotExecute() {
|
||||
rddToDelete = mockEmptyHoodieKeyRdd();
|
||||
config = newWriteConfig(WITHOUT_COMBINE);
|
||||
|
||||
SparkDeleteHelper.newInstance().execute("test-time", rddToDelete, context, config, table, executor);
|
||||
|
||||
verify(rddToDelete, never()).repartition(DELETE_PARALLELISM);
|
||||
verifyNoDeleteExecution();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void deleteWithoutCombineShouldRepartitionForNonEmptyRdd() {
|
||||
rddToDelete = newHoodieKeysRddMock(2, CombineTestMode.None);
|
||||
config = newWriteConfig(WITHOUT_COMBINE);
|
||||
|
||||
SparkDeleteHelper.newInstance().execute("test-time", rddToDelete, context, config, table, executor);
|
||||
|
||||
verify(rddToDelete, times(1)).repartition(DELETE_PARALLELISM);
|
||||
verifyDeleteExecution();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void deleteWithCombineShouldRepartitionForNonEmptyRddAndNonGlobalIndex() {
|
||||
rddToDelete = newHoodieKeysRddMock(2, CombineTestMode.NoneGlobalIndex);
|
||||
config = newWriteConfig(WITH_COMBINE);
|
||||
|
||||
SparkDeleteHelper.newInstance().execute("test-time", rddToDelete, context, 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);
|
||||
|
||||
SparkDeleteHelper.newInstance().execute("test-time", rddToDelete, context, 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(HoodieJavaRDD.of(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());
|
||||
|
||||
doReturn(HoodieJavaRDD.of(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;
|
||||
}
|
||||
|
||||
}
|
||||
@@ -21,6 +21,7 @@ package org.apache.hudi.table.action.compact;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.client.SparkRDDWriteClient;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.data.HoodieData;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
@@ -195,12 +196,12 @@ public class TestHoodieCompactor extends HoodieClientTestHarness {
|
||||
String compactionInstantTime = "102";
|
||||
table.scheduleCompaction(context, compactionInstantTime, Option.empty());
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
JavaRDD<WriteStatus> result = (JavaRDD<WriteStatus>) table.compact(
|
||||
HoodieData<WriteStatus> result = (HoodieData<WriteStatus>) table.compact(
|
||||
context, compactionInstantTime).getWriteStatuses();
|
||||
|
||||
// Verify that all partition paths are present in the WriteStatus result
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
List<WriteStatus> writeStatuses = result.collect();
|
||||
List<WriteStatus> writeStatuses = result.collectAsList();
|
||||
assertTrue(writeStatuses.stream()
|
||||
.filter(writeStatus -> writeStatus.getStat().getPartitionPath().contentEquals(partitionPath)).count() > 0);
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user