1
0

[HUDI-238] Make Hudi support Scala 2.12 (#1226)

* [HUDI-238] Rename scala related artifactId & add maven profile to support Scala 2.12
This commit is contained in:
wenningd
2020-01-17 14:02:21 -08:00
committed by Balaji Varadarajan
parent 923e2b4a1e
commit 292c1e2ff4
19 changed files with 261 additions and 153 deletions

View File

@@ -28,14 +28,14 @@ import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
import org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen.CheckpointUtils;
import org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen.Config;
import kafka.common.TopicAndPartition;
import org.apache.avro.generic.GenericRecord;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.common.TopicPartition;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset;
import org.apache.spark.streaming.kafka.KafkaTestUtils;
import org.apache.spark.streaming.kafka.OffsetRange;
import org.apache.spark.streaming.kafka010.KafkaTestUtils;
import org.apache.spark.streaming.kafka010.OffsetRange;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
@@ -44,6 +44,7 @@ import org.junit.Test;
import java.io.IOException;
import java.util.HashMap;
import java.util.UUID;
import static org.junit.Assert.assertEquals;
@@ -84,13 +85,12 @@ public class TestKafkaSource extends UtilitiesTestBase {
private TypedProperties createPropsForJsonSource(Long maxEventsToReadFromKafkaSource) {
TypedProperties props = new TypedProperties();
props.setProperty("hoodie.deltastreamer.source.kafka.topic", TEST_TOPIC_NAME);
props.setProperty("metadata.broker.list", testUtils.brokerAddress());
props.setProperty("auto.offset.reset", "smallest");
props.setProperty("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
props.setProperty("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
props.setProperty("bootstrap.servers", testUtils.brokerAddress());
props.setProperty("auto.offset.reset", "earliest");
props.setProperty("hoodie.deltastreamer.kafka.source.maxEvents",
maxEventsToReadFromKafkaSource != null ? String.valueOf(maxEventsToReadFromKafkaSource) :
String.valueOf(Config.maxEventsFromKafkaSource));
props.setProperty(ConsumerConfig.GROUP_ID_CONFIG, UUID.randomUUID().toString());
return props;
}
@@ -214,10 +214,10 @@ public class TestKafkaSource extends UtilitiesTestBase {
assertEquals(Option.empty(), fetch6.getBatch());
}
private static HashMap<TopicAndPartition, LeaderOffset> makeOffsetMap(int[] partitions, long[] offsets) {
HashMap<TopicAndPartition, LeaderOffset> map = new HashMap<>();
private static HashMap<TopicPartition, Long> makeOffsetMap(int[] partitions, long[] offsets) {
HashMap<TopicPartition, Long> map = new HashMap<>();
for (int i = 0; i < partitions.length; i++) {
map.put(new TopicAndPartition(TEST_TOPIC_NAME, partitions[i]), new LeaderOffset("", -1, offsets[i]));
map.put(new TopicPartition(TEST_TOPIC_NAME, partitions[i]), offsets[i]);
}
return map;
}

View File

@@ -25,6 +25,6 @@ hoodie.deltastreamer.schemaprovider.registry.url=http://localhost:8081/subjects/
#hoodie.deltastreamer.source.kafka.topic=uber_trips
hoodie.deltastreamer.source.kafka.topic=impressions
#Kafka props
metadata.broker.list=localhost:9092
auto.offset.reset=smallest
bootstrap.servers=localhost:9092
auto.offset.reset=earliest
schema.registry.url=http://localhost:8081