|
@@ -18,23 +18,50 @@
|
|
|
|
|
|
package org.apache.hadoop.fs.azurebfs.services;
|
|
|
|
|
|
+import java.util.concurrent.atomic.AtomicLong;
|
|
|
+
|
|
|
+import org.apache.hadoop.fs.statistics.IOStatistics;
|
|
|
+import org.apache.hadoop.fs.statistics.StreamStatisticNames;
|
|
|
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
|
|
|
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
|
|
|
+
|
|
|
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.ACTION_HTTP_GET_REQUEST;
|
|
|
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MEAN;
|
|
|
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore;
|
|
|
+
|
|
|
/**
|
|
|
* Stats for the AbfsInputStream.
|
|
|
*/
|
|
|
public class AbfsInputStreamStatisticsImpl
|
|
|
implements AbfsInputStreamStatistics {
|
|
|
- private long seekOperations;
|
|
|
- private long forwardSeekOperations;
|
|
|
- private long backwardSeekOperations;
|
|
|
- private long bytesRead;
|
|
|
- private long bytesSkippedOnSeek;
|
|
|
- private long bytesBackwardsOnSeek;
|
|
|
- private long seekInBuffer;
|
|
|
- private long readOperations;
|
|
|
- private long bytesReadFromBuffer;
|
|
|
- private long remoteReadOperations;
|
|
|
- private long readAheadBytesRead;
|
|
|
- private long remoteBytesRead;
|
|
|
+
|
|
|
+ private final IOStatisticsStore ioStatisticsStore = iostatisticsStore()
|
|
|
+ .withCounters(
|
|
|
+ StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS,
|
|
|
+ StreamStatisticNames.STREAM_READ_SEEK_FORWARD_OPERATIONS,
|
|
|
+ StreamStatisticNames.STREAM_READ_SEEK_BACKWARD_OPERATIONS,
|
|
|
+ StreamStatisticNames.STREAM_READ_BYTES,
|
|
|
+ StreamStatisticNames.STREAM_READ_SEEK_BYTES_SKIPPED,
|
|
|
+ StreamStatisticNames.STREAM_READ_OPERATIONS,
|
|
|
+ StreamStatisticNames.STREAM_READ_SEEK_BYTES_BACKWARDS,
|
|
|
+ StreamStatisticNames.SEEK_IN_BUFFER,
|
|
|
+ StreamStatisticNames.BYTES_READ_BUFFER,
|
|
|
+ StreamStatisticNames.REMOTE_READ_OP,
|
|
|
+ StreamStatisticNames.READ_AHEAD_BYTES_READ,
|
|
|
+ StreamStatisticNames.REMOTE_BYTES_READ
|
|
|
+ )
|
|
|
+ .withDurationTracking(ACTION_HTTP_GET_REQUEST)
|
|
|
+ .build();
|
|
|
+
|
|
|
+ /* Reference to the atomic counter for frequently updated counters to avoid
|
|
|
+ * cost of the map lookup on every increment.
|
|
|
+ */
|
|
|
+ private final AtomicLong bytesRead =
|
|
|
+ ioStatisticsStore.getCounterReference(StreamStatisticNames.STREAM_READ_BYTES);
|
|
|
+ private final AtomicLong readOps =
|
|
|
+ ioStatisticsStore.getCounterReference(StreamStatisticNames.STREAM_READ_OPERATIONS);
|
|
|
+ private final AtomicLong seekOps =
|
|
|
+ ioStatisticsStore.getCounterReference(StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS);
|
|
|
|
|
|
/**
|
|
|
* Seek backwards, incrementing the seek and backward seek counters.
|
|
@@ -44,9 +71,9 @@ public class AbfsInputStreamStatisticsImpl
|
|
|
*/
|
|
|
@Override
|
|
|
public void seekBackwards(long negativeOffset) {
|
|
|
- seekOperations++;
|
|
|
- backwardSeekOperations++;
|
|
|
- bytesBackwardsOnSeek -= negativeOffset;
|
|
|
+ seekOps.incrementAndGet();
|
|
|
+ ioStatisticsStore.incrementCounter(StreamStatisticNames.STREAM_READ_SEEK_BACKWARD_OPERATIONS);
|
|
|
+ ioStatisticsStore.incrementCounter(StreamStatisticNames.STREAM_READ_SEEK_BYTES_BACKWARDS, negativeOffset);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -58,11 +85,9 @@ public class AbfsInputStreamStatisticsImpl
|
|
|
*/
|
|
|
@Override
|
|
|
public void seekForwards(long skipped) {
|
|
|
- seekOperations++;
|
|
|
- forwardSeekOperations++;
|
|
|
- if (skipped > 0) {
|
|
|
- bytesSkippedOnSeek += skipped;
|
|
|
- }
|
|
|
+ seekOps.incrementAndGet();
|
|
|
+ ioStatisticsStore.incrementCounter(StreamStatisticNames.STREAM_READ_SEEK_FORWARD_OPERATIONS);
|
|
|
+ ioStatisticsStore.incrementCounter(StreamStatisticNames.STREAM_READ_SEEK_BYTES_SKIPPED, skipped);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -90,9 +115,7 @@ public class AbfsInputStreamStatisticsImpl
|
|
|
*/
|
|
|
@Override
|
|
|
public void bytesRead(long bytes) {
|
|
|
- if (bytes > 0) {
|
|
|
- bytesRead += bytes;
|
|
|
- }
|
|
|
+ bytesRead.addAndGet(bytes);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -104,9 +127,7 @@ public class AbfsInputStreamStatisticsImpl
|
|
|
*/
|
|
|
@Override
|
|
|
public void bytesReadFromBuffer(long bytes) {
|
|
|
- if (bytes > 0) {
|
|
|
- bytesReadFromBuffer += bytes;
|
|
|
- }
|
|
|
+ ioStatisticsStore.incrementCounter(StreamStatisticNames.BYTES_READ_BUFFER, bytes);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -116,18 +137,15 @@ public class AbfsInputStreamStatisticsImpl
|
|
|
*/
|
|
|
@Override
|
|
|
public void seekInBuffer() {
|
|
|
- seekInBuffer++;
|
|
|
+ ioStatisticsStore.incrementCounter(StreamStatisticNames.SEEK_IN_BUFFER);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* A {@code read(byte[] buf, int off, int len)} operation has started.
|
|
|
- *
|
|
|
- * @param pos starting position of the read.
|
|
|
- * @param len length of bytes to read.
|
|
|
*/
|
|
|
@Override
|
|
|
- public void readOperationStarted(long pos, long len) {
|
|
|
- readOperations++;
|
|
|
+ public void readOperationStarted() {
|
|
|
+ readOps.incrementAndGet();
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -137,9 +155,7 @@ public class AbfsInputStreamStatisticsImpl
|
|
|
*/
|
|
|
@Override
|
|
|
public void readAheadBytesRead(long bytes) {
|
|
|
- if (bytes > 0) {
|
|
|
- readAheadBytesRead += bytes;
|
|
|
- }
|
|
|
+ ioStatisticsStore.incrementCounter(StreamStatisticNames.READ_AHEAD_BYTES_READ, bytes);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -149,9 +165,7 @@ public class AbfsInputStreamStatisticsImpl
|
|
|
*/
|
|
|
@Override
|
|
|
public void remoteBytesRead(long bytes) {
|
|
|
- if (bytes > 0) {
|
|
|
- remoteBytesRead += bytes;
|
|
|
- }
|
|
|
+ ioStatisticsStore.incrementCounter(StreamStatisticNames.REMOTE_BYTES_READ, bytes);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -161,55 +175,88 @@ public class AbfsInputStreamStatisticsImpl
|
|
|
*/
|
|
|
@Override
|
|
|
public void remoteReadOperation() {
|
|
|
- remoteReadOperations++;
|
|
|
+ ioStatisticsStore.incrementCounter(StreamStatisticNames.REMOTE_READ_OP);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Getter for IOStatistics instance used.
|
|
|
+ * @return IOStatisticsStore instance which extends IOStatistics.
|
|
|
+ */
|
|
|
+ @Override
|
|
|
+ public IOStatistics getIOStatistics() {
|
|
|
+ return ioStatisticsStore;
|
|
|
}
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
public long getSeekOperations() {
|
|
|
- return seekOperations;
|
|
|
+ return ioStatisticsStore.counters().get(StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS);
|
|
|
}
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
public long getForwardSeekOperations() {
|
|
|
- return forwardSeekOperations;
|
|
|
+ return ioStatisticsStore.counters().get(StreamStatisticNames.STREAM_READ_SEEK_FORWARD_OPERATIONS);
|
|
|
}
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
public long getBackwardSeekOperations() {
|
|
|
- return backwardSeekOperations;
|
|
|
+ return ioStatisticsStore.counters().get(StreamStatisticNames.STREAM_READ_SEEK_BACKWARD_OPERATIONS);
|
|
|
}
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
public long getBytesRead() {
|
|
|
- return bytesRead;
|
|
|
+ return ioStatisticsStore.counters().get(StreamStatisticNames.STREAM_READ_BYTES);
|
|
|
}
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
public long getBytesSkippedOnSeek() {
|
|
|
- return bytesSkippedOnSeek;
|
|
|
+ return ioStatisticsStore.counters().get(StreamStatisticNames.STREAM_READ_SEEK_BYTES_SKIPPED);
|
|
|
}
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
public long getBytesBackwardsOnSeek() {
|
|
|
- return bytesBackwardsOnSeek;
|
|
|
+ return ioStatisticsStore.counters().get(StreamStatisticNames.STREAM_READ_SEEK_BYTES_BACKWARDS);
|
|
|
}
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
public long getSeekInBuffer() {
|
|
|
- return seekInBuffer;
|
|
|
+ return ioStatisticsStore.counters().get(StreamStatisticNames.SEEK_IN_BUFFER);
|
|
|
+
|
|
|
}
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
public long getReadOperations() {
|
|
|
- return readOperations;
|
|
|
+ return ioStatisticsStore.counters().get(StreamStatisticNames.STREAM_READ_OPERATIONS);
|
|
|
}
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
public long getBytesReadFromBuffer() {
|
|
|
- return bytesReadFromBuffer;
|
|
|
+ return ioStatisticsStore.counters().get(StreamStatisticNames.BYTES_READ_BUFFER);
|
|
|
}
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
public long getRemoteReadOperations() {
|
|
|
- return remoteReadOperations;
|
|
|
+ return ioStatisticsStore.counters().get(StreamStatisticNames.REMOTE_READ_OP);
|
|
|
}
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
public long getReadAheadBytesRead() {
|
|
|
- return readAheadBytesRead;
|
|
|
+ return ioStatisticsStore.counters().get(StreamStatisticNames.READ_AHEAD_BYTES_READ);
|
|
|
}
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
public long getRemoteBytesRead() {
|
|
|
- return remoteBytesRead;
|
|
|
+ return ioStatisticsStore.counters().get(StreamStatisticNames.REMOTE_BYTES_READ);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Getter for the mean value of the time taken to complete a HTTP GET
|
|
|
+ * request by AbfsInputStream.
|
|
|
+ * @return mean value.
|
|
|
+ */
|
|
|
+ @VisibleForTesting
|
|
|
+ public double getActionHttpGetRequest() {
|
|
|
+ return ioStatisticsStore.meanStatistics().
|
|
|
+ get(ACTION_HTTP_GET_REQUEST + SUFFIX_MEAN).mean();
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -223,18 +270,7 @@ public class AbfsInputStreamStatisticsImpl
|
|
|
public String toString() {
|
|
|
final StringBuilder sb = new StringBuilder(
|
|
|
"StreamStatistics{");
|
|
|
- sb.append(", SeekOperations=").append(seekOperations);
|
|
|
- sb.append(", ForwardSeekOperations=").append(forwardSeekOperations);
|
|
|
- sb.append(", BackwardSeekOperations=").append(backwardSeekOperations);
|
|
|
- sb.append(", BytesSkippedOnSeek=").append(bytesSkippedOnSeek);
|
|
|
- sb.append(", BytesBackwardsOnSeek=").append(bytesBackwardsOnSeek);
|
|
|
- sb.append(", seekInBuffer=").append(seekInBuffer);
|
|
|
- sb.append(", BytesRead=").append(bytesRead);
|
|
|
- sb.append(", ReadOperations=").append(readOperations);
|
|
|
- sb.append(", bytesReadFromBuffer=").append(bytesReadFromBuffer);
|
|
|
- sb.append(", remoteReadOperations=").append(remoteReadOperations);
|
|
|
- sb.append(", readAheadBytesRead=").append(readAheadBytesRead);
|
|
|
- sb.append(", remoteBytesRead=").append(remoteBytesRead);
|
|
|
+ sb.append(ioStatisticsStore.toString());
|
|
|
sb.append('}');
|
|
|
return sb.toString();
|
|
|
}
|