|
@@ -20,20 +20,23 @@ package org.apache.hadoop.fs.azurebfs.services;
|
|
|
|
|
|
import java.util.Timer;
|
|
import java.util.Timer;
|
|
import java.util.TimerTask;
|
|
import java.util.TimerTask;
|
|
|
|
+import java.util.concurrent.atomic.AtomicBoolean;
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
import java.util.concurrent.atomic.AtomicLong;
|
|
import java.util.concurrent.atomic.AtomicLong;
|
|
import java.util.concurrent.atomic.AtomicReference;
|
|
import java.util.concurrent.atomic.AtomicReference;
|
|
|
|
|
|
-import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
|
|
|
|
-import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
|
|
|
|
|
|
+import org.apache.hadoop.classification.VisibleForTesting;
|
|
|
|
+import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
|
|
|
|
+import org.apache.hadoop.util.Preconditions;
|
|
import org.apache.commons.lang3.StringUtils;
|
|
import org.apache.commons.lang3.StringUtils;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.LoggerFactory;
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
|
|
+import static org.apache.hadoop.util.Time.now;
|
|
|
|
+
|
|
class AbfsClientThrottlingAnalyzer {
|
|
class AbfsClientThrottlingAnalyzer {
|
|
private static final Logger LOG = LoggerFactory.getLogger(
|
|
private static final Logger LOG = LoggerFactory.getLogger(
|
|
AbfsClientThrottlingAnalyzer.class);
|
|
AbfsClientThrottlingAnalyzer.class);
|
|
- private static final int DEFAULT_ANALYSIS_PERIOD_MS = 10 * 1000;
|
|
|
|
private static final int MIN_ANALYSIS_PERIOD_MS = 1000;
|
|
private static final int MIN_ANALYSIS_PERIOD_MS = 1000;
|
|
private static final int MAX_ANALYSIS_PERIOD_MS = 30000;
|
|
private static final int MAX_ANALYSIS_PERIOD_MS = 30000;
|
|
private static final double MIN_ACCEPTABLE_ERROR_PERCENTAGE = .1;
|
|
private static final double MIN_ACCEPTABLE_ERROR_PERCENTAGE = .1;
|
|
@@ -50,42 +53,38 @@ class AbfsClientThrottlingAnalyzer {
|
|
private String name = null;
|
|
private String name = null;
|
|
private Timer timer = null;
|
|
private Timer timer = null;
|
|
private AtomicReference<AbfsOperationMetrics> blobMetrics = null;
|
|
private AtomicReference<AbfsOperationMetrics> blobMetrics = null;
|
|
|
|
+ private AtomicLong lastExecutionTime = null;
|
|
|
|
+ private final AtomicBoolean isOperationOnAccountIdle = new AtomicBoolean(false);
|
|
|
|
+ private AbfsConfiguration abfsConfiguration = null;
|
|
|
|
+ private boolean accountLevelThrottlingEnabled = true;
|
|
|
|
|
|
private AbfsClientThrottlingAnalyzer() {
|
|
private AbfsClientThrottlingAnalyzer() {
|
|
// hide default constructor
|
|
// hide default constructor
|
|
}
|
|
}
|
|
|
|
|
|
- /**
|
|
|
|
- * Creates an instance of the <code>AbfsClientThrottlingAnalyzer</code> class with
|
|
|
|
- * the specified name.
|
|
|
|
- *
|
|
|
|
- * @param name a name used to identify this instance.
|
|
|
|
- * @throws IllegalArgumentException if name is null or empty.
|
|
|
|
- */
|
|
|
|
- AbfsClientThrottlingAnalyzer(String name) throws IllegalArgumentException {
|
|
|
|
- this(name, DEFAULT_ANALYSIS_PERIOD_MS);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
/**
|
|
/**
|
|
* Creates an instance of the <code>AbfsClientThrottlingAnalyzer</code> class with
|
|
* Creates an instance of the <code>AbfsClientThrottlingAnalyzer</code> class with
|
|
* the specified name and period.
|
|
* the specified name and period.
|
|
*
|
|
*
|
|
* @param name A name used to identify this instance.
|
|
* @param name A name used to identify this instance.
|
|
- * @param period The frequency, in milliseconds, at which metrics are
|
|
|
|
- * analyzed.
|
|
|
|
|
|
+ * @param abfsConfiguration The configuration set.
|
|
* @throws IllegalArgumentException If name is null or empty.
|
|
* @throws IllegalArgumentException If name is null or empty.
|
|
* If period is less than 1000 or greater than 30000 milliseconds.
|
|
* If period is less than 1000 or greater than 30000 milliseconds.
|
|
*/
|
|
*/
|
|
- AbfsClientThrottlingAnalyzer(String name, int period)
|
|
|
|
|
|
+ AbfsClientThrottlingAnalyzer(String name, AbfsConfiguration abfsConfiguration)
|
|
throws IllegalArgumentException {
|
|
throws IllegalArgumentException {
|
|
Preconditions.checkArgument(
|
|
Preconditions.checkArgument(
|
|
StringUtils.isNotEmpty(name),
|
|
StringUtils.isNotEmpty(name),
|
|
"The argument 'name' cannot be null or empty.");
|
|
"The argument 'name' cannot be null or empty.");
|
|
|
|
+ int period = abfsConfiguration.getAnalysisPeriod();
|
|
Preconditions.checkArgument(
|
|
Preconditions.checkArgument(
|
|
period >= MIN_ANALYSIS_PERIOD_MS && period <= MAX_ANALYSIS_PERIOD_MS,
|
|
period >= MIN_ANALYSIS_PERIOD_MS && period <= MAX_ANALYSIS_PERIOD_MS,
|
|
"The argument 'period' must be between 1000 and 30000.");
|
|
"The argument 'period' must be between 1000 and 30000.");
|
|
this.name = name;
|
|
this.name = name;
|
|
- this.analysisPeriodMs = period;
|
|
|
|
|
|
+ this.abfsConfiguration = abfsConfiguration;
|
|
|
|
+ this.accountLevelThrottlingEnabled = abfsConfiguration.accountThrottlingEnabled();
|
|
|
|
+ this.analysisPeriodMs = abfsConfiguration.getAnalysisPeriod();
|
|
|
|
+ this.lastExecutionTime = new AtomicLong(now());
|
|
this.blobMetrics = new AtomicReference<AbfsOperationMetrics>(
|
|
this.blobMetrics = new AtomicReference<AbfsOperationMetrics>(
|
|
new AbfsOperationMetrics(System.currentTimeMillis()));
|
|
new AbfsOperationMetrics(System.currentTimeMillis()));
|
|
this.timer = new Timer(
|
|
this.timer = new Timer(
|
|
@@ -95,6 +94,47 @@ class AbfsClientThrottlingAnalyzer {
|
|
analysisPeriodMs);
|
|
analysisPeriodMs);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Resumes the timer if it was stopped.
|
|
|
|
+ */
|
|
|
|
+ private void resumeTimer() {
|
|
|
|
+ blobMetrics = new AtomicReference<AbfsOperationMetrics>(
|
|
|
|
+ new AbfsOperationMetrics(System.currentTimeMillis()));
|
|
|
|
+ timer.schedule(new TimerTaskImpl(),
|
|
|
|
+ analysisPeriodMs,
|
|
|
|
+ analysisPeriodMs);
|
|
|
|
+ isOperationOnAccountIdle.set(false);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Synchronized method to suspend or resume timer.
|
|
|
|
+ * @param timerFunctionality resume or suspend.
|
|
|
|
+ * @param timerTask The timertask object.
|
|
|
|
+ * @return true or false.
|
|
|
|
+ */
|
|
|
|
+ private synchronized boolean timerOrchestrator(TimerFunctionality timerFunctionality,
|
|
|
|
+ TimerTask timerTask) {
|
|
|
|
+ switch (timerFunctionality) {
|
|
|
|
+ case RESUME:
|
|
|
|
+ if (isOperationOnAccountIdle.get()) {
|
|
|
|
+ resumeTimer();
|
|
|
|
+ }
|
|
|
|
+ break;
|
|
|
|
+ case SUSPEND:
|
|
|
|
+ if (accountLevelThrottlingEnabled && (System.currentTimeMillis()
|
|
|
|
+ - lastExecutionTime.get() >= getOperationIdleTimeout())) {
|
|
|
|
+ isOperationOnAccountIdle.set(true);
|
|
|
|
+ timerTask.cancel();
|
|
|
|
+ timer.purge();
|
|
|
|
+ return true;
|
|
|
|
+ }
|
|
|
|
+ break;
|
|
|
|
+ default:
|
|
|
|
+ break;
|
|
|
|
+ }
|
|
|
|
+ return false;
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Updates metrics with results from the current storage operation.
|
|
* Updates metrics with results from the current storage operation.
|
|
*
|
|
*
|
|
@@ -104,12 +144,13 @@ class AbfsClientThrottlingAnalyzer {
|
|
public void addBytesTransferred(long count, boolean isFailedOperation) {
|
|
public void addBytesTransferred(long count, boolean isFailedOperation) {
|
|
AbfsOperationMetrics metrics = blobMetrics.get();
|
|
AbfsOperationMetrics metrics = blobMetrics.get();
|
|
if (isFailedOperation) {
|
|
if (isFailedOperation) {
|
|
- metrics.bytesFailed.addAndGet(count);
|
|
|
|
- metrics.operationsFailed.incrementAndGet();
|
|
|
|
|
|
+ metrics.addBytesFailed(count);
|
|
|
|
+ metrics.incrementOperationsFailed();
|
|
} else {
|
|
} else {
|
|
- metrics.bytesSuccessful.addAndGet(count);
|
|
|
|
- metrics.operationsSuccessful.incrementAndGet();
|
|
|
|
|
|
+ metrics.addBytesSuccessful(count);
|
|
|
|
+ metrics.incrementOperationsSuccessful();
|
|
}
|
|
}
|
|
|
|
+ blobMetrics.set(metrics);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -117,6 +158,8 @@ class AbfsClientThrottlingAnalyzer {
|
|
* @return true if Thread sleeps(Throttling occurs) else false.
|
|
* @return true if Thread sleeps(Throttling occurs) else false.
|
|
*/
|
|
*/
|
|
public boolean suspendIfNecessary() {
|
|
public boolean suspendIfNecessary() {
|
|
|
|
+ lastExecutionTime.set(now());
|
|
|
|
+ timerOrchestrator(TimerFunctionality.RESUME, null);
|
|
int duration = sleepDuration;
|
|
int duration = sleepDuration;
|
|
if (duration > 0) {
|
|
if (duration > 0) {
|
|
try {
|
|
try {
|
|
@@ -134,19 +177,27 @@ class AbfsClientThrottlingAnalyzer {
|
|
return sleepDuration;
|
|
return sleepDuration;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ int getOperationIdleTimeout() {
|
|
|
|
+ return abfsConfiguration.getAccountOperationIdleTimeout();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ AtomicBoolean getIsOperationOnAccountIdle() {
|
|
|
|
+ return isOperationOnAccountIdle;
|
|
|
|
+ }
|
|
|
|
+
|
|
private int analyzeMetricsAndUpdateSleepDuration(AbfsOperationMetrics metrics,
|
|
private int analyzeMetricsAndUpdateSleepDuration(AbfsOperationMetrics metrics,
|
|
int sleepDuration) {
|
|
int sleepDuration) {
|
|
final double percentageConversionFactor = 100;
|
|
final double percentageConversionFactor = 100;
|
|
- double bytesFailed = metrics.bytesFailed.get();
|
|
|
|
- double bytesSuccessful = metrics.bytesSuccessful.get();
|
|
|
|
- double operationsFailed = metrics.operationsFailed.get();
|
|
|
|
- double operationsSuccessful = metrics.operationsSuccessful.get();
|
|
|
|
|
|
+ double bytesFailed = metrics.getBytesFailed().get();
|
|
|
|
+ double bytesSuccessful = metrics.getBytesSuccessful().get();
|
|
|
|
+ double operationsFailed = metrics.getOperationsFailed().get();
|
|
|
|
+ double operationsSuccessful = metrics.getOperationsSuccessful().get();
|
|
double errorPercentage = (bytesFailed <= 0)
|
|
double errorPercentage = (bytesFailed <= 0)
|
|
? 0
|
|
? 0
|
|
: (percentageConversionFactor
|
|
: (percentageConversionFactor
|
|
* bytesFailed
|
|
* bytesFailed
|
|
/ (bytesFailed + bytesSuccessful));
|
|
/ (bytesFailed + bytesSuccessful));
|
|
- long periodMs = metrics.endTime - metrics.startTime;
|
|
|
|
|
|
+ long periodMs = metrics.getEndTime() - metrics.getStartTime();
|
|
|
|
|
|
double newSleepDuration;
|
|
double newSleepDuration;
|
|
|
|
|
|
@@ -238,10 +289,13 @@ class AbfsClientThrottlingAnalyzer {
|
|
}
|
|
}
|
|
|
|
|
|
long now = System.currentTimeMillis();
|
|
long now = System.currentTimeMillis();
|
|
- if (now - blobMetrics.get().startTime >= analysisPeriodMs) {
|
|
|
|
|
|
+ if (timerOrchestrator(TimerFunctionality.SUSPEND, this)) {
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
|
|
+ if (now - blobMetrics.get().getStartTime() >= analysisPeriodMs) {
|
|
AbfsOperationMetrics oldMetrics = blobMetrics.getAndSet(
|
|
AbfsOperationMetrics oldMetrics = blobMetrics.getAndSet(
|
|
new AbfsOperationMetrics(now));
|
|
new AbfsOperationMetrics(now));
|
|
- oldMetrics.endTime = now;
|
|
|
|
|
|
+ oldMetrics.setEndTime(now);
|
|
sleepDuration = analyzeMetricsAndUpdateSleepDuration(oldMetrics,
|
|
sleepDuration = analyzeMetricsAndUpdateSleepDuration(oldMetrics,
|
|
sleepDuration);
|
|
sleepDuration);
|
|
}
|
|
}
|
|
@@ -252,24 +306,4 @@ class AbfsClientThrottlingAnalyzer {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Stores Abfs operation metrics during each analysis period.
|
|
|
|
- */
|
|
|
|
- static class AbfsOperationMetrics {
|
|
|
|
- private AtomicLong bytesFailed;
|
|
|
|
- private AtomicLong bytesSuccessful;
|
|
|
|
- private AtomicLong operationsFailed;
|
|
|
|
- private AtomicLong operationsSuccessful;
|
|
|
|
- private long endTime;
|
|
|
|
- private long startTime;
|
|
|
|
-
|
|
|
|
- AbfsOperationMetrics(long startTime) {
|
|
|
|
- this.startTime = startTime;
|
|
|
|
- this.bytesFailed = new AtomicLong();
|
|
|
|
- this.bytesSuccessful = new AtomicLong();
|
|
|
|
- this.operationsFailed = new AtomicLong();
|
|
|
|
- this.operationsSuccessful = new AtomicLong();
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
}
|
|
}
|