Skip to content
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-30093] [formats] Protobuf Timestamp Compile Error #21436

Open
wants to merge 1 commit into
base: master
Choose a base branch
from

Conversation

hdulay
Copy link

@hdulay hdulay commented Nov 30, 2022

FLINK-30093

What is the purpose of the change

This PR enables support for Google Protobuf Timestamp datatypes and possibly other Google Protobuf datatypes.

Brief change log

  • Adding <includeMavenTypes>direct</includeMavenTypes> Specifies whether to extract .proto files from Maven dependencies and add them to the protoc import path. Options: "none" (do not extract any proto files), "direct" (extract only from direct dependencies), "transitive" (extract from direct and transitive dependencies)
  • Ensuring the generated code provides the correct package name for all google proto types.

Verifying this change

  • Added Google Protobuf Timestamp to existing Proto tests.

Does this pull request potentially affect one of the following parts:

  • Dependencies (does it add or upgrade a dependency): (no)
  • The public API, i.e., is any changed class annotated with @Public(Evolving): (yno)
  • The serializers: (no)
  • The runtime per-record code paths (performance sensitive): (don't know)
  • Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: (no)
  • The S3 file system connector: (no)

Documentation

  • Does this pull request introduce a new feature? (no)
  • If yes, how is the feature documented? (not applicable)

@hdulay
Copy link
Author

hdulay commented Nov 30, 2022

Hi @maosuhan . Since you were the originator of this code, I was hoping you could check this enhancement to the protobuf work. Thanks

@flinkbot
Copy link
Collaborator

flinkbot commented Nov 30, 2022

CI report:

Bot commands The @flinkbot bot supports the following commands:
  • @flinkbot run azure re-run the last Azure build

@MartijnVisser
Copy link
Contributor

@hdulay Thanks for opening your first PR. Please check out the code style & quality guide for naming conventions on commits https://flink.apache.org/contributing/code-style-and-quality-preamble.html

@hdulay hdulay changed the title FLINK-30093 Formats Protobuf Timestamp [FLINK-30093] [formats] Protobuf Timestamp Nov 30, 2022
@hdulay hdulay changed the title [FLINK-30093] [formats] Protobuf Timestamp [FLINK-30093] [formats] Protobuf Timestamp Compile Error Nov 30, 2022
@rmetzger
Copy link
Contributor

rmetzger commented Dec 1, 2022

Thx for the PR. Can you also update the commit message, for example to the PR title?

@rmetzger
Copy link
Contributor

rmetzger commented Dec 1, 2022

The CI failure seems to be caused by a known instability: https://issues.apache.org/jira/browse/FLINK-30257

@hdulay
Copy link
Author

hdulay commented Dec 1, 2022

Updated commit message. Tnx

@maosuhan
Copy link
Contributor

maosuhan commented Dec 2, 2022

@hdulay Thanks for raising this issue and your fix PR. I will help to review this PR. cc @MartijnVisser

@@ -34,6 +35,15 @@ public static String getFullJavaName(Descriptors.Descriptor descriptor, String o
String parentJavaFullName =
getFullJavaName(descriptor.getContainingType(), outerProtoName);
return parentJavaFullName + "." + descriptor.getName();
} else if (descriptor.getFullName().contains("google")) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@hdulay The compilation error is caused by a third-party import, right? Could you make it more general to adapt to all the import cases?

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes

@@ -85,6 +88,8 @@ public void testDeserialization() throws Exception {

assertEquals(1, rowData.getInt(0));
assertEquals(2L, rowData.getLong(1));

assertEquals(ts.getNanos(), row.getTimestamp(11, 3).getNanoOfMillisecond());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm curious how does Timestamp work because I did not define Timestamp type for flink. I guess Timestamp is a message type in protobuf, how come it will convert to flink Timestamp automatically?

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The org.apache.flink.table.data.binary.BinaryRowData.getTimestamp() method from the flink-table module does the translation to org.apache.flink.table.data.TimestampData.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@hdulay I think there is still gap between protobuf and flink internal memory layout. I tried to build a concrete Timestamp like Timestamp ts =Timestamp.newBuilder().setSeconds(xxx).setNanos(yyy).build() in unit test. And you can easily find the difference.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@maosuhan Yes as I stepped through the ProtobufSQLITCaseTest to add a timestamp field, I realized this issue goes beyond just source generation and compilation errors. Is there an example of how to add new data type? I'm assuming I'll need to somehow map it to the TimestampType logical type. Thanks

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just to step in: adding a new datatype requires a FLIP and a discussion. Why can't any of the existing datatypes be reused?

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@MartijnVisser Hi. I'm suggesting to add support for 3rd party protobuf datatype. Which needs a mapping to either a RowType or a SimpleType in the protobuf codegen components. I'm assuming this is the correct approach. Thanks

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@hdulay Yes, I agree with your idea. RowType must be doable to reflect a Timestamp pb type. If you want to make it more easy to use, you can treat google.protobuf.Timestamp a special type and manually convert Timestamp to TimestampData in code.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks. Let me know if it's appropriate to convert this PR to a draft. Thanks for all the guidance.

@@ -33,6 +34,7 @@ message Pb3Test {
bytes i = 9;
map<string, string> map1 = 10;
map<string, InnerMessageTest> map2 = 11;
google.protobuf.Timestamp ts = 12;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Also test in ProtobufSQLITCaseTest for end to end correctness.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
5 participants