-
Notifications
You must be signed in to change notification settings - Fork 13.4k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[FLINK-18202][PB format] New Format of protobuf #14376
Conversation
Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community Automated ChecksLast check on commit 85d79ad (Fri May 28 11:05:33 UTC 2021) Warnings:
Mention the bot in a comment to re-run the automated checks. Review Progress
Please see the Pull Request Review Guide for a full explanation of the review process. The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required Bot commandsThe @flinkbot bot supports the following commands:
|
...ormats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbSchemaValidator.java
Outdated
Show resolved
Hide resolved
...ormats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbSchemaValidator.java
Outdated
Show resolved
Hide resolved
...src/main/java/org/apache/flink/formats/protobuf/deserialize/PbCodegenSimpleDeserializer.java
Outdated
Show resolved
Hide resolved
Hi @maosuhan , the community changed the code format recently. Could you rebase your branch to the lastest master and update the format according to this doc? https://ci.apache.org/projects/flink/flink-docs-master/flinkDev/ide_setup.html#code-formatting cc @libenchao , do you have time to review this PR? |
a9a50e7
to
80e42c1
Compare
@wuchong , I have updated the code formatting in my PR. |
Any progress? |
@wuchong Sorry for the late reply, being a little recently. |
We are supporting protobuf in the pulsar connector and expect this pr to be merged. |
Hi, @maosuhan thanks a lot for your work. I have used your code in flink 1.11.2. There seems to be some problems for me:
For problem 1, I set the scope of the protobuf-java in flink-protobuf module to <dependency>
<groupId>com.google.protobuf</groupId>
<artifactId>protobuf-java</artifactId>
<version>${protoc.version}</version>
<scope>provided</scope>
</dependency> For problem 2, it seems that the @Override
public RowData deserialize(byte[] message) throws IOException {
try {
if(protoToRowConverter == null) {
LOG.info(String.format("[protobuf new]: %s %s %s", messageClassName, rowType.toString(), "" + readDefaultValues));
protoToRowConverter = new ProtoToRowConverter(messageClassName, rowType, readDefaultValues);
}
return protoToRowConverter.convertProtoBinaryToRow(message);
} catch (Throwable t) {
if (ignoreParseErrors) {
return null;
}
LOG.error("Failed to deserialize PB object.", t);
throw new IOException("Failed to deserialize PB object.", t);
}
} Will the same problems occur in the new flink version ? Thanks for reading. |
@ruanhang1993 Many thanks for finding issues of this PR.
|
@maosuhan I am honored to receive your reply. I use the fixed version for problem 1 in flink 1.12.1 recently. With the fixed code for problem 1 in flink 1.12.1, I got the error like this:
I generate the protobuf message class and package like this, and use it when submitting the flink job. protoc -I=$PROTO_FILE_PATH --java_out=$JAVA_OUTPUT/src $PROTO_FILE
javac -cp $JAVA_OUTPUT/libs/protobuf-java.jar -d $JAVA_OUTPUT/target $JAVA_OUTPUT/src/$PACKAGE_PATH/*.java
cd $JAVA_OUTPUT
jar -cvf $JAR_NAME -C $JAVA_OUTPUT/target/ . The problem is that the public static Descriptors.Descriptor getDescriptor(String className) {
try {
Class<?> pbClass = Class.forName(className);
return (Descriptors.Descriptor)
pbClass.getMethod(PbConstant.PB_METHOD_GET_DESCRIPTOR).invoke(null);
} catch (Exception y) {
throw new IllegalArgumentException(
String.format("get %s descriptors error!", className), y);
}
} Do I need to relocate the class like you when generating the protobuf message class? ps: The setting in |
@ruanhang1993
I can successfully package flink project. Could you provide the package command when you see conflict? |
There is no problem when packaging flink project. The problem occurs when submitting job by flink cli. Caused by: java.lang.ClassCastException: com.google.protobuf.Descriptors$Descriptor cannot be cast to com.google.protobuf.Descriptors$Descriptor
at org.apache.flink.formats.protobuf.PbFormatUtils.getDescriptor(PbFormatUtils.java:81) ~[?:?]
...... The flink directory flink run -m yarn-cluster \
-yd ...... -yt protobufMessage.jar \
-c package.Main myJob.jar jobParams \ After a few tests, I think the problem is about the class loading in flink, not conflicting with other modules as I thought. I need to place the It seems that I use the jar in a wrong way. Thanks a lot for your answer. |
@maosuhan I just pulled your code into my local repo and there are many check-style violations, could you resolve these problems first and make the CI passed? |
0768853
to
d7b612a
Compare
@libenchao I have fixed check-style errors. They're all related to java doc issue. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@maosuhan Thanks for your great work, the code is already in a very good shape.
I left some comments for some details.
And there are some concerns I want to raise:
- Regarding to serialization, we actually don't allow
null
for almost all types in pb. however in Flink, any type could produce null. Your current implementation is setting default value for these cases. - Regarding to code generating, I see you developed a mini framework to to it, and it works well now, except for the readability, both for the generating code and generated code.
For 1, failing the job are fulling default values for null values are both choices, I don't have a preference for now, just point it out to have more discussions.
For 2, have you ever tried to find a mature codegen framework? If there is no other better choice, I'm ok with current one too.
CC @wuchong
...flink-protobuf/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
Show resolved
Hide resolved
...formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbDecodingFormat.java
Outdated
Show resolved
Hide resolved
.../main/java/org/apache/flink/formats/protobuf/deserialize/PbRowDataDeserializationSchema.java
Outdated
Show resolved
Hide resolved
.../main/java/org/apache/flink/formats/protobuf/deserialize/PbRowDataDeserializationSchema.java
Outdated
Show resolved
Hide resolved
.../main/java/org/apache/flink/formats/protobuf/deserialize/PbRowDataDeserializationSchema.java
Outdated
Show resolved
Hide resolved
...-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/RowToProtoConverter.java
Outdated
Show resolved
Hide resolved
...buf/src/main/java/org/apache/flink/formats/protobuf/serialize/PbCodegenSimpleSerializer.java
Outdated
Show resolved
Hide resolved
...otobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/PbCodegenRowSerializer.java
Outdated
Show resolved
Hide resolved
...otobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/PbCodegenRowSerializer.java
Outdated
Show resolved
Hide resolved
...ats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbRowTypeInformation.java
Outdated
Show resolved
Hide resolved
@libenchao Hi Benchao, very thanks for your review effort and it is a great appreciate that you give many detailed suggestions.
|
@wuchong What's your opinion for these issues? |
Regarding to the topics @libenchao raised,
|
@wuchong Thanks for your opinion.
Does it sound good? @wuchong @libenchao |
I think we can simplify the desgin to just allow users change default value for types instead of columns. HBase connector also has a similar option |
Thanks a lot for working on a Protobuf format! We also use that serialization mechanism at my company and will need such format soon, this is very useful. I have a different understanding regarding the handling of missing values by protobuf 3 and I believe we should be able to provide Fllink with nulls in case of missing value in pb. @maosuhan, if you want I'd be happy to collaborate with you on this and add support for this case. As I understand it:
The way we approach this where I work is:
Here's a quick example, generating java classes with pb Given this schema:
and this Parent roundTrip(Parent parent) throws InvalidProtocolBufferException {
return Parent.parser().parseFrom(parent.toByteArray());
} Those assertions show that the missing var withoutName = Parent.newBuilder()
.setId("theId")
.setInnerValue(
Parent.Inner.newBuilder()
.setF1("theF1")
.setF2("theF1")
.build()
)
.build();
assert ! withoutName.hasName();
assert ! roundTrip(withoutName).hasName();
assert withoutName.hasInnerValue();
assert roundTrip(withoutName).hasInnerValue();
assert withoutName.getInnerValue().getF1().equals("theF1");
assert roundTrip(withoutName).getInnerValue().getF1().equals("theF1");
//assert ! hasNoName.hasId(); // does not compile: hasXYZ() does not exist if XYZ is a scalar Similarly, this instance with a missing nested field can be interpreted correctly as well by a reader: var withoutInner = Parent.newBuilder()
.setId("theId")
.setName(StringValue.newBuilder().setValue("theName").build())
.build();
assert ! withoutInner.hasInnerValue();
assert ! roundTrip(withoutInner).hasInnerValue();
assert withoutInner.hasName();
assert roundTrip(withoutInner).hasName();
assert withoutInner.getName().getValue().equals("theName");
assert roundTrip(withoutInner).getName().getValue().equals("theName"); |
…hema&PbRowDataSerializationSchema
61f4ffb
to
5cf1de8
Compare
flink-formats/flink-sql-protobuf/src/main/resources/META-INF/NOTICE
Outdated
Show resolved
Hide resolved
This product includes software developed at | ||
The Apache Software Foundation (http://www.apache.org/). | ||
|
||
This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Besides, 'protobuf-java' does not use 'Apache Software License 2.0', it uses 'BSD-3 License'
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@libenchao thanks for your comment. I have fixed it.
…OTICE update copyright year Co-authored-by: Benchao Li <[email protected]>
@maosuhan Thanks for that! |
@maosuhan Thanks for your hard work. I'll do a final round of review to confirm the issues mentioned above has been addressed. |
@flinkbot run azure |
The current latest commit in this repo is 9504205 which has a CI that has completed successfully at https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=38767&view=results |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@maosuhan The PR LGTM now, thanks for your contribution. And thanks @MartijnVisser for pushing this forward.
I'll merge this after 24 hours, if there is no other objections.
@libenchao I think we can already go ahead and merge this since you've approved it and also numerous others have looked at it. Really looking forward to getting this in :) |
@libenchao Huge appreciate to your review work to get this in, it is very helpful and professional. I will finish a doc soon to introduce how to use protobuf. |
* [FLINK-18202][protobuf] Introduce protobuf format This closes apache#14376 * [FLINK-29062][build] Fix protobuf plugin proxy issue on flink-protobuf module. * [FLINK-30093][protobuf] Fix compile errors for google.protobuf.Timestamp type Close apache#21613 * add schema as a format option * bump to 1.15-SNAPSHOT --------- Co-authored-by: maosuhan <[email protected]> Co-authored-by: jiabao.sun <[email protected]> Co-authored-by: laughingman7743 <[email protected]>
What is the purpose of the change
Protobuf is a structured data format introduced by google. Compared to json, protobuf is more efficient of space and computing. Nowadays, many companies use protobuf instead of json as data format in kafka and streaming processing.
So, we will introduce a new format which can derialize/serialize protobuf data in a fast speed.
User can use this format in SQL or Table API.
Verifying this change
Tests
Add many unit tests to test of des/ser for different data type/structures of protobuf.
Benchmark
Performance test for pb object containing 200+ fields. Below is the consumed time of processing 10M rows.
Does this pull request potentially affect one of the following parts:
Documentation
Connector params:
must be located in the classpath both in client and task side.
Notice
default values
As you know, if the protobuf syntax is proto2, the generated pb class has validity bit flags to indicate whether a field is set or not. We can use
pbObject.hasXXX()
method to know whether the field is set or not. So if syntax=2,the decoded flink row may contain null values. User can also setprotobuf.read-default-values
to control the behavior of handling null values.But if the syntax is proto3, the generated protobuf class does not have
pbObject.hasXXX()
method and does not hold validity bit flags, so there is no way to tell if a field is set or not if it is equals to default value. For example, ifpbObj.getDim1()
returns 0, there's no way to tell if dim1 is set by 0 or it is not set anyway. So if message class is from proto3 syntax, the decoded flink row will not contain any null values.Also pb does not permit null in key/value of map and array. We need to generate default value for them.
OneOf field
In serialization process, there's no guarantee that the flink row fields of one-of group only contains at least one non-null value.
So in serialization, we set each field in the order of flink schema, so the field in high position will override then field of low position in the same one-of group.
Enum type
Enum value of pb will be converted to String and vice versa in the name of enum value definition in pb.