Skip to content
Merged
Show file tree
Hide file tree
Changes from 3 commits
Commits
Show all changes
38 commits
Select commit Hold shift + click to select a range
f2e8409
AbfsBackoffMetrics new implementation
Sep 16, 2024
10906cd
ABFS Backoff metrics sample usecase implementation
Sep 18, 2024
db9ec05
ABFS Backoff metrics using IOStatistics
Sep 19, 2024
6531e83
AbfsBackoff metrics changes
Sep 24, 2024
7c6e0ff
Move AbfsBackoffMetrics to service directory
Oct 1, 2024
b7f7fbe
Hadoop-19311 Abfs Backoff and read footer metrics using IOStatistics
Oct 17, 2024
dabc171
Added license to newly added file
Oct 17, 2024
3692134
pulled latest trunk code
Oct 17, 2024
c24f0eb
Revert AbfsClient changes
Oct 17, 2024
8a87b39
Abfs Client format fix
Oct 17, 2024
fcaf09f
Files refactor
Oct 17, 2024
04c45a6
Fix checkstyle and added license
Oct 18, 2024
53b6623
Fix checkstyle errors
Oct 18, 2024
231a434
Added java doc for each classes and methods
Oct 18, 2024
e56e2e0
Added test case for back off metrics
Oct 18, 2024
dfbc356
Fix magic value checkstyle error
Oct 19, 2024
26ec83e
abfs client unnecessary changes reverted
bhattmanish98 Oct 21, 2024
5ecce29
Merge branch 'trunk' of https://github.com/bhattmanish98/hadoop into …
Oct 23, 2024
361af92
Test case fix
Oct 23, 2024
8feccf7
Merge branch 'trunk' of https://github.com/apache/hadoop into HADOOP-…
Nov 8, 2024
327a0a6
Comments for read footer metrics toString method
Nov 8, 2024
449b2b6
Changes after review
Nov 18, 2024
e4b680a
Checkstype fixes
Nov 18, 2024
3cc57f7
Add private constructor in StringUtils class
Nov 18, 2024
24e49f8
Merged latest trunk code
Dec 9, 2024
f636f41
Changes based on comments given
Dec 9, 2024
154a4ae
Use of Mean statistics in read footer metrics
Dec 9, 2024
7d80bd2
Rename update read method name
Dec 10, 2024
4feaac7
Fixed test cases and checkstyle failures
Dec 10, 2024
bc6f9cb
Read Length correction in test case
Dec 10, 2024
98c8716
Added metrics config in account template
Dec 11, 2024
bd4b131
Move constant to metrics constant file
Dec 17, 2024
b9ec4aa
Created common method for getPrecisionMetrics
bhattmanish98 Dec 18, 2024
3702356
Created common method for getPrecisionMetrics
bhattmanish98 Dec 18, 2024
d4f1e09
Removed entire package name from comments in retryValue and fileType …
bhattmanish98 Dec 18, 2024
65b64a9
Add commit to retrigger yetus build
bhattmanish98 Dec 18, 2024
ef2b169
Merge branch 'trunk' of https://github.com/apache/hadoop into HADOOP-…
bhattmanish98 Jan 6, 2025
5462967
Changes as well comments given
bhattmanish98 Jan 6, 2025
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -26,12 +26,12 @@
* Enum representing various ABFS read footer metrics.
*/
public enum AbfsReadFooterMetricsEnum {
TOTAL_FILES("totalFiles", "Total files in a file system", FILE, TYPE_COUNTER),
TOTAL_FILES("totalFiles", "Total files read", FILE, TYPE_COUNTER),
FILE_LENGTH("fileLength", "File length", FILE, TYPE_GAUGE),
SIZE_READ_BY_FIRST_READ("sizeReadByFirstRead", "Size read by first read", FILE, TYPE_GAUGE),
OFFSET_DIFF_BETWEEN_FIRST_AND_SECOND_READ("offsetDiffBetweenFirstAndSecondRead", "Offset difference between first and second read", FILE, TYPE_GAUGE),
READ_LEN_REQUESTED("readLenRequested", "Read length requested", FILE, TYPE_GAUGE),
READ_COUNT("readCount", "Read count", FILE, TYPE_COUNTER),
READ_COUNT("readCount", "Number of total reads", FILE, TYPE_COUNTER),
FIRST_OFFSET_DIFF("firstOffsetDiff", "First offset difference", FILE, TYPE_GAUGE),
SECOND_OFFSET_DIFF("secondOffsetDiff", "Second offset difference", FILE, TYPE_GAUGE);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import org.apache.hadoop.fs.azurebfs.statistics.AbstractAbfsStatisticsSource;
import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;

import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING;
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.HUNDRED;
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.THOUSAND;
import static org.apache.hadoop.fs.azurebfs.constants.MetricsConstants.RETRY;
Expand All @@ -55,6 +56,7 @@
import static org.apache.hadoop.fs.azurebfs.enums.RetryValue.TWENTY_FIVE_AND_ABOVE;
import static org.apache.hadoop.fs.azurebfs.enums.StatisticTypeEnum.TYPE_COUNTER;
import static org.apache.hadoop.fs.azurebfs.enums.StatisticTypeEnum.TYPE_GAUGE;
import static org.apache.hadoop.fs.azurebfs.utils.StringUtils.formatWithPrecision;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore;

/**
Expand Down Expand Up @@ -202,51 +204,68 @@ public void setMetricValue(AbfsBackoffMetricsEnum metric, long value) {
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() {
if (getMetricValue(TOTAL_NUMBER_OF_REQUESTS) == 0) {
return "";
private void getRetryMetrics(StringBuilder metricBuilder) {
for (RetryValue retryCount : RETRY_LIST) {
long totalRequests = getMetricValue(TOTAL_REQUESTS, retryCount);
metricBuilder.append("$RCTSI$_").append(retryCount.getValue())
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we can create a separate constants file for all these acronyms to make it easier to modify if needed later

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1

All these constants are used specifically for Metric work. They qualify to be added into MetricConstants class. It make sesne to retain that class and add all these constants there itself.

.append("R=").append(getMetricValue(NUMBER_OF_REQUESTS_SUCCEEDED, retryCount));

if (totalRequests > 0) {
metricBuilder.append("$MMA$_").append(retryCount.getValue())
.append("R=").append(formatWithPrecision((double) getMetricValue(MIN_BACK_OFF, retryCount) / THOUSAND)).append("s")
.append(formatWithPrecision((double) getMetricValue(MAX_BACK_OFF, retryCount) / THOUSAND)).append("s")
.append(formatWithPrecision((double) getMetricValue(TOTAL_BACK_OFF, retryCount) / totalRequests / THOUSAND)).append("s");
} else {
metricBuilder.append("$MMA$_").append(retryCount.getValue()).append("R=0s");
}
}
StringBuilder metricString = new StringBuilder();
}

/*
Acronyms :-
1.BWT :- Number of Bandwidth throttled requests
2.IT :- Number of IOPS throttled requests
3.OT :- Number of Other throttled requests
4.NFR :- Number of requests which failed due to network errors
5.%RT :- Percentage of requests that are throttled
6.TRNR :- Total number of requests which succeeded without retrying
7.TRF :- Total number of requests which failed
8.TR :- Total number of requests which were made
9.MRC :- Max retry count across all requests
*/
private void getMmaMetrics(StringBuilder metricBuilder) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This function would only be catering to the backoff or sleep time (like mentioned above for MMA) or the number of requests as well (like here in the function)?

long totalRequestsThrottled = getMetricValue(NUMBER_OF_NETWORK_FAILED_REQUESTS)
+ getMetricValue(NUMBER_OF_IOPS_THROTTLED_REQUESTS)
+ getMetricValue(NUMBER_OF_OTHER_THROTTLED_REQUESTS)
+ getMetricValue(NUMBER_OF_BANDWIDTH_THROTTLED_REQUESTS);
double percentageOfRequestsThrottled = ((double) totalRequestsThrottled / getMetricValue(TOTAL_NUMBER_OF_REQUESTS)) * HUNDRED;

for (RetryValue retryCount : RETRY_LIST) {
long totalRequests = getMetricValue(TOTAL_REQUESTS, retryCount);
metricString.append("$RCTSI$_").append(retryCount.getValue()).append("R=").append(getMetricValue(NUMBER_OF_REQUESTS_SUCCEEDED, retryCount));
if (totalRequests > 0) {
metricString.append("$MMA$_").append(retryCount.getValue()).append("R=")
.append(String.format("%.3f", (double) getMetricValue(MIN_BACK_OFF, retryCount) / THOUSAND)).append("s")
.append(String.format("%.3f", (double) getMetricValue(MAX_BACK_OFF, retryCount) / THOUSAND)).append("s")
.append(String.format("%.3f", (double) getMetricValue(TOTAL_BACK_OFF, retryCount) / totalRequests / THOUSAND)).append("s");
} else {
metricString.append("$MMA$_").append(retryCount.getValue()).append("R=0s");
}
}
metricString.append("$BWT=").append(getMetricValue(NUMBER_OF_BANDWIDTH_THROTTLED_REQUESTS))
metricBuilder.append("$BWT=").append(getMetricValue(NUMBER_OF_BANDWIDTH_THROTTLED_REQUESTS))
.append("$IT=").append(getMetricValue(NUMBER_OF_IOPS_THROTTLED_REQUESTS))
.append("$OT=").append(getMetricValue(NUMBER_OF_OTHER_THROTTLED_REQUESTS))
.append("$RT=").append(String.format("%.3f", percentageOfRequestsThrottled))
.append("$RT=").append(formatWithPrecision(percentageOfRequestsThrottled))
.append("$NFR=").append(getMetricValue(NUMBER_OF_NETWORK_FAILED_REQUESTS))
.append("$TRNR=").append(getMetricValue(NUMBER_OF_REQUESTS_SUCCEEDED_WITHOUT_RETRYING))
.append("$TRF=").append(getMetricValue(NUMBER_OF_REQUESTS_FAILED))
.append("$TR=").append(getMetricValue(TOTAL_NUMBER_OF_REQUESTS))
.append("$MRC=").append(getMetricValue(MAX_RETRY_COUNT));
}

