-
Notifications
You must be signed in to change notification settings - Fork 588
HDDS-11376. Improve ReplicationSupervisor to record replication metrics #7140
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
Merged
Merged
Changes from 8 commits
Commits
Show all changes
10 commits
Select commit
Hold shift + click to select a range
7d04590
HDDS-11376. Improve ReplicationSupervisor to record replication metrics
jianghuazhu 9627505
Fix checkstyle.
jianghuazhu e25bb91
Improve requestCounter to record metrics for different replication ty…
jianghuazhu 9e6ff29
Update some code.
jianghuazhu d143ace
Fix some checkstyle
jianghuazhu ce67b30
Fix some checkstyle
jianghuazhu f03b420
Fix some checkstyle
jianghuazhu ae8fc2c
Improve some abstract methods.
jianghuazhu 06f3db3
Simplify some code.
jianghuazhu 8e06209
Improve some code.
jianghuazhu File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change | ||||||||
|---|---|---|---|---|---|---|---|---|---|---|
|
|
@@ -26,6 +26,7 @@ | |||||||||
| import java.util.Objects; | ||||||||||
| import java.util.OptionalLong; | ||||||||||
| import java.util.Set; | ||||||||||
| import java.util.Collections; | ||||||||||
| import java.util.concurrent.ConcurrentHashMap; | ||||||||||
| import java.util.concurrent.ExecutorService; | ||||||||||
| import java.util.concurrent.PriorityBlockingQueue; | ||||||||||
|
|
@@ -71,11 +72,17 @@ public final class ReplicationSupervisor { | |||||||||
| private final StateContext context; | ||||||||||
| private final Clock clock; | ||||||||||
|
|
||||||||||
| private final AtomicLong requestCounter = new AtomicLong(); | ||||||||||
| private final AtomicLong successCounter = new AtomicLong(); | ||||||||||
| private final AtomicLong failureCounter = new AtomicLong(); | ||||||||||
| private final AtomicLong timeoutCounter = new AtomicLong(); | ||||||||||
| private final AtomicLong skippedCounter = new AtomicLong(); | ||||||||||
| private final Map<String, AtomicLong> requestCounter = new ConcurrentHashMap<>(); | ||||||||||
| private final Map<String, AtomicLong> successCounter = new ConcurrentHashMap<>(); | ||||||||||
| private final Map<String, AtomicLong> failureCounter = new ConcurrentHashMap<>(); | ||||||||||
| private final Map<String, AtomicLong> timeoutCounter = new ConcurrentHashMap<>(); | ||||||||||
| private final Map<String, AtomicLong> skippedCounter = new ConcurrentHashMap<>(); | ||||||||||
|
|
||||||||||
| private static final Map<String, String> METRICS_MAP; | ||||||||||
|
|
||||||||||
| static { | ||||||||||
| METRICS_MAP = new HashMap<>(); | ||||||||||
| } | ||||||||||
|
|
||||||||||
| /** | ||||||||||
| * A set of container IDs that are currently being downloaded | ||||||||||
|
|
@@ -188,6 +195,10 @@ public static Builder newBuilder() { | |||||||||
| return new Builder(); | ||||||||||
| } | ||||||||||
|
|
||||||||||
| public static Map<String, String> getMetricsMap() { | ||||||||||
| return Collections.unmodifiableMap(METRICS_MAP); | ||||||||||
| } | ||||||||||
|
|
||||||||||
| private ReplicationSupervisor(StateContext context, ExecutorService executor, | ||||||||||
| ReplicationConfig replicationConfig, DatanodeConfiguration datanodeConfig, | ||||||||||
| Clock clock, IntConsumer executorThreadUpdater) { | ||||||||||
|
|
@@ -221,6 +232,19 @@ public void addTask(AbstractReplicationTask task) { | |||||||||
| return; | ||||||||||
| } | ||||||||||
|
|
||||||||||
| if (requestCounter.get(task.getMetricName()) == null) { | ||||||||||
| synchronized (this) { | ||||||||||
| if (requestCounter.get(task.getMetricName()) == null) { | ||||||||||
| requestCounter.put(task.getMetricName(), new AtomicLong(0)); | ||||||||||
| successCounter.put(task.getMetricName(), new AtomicLong(0)); | ||||||||||
| failureCounter.put(task.getMetricName(), new AtomicLong(0)); | ||||||||||
| timeoutCounter.put(task.getMetricName(), new AtomicLong(0)); | ||||||||||
| skippedCounter.put(task.getMetricName(), new AtomicLong(0)); | ||||||||||
| METRICS_MAP.put(task.getMetricName(), task.getMetricDescriptionSegment()); | ||||||||||
| } | ||||||||||
| } | ||||||||||
| } | ||||||||||
|
|
||||||||||
| if (inFlight.add(task)) { | ||||||||||
| if (task.getPriority() != ReplicationCommandPriority.LOW) { | ||||||||||
| // Low priority tasks are not included in the replication queue sizes | ||||||||||
|
|
@@ -330,14 +354,14 @@ public TaskRunner(AbstractReplicationTask task) { | |||||||||
| @Override | ||||||||||
| public void run() { | ||||||||||
| try { | ||||||||||
| requestCounter.incrementAndGet(); | ||||||||||
| requestCounter.get(task.getMetricName()).incrementAndGet(); | ||||||||||
|
|
||||||||||
| final long now = clock.millis(); | ||||||||||
| final long deadline = task.getDeadline(); | ||||||||||
| if (deadline > 0 && now > deadline) { | ||||||||||
| LOG.info("Ignoring {} since the deadline has passed ({} < {})", | ||||||||||
| this, Instant.ofEpochMilli(deadline), Instant.ofEpochMilli(now)); | ||||||||||
| timeoutCounter.incrementAndGet(); | ||||||||||
| timeoutCounter.get(task.getMetricName()).incrementAndGet(); | ||||||||||
| return; | ||||||||||
| } | ||||||||||
|
|
||||||||||
|
|
@@ -364,18 +388,18 @@ public void run() { | |||||||||
| task.runTask(); | ||||||||||
| if (task.getStatus() == Status.FAILED) { | ||||||||||
| LOG.warn("Failed {}", this); | ||||||||||
| failureCounter.incrementAndGet(); | ||||||||||
| failureCounter.get(task.getMetricName()).incrementAndGet(); | ||||||||||
| } else if (task.getStatus() == Status.DONE) { | ||||||||||
| LOG.info("Successful {}", this); | ||||||||||
| successCounter.incrementAndGet(); | ||||||||||
| successCounter.get(task.getMetricName()).incrementAndGet(); | ||||||||||
| } else if (task.getStatus() == Status.SKIPPED) { | ||||||||||
| LOG.info("Skipped {}", this); | ||||||||||
| skippedCounter.incrementAndGet(); | ||||||||||
| skippedCounter.get(task.getMetricName()).incrementAndGet(); | ||||||||||
| } | ||||||||||
| } catch (Exception e) { | ||||||||||
| task.setStatus(Status.FAILED); | ||||||||||
| LOG.warn("Failed {}", this, e); | ||||||||||
| failureCounter.incrementAndGet(); | ||||||||||
| failureCounter.get(task.getMetricName()).incrementAndGet(); | ||||||||||
| } finally { | ||||||||||
| inFlight.remove(task); | ||||||||||
| decrementTaskCounter(task); | ||||||||||
|
|
@@ -419,7 +443,12 @@ public boolean equals(Object o) { | |||||||||
| } | ||||||||||
|
|
||||||||||
| public long getReplicationRequestCount() { | ||||||||||
| return requestCounter.get(); | ||||||||||
| return getCount(requestCounter); | ||||||||||
| } | ||||||||||
|
|
||||||||||
| public long getReplicationRequestCount(String metricsName) { | ||||||||||
| AtomicLong counter = requestCounter.get(metricsName); | ||||||||||
| return counter != null ? counter.get() : 0; | ||||||||||
| } | ||||||||||
|
|
||||||||||
| public long getQueueSize() { | ||||||||||
|
|
@@ -438,20 +467,51 @@ public long getMaxReplicationStreams() { | |||||||||
| } | ||||||||||
| } | ||||||||||
|
|
||||||||||
| private long getCount(Map<String, AtomicLong> counter) { | ||||||||||
| if (counter.isEmpty()) { | ||||||||||
| return 0; | ||||||||||
| } | ||||||||||
| AtomicLong total = new AtomicLong(0); | ||||||||||
| counter.forEach((key, value) -> { | ||||||||||
| total.set(total.get() + value.get()); | ||||||||||
| }); | ||||||||||
|
||||||||||
| counter.forEach((key, value) -> { | |
| total.set(total.get() + value.get()); | |
| }); | |
| counter.forEach((key, value) -> total.addAndGet(value.get())); |
Contributor
Author
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 @errose28 .
I have updated it.
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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 does not need to be an AtomicLong - a simple Long is fine. This is a local variable and we return only its value, so it does not need to be thread safe.
Not particularly important, but you could also remove the
if (counter.isEmpty())section: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 @sodonnel for your comment and review.
I have updated it, can you review it again?
Thanks.