Skip to content

Commit

Permalink
[flink protobuf] add IT test
Browse files Browse the repository at this point in the history
  • Loading branch information
maosuhan committed Apr 28, 2021
1 parent e3f953d commit 85d79ad
Show file tree
Hide file tree
Showing 4 changed files with 56 additions and 9 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -59,10 +59,11 @@ public void testSource() {
}

@Test
public void testSink() throws Exception {
public void testSourceWithDefaultValue() {
MapTest mapTest = MapTest.newBuilder().build();

TestProtobufSourceFunction.messages.clear();
TestProtobufSourceFunction.messages.add(getProtoTestObject());
TestProtobufSinkFunction.results.clear();
TestProtobufSourceFunction.messages.add(mapTest);

env().setParallelism(1);
String sql =
Expand All @@ -73,10 +74,23 @@ public void testSink() throws Exception {
+ ") with ("
+ " 'connector' = 'protobuf-test-connector', "
+ " 'format' = 'protobuf', "
+ " 'protobuf.message-class-name' = 'org.apache.flink.formats.protobuf.testproto.MapTest'"
+ " 'protobuf.message-class-name' = 'org.apache.flink.formats.protobuf.testproto.MapTest', "
+ " 'protobuf.read-default-values' = 'true' "
+ ")";
tEnv().executeSql(sql);
sql =
TableResult result = tEnv().executeSql("select * from bigdata_source");
Row row = result.collect().next();
assertEquals(0, (int) row.getField(0));
}

@Test
public void testSink() throws Exception {
TestProtobufSourceFunction.messages.clear();
TestProtobufSourceFunction.messages.add(getProtoTestObject());
TestProtobufSinkFunction.results.clear();

env().setParallelism(1);
String sql =
"create table bigdata_sink ( "
+ " a int, "
+ " map1 map<string,string>,"
Expand All @@ -88,7 +102,8 @@ public void testSink() throws Exception {
+ ")";
tEnv().executeSql(sql);
TableResult tableResult =
tEnv().executeSql("insert into bigdata_sink select * from bigdata_source");
tEnv().executeSql(
"insert into bigdata_sink select 1, map['a', 'b', 'c', 'd'], map['f', row(1,cast(2 as bigint))] ");
tableResult.await();

byte[] bytes = TestProtobufSinkFunction.results.get(0);
Expand All @@ -100,4 +115,36 @@ public void testSink() throws Exception {
assertEquals(1, innerMessageTest.getA());
assertEquals(2L, innerMessageTest.getB());
}

@Test
public void testSinkWithNullLiteral() throws Exception {
TestProtobufSourceFunction.messages.clear();
TestProtobufSourceFunction.messages.add(getProtoTestObject());
TestProtobufSinkFunction.results.clear();

env().setParallelism(1);
String sql =
"create table bigdata_sink ( "
+ " a int, "
+ " map1 map<string,string>,"
+ " map2 map<string, row<a int, b bigint>>"
+ ") with ("
+ " 'connector' = 'protobuf-test-connector', "
+ " 'format' = 'protobuf', "
+ " 'protobuf.message-class-name' = 'org.apache.flink.formats.protobuf.testproto.MapTest', "
+ " 'protobuf.write-null-string-literal' = 'NULL' "
+ ")";
tEnv().executeSql(sql);
TableResult tableResult =
tEnv().executeSql(
"insert into bigdata_sink select 1, map['a', null], map['b', cast(null as row<a int, b bigint>)]");
tableResult.await();

byte[] bytes = TestProtobufSinkFunction.results.get(0);
MapTest mapTest = MapTest.parseFrom(bytes);
assertEquals(1, mapTest.getA());
assertEquals("NULL", mapTest.getMap1Map().get("a"));
MapTest.InnerMessageTest innerMessageTest = mapTest.getMap2Map().get("b");
assertEquals(MapTest.InnerMessageTest.getDefaultInstance(), innerMessageTest);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
import java.util.HashSet;
import java.util.Set;

/** */
/** Test protobuf table factory. Only used in flink ptotobuf test module. */
public class TestProtobufTableFactory
implements DynamicTableSourceFactory, DynamicTableSinkFactory {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,7 @@
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.types.DataType;

/** */
/** Table sink for protobuf table factory test. */
public class TestProtobufTableSink implements DynamicTableSink {
private final EncodingFormat<SerializationSchema<RowData>> encodingFormat;
private final DataType dataType;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,7 @@
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.types.DataType;

/** */
/** Table source for protobuf table factory test. */
public class TestProtobufTableSource implements ScanTableSource {
private final DecodingFormat<DeserializationSchema<RowData>> decodingFormat;
private final DataType producedDataType;
Expand Down

0 comments on commit 85d79ad

Please sign in to comment.