-
Notifications
You must be signed in to change notification settings - Fork 4.3k
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
[BEAM-12384] Set output typeDescriptor explictly in Read.Bounded transform #14854
[BEAM-12384] Set output typeDescriptor explictly in Read.Bounded transform #14854
Conversation
@@ -151,7 +150,8 @@ private Bounded(@Nullable String name, BoundedSource<T> source) { | |||
.apply(ParDo.of(new OutputSingleSource<>(source))) | |||
.setCoder(SerializableCoder.of(new TypeDescriptor<BoundedSource<T>>() {})) | |||
.apply(ParDo.of(new BoundedSourceAsSDFWrapperFn<>())) | |||
.setCoder(source.getOutputCoder()); | |||
.setCoder(source.getOutputCoder()) | |||
.setTypeDescriptor(source.getOutputCoder().getEncodedTypeDescriptor()); |
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.
Do we maintain the TypeDescriptor
information before for Read
? I was under impression that for most of cases we only set Coder
for a output PCollection
.
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.
You are right and I don't know why we don't pay more attention to this. Probably because coders seem to include the TypeDescriptor, any ideas @kennknowles ? is this redundant somehow?
In any case having this information seems important for the downstream transforms.
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.
It seems like the typeDescriptor
can be inferred from Coder.getEncodedTypeDescriptor()
. If we really want to populate this information in a consistent way, probably we can consider changing PCollection.getTypeDescriptor()
to infer the typeDescriptor
from Coder
if the typeDescriptor
is set.
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.
Yes, the expected use of this method is to set the type descriptor but not the coder. This way, the coder registry still can choose the coder.
Setting both is redundant, in theory. Setting just the coder should suffice. Maybe some plumbing needed? It was not really expected to look at either one in this way.
Another angle to consider is that type descriptor is Java-specific, while coder is the portable "type" of the data. I don't know if that matters here.
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.
I'm thinking about changes like: #14870
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.
I like @boyuanzz fix because even in the presence of different Coders the TypeDescriptor is commonly preserved inside of the Coders. WDYT @kennknowles can you spot some particular issues about it?
I can rebase this PR targetting a generic implementation like the one on #14870 but I did not do it like that because I was not really familiar with the reasoning behind not relying on the coder typeDescriptor.
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.
Yea makes lots of sense.
Run Java PreCommit |
Run Java_Examples_Dataflow PreCommit |
Run Java_Examples_Dataflow_Java11 PreCommit |
2857fa7
to
e2bfde9
Compare
Run Java PreCommit |
Run Java_Examples_Dataflow_Java11 PreCommit |
e2bfde9
to
83bccf9
Compare
I cherry-picked @boyuanzz commit from the other PR notice however that I could not get the Type to be preserved after the two DoFns are applied. In @Override
public TypeDescriptor<SourceT> getOutputTypeDescriptor() {
return (TypeDescriptor<SourceT>)
new TypeDescriptor<Source<T>>(getClass()) {}.where(
new TypeParameter<T>() {}, source.getOutputCoder().getEncodedTypeDescriptor());
} but in the second DoFn I did not find a way to recover the real type of T when overwriting |
I would say, let's unblock you first : ) Can you file a JIRA issue on this? I think we should find a ultimate solution there. |
Run Java PreCommit |
Run Java_Examples_Dataflow PreCommit |
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.
Thanks!
Thanks @boyuanzz I filled BEAM-12420 TypeDescriptor information gets lost when applying multiple DoFn on Composite Transform as a follow up. |
R: @boyuanzz @kennknowles