-
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
make FieldValueTypeInformation creators take a TypeDescriptor parameter #32081
Changes from all commits
ce83352
d2e3478
ce5ff31
70be685
6ef957a
0e7018d
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -19,6 +19,7 @@ | |
|
||
import java.io.Serializable; | ||
import org.apache.beam.sdk.annotations.Internal; | ||
import org.checkerframework.checker.nullness.qual.NonNull; | ||
import org.checkerframework.checker.nullness.qual.Nullable; | ||
|
||
/** | ||
|
@@ -29,7 +30,7 @@ | |
* <p>Implementations of this interface are generated at runtime to map object fields to Row fields. | ||
*/ | ||
@Internal | ||
public interface FieldValueGetter<ObjectT, ValueT> extends Serializable { | ||
public interface FieldValueGetter<ObjectT extends @NonNull Object, ValueT> extends Serializable { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. why do you have ObjectT extends Object? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. so apparently there's a difference (I don't understand why it would be so, though) between a
with a signature as above, we can only write:
or alternatively:
The checkerframework will make sure that we propagate the non-nullness of the parameter.
nor will this:
but it will work if I change the signature of
in other words, the There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. yeah, at least not when we're talking type parameters, for example:
gives me
|
||
@Nullable | ||
ValueT get(ObjectT object); | ||
|
||
|
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.
what are these new tags for?
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.
so in
RowValueGettersFactory
there's this code:since we're in the constructor, the
this
parameter is still not fully initialized, so by using this annotation we can temporarily (for the duration of the initalization of the CachingFactory object) allow assigning theinnerFactory
field a value which itself is under initialization (or indeed whose initialization status is unknown, as the signature of the CachingFactory constructor tells us)see also https://checkerframework.org/manual/#circular-initialization