-
Notifications
You must be signed in to change notification settings - Fork 13.5k
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-14816] Add thread dump feature for taskmanager #11887
Conversation
Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community Automated ChecksLast check on commit 04664a7 (Thu Apr 23 16:26:27 UTC 2020) Warnings:
Mention the bot in a comment to re-run the automated checks. Review Progress
Please see the Pull Request Review Guide for a full explanation of the review process. The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required Bot commandsThe @flinkbot bot supports the following commands:
|
hi @tillrohrmann, thanks for doing this, 💯 , but |
Thanks for the pointer @lamber-ken. I will update the PR to fix the |
Instead of serving plain text, this commit changes the TaskManagerThreadDumpHandler to return a JSON response. This allows to further extend this handler to not only generate a thread dump of all threads but also for a sub set. Morever, it allows to return a more structured return value in the future. This closes apache#11887.
04664a7
to
9c2c469
Compare
Instead of serving plain text, this commit changes the TaskManagerThreadDumpHandler to return a JSON response. This allows to further extend this handler to not only generate a thread dump of all threads but also for a sub set. Morever, it allows to return a more structured return value in the future. This closes apache#11887.
9c2c469
to
c81293a
Compare
Instead of serving plain text, this commit changes the TaskManagerThreadDumpHandler to return a JSON response. This allows to further extend this handler to not only generate a thread dump of all threads but also for a sub set. Morever, it allows to return a more structured return value in the future. This closes apache#11887.
c81293a
to
b84b3cb
Compare
cc @zentol: The |
Instead of serving plain text, this commit changes the TaskManagerThreadDumpHandler to return a JSON response. This allows to further extend this handler to not only generate a thread dump of all threads but also for a sub set. Morever, it allows to return a more structured return value in the future. This closes apache#11887.
852c752
to
bc5fd41
Compare
private CompletableFuture<TransientBlobKey> putTransientBlobStream(InputStream inputStream, String fileTag) { | ||
final TransientBlobCache transientBlobService = blobCacheService.getTransientBlobService(); | ||
final TransientBlobKey transientBlobKey; | ||
|
||
try { | ||
transientBlobKey = transientBlobService.putTransient(inputStream); | ||
} catch (IOException e) { | ||
log.debug("Could not upload file {}.", fileTag, e); | ||
return FutureUtils.completedExceptionally(new FlinkException("Could not upload file " + fileTag + '.', e)); | ||
} | ||
return CompletableFuture.completedFuture(transientBlobKey); | ||
} | ||
|
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.
hi @tillrohrmann, since doesn't make thread dump as stream in your design right now, we can revert it.
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 think this change is still good since it breaks up a longer method into smaller units.
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.
They still belong into a separate commit though.
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 think this change is still good since it breaks up a longer method into smaller units.
ok
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.
Technically yes but I think it is ok to not twiddle with this commit because the benefit would be marginal.
@@ -22,6 +22,13 @@ | |||
* Different file types to request from the {@link TaskExecutor}. | |||
*/ | |||
public enum FileType { | |||
/** | |||
* the log file type for taskmanager |
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 seems unrelated?
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.
Somewhat. But I believe that the extra effort splitting this change into a separate commit won't pay off. Hence, I would like to keep it as is.
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.
Per the code-style guide:
Pull Requests must put cleanup, refactoring, and core changes into separate commits.
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 Chesnay. I will refactor the changes into separate commits in order to follow the code style guide.
final Collection<ThreadInfo> threadDump = JvmUtils.createThreadDump(); | ||
|
||
final Collection<ThreadDumpInfo.ThreadInfo> threadInfos = threadDump.stream() | ||
.map(threadInfo -> ThreadDumpInfo.ThreadInfo.create(threadInfo.getThreadName(), threadInfo.toString())) |
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 wondering whether this conversion should really be done on the TaskExecutor.
I'd prefer if conversions to REST responses are done by the handler just to separate things a 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.
As soon as the Rest response and the internal cluster message differ I will separate the classes. As long as they are identical I think it is fine to use the same class here.
public static final String FIELD_NAME_THREAD_INFOS = "threadInfos"; | ||
|
||
@JsonProperty(FIELD_NAME_THREAD_INFOS) | ||
private final Collection<ThreadInfo> threadInfos; |
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 not a map thread_name -> thread_info? I think this would be more generally usable.
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.
The main reason was the duplicated thread name information as a key and as part of the ThreadInfo
class. But I guess this is premature optimization. I will change it.
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.
Actually, I think whether it is stored in a map or not is an implementation detail. Moreover, we don't offer a method getThreadInfo(String threadName)
yet. Hence, there is actually no reason to store them internally as a map. Once the requirement changes, I'm happy to adapt the class.
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 wasn't talking about changing the internal storage, but to expose it as a map instead (i.e., changing the JOSN representation)
Assuming a user is interested in a particular thread, then with a map one can do threadDump['myThreadName']
, instead of having to iterate over the array.
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 point. I will take another look.
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.
After having looked at it again, I still believe that the response should be a collection of ThreadInfos
. If the user needs to do a named lookup, then she can still create a map from the collection. It should not be the responsibility response class to present the elements in a certain order. Moreover, we don't have this requirement at the moment. Hence, I won't do the proposed change.
private CompletableFuture<TransientBlobKey> putTransientBlobStream(InputStream inputStream, String fileTag) { | ||
final TransientBlobCache transientBlobService = blobCacheService.getTransientBlobService(); | ||
final TransientBlobKey transientBlobKey; | ||
|
||
try { | ||
transientBlobKey = transientBlobService.putTransient(inputStream); | ||
} catch (IOException e) { | ||
log.debug("Could not upload file {}.", fileTag, e); | ||
return FutureUtils.completedExceptionally(new FlinkException("Could not upload file " + fileTag + '.', e)); | ||
} | ||
return CompletableFuture.completedFuture(transientBlobKey); | ||
} | ||
|
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.
They still belong into a separate commit though.
Thanks for the review @zentol. I answered all your comments. Please take another look. |
Instead of serving plain text, this commit changes the TaskManagerThreadDumpHandler to return a JSON response. This allows to further extend this handler to not only generate a thread dump of all threads but also for a sub set. Morever, it allows to return a more structured return value in the future. This closes apache#11887.
bc5fd41
to
5ecc88f
Compare
Instead of serving plain text, this commit changes the TaskManagerThreadDumpHandler to return a JSON response. This allows to further extend this handler to not only generate a thread dump of all threads but also for a sub set. Morever, it allows to return a more structured return value in the future. This closes apache#11887.
5ecc88f
to
89f3c0d
Compare
Thanks for the review @lamber-ken. @zentol I've split the unrelated changes into separate commits. I won't change the JSON format though. I will merge this PR once CI gives green light. |
Extension of #10228.