-
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
Conversation
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
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.
Just one comment, more of a doubt.
Rest LGTM
if (abfsStore.getClient().isMetricCollectionEnabled()) { | ||
TracingContext tracingMetricContext = new TracingContext( | ||
clientCorrelationId, | ||
fileSystemId, FSOperationType.GET_ATTR, true, |
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.
Here for metric tracing context we are using fileSystemId only, where as the metric tracing context created in ABfsClient, we are suing hostname...
Any reason for this difference?
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 mainly because the fileystem id is private to the fileystem class, hence cannot be used in the client class for a unique identifier. Hence we have used hostname for the identifier in the client class. If this looks as a disparity, we can make both the tracing contexts use the hostname.
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.
Looks good this way only.
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.
have a look at the CommonAuditContext
; it includes things like jobid, kerberos principal and more...so letting you identify jobs and people/applications
Hi @steveloughran @mukund-thakur, requesting you to kindly review this PR. |
Hi @steveloughran @mukund-thakur @mehakmeet, requesting you to kindly review this PR. |
💔 -1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
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.
well this is quite the patch, isn't it!
commented.
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.HUNDRED; | ||
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.THOUSAND; | ||
|
||
public class AbfsBackoffMetrics { |
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.
= new ConcurrentHashMap<>(); | ||
|
||
public AbfsBackoffMetrics() { | ||
initializeMap(); |
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'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 S3AInstrumentation.InputStreamStatistics
as an example of this
this.numberOfNetworkFailedRequests = new AtomicLong(); | ||
} | ||
|
||
public AbfsBackoffMetrics(String retryCount) { |
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 only partially initialises the metrics
|
||
private AtomicLong numberOfRequestsSucceeded; | ||
|
||
private AtomicLong minBackoff; |
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.
are these timings? if so should include time unit in name or at least use millis and javadoc it
+ 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 comment
The reason will be displayed to describe this comment to others. Learn more.
this is going to blow up if you used the string constructor.
- consider adding a unit test calling .toString() immediately after creating the object.
- if you use IOStatistics, as I've proposed, you can just use ioStatisticsToString to print these things
|
||
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_KB; | ||
|
||
public class AbfsReadFooterMetrics { |
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, I think a lot of the numbers could be collected in IOStatisticsStore so easily printed and marshalled
|
||
|
||
@Override | ||
public String 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 think this should be an explicit method, not toString() which is generally assumed to be flor logging and lower cost.
if (abfsReadFooterMetrics.getIsParquetFile()) { | ||
isParquetList.add(abfsReadFooterMetrics); | ||
} else { | ||
if (abfsReadFooterMetrics.getReadCount() >= 2) { |
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.
so it only goes into the non parquet file if its read the footer more than once?
@@ -60,6 +61,9 @@ public enum AzureServiceErrorCode { | |||
private final String errorCode; | |||
private final int httpStatusCode; | |||
private final String errorMessage; | |||
|
|||
private static final Logger LOG1 = LoggerFactory.getLogger(AzureServiceErrorCode.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.
call it LOG
|
||
private AtomicLong numberOfRequestsFailed; | ||
|
||
private final Map<String, AbfsBackoffMetrics> metricsMap |
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 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.
@anmolanmol1234 you got any timeline for changes here? |
Hi @steveloughran, since the changes requested would need a design refactor and scale tests from our side, I would say July end sounds feasible to me. |
well lets that as a next step and we can merge this in? makes sense. but it will be your next piece of homework... |
Sure @steveloughran, sounds good. |
🎊 +1 overall
This message was automatically generated. |
ok, fix those 2 checkstyles
+1 pending that |
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.
+1
will merge once the build finishes
🎊 +1 overall
This message was automatically generated. |
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.
LGTM
+1
fixed in 3.5, submit a PR for this against 3.4. regarding my outstanding comments, i will create a followup |
Sure will do so. |
…apache#6314) Adds support for metric collection at the filesystem instance level. Metrics are pushed to the store upon the closure of a filesystem instance, encompassing all operations that utilized that specific instance. Collected Metrics: - Number of successful requests without any retries. - Count of requests that succeeded after a specified number of retries (x retries). - Request count subjected to throttling. - Number of requests that failed despite exhausting all retry attempts. etc. Implementation Details: Incorporated logic in the AbfsClient to facilitate metric pushing through an additional request. This occurs in scenarios where no requests are sent to the backend for a defined idle period. By implementing these enhancements, we ensure comprehensive monitoring and analysis of filesystem interactions, enabling a deeper understanding of success rates, retry scenarios, throttling instances, and exhaustive failure scenarios. Additionally, the AbfsClient logic ensures that metrics are proactively pushed even during idle periods, maintaining a continuous and accurate representation of filesystem performance. Contributed by Anmol Asrani
We have introduced support for metric collection at the filesystem instance level.
Metrics are pushed to the store upon the closure of a filesystem instance, encompassing all operations that utilized that specific instance.
Collected Metrics:
3.Request count subjected to throttling.
4.Number of requests that failed despite exhausting all retry attempts. etc.
Implementation Details:
By implementing these enhancements, we ensure comprehensive monitoring and analysis of filesystem interactions, enabling a deeper understanding of success rates, retry scenarios, throttling instances, and exhaustive failure scenarios. Additionally, the AbfsClient logic ensures that metrics are proactively pushed even during idle periods, maintaining a continuous and accurate representation of filesystem performance.