001/** 002 * 003 * Licensed to the Apache Software Foundation (ASF) under one 004 * or more contributor license agreements. See the NOTICE file 005 * distributed with this work for additional information 006 * regarding copyright ownership. The ASF licenses this file 007 * to you under the Apache License, Version 2.0 (the 008 * "License"); you may not use this file except in compliance 009 * with the License. You may obtain a copy of the License at 010 * 011 * http://www.apache.org/licenses/LICENSE-2.0 012 * 013 * Unless required by applicable law or agreed to in writing, software 014 * distributed under the License is distributed on an "AS IS" BASIS, 015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 016 * See the License for the specific language governing permissions and 017 * limitations under the License. 018 */ 019package org.apache.hadoop.hbase.util; 020 021import java.util.concurrent.ThreadLocalRandom; 022import java.util.concurrent.TimeUnit; 023 024import org.apache.yetus.audience.InterfaceAudience; 025import org.slf4j.Logger; 026import org.slf4j.LoggerFactory; 027 028import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; 029 030/** 031 * Operation retry accounting. 032 * Use to calculate wait period, {@link #getBackoffTimeAndIncrementAttempts()}}, or for performing 033 * wait, {@link #sleepUntilNextRetry()}, in accordance with a {@link RetryConfig}, initial 034 * settings, and a Retry Policy, (See org.apache.hadoop.io.retry.RetryPolicy). 035 * Like <a href=https://github.com/rholder/guava-retrying>guava-retrying</a>. 036 * @since 0.92.0 037 * @see RetryCounterFactory 038 */ 039@InterfaceAudience.Private 040public class RetryCounter { 041 /** 042 * Configuration for a retry counter 043 */ 044 public static class RetryConfig { 045 private int maxAttempts; 046 private long sleepInterval; 047 private long maxSleepTime; 048 private TimeUnit timeUnit; 049 private BackoffPolicy backoffPolicy; 050 private float jitter; 051 052 private static final BackoffPolicy DEFAULT_BACKOFF_POLICY = new ExponentialBackoffPolicy(); 053 054 public RetryConfig() { 055 maxAttempts = 1; 056 sleepInterval = 1000; 057 maxSleepTime = -1; 058 timeUnit = TimeUnit.MILLISECONDS; 059 backoffPolicy = DEFAULT_BACKOFF_POLICY; 060 jitter = 0.0f; 061 } 062 063 public RetryConfig(int maxAttempts, long sleepInterval, long maxSleepTime, 064 TimeUnit timeUnit, BackoffPolicy backoffPolicy) { 065 this.maxAttempts = maxAttempts; 066 this.sleepInterval = sleepInterval; 067 this.maxSleepTime = maxSleepTime; 068 this.timeUnit = timeUnit; 069 this.backoffPolicy = backoffPolicy; 070 } 071 072 public RetryConfig setBackoffPolicy(BackoffPolicy backoffPolicy) { 073 this.backoffPolicy = backoffPolicy; 074 return this; 075 } 076 077 public RetryConfig setMaxAttempts(int maxAttempts) { 078 this.maxAttempts = maxAttempts; 079 return this; 080 } 081 082 public RetryConfig setMaxSleepTime(long maxSleepTime) { 083 this.maxSleepTime = maxSleepTime; 084 return this; 085 } 086 087 public RetryConfig setSleepInterval(long sleepInterval) { 088 this.sleepInterval = sleepInterval; 089 return this; 090 } 091 092 public RetryConfig setTimeUnit(TimeUnit timeUnit) { 093 this.timeUnit = timeUnit; 094 return this; 095 } 096 097 public RetryConfig setJitter(float jitter) { 098 Preconditions.checkArgument(jitter >= 0.0f && jitter < 1.0f, 099 "Invalid jitter: %s, should be in range [0.0, 1.0)", jitter); 100 this.jitter = jitter; 101 return this; 102 } 103 104 public int getMaxAttempts() { 105 return maxAttempts; 106 } 107 108 public long getMaxSleepTime() { 109 return maxSleepTime; 110 } 111 112 public long getSleepInterval() { 113 return sleepInterval; 114 } 115 116 public TimeUnit getTimeUnit() { 117 return timeUnit; 118 } 119 120 public float getJitter() { 121 return jitter; 122 } 123 124 public BackoffPolicy getBackoffPolicy() { 125 return backoffPolicy; 126 } 127 } 128 129 private static long addJitter(long interval, float jitter) { 130 long jitterInterval = (long) (interval * ThreadLocalRandom.current().nextFloat() * jitter); 131 return interval + jitterInterval; 132 } 133 134 /** 135 * Policy for calculating sleeping intervals between retry attempts 136 */ 137 public static class BackoffPolicy { 138 public long getBackoffTime(RetryConfig config, int attempts) { 139 return addJitter(config.getSleepInterval(), config.getJitter()); 140 } 141 } 142 143 public static class ExponentialBackoffPolicy extends BackoffPolicy { 144 @Override 145 public long getBackoffTime(RetryConfig config, int attempts) { 146 long backoffTime = (long) (config.getSleepInterval() * Math.pow(2, attempts)); 147 return addJitter(backoffTime, config.getJitter()); 148 } 149 } 150 151 public static class ExponentialBackoffPolicyWithLimit extends ExponentialBackoffPolicy { 152 @Override 153 public long getBackoffTime(RetryConfig config, int attempts) { 154 long backoffTime = super.getBackoffTime(config, attempts); 155 return config.getMaxSleepTime() > 0 ? Math.min(backoffTime, config.getMaxSleepTime()) : backoffTime; 156 } 157 } 158 159 private static final Logger LOG = LoggerFactory.getLogger(RetryCounter.class); 160 161 private RetryConfig retryConfig; 162 private int attempts; 163 164 public RetryCounter(int maxAttempts, long sleepInterval, TimeUnit timeUnit) { 165 this(new RetryConfig(maxAttempts, sleepInterval, -1, timeUnit, new ExponentialBackoffPolicy())); 166 } 167 168 public RetryCounter(RetryConfig retryConfig) { 169 this.attempts = 0; 170 this.retryConfig = retryConfig; 171 } 172 173 public int getMaxAttempts() { 174 return retryConfig.getMaxAttempts(); 175 } 176 177 /** 178 * Sleep for a back off time as supplied by the backoff policy, and increases the attempts 179 */ 180 public void sleepUntilNextRetry() throws InterruptedException { 181 int attempts = getAttemptTimes(); 182 long sleepTime = getBackoffTime(); 183 LOG.trace("Sleeping {} ms before retry #{}...", sleepTime, attempts); 184 retryConfig.getTimeUnit().sleep(sleepTime); 185 useRetry(); 186 } 187 188 public boolean shouldRetry() { 189 return attempts < retryConfig.getMaxAttempts(); 190 } 191 192 public void useRetry() { 193 attempts++; 194 } 195 196 public boolean isRetry() { 197 return attempts > 0; 198 } 199 200 public int getAttemptTimes() { 201 return attempts; 202 } 203 204 public long getBackoffTime() { 205 return this.retryConfig.backoffPolicy.getBackoffTime(this.retryConfig, getAttemptTimes()); 206 } 207 208 public long getBackoffTimeAndIncrementAttempts() { 209 long backoffTime = getBackoffTime(); 210 useRetry(); 211 return backoffTime; 212 } 213}