[HUDI-2439] Remove SparkBoundedInMemoryExecutor (#4860)
This commit is contained in:
@@ -22,12 +22,15 @@ import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
||||
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.spark.TaskContext;
|
||||
import org.apache.spark.TaskContext$;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
@@ -44,7 +47,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
public class TestSparkBoundedInMemoryExecutor extends HoodieClientTestHarness {
|
||||
public class TestBoundedInMemoryExecutorInSpark extends HoodieClientTestHarness {
|
||||
|
||||
private final String instantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
|
||||
@@ -58,6 +61,11 @@ public class TestSparkBoundedInMemoryExecutor extends HoodieClientTestHarness {
|
||||
cleanupResources();
|
||||
}
|
||||
|
||||
private Runnable getPreExecuteRunnable() {
|
||||
final TaskContext taskContext = TaskContext.get();
|
||||
return () -> TaskContext$.MODULE$.setTaskContext(taskContext);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testExecutor() {
|
||||
|
||||
@@ -85,10 +93,10 @@ public class TestSparkBoundedInMemoryExecutor extends HoodieClientTestHarness {
|
||||
}
|
||||
};
|
||||
|
||||
SparkBoundedInMemoryExecutor<HoodieRecord, Tuple2<HoodieRecord, Option<IndexedRecord>>, Integer> executor = null;
|
||||
BoundedInMemoryExecutor<HoodieRecord, Tuple2<HoodieRecord, Option<IndexedRecord>>, Integer> executor = null;
|
||||
try {
|
||||
executor = new SparkBoundedInMemoryExecutor(hoodieWriteConfig, hoodieRecords.iterator(), consumer,
|
||||
getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA));
|
||||
executor = new BoundedInMemoryExecutor(hoodieWriteConfig.getWriteBufferLimitBytes(), hoodieRecords.iterator(), consumer,
|
||||
getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), getPreExecuteRunnable());
|
||||
int result = executor.execute();
|
||||
// It should buffer and write 100 records
|
||||
assertEquals(100, result);
|
||||
@@ -131,11 +139,11 @@ public class TestSparkBoundedInMemoryExecutor extends HoodieClientTestHarness {
|
||||
}
|
||||
};
|
||||
|
||||
SparkBoundedInMemoryExecutor<HoodieRecord, Tuple2<HoodieRecord, Option<IndexedRecord>>, Integer> executor = null;
|
||||
BoundedInMemoryExecutor<HoodieRecord, Tuple2<HoodieRecord, Option<IndexedRecord>>, Integer> executor = null;
|
||||
try {
|
||||
executor = new SparkBoundedInMemoryExecutor(hoodieWriteConfig, hoodieRecords.iterator(), consumer,
|
||||
getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA));
|
||||
SparkBoundedInMemoryExecutor<HoodieRecord, Tuple2<HoodieRecord, Option<IndexedRecord>>, Integer> finalExecutor = executor;
|
||||
executor = new BoundedInMemoryExecutor(hoodieWriteConfig.getWriteBufferLimitBytes(), hoodieRecords.iterator(), consumer,
|
||||
getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), getPreExecuteRunnable());
|
||||
BoundedInMemoryExecutor<HoodieRecord, Tuple2<HoodieRecord, Option<IndexedRecord>>, Integer> finalExecutor = executor;
|
||||
|
||||
Thread.currentThread().interrupt();
|
||||
|
||||
Reference in New Issue
Block a user