1717
1818package com .bytedance .bitsail .connector .legacy .kafka .source ;
1919
20+ import com .bytedance .bitsail .batch .file .parser .PbBytesParser ;
2021import com .bytedance .bitsail .common .configuration .BitSailConfiguration ;
22+ import com .bytedance .bitsail .parser .option .RowParserOptions ;
2123import com .bytedance .bitsail .test .connector .test .EmbeddedFlinkCluster ;
2224import com .bytedance .bitsail .test .connector .test .testcontainers .kafka .KafkaCluster ;
2325import com .bytedance .bitsail .test .connector .test .utils .JobConfUtils ;
2426
2527import com .alibaba .fastjson .JSONObject ;
2628import com .google .common .collect .Maps ;
29+ import com .google .protobuf .ByteString ;
30+ import com .google .protobuf .Descriptors ;
31+ import com .google .protobuf .DynamicMessage ;
32+ import org .apache .commons .io .IOUtils ;
2733import org .apache .kafka .clients .consumer .ConsumerConfig ;
2834import org .apache .kafka .clients .producer .KafkaProducer ;
35+ import org .apache .kafka .clients .producer .ProducerConfig ;
2936import org .apache .kafka .clients .producer .ProducerRecord ;
37+ import org .apache .kafka .common .serialization .ByteArraySerializer ;
38+ import org .apache .kafka .common .serialization .StringSerializer ;
3039import org .junit .After ;
3140import org .junit .Before ;
3241import org .junit .Test ;
3342import org .slf4j .Logger ;
3443import org .slf4j .LoggerFactory ;
44+ import org .testcontainers .shaded .com .google .common .collect .ImmutableMap ;
3545
46+ import java .io .File ;
47+ import java .net .URI ;
48+ import java .util .Base64 ;
49+ import java .util .List ;
3650import java .util .Map ;
3751import java .util .concurrent .ScheduledThreadPoolExecutor ;
3852import java .util .concurrent .TimeUnit ;
@@ -46,6 +60,7 @@ public class KafkaSourceITCase {
4660 private static final int TOTAL_SEND_COUNT = 300 ;
4761 private final String topicName = "testTopic" ;
4862 private final KafkaCluster kafkaCluster = new KafkaCluster ();
63+ private ScheduledThreadPoolExecutor produceService ;
4964
5065 private static String constructARecord (int index ) {
5166 JSONObject jsonObject = new JSONObject ();
@@ -59,34 +74,120 @@ private static String constructARecord(int index) {
5974 public void before () {
6075 kafkaCluster .startService ();
6176 kafkaCluster .createTopic (topicName );
62- startSendDataToKafka ( );
77+ produceService = new ScheduledThreadPoolExecutor ( 1 );
6378 }
6479
65- private void startSendDataToKafka () {
80+ private void startSendJsonDataToKafka () {
6681 KafkaProducer <String , String > producer = kafkaCluster .getProducer (topicName );
67- ScheduledThreadPoolExecutor produceService = new ScheduledThreadPoolExecutor (1 );
6882 AtomicInteger sendCount = new AtomicInteger (0 );
6983 produceService .scheduleAtFixedRate (() -> {
7084 try {
71- for (int i = 0 ; i < 5000 ; ++i ) {
72- String record = constructARecord (sendCount .getAndIncrement ());
73- producer .send (new ProducerRecord (topicName , record ));
74- }
85+ for (int i = 0 ; i < 5000 ; ++i ) {
86+ String record = constructARecord (sendCount .getAndIncrement ());
87+ producer .send (new ProducerRecord (topicName , record ));
88+ }
7589 } catch (Exception e ) {
76- LOG .error ("failed to send a record" );
90+ LOG .error ("failed to send a record" );
7791 } finally {
78- LOG .info (">>> kafka produce count: {}" , sendCount .get ());
92+ LOG .info (">>> kafka produce count: {}" , sendCount .get ());
93+ }
94+ }, 0 , 1 , TimeUnit .SECONDS );
95+ }
96+
97+ private void startSendPbDataToKafka (BitSailConfiguration configuration ) throws Exception {
98+ PbBytesParser parser = new PbBytesParser (configuration );
99+ List <Descriptors .FieldDescriptor > fields = parser .getDescriptor ().getFields ();
100+ Descriptors .Descriptor type = parser .getDescriptor ();
101+
102+ KafkaProducer <String , byte []> producer = new KafkaProducer <>(
103+ ImmutableMap .of (
104+ ProducerConfig .BOOTSTRAP_SERVERS_CONFIG , KafkaCluster .getBootstrapServer (),
105+ ProducerConfig .CLIENT_ID_CONFIG , "producer"
106+ ),
107+ new StringSerializer (),
108+ new ByteArraySerializer ()
109+ );
110+ AtomicInteger sendCount = new AtomicInteger (0 );
111+ produceService .scheduleAtFixedRate (() -> {
112+ try {
113+ for (int i = 0 ; i < 5000 ; ++i ) {
114+ DynamicMessage .Builder builder = DynamicMessage .newBuilder (type );
115+ for (Descriptors .FieldDescriptor field : fields ) {
116+ switch (field .getJavaType ()) {
117+ case INT :
118+ builder .setField (field , i );
119+ break ;
120+ case LONG :
121+ builder .setField (field , (long ) i );
122+ break ;
123+ case FLOAT :
124+ builder .setField (field , (float ) i );
125+ break ;
126+ case DOUBLE :
127+ builder .setField (field , (double ) i );
128+ break ;
129+ case BOOLEAN :
130+ builder .setField (field , i % 2 == 0 );
131+ break ;
132+ case BYTE_STRING :
133+ builder .setField (field , ByteString .copyFrom (("bytes_" + i ).getBytes ()));
134+ break ;
135+ case MESSAGE :
136+ case STRING :
137+ case ENUM :
138+ default :
139+ builder .setField (field , "text_" + i );
140+ break ;
141+ }
142+ }
143+ DynamicMessage message = builder .build ();
144+ producer .send (new ProducerRecord (topicName , message .toByteArray ()));
145+ sendCount .getAndIncrement ();
146+ }
147+ } catch (Exception e ) {
148+ LOG .error ("failed to send a record" );
149+ } finally {
150+ LOG .info (">>> kafka produce count: {}" , sendCount .get ());
79151 }
80152 }, 0 , 1 , TimeUnit .SECONDS );
81153 }
82154
83155 @ Test
84- public void testKafkaSource () throws Exception {
156+ public void testKafkaSourceJsonFormat () throws Exception {
157+ startSendJsonDataToKafka ();
85158 BitSailConfiguration configuration = JobConfUtils .fromClasspath ("kafka_to_print.json" );
86159 updateConfiguration (configuration );
87160 EmbeddedFlinkCluster .submitJob (configuration );
88161 }
89162
163+ @ Test
164+ public void testKafkaSourcePbFormat () throws Exception {
165+ BitSailConfiguration configuration = JobConfUtils .fromClasspath ("kafka_to_print_pb_format.json" );
166+ URI proto = KafkaSourceITCase .class .getClassLoader ().getResource ("kafka.fds" ).toURI ();
167+ byte [] descriptor = IOUtils .toByteArray (new File (proto ).toURI ());
168+
169+ String protoDescriptor = Base64 .getEncoder ().encodeToString (descriptor );
170+ configuration .set (RowParserOptions .PROTO_DESCRIPTOR , protoDescriptor );
171+ configuration .set (RowParserOptions .PROTO_CLASS_NAME , "ProtoTest" );
172+ startSendPbDataToKafka (configuration );
173+ updateConfiguration (configuration );
174+ EmbeddedFlinkCluster .submitJob (configuration );
175+ }
176+
177+ @ Test
178+ public void testKafkaSourcePbFormatFullTypes () throws Exception {
179+ BitSailConfiguration configuration = JobConfUtils .fromClasspath ("kafka_to_print_pb_format_full_types.json" );
180+ URI proto = KafkaSourceITCase .class .getClassLoader ().getResource ("kafka_full_types.fds" ).toURI ();
181+ byte [] descriptor = IOUtils .toByteArray (new File (proto ).toURI ());
182+
183+ String protoDescriptor = Base64 .getEncoder ().encodeToString (descriptor );
184+ configuration .set (RowParserOptions .PROTO_DESCRIPTOR , protoDescriptor );
185+ configuration .set (RowParserOptions .PROTO_CLASS_NAME , "ProtoTest" );
186+ startSendPbDataToKafka (configuration );
187+ updateConfiguration (configuration );
188+ EmbeddedFlinkCluster .submitJob (configuration );
189+ }
190+
90191 protected void updateConfiguration (BitSailConfiguration jobConfiguration ) {
91192 // jobConfiguration.set(FakeReaderOptions.TOTAL_COUNT, TOTAL_SEND_COUNT);
92193
@@ -98,6 +199,7 @@ protected void updateConfiguration(BitSailConfiguration jobConfiguration) {
98199
99200 @ After
100201 public void after () {
202+ produceService .shutdown ();
101203 kafkaCluster .stopService ();
102204 }
103205
0 commit comments