-
Notifications
You must be signed in to change notification settings - Fork 9.1k
HADOOP-18325: ABFS: Add correlated metric support for ABFS operations #6314
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
Changes from all commits
98d2901
16612d3
fb2295e
986d2e6
23d9b47
d57a746
4476d33
d7b6be8
324d2c0
b9273d3
5fc1417
b185269
ef9691a
3bddb11
4afba69
0edb368
791b84e
f6bad55
9230bd8
e1c4c61
07aa9af
7686cce
f9de500
f66bfe3
d0c0f68
637060f
a477400
0f44a9b
312853c
bca9c4e
4e99643
a5ca55f
1d99d7b
3b5ba52
139b528
4e9cd93
137f337
15337ff
5706ca0
3abca57
4889c59
6cc7d37
05acafc
388d837
da41836
d55aa47
e26ee5f
5f680d0
a0d3ef9
63abb7f
eb381e1
386da87
40868de
1f6827f
d22b897
899d11a
4a9fafa
96855b7
3cc1c74
241410a
b4b273a
dbcb447
c88d902
c4c15c9
41ffe2b
1dc49f4
62d5889
c000464
84bce79
758dfea
98066c3
8e0f143
96b91a5
b93a891
adc8049
e9a7c7c
831201a
d1c3c57
1609cf0
f97e43a
eca298e
bf1c409
f45c3db
2875fcc
901f670
0395e94
b3acadb
9e8acc9
73ab52b
081bce4
3e21c2d
0f4068f
f51f756
ea92172
e514c88
7c933fd
b2ae082
ab9ecdc
5696ba6
7d41cee
99495d3
646a81c
09c2674
c416fb0
e944dc6
290dbaa
d5fbf4a
e18b2f5
8147b27
18205c1
1479da8
ac3cb16
f8df32f
05edb95
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 |
---|---|---|
@@ -0,0 +1,312 @@ | ||
/** | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.hadoop.fs.azurebfs; | ||
|
||
import java.util.ArrayList; | ||
import java.util.Arrays; | ||
import java.util.Map; | ||
import java.util.concurrent.ConcurrentHashMap; | ||
import java.util.concurrent.atomic.AtomicLong; | ||
|
||
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.HUNDRED; | ||
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.THOUSAND; | ||
|
||
public class AbfsBackoffMetrics { | ||
|
||
private AtomicLong numberOfRequestsSucceeded; | ||
|
||
private AtomicLong minBackoff; | ||
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. are these timings? if so should include time unit in name or at least use millis and javadoc it |
||
|
||
private AtomicLong maxBackoff; | ||
|
||
private AtomicLong totalRequests; | ||
|
||
private AtomicLong totalBackoff; | ||
|
||
private String retryCount; | ||
|
||
private AtomicLong numberOfIOPSThrottledRequests; | ||
|
||
private AtomicLong numberOfBandwidthThrottledRequests; | ||
|
||
private AtomicLong numberOfOtherThrottledRequests; | ||
|
||
private AtomicLong numberOfNetworkFailedRequests; | ||
|
||
private AtomicLong maxRetryCount; | ||
|
||
private AtomicLong totalNumberOfRequests; | ||
|
||
private AtomicLong numberOfRequestsSucceededWithoutRetrying; | ||
|
||
private AtomicLong numberOfRequestsFailed; | ||
|
||
private final Map<String, AbfsBackoffMetrics> metricsMap | ||
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. this is mixing up a metrics class "stats on use" with the map of filenames -> instances. it explains why there are two constructors and one only does partial init and whose toString() will NPE -but it doesn't justify this design. Proposed: split out the metrics (which should use IOStatisticsStore for its structure, and implement IOStatisticsSource serving this) from the map managing the metrics. that must be at most one per client instance and needs a good cleanup story so it scales well. |
||
= new ConcurrentHashMap<>(); | ||
|
||
public AbfsBackoffMetrics() { | ||
initializeMap(); | ||
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. I'd suggest you actually use an {{IOStatisticsStoreBuilder}} to build an IOStatisticsStore, from which you can then extract the counters for direct access -but still be able to snapshot, aggregate and share the stats though public APIs. See |
||
this.numberOfIOPSThrottledRequests = new AtomicLong(); | ||
this.numberOfBandwidthThrottledRequests = new AtomicLong(); | ||
this.numberOfOtherThrottledRequests = new AtomicLong(); | ||
this.totalNumberOfRequests = new AtomicLong(); | ||
this.maxRetryCount = new AtomicLong(); | ||
this.numberOfRequestsSucceededWithoutRetrying = new AtomicLong(); | ||
this.numberOfRequestsFailed = new AtomicLong(); | ||
this.numberOfNetworkFailedRequests = new AtomicLong(); | ||
} | ||
|
||
public AbfsBackoffMetrics(String retryCount) { | ||
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. this only partially initialises the metrics |
||
this.retryCount = retryCount; | ||
this.numberOfRequestsSucceeded = new AtomicLong(); | ||
this.minBackoff = new AtomicLong(Long.MAX_VALUE); | ||
this.maxBackoff = new AtomicLong(); | ||
this.totalRequests = new AtomicLong(); | ||
this.totalBackoff = new AtomicLong(); | ||
} | ||
|
||
private void initializeMap() { | ||
ArrayList<String> retryCountList = new ArrayList<String>( | ||
Arrays.asList("1", "2", "3", "4", "5_15", "15_25", "25AndAbove")); | ||
for (String s : retryCountList) { | ||
metricsMap.put(s, new AbfsBackoffMetrics(s)); | ||
} | ||
} | ||
|
||
public long getNumberOfRequestsSucceeded() { | ||
return this.numberOfRequestsSucceeded.get(); | ||
} | ||
|
||
public void setNumberOfRequestsSucceeded(long numberOfRequestsSucceeded) { | ||
this.numberOfRequestsSucceeded.set(numberOfRequestsSucceeded); | ||
} | ||
|
||
public void incrementNumberOfRequestsSucceeded() { | ||
this.numberOfRequestsSucceeded.getAndIncrement(); | ||
} | ||
|
||
public long getMinBackoff() { | ||
return this.minBackoff.get(); | ||
} | ||
|
||
public void setMinBackoff(long minBackoff) { | ||
this.minBackoff.set(minBackoff); | ||
} | ||
|
||
public long getMaxBackoff() { | ||
return this.maxBackoff.get(); | ||
} | ||
|
||
public void setMaxBackoff(long maxBackoff) { | ||
this.maxBackoff.set(maxBackoff); | ||
} | ||
|
||
public long getTotalRequests() { | ||
return this.totalRequests.get(); | ||
} | ||
|
||
public void incrementTotalRequests() { | ||
this.totalRequests.incrementAndGet(); | ||
} | ||
|
||
public void setTotalRequests(long totalRequests) { | ||
this.totalRequests.set(totalRequests); | ||
} | ||
|
||
public long getTotalBackoff() { | ||
return this.totalBackoff.get(); | ||
} | ||
|
||
public void setTotalBackoff(long totalBackoff) { | ||
this.totalBackoff.set(totalBackoff); | ||
} | ||
|
||
public String getRetryCount() { | ||
return this.retryCount; | ||
} | ||
|
||
public long getNumberOfIOPSThrottledRequests() { | ||
return this.numberOfIOPSThrottledRequests.get(); | ||
} | ||
|
||
public void setNumberOfIOPSThrottledRequests(long numberOfIOPSThrottledRequests) { | ||
this.numberOfIOPSThrottledRequests.set(numberOfIOPSThrottledRequests); | ||
} | ||
|
||
public void incrementNumberOfIOPSThrottledRequests() { | ||
this.numberOfIOPSThrottledRequests.getAndIncrement(); | ||
} | ||
|
||
public long getNumberOfBandwidthThrottledRequests() { | ||
return this.numberOfBandwidthThrottledRequests.get(); | ||
} | ||
|
||
public void setNumberOfBandwidthThrottledRequests(long numberOfBandwidthThrottledRequests) { | ||
this.numberOfBandwidthThrottledRequests.set(numberOfBandwidthThrottledRequests); | ||
} | ||
|
||
public void incrementNumberOfBandwidthThrottledRequests() { | ||
this.numberOfBandwidthThrottledRequests.getAndIncrement(); | ||
} | ||
|
||
public long getNumberOfOtherThrottledRequests() { | ||
return this.numberOfOtherThrottledRequests.get(); | ||
} | ||
|
||
public void setNumberOfOtherThrottledRequests(long numberOfOtherThrottledRequests) { | ||
this.numberOfOtherThrottledRequests.set(numberOfOtherThrottledRequests); | ||
} | ||
|
||
public void incrementNumberOfOtherThrottledRequests() { | ||
this.numberOfOtherThrottledRequests.getAndIncrement(); | ||
} | ||
|
||
public long getMaxRetryCount() { | ||
return this.maxRetryCount.get(); | ||
} | ||
|
||
public void setMaxRetryCount(long maxRetryCount) { | ||
this.maxRetryCount.set(maxRetryCount); | ||
} | ||
|
||
public void incrementMaxRetryCount() { | ||
this.maxRetryCount.getAndIncrement(); | ||
} | ||
|
||
public long getTotalNumberOfRequests() { | ||
return this.totalNumberOfRequests.get(); | ||
} | ||
|
||
public void setTotalNumberOfRequests(long totalNumberOfRequests) { | ||
this.totalNumberOfRequests.set(totalNumberOfRequests); | ||
} | ||
|
||
public void incrementTotalNumberOfRequests() { | ||
this.totalNumberOfRequests.getAndIncrement(); | ||
} | ||
|
||
public Map<String, AbfsBackoffMetrics> getMetricsMap() { | ||
return metricsMap; | ||
} | ||
|
||
public long getNumberOfRequestsSucceededWithoutRetrying() { | ||
return this.numberOfRequestsSucceededWithoutRetrying.get(); | ||
} | ||
|
||
public void setNumberOfRequestsSucceededWithoutRetrying(long numberOfRequestsSucceededWithoutRetrying) { | ||
this.numberOfRequestsSucceededWithoutRetrying.set(numberOfRequestsSucceededWithoutRetrying); | ||
} | ||
|
||
public void incrementNumberOfRequestsSucceededWithoutRetrying() { | ||
this.numberOfRequestsSucceededWithoutRetrying.getAndIncrement(); | ||
} | ||
|
||
public long getNumberOfRequestsFailed() { | ||
return this.numberOfRequestsFailed.get(); | ||
} | ||
|
||
public void setNumberOfRequestsFailed(long numberOfRequestsFailed) { | ||
this.numberOfRequestsFailed.set(numberOfRequestsFailed); | ||
} | ||
|
||
public void incrementNumberOfRequestsFailed() { | ||
this.numberOfRequestsFailed.getAndIncrement(); | ||
} | ||
|
||
public long getNumberOfNetworkFailedRequests() { | ||
return this.numberOfNetworkFailedRequests.get(); | ||
} | ||
|
||
public void setNumberOfNetworkFailedRequests(long numberOfNetworkFailedRequests) { | ||
this.numberOfNetworkFailedRequests.set(numberOfNetworkFailedRequests); | ||
} | ||
|
||
public void incrementNumberOfNetworkFailedRequests() { | ||
this.numberOfNetworkFailedRequests.getAndIncrement(); | ||
} | ||
|
||
/* | ||
Acronyms :- | ||
1.RCTSI :- Request count that succeeded in x retries | ||
2.MMA :- Min Max Average (This refers to the backoff or sleep time between 2 requests) | ||
3.s :- seconds | ||
4.BWT :- Number of Bandwidth throttled requests | ||
5.IT :- Number of IOPS throttled requests | ||
6.OT :- Number of Other throttled requests | ||
7.NFR :- Number of requests which failed due to network errors | ||
8.%RT :- Percentage of requests that are throttled | ||
9.TRNR :- Total number of requests which succeeded without retrying | ||
10.TRF :- Total number of requests which failed | ||
11.TR :- Total number of requests which were made | ||
12.MRC :- Max retry count across all requests | ||
*/ | ||
@Override | ||
public String toString() { | ||
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. What is the goal here? something for logging or to submit as a header in a message. And do you expect the log entries to be structured enough for parsing later? If so, I would propose having a toLogString() message and highlight that it must not be changed |
||
StringBuilder metricString = new StringBuilder(); | ||
long totalRequestsThrottled = getNumberOfBandwidthThrottledRequests() | ||
+ getNumberOfIOPSThrottledRequests() | ||
+ getNumberOfOtherThrottledRequests(); | ||
double percentageOfRequestsThrottled = | ||
((double) totalRequestsThrottled / getTotalNumberOfRequests()) * HUNDRED; | ||
for (Map.Entry<String, AbfsBackoffMetrics> entry : metricsMap.entrySet()) { | ||
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. this is going to blow up if you used the string constructor.
|
||
metricString.append("$RCTSI$_").append(entry.getKey()) | ||
.append("R_").append("=") | ||
.append(entry.getValue().getNumberOfRequestsSucceeded()); | ||
long totalRequests = entry.getValue().getTotalRequests(); | ||
if (totalRequests > 0) { | ||
metricString.append("$MMA$_").append(entry.getKey()) | ||
.append("R_").append("=") | ||
.append(String.format("%.3f", | ||
(double) entry.getValue().getMinBackoff() / THOUSAND)) | ||
.append("s") | ||
.append(String.format("%.3f", | ||
(double) entry.getValue().getMaxBackoff() / THOUSAND)) | ||
.append("s") | ||
.append(String.format("%.3f", | ||
((double) entry.getValue().getTotalBackoff() / totalRequests) | ||
/ THOUSAND)) | ||
.append("s"); | ||
} else { | ||
metricString.append("$MMA$_").append(entry.getKey()) | ||
.append("R_").append("=0s"); | ||
} | ||
} | ||
metricString.append("$BWT=") | ||
.append(getNumberOfBandwidthThrottledRequests()) | ||
.append("$IT=") | ||
.append(getNumberOfIOPSThrottledRequests()) | ||
.append("$OT=") | ||
.append(getNumberOfOtherThrottledRequests()) | ||
.append("$RT=") | ||
.append(String.format("%.3f", percentageOfRequestsThrottled)) | ||
.append("$NFR=") | ||
.append(getNumberOfNetworkFailedRequests()) | ||
.append("$TRNR=") | ||
.append(getNumberOfRequestsSucceededWithoutRetrying()) | ||
.append("$TRF=") | ||
.append(getNumberOfRequestsFailed()) | ||
.append("$TR=") | ||
.append(getTotalNumberOfRequests()) | ||
.append("$MRC=") | ||
.append(getMaxRetryCount()); | ||
|
||
return metricString + ""; | ||
} | ||
} | ||
|
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.
can you add javadocs for each of these and the class.