Skip to content

Commit

Permalink
add comment
Browse files Browse the repository at this point in the history
  • Loading branch information
maosuhan committed Jan 7, 2021
1 parent 5a9464d commit 80e42c1
Show file tree
Hide file tree
Showing 5 changed files with 10 additions and 0 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,8 @@ public PbCodegenArrayDeserializer(
public String codegen(
String returnVarName,
String messageGetStr) throws PbCodegenException {
// The type of messageGetStr is a native List object,
// it should be converted to ArrayData of flink internal type
PbCodegenVarId varUid = PbCodegenVarId.getInstance();
int uid = varUid.getAndIncrement();
String protoTypeStr = PbCodegenUtils.getTypeStrFromProto(fd, false);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,8 @@ public static PbCodegenDeserializer getPbCodegenDes(
Descriptors.FieldDescriptor fd,
LogicalType type,
boolean readDefaultValues) throws PbCodegenException {
// We do not use FieldDescriptor to check because when FieldDescriptor is an element type in array,
// FieldDescriptor.isRepeated() is still true
if (type instanceof RowType) {
return new PbCodegenRowDeserializer(fd.getMessageType(), (RowType) type,
readDefaultValues);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,8 @@ public PbCodegenMapDeserializer(
public String codegen(
String returnVarName,
String messageGetStr) throws PbCodegenException {
// The type of messageGetStr is a native Map object,
// it should be converted to MapData of flink internal type
PbCodegenVarId varUid = PbCodegenVarId.getInstance();
int uid = varUid.getAndIncrement();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,8 @@ public PbCodegenRowDeserializer(
public String codegen(
String returnVarName,
String messageGetStr) throws PbCodegenException {
// The type of messageGetStr is a native pb object,
// it should be converted to RowData of flink internal type
PbCodegenVarId varUid = PbCodegenVarId.getInstance();
int uid = varUid.getAndIncrement();
String pbMessageVar = "message" + uid;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,8 @@ public PbCodegenSimpleDeserializer(

@Override
public String codegen(String returnVarName, String messageGetStr) {
// the type of messageGetStr must not be primitive type,
// it should convert to internal flink row type like StringData.
StringBuilder sb = new StringBuilder();
switch (fd.getJavaType()) {
case INT:
Expand Down

0 comments on commit 80e42c1

Please sign in to comment.