[HUDI-3569] Introduce ChainedJsonKafkaSourePostProcessor to support setting multi processors at once (#4969)
This commit is contained in:
@@ -30,6 +30,8 @@ import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
import static org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen.Config.JSON_KAFKA_PROCESSOR_CLASS_OPT;
|
||||
import static org.apache.hudi.utilities.testutils.UtilitiesTestBase.Helpers.jsonifyRecords;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
@@ -96,6 +98,28 @@ public class TestJsonKafkaSourcePostProcessor extends TestJsonKafkaSource {
|
||||
() -> kafkaSource.fetchNewDataInAvroFormat(Option.empty(), 900));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testChainedJsonKafkaSourcePostProcessor() {
|
||||
// topic setup.
|
||||
final String topic = TEST_TOPIC_PREFIX + "testChainedJsonKafkaSourcePostProcessor";
|
||||
testUtils.createTopic(topic, 2);
|
||||
|
||||
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
|
||||
TypedProperties props = createPropsForJsonSource(topic, null, "earliest");
|
||||
|
||||
// processor class name setup
|
||||
props.setProperty(JSON_KAFKA_PROCESSOR_CLASS_OPT.key(), SampleJsonKafkaSourcePostProcessor.class.getName()
|
||||
+ "," + DummyJsonKafkaSourcePostProcessor.class.getName());
|
||||
|
||||
Source jsonSource = new JsonKafkaSource(props, jsc(), spark(), schemaProvider, metrics);
|
||||
SourceFormatAdapter kafkaSource = new SourceFormatAdapter(jsonSource);
|
||||
|
||||
testUtils.sendMessages(topic, jsonifyRecords(dataGenerator.generateInserts("000", 1000)));
|
||||
InputBatch<JavaRDD<GenericRecord>> fetch1 = kafkaSource.fetchNewDataInAvroFormat(Option.empty(), 900);
|
||||
|
||||
assertEquals(0, fetch1.getBatch().get().count());
|
||||
}
|
||||
|
||||
/**
|
||||
* JsonKafkaSourcePostProcessor that return a sub RDD of the incoming data which get the data from incoming data using
|
||||
* {org.apache.spark.api.java.JavaRDD#sample(boolean, double, long)} method.
|
||||
@@ -112,4 +136,16 @@ public class TestJsonKafkaSourcePostProcessor extends TestJsonKafkaSource {
|
||||
}
|
||||
}
|
||||
|
||||
public static class DummyJsonKafkaSourcePostProcessor extends JsonKafkaSourcePostProcessor {
|
||||
public DummyJsonKafkaSourcePostProcessor(TypedProperties props) {
|
||||
super(props);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<String> process(JavaRDD<String> inputJsonRecords) {
|
||||
// return empty RDD
|
||||
return inputJsonRecords.map(x -> "").filter(x -> !Objects.equals(x, ""));
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user