HBASE-20506 Add doc and test for unused RetryCounter, useful-looking utility
This commit is contained in:
parent
5a071dbe2b
commit
7790ab156e
|
@ -24,9 +24,17 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* Operation retry accounting.
|
||||
* Use to calculate wait period, {@link #getBackoffTimeAndIncrementAttempts()}}, or for performing
|
||||
* wait, {@link #sleepUntilNextRetry()}, in accordance with a {@link RetryConfig}, initial
|
||||
* settings, and a Retry Policy, (See org.apache.hadoop.io.retry.RetryPolicy).
|
||||
* Like <a href=https://github.com/rholder/guava-retrying>guava-retrying</a>.
|
||||
* @since 0.92.0
|
||||
* @see RetryCounterFactory
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class RetryCounter {
|
||||
|
||||
/**
|
||||
* Configuration for a retry counter
|
||||
*/
|
||||
|
@ -151,10 +159,8 @@ public class RetryCounter {
|
|||
*/
|
||||
public void sleepUntilNextRetry() throws InterruptedException {
|
||||
int attempts = getAttemptTimes();
|
||||
long sleepTime = retryConfig.backoffPolicy.getBackoffTime(retryConfig, attempts);
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Sleeping " + sleepTime + "ms before retry #" + attempts + "...");
|
||||
}
|
||||
long sleepTime = getBackoffTime();
|
||||
LOG.trace("Sleeping {} ms before retry #{}...", sleepTime, attempts);
|
||||
retryConfig.getTimeUnit().sleep(sleepTime);
|
||||
useRetry();
|
||||
}
|
||||
|
@ -174,4 +180,14 @@ public class RetryCounter {
|
|||
public int getAttemptTimes() {
|
||||
return attempts;
|
||||
}
|
||||
|
||||
public long getBackoffTime() {
|
||||
return this.retryConfig.backoffPolicy.getBackoffTime(this.retryConfig, getAttemptTimes());
|
||||
}
|
||||
|
||||
public long getBackoffTimeAndIncrementAttempts() {
|
||||
long backoffTime = getBackoffTime();
|
||||
useRetry();
|
||||
return backoffTime;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,55 @@
|
|||
/**
|
||||
* 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.hbase.util;
|
||||
|
||||
import static junit.framework.TestCase.assertTrue;
|
||||
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.junit.ClassRule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
||||
/**
|
||||
* Basic test for some old functionality we don't seem to have used but that looks nice.
|
||||
*/
|
||||
@Category({MiscTests.class, SmallTests.class})
|
||||
public class TestRetryCounter {
|
||||
@ClassRule
|
||||
public static final HBaseClassTestRule CLASS_RULE =
|
||||
HBaseClassTestRule.forClass(TestRetryCounter.class);
|
||||
private static final Logger LOG = LoggerFactory.getLogger(TestRetryCounter.class);
|
||||
|
||||
@Test
|
||||
public void testBasics() throws InterruptedException {
|
||||
int maxAttempts = 10;
|
||||
RetryCounterFactory factory =
|
||||
new RetryCounterFactory(maxAttempts, 10, 1000);
|
||||
RetryCounter retryCounter = factory.create();
|
||||
while (retryCounter.shouldRetry()) {
|
||||
LOG.info("Attempt={}, backoffTime={}", retryCounter.getAttemptTimes(),
|
||||
retryCounter.getBackoffTime());
|
||||
retryCounter.sleepUntilNextRetry();
|
||||
}
|
||||
assertTrue(retryCounter.getAttemptTimes() == maxAttempts);
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue