-
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
Add support for StringSet metric in Java SDK to track set of unique s… #31789
Conversation
R: @robertwb |
Stopping reviewer notifications for this pull request: review requested by someone other than the bot, ceding control |
04d542f
to
35ccdb1
Compare
35ccdb1
to
44a3901
Compare
ecfa14e
to
67ef5bd
Compare
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, this looks realy good.
} | ||
|
||
/** | ||
* Return a {@code StringSetCell} named {@code metricName}.If it doesn't exist, return {@code |
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.
Nit: space after period.
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.
Done.
public static ByteString encodeStringSet(StringSetData data) { | ||
try (ByteStringOutputStream output = new ByteStringOutputStream()) { | ||
// encode the length of set | ||
STRING_CODER.encode(String.valueOf(data.stringSet().size()), output); |
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.
Use IterableCoder.of(StringsUtf8Coder.of())
rather than encoding the length as a string.
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.
This is nice!!! Thank you.
(I will work on this and other review comments tomorrow. Didn't realize the difference between "Add single comment" and "start review" in previous comment)
|
||
@Override | ||
public void add(String value) { | ||
update(StringSetData.create(new HashSet<>(Collections.singletonList(value)))); |
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.
Perhaps optimized for the case where value is already in the set by skipping the update (and creation of sets and just to do a no-op merge, and setting the dirty bit).
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.
Good idea. Thanks.
@AutoValue | ||
public abstract class StringSetData implements Serializable { | ||
|
||
public abstract Set<String> stringSet(); |
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.
Perhaps here and below we should document whether the sets are (expected to be?) immutable or not. E.g. is one allowed ot modify the set after passing it to create? Modify the set returned from stringSet()?
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.
Done.
Also I see I missed to set well defined mutability behavior for StringSetData and Result. Fixed it now. Result needs to immutable. Data I think we can have either ways, having mutable will allow being able combine more efficiently but can lead to confusing contract specially for EmptyStringSetData. I am going to make StringSetData immutable and they can be combined by only copying
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.
Note: We ended up changing StringSetData to be mutable because in certain IOs such as TextIOs the number lineage records (per file) are very large and immutable hashset copy cost became huge
|
||
StringSetCell differentDirty = new StringSetCell(MetricName.named("namespace", "name")); | ||
differentDirty.getDirty().afterModification(); | ||
Assert.assertNotEquals(stringSetCell, differentDirty); |
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'll admit this feels a bit odd. But when do cells need to be compared (or be hashable)? (If it's just following the convention of what's done elsewhere, that's fine.)
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.
This is just conventional. I did not find any direct (or even indirect) usage in my work so far.
if (metricUpdate.getSet() == null) { | ||
return StringSetResult.empty(); | ||
} | ||
return StringSetResult.create(new HashSet<>(((ArrayList) metricUpdate.getSet()))); |
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.
ImmutableSet.copyOf(...) is likely faster, especially for sets of {0,1} elements. (Might be worth using elsewhere too.)
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.
Oh ya true. Thanks.
@@ -184,6 +189,13 @@ private MetricUpdate makeCounterMetricUpdate( | |||
return setStructuredName(update, name, namespace, step, tentative); | |||
} | |||
|
|||
private MetricUpdate makeStringSetMetricUpdate( | |||
String name, String namespace, String step, List<String> setValues, boolean tentative) { |
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.
Why is this a List of values? Should it be a Set or Container?
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.
Ah you are right it can be set. Fixed. Earlier the documentation tripped me https://screenshot.googleplex.com/4npsge3GVUo3zRs I thought it needs to be "list" .
|
||
@Override | ||
public void add(String value) { | ||
stringSetData.combine(StringSetData.create(ImmutableSet.of("ab"))); |
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.
Again, perhaps worth optimizing for the case where value is already present.
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.
Fixed. Also fixed my typo of ignoring value and using "ab".
public StringSetData combine(Iterable<StringSetData> updates) { | ||
StringSetData result = StringSetData.empty(); | ||
for (StringSetData update : updates) { | ||
result = result.combine(update); |
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.
Doing a one-at-a-time binary combine of multiple sets might be inefficient (vs. creating a single set and adding everything to it). I don't know how often this method is called thoug.
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.
Ah good point. Done.
ad17a69
to
0ca3bff
Compare
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.
@robertwb
Thanks for the prompt review. Fixed review comment. Please have another look.
Thank you.
|
||
@Override | ||
public void add(String value) { | ||
update(StringSetData.create(new HashSet<>(Collections.singletonList(value)))); |
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.
Good idea. Thanks.
@AutoValue | ||
public abstract class StringSetData implements Serializable { | ||
|
||
public abstract Set<String> stringSet(); |
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.
Done.
Also I see I missed to set well defined mutability behavior for StringSetData and Result. Fixed it now. Result needs to immutable. Data I think we can have either ways, having mutable will allow being able combine more efficiently but can lead to confusing contract specially for EmptyStringSetData. I am going to make StringSetData immutable and they can be combined by only copying
|
||
StringSetCell differentDirty = new StringSetCell(MetricName.named("namespace", "name")); | ||
differentDirty.getDirty().afterModification(); | ||
Assert.assertNotEquals(stringSetCell, differentDirty); |
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.
This is just conventional. I did not find any direct (or even indirect) usage in my work so far.
public StringSetData combine(Iterable<StringSetData> updates) { | ||
StringSetData result = StringSetData.empty(); | ||
for (StringSetData update : updates) { | ||
result = result.combine(update); |
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.
Ah good point. Done.
if (metricUpdate.getSet() == null) { | ||
return StringSetResult.empty(); | ||
} | ||
return StringSetResult.create(new HashSet<>(((ArrayList) metricUpdate.getSet()))); |
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.
Oh ya true. Thanks.
|
||
@Override | ||
public void add(String value) { | ||
stringSetData.combine(StringSetData.create(ImmutableSet.of("ab"))); |
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.
Fixed. Also fixed my typo of ignoring value and using "ab".
@@ -184,6 +189,13 @@ private MetricUpdate makeCounterMetricUpdate( | |||
return setStructuredName(update, name, namespace, step, tentative); | |||
} | |||
|
|||
private MetricUpdate makeStringSetMetricUpdate( | |||
String name, String namespace, String step, List<String> setValues, boolean tentative) { |
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.
Ah you are right it can be set. Fixed. Earlier the documentation tripped me https://screenshot.googleplex.com/4npsge3GVUo3zRs I thought it needs to be "list" .
* Null-containing sets don't need to be tested as they can no longer be constructed. * Use vendered guava.
Add support for StringSet metric in Java SDK to track set of unique string as metric. addresses apache#31788 * Add support for StringSet metric in Java SDK to track set of unique string as metric. * Fix compilation and tests * Add support for StringSet in PortableRunner and fix some spotless java checks * Add support for StringSet in JetRunner * Fix precommit errors * Fixes for review comments * Other fixes * Fixes for spotless java * Fix a couple of tests. * Null-containing sets don't need to be tested as they can no longer be constructed. * Use vendered guava. * unused imports --------- Co-authored-by: Robert Bradshaw <[email protected]>
Add support for StringSet metric in Java SDK to track set of unique string as metric. addresses apache#31788 * Add support for StringSet metric in Java SDK to track set of unique string as metric. * Fix compilation and tests * Add support for StringSet in PortableRunner and fix some spotless java checks * Add support for StringSet in JetRunner * Fix precommit errors * Fixes for review comments * Other fixes * Fixes for spotless java * Fix a couple of tests. * Null-containing sets don't need to be tested as they can no longer be constructed. * Use vendered guava. * unused imports --------- Co-authored-by: Robert Bradshaw <[email protected]>
Add support for StringSet metric in Java SDK to track set of unique string as metric.
addresses #31788
Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
addresses #123
), if applicable. This will automatically add a link to the pull request in the issue. If you would like the issue to automatically close on merging the pull request, commentfixes #<ISSUE NUMBER>
instead.CHANGES.md
with noteworthy changes.See the Contributor Guide for more tips on how to make review process smoother.
To check the build health, please visit https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md
GitHub Actions Tests Status (on master branch)
See CI.md for more information about GitHub Actions CI or the workflows README to see a list of phrases to trigger workflows.