return metricString.toString();
/**
* Retrieves the string representation of the metrics.
*
* @return the string representation of the metrics
*/
@Override
public String toString() {
if (getMetricValue(TOTAL_NUMBER_OF_REQUESTS) == 0) {
return EMPTY_STRING;
}
StringBuilder metricBuilder = new StringBuilder();
getRetryMetrics(metricBuilder);
getMmaMetrics(metricBuilder);
return metricBuilder.toString();
}

@VisibleForTesting
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@
import static org.apache.hadoop.fs.azurebfs.enums.FileType.NON_PARQUET;
import static org.apache.hadoop.fs.azurebfs.enums.StatisticTypeEnum.TYPE_COUNTER;
import static org.apache.hadoop.fs.azurebfs.enums.StatisticTypeEnum.TYPE_GAUGE;
import static org.apache.hadoop.fs.azurebfs.utils.StringUtils.formatWithPrecision;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore;

/**
Expand All @@ -58,7 +59,7 @@ public class AbfsReadFooterMetrics extends AbstractAbfsStatisticsSource {
/**
* Inner class to handle file type checks.
*/
private static final class CheckFileType {
private static final class FileTypeMetrics {
private final AtomicBoolean collectMetrics;
private final AtomicBoolean collectMetricsForNextRead;
private final AtomicBoolean collectLenMetrics;
Expand All @@ -68,7 +69,7 @@ private static final class CheckFileType {
private String sizeReadByFirstRead;
private String offsetDiffBetweenFirstAndSecondRead;

private CheckFileType() {
private FileTypeMetrics() {
collectMetrics = new AtomicBoolean(false);
collectMetricsForNextRead = new AtomicBoolean(false);
collectLenMetrics = new AtomicBoolean(false);
Expand Down Expand Up @@ -151,7 +152,7 @@ private FileType getFileType() {
}
}

private final Map<String, CheckFileType> checkFileMap = new HashMap<>();
private final Map<String, FileTypeMetrics> fileTypeMetricsMap = new HashMap<>();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

updateMap, updateReadMetrics call fileTypeMetricsMap.computeIfAbsent which is not threadsafe. Change it to ConcurrentHashMap

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Taken!


/**
* Constructor to initialize the IOStatisticsStore with counters and gauges.
Expand Down Expand Up @@ -230,7 +231,7 @@ public Long getTotalReadCount() {
* @param filePathIdentifier the file path identifier
*/
public void updateMap(String filePathIdentifier) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Where are the changes in AbfsInputStream method to call this method ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There is no change in AbfsInputStream related to this method.

checkFileMap.computeIfAbsent(filePathIdentifier, key -> new CheckFileType());
fileTypeMetricsMap.computeIfAbsent(filePathIdentifier, key -> new FileTypeMetrics());
}

/**
Expand All @@ -242,68 +243,104 @@ public void updateMap(String filePathIdentifier) {
* @param nextReadPos the position of the next read
*/
public void checkMetricUpdate(final String filePathIdentifier, final int len, final long contentLength, final long nextReadPos) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: (optional) method can be renamed to updateReadMetrics

CheckFileType checkFileType = checkFileMap.computeIfAbsent(filePathIdentifier, key -> new CheckFileType());
if (checkFileType.getReadCount() == 0 || (checkFileType.getReadCount() >= 1 && checkFileType.getCollectMetrics())) {
updateMetrics(checkFileType, len, contentLength, nextReadPos);
FileTypeMetrics fileTypeMetrics = fileTypeMetricsMap.computeIfAbsent(filePathIdentifier, key -> new FileTypeMetrics());
if (fileTypeMetrics.getReadCount() == 0 || (fileTypeMetrics.getReadCount() >= 1 && fileTypeMetrics.getCollectMetrics())) {
updateMetrics(fileTypeMetrics, len, contentLength, nextReadPos);
}
}

/**
* Updates metrics for a specific file identified by filePathIdentifier.
*
* @param checkFileType File metadata to know file type.
* @param fileTypeMetrics File metadata to know file type.
* @param len The length of the read operation.
* @param contentLength The total content length of the file.
* @param nextReadPos The position of the next read operation.
*/
private void updateMetrics(CheckFileType checkFileType, int len, long contentLength, long nextReadPos) {
private void updateMetrics(FileTypeMetrics fileTypeMetrics, int len, long contentLength, long nextReadPos) {
synchronized (this) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

do we need synchronized block here? readCount is of type AtomicLong and hence threadsafe

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Make sense, removed synchronized block.

checkFileType.incrementReadCount();
fileTypeMetrics.incrementReadCount();
}

long readCount = checkFileType.getReadCount();
long readCount = fileTypeMetrics.getReadCount();

if (readCount == 1) {
handleFirstRead(checkFileType, nextReadPos, len, contentLength);
handleFirstRead(fileTypeMetrics, nextReadPos, len, contentLength);
} else if (readCount == 2) {
handleSecondRead(checkFileType, nextReadPos, len, contentLength);
handleSecondRead(fileTypeMetrics, nextReadPos, len, contentLength);
} else {
handleFurtherRead(checkFileType, len);
handleFurtherRead(fileTypeMetrics, len);
}
}

private void handleFirstRead(CheckFileType checkFileType, long nextReadPos, int len, long contentLength) {
/**
* Handles the first read operation by checking if the current read position is near the end of the file.
* If it is, updates the {@link FileTypeMetrics} object to enable metrics collection and records the first read's
* offset and size.
*
* @param fileTypeMetrics The {@link FileTypeMetrics} object to update with metrics and read details.
* @param nextReadPos The position where the next read will start.
* @param len The length of the current read operation.
* @param contentLength The total length of the file content.
*/
private void handleFirstRead(FileTypeMetrics fileTypeMetrics, long nextReadPos, int len, long contentLength) {
if (nextReadPos >= contentLength - (long) Integer.parseInt(FOOTER_LENGTH) * ONE_KB) {
checkFileType.setCollectMetrics(true);
checkFileType.setCollectMetricsForNextRead(true);
checkFileType.setOffsetOfFirstRead(nextReadPos);
checkFileType.setSizeReadByFirstRead(len + "_" + Math.abs(contentLength - nextReadPos));
fileTypeMetrics.setCollectMetrics(true);
fileTypeMetrics.setCollectMetricsForNextRead(true);
fileTypeMetrics.setOffsetOfFirstRead(nextReadPos);
fileTypeMetrics.setSizeReadByFirstRead(len + "_" + Math.abs(contentLength - nextReadPos));
}
}

private void handleSecondRead(CheckFileType checkFileType, long nextReadPos, int len, long contentLength) {
if (checkFileType.getCollectMetricsForNextRead()) {
long offsetDiff = Math.abs(nextReadPos - checkFileType.getOffsetOfFirstRead());
checkFileType.setOffsetDiffBetweenFirstAndSecondRead(len + "_" + offsetDiff);
checkFileType.setCollectLenMetrics(true);
checkFileType.updateFileType();
updateMetricsData(checkFileType, len, contentLength);
/**
* Handles the second read operation by checking if metrics collection is enabled for the next read.
* If it is, calculates the offset difference between the first and second reads, updates the {@link FileTypeMetrics}
* object with this information, and sets the file type. Then, updates the metrics data.
*
* @param fileTypeMetrics The {@link FileTypeMetrics} object to update with metrics and read details.
* @param nextReadPos The position where the next read will start.
* @param len The length of the current read operation.
* @param contentLength The total length of the file content.
*/
private void handleSecondRead(FileTypeMetrics fileTypeMetrics, long nextReadPos, int len, long contentLength) {
if (fileTypeMetrics.getCollectMetricsForNextRead()) {
long offsetDiff = Math.abs(nextReadPos - fileTypeMetrics.getOffsetOfFirstRead());
fileTypeMetrics.setOffsetDiffBetweenFirstAndSecondRead(len + "_" + offsetDiff);
fileTypeMetrics.setCollectLenMetrics(true);
fileTypeMetrics.updateFileType();
updateMetricsData(fileTypeMetrics, len, contentLength);
}
}

private synchronized void handleFurtherRead(CheckFileType checkFileType, int len) {
if (checkFileType.getCollectLenMetrics() && checkFileType.getFileType() != null) {
FileType fileType = checkFileType.getFileType();
/**
* Handles further read operations beyond the second read. If metrics collection is enabled and the file type is set,
* updates the read length requested and increments the read count for the specific file type.
*
* @param fileTypeMetrics The {@link FileTypeMetrics} object containing metrics and read details.
* @param len The length of the current read operation.
*/
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

do we need synchronized here?

The call to addMeanStatisticSample does not require synchronization since IOStatisticsStoreImpl.meanStatisticMap is a ConcurrentHashMap and MeanStatistic.addSample is already synchronized.

If synchronization is needed for any other reason, it should be synchronized on fileTypeMetrics instead of this

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, we need synchronization only to update Mean Statistic, and as rightly mentioned it is already Concurrent so we don't need this. Reverted this.

private synchronized void handleFurtherRead(FileTypeMetrics fileTypeMetrics, int len) {
if (fileTypeMetrics.getCollectLenMetrics() && fileTypeMetrics.getFileType() != null) {
FileType fileType = fileTypeMetrics.getFileType();
updateMetricValue(fileType, READ_LEN_REQUESTED, len);
incrementMetricValue(fileType, READ_COUNT);
}
}

private synchronized void updateMetricsData(CheckFileType checkFileType, int len, long contentLength) {
long sizeReadByFirstRead = Long.parseLong(checkFileType.getSizeReadByFirstRead().split("_")[0]);
long firstOffsetDiff = Long.parseLong(checkFileType.getSizeReadByFirstRead().split("_")[1]);
long secondOffsetDiff = Long.parseLong(checkFileType.getOffsetDiffBetweenFirstAndSecondRead().split("_")[1]);
FileType fileType = checkFileType.getFileType();
/**
* Updates the metrics data for a specific file identified by the {@link FileTypeMetrics} object.
* This method calculates and updates various metrics such as read length requested, file length,
* size read by the first read, and offset differences between reads.
*
* @param fileTypeMetrics The {@link FileTypeMetrics} object containing metrics and read details.
* @param len The length of the current read operation.
* @param contentLength The total length of the file content.
*/
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

synchronized on fileTypeMetrics instead on this

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As mentioned above, no need of this, reverted this.

private synchronized void updateMetricsData(FileTypeMetrics fileTypeMetrics, int len, long contentLength) {
long sizeReadByFirstRead = Long.parseLong(fileTypeMetrics.getSizeReadByFirstRead().split("_")[0]);
long firstOffsetDiff = Long.parseLong(fileTypeMetrics.getSizeReadByFirstRead().split("_")[1]);
long secondOffsetDiff = Long.parseLong(fileTypeMetrics.getOffsetDiffBetweenFirstAndSecondRead().split("_")[1]);
FileType fileType = fileTypeMetrics.getFileType();

updateMetricValue(fileType, READ_LEN_REQUESTED, len + sizeReadByFirstRead);
updateMetricValue(fileType, FILE_LENGTH, contentLength);
Expand All @@ -314,39 +351,27 @@ private synchronized void updateMetricsData(CheckFileType checkFileType, int len
incrementMetricValue(fileType, TOTAL_FILES);
}

/**
* Returns the read footer metrics for a given file type.
*
* @param fileType the type of the file
* @return the read footer metrics as a string
*/
public String getReadFooterMetrics(FileType fileType) {
StringBuilder readFooterMetric = new StringBuilder();
appendMetrics(readFooterMetric, fileType);
return readFooterMetric.toString();
}

private void appendMetrics(StringBuilder metricBuilder, FileType fileType) {
long totalFiles = getMetricValue(fileType, TOTAL_FILES);
long readCount = getMetricValue(fileType, READ_COUNT);
if (totalFiles <= 0 || readCount <= 0) {
return;
}

String sizeReadByFirstRead = String.format("%.3f", getMetricValue(fileType, SIZE_READ_BY_FIRST_READ) / (double) totalFiles);
String offsetDiffBetweenFirstAndSecondRead = String.format("%.3f",
getMetricValue(fileType, OFFSET_DIFF_BETWEEN_FIRST_AND_SECOND_READ) / (double) totalFiles);
String sizeReadByFirstRead = formatWithPrecision(getMetricValue(fileType, SIZE_READ_BY_FIRST_READ) / (double) totalFiles);
String offsetDiffBetweenFirstAndSecondRead = formatWithPrecision(getMetricValue(fileType,
OFFSET_DIFF_BETWEEN_FIRST_AND_SECOND_READ) / (double) totalFiles);

if (NON_PARQUET.equals(fileType)) {
sizeReadByFirstRead += "_" + String.format("%.3f", getMetricValue(fileType, FIRST_OFFSET_DIFF) / (double) totalFiles);
offsetDiffBetweenFirstAndSecondRead += "_" + String.format("%.3f", getMetricValue(fileType, SECOND_OFFSET_DIFF) / (double) totalFiles);
sizeReadByFirstRead += "_" + formatWithPrecision(getMetricValue(fileType, FIRST_OFFSET_DIFF) / (double) totalFiles);
offsetDiffBetweenFirstAndSecondRead += "_" + formatWithPrecision(getMetricValue(fileType, SECOND_OFFSET_DIFF) / (double) totalFiles);
}

metricBuilder.append("$").append(fileType)
.append(":$FR=").append(sizeReadByFirstRead)
.append("$SR=").append(offsetDiffBetweenFirstAndSecondRead)
.append("$FL=").append(String.format("%.3f", getMetricValue(fileType, FILE_LENGTH) / (double) totalFiles))
.append("$RL=").append(String.format("%.3f", getMetricValue(fileType, READ_LEN_REQUESTED) / (double) readCount));
.append("$FL=").append(formatWithPrecision(getMetricValue(fileType, FILE_LENGTH) / (double) totalFiles))
.append("$RL=").append(formatWithPrecision(getMetricValue(fileType, READ_LEN_REQUESTED) / (double) readCount));
}

/**
Expand Down
Loading