afedulov commented on code in PR #19228:
URL: https://github.com/apache/flink/pull/19228#discussion_r855628580
##########
flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/threadinfo/JobVertexThreadInfoStats.java:
##########
@@ -44,7 +45,7 @@
private final long endTime;
/** Map of thread info samples by execution ID. */
- private final Map<ExecutionAttemptID, List<ThreadInfoSample>>
samplesBySubtask;
+ private final Map<Set<ExecutionAttemptID>, List<ThreadInfoSample>>
samplesBySubtask;
Review Comment:
Hmm... I am not sure about that:
1. There is a Set-to-List relationship. So we'd need to know which element
from the list "belongs" to which element in the set. That means passing down
not just Thread IDs, but the (ExecutionAttemptID,ThreadID) bundles all the way
down to the MXBeans JVM sampler, plus preserving this relation in the result
(`ThreadInfoSample` will have to contain the ExecutionAttemptID).
2. The changes in `matchExecutionsWithGateways` are sort of the core of the
optimization, because this is where "bundling" requests by their `TaskManager`
actually happens.
What are the concerns with the defaults Set's hashCode?
- Is it the concern that elements could be added into the collection after
it was "hashed" into a Map?
=> We could wrap it in a `Collections.unmodifiableSet(set)`
- Is it the default implementation that would potentially yield the same
results for the "anagrams"?
[AbstractSet.java](https://www.cc.gatech.edu/computing/pag/tmp/html_dir/java/util/AbstractSet.java.html)
=> In that case we could implement a custom key type and go full Joshua
Bloch on it (Chapter 11 in the 3rd edition and add multiplication with the
previous "result" for each element while iterating)
- Is it the performance of recomputing the hashCode?
=> We could cache it
- Is there anything else I missed in terms of surprises?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]