private void unlockHbck() { if (isExclusive() && hbckLockCleanup.compareAndSet(true, false)) { RetryCounter retryCounter = lockFileRetryCounterFactory.create(); do { try { IOUtils.closeQuietly(hbckOutFd); FSUtils.delete(FSUtils.getCurrentFileSystem(getConf()), HBCK_LOCK_PATH, true); LOG.info("Finishing hbck"); return; } catch (IOException ioe) { LOG.info("Failed to delete " + HBCK_LOCK_PATH + ", try=" + (retryCounter.getAttemptTimes() + 1) + " of " + retryCounter.getMaxAttempts()); LOG.debug("Failed to delete " + HBCK_LOCK_PATH, ioe); try { retryCounter.sleepUntilNextRetry(); } catch (InterruptedException ie) { Thread.currentThread().interrupt(); LOG.warn("Interrupted while deleting lock file" + HBCK_LOCK_PATH); return; } } } while (retryCounter.shouldRetry()); } }
checkAndMarkRunningHbck(getConf(), this.lockFileRetryCounterFactory.create()); HBCK_LOCK_PATH = pair.getFirst(); this.hbckOutFd = pair.getSecond();
RetryCounter retryCounter = createZNodeRetryCounterFactory.create(); hbckEphemeralNodePath = ZNodePaths.joinZNode( zkw.getZNodePaths().masterMaintZNode,
/** * @return True if region is online and scannable else false if an error or shutdown (Otherwise * we just block in here holding up all forward-progess). */ private boolean isRegionOnline(RegionInfo ri) throws InterruptedException { RetryCounter rc = null; while (!isStopped()) { RegionState rs = this.assignmentManager.getRegionStates().getRegionState(ri); if (rs.isOpened()) { if (this.getServerManager().isServerOnline(rs.getServerName())) { return true; } } // Region is not OPEN. Optional<Procedure<MasterProcedureEnv>> optProc = this.procedureExecutor.getProcedures(). stream().filter(p -> p instanceof ServerCrashProcedure).findAny(); // TODO: Add a page to refguide on how to do repair. Have this log message point to it. // Page will talk about loss of edits, how to schedule at least the meta WAL recovery, and // then how to assign including how to break region lock if one held. LOG.warn("{} is NOT online; state={}; ServerCrashProcedures={}. Master startup cannot " + "progress, in holding-pattern until region onlined.", ri.getRegionNameAsString(), rs, optProc.isPresent()); // Check once-a-minute. if (rc == null) { rc = new RetryCounterFactory(1000).create(); } Threads.sleep(rc.getBackoffTimeAndIncrementAttempts()); } return false; }
private Pair<Integer, String> execWithRetries(String hostname, ServiceType service, String... cmd) throws IOException { RetryCounter retryCounter = retryCounterFactory.create(); while (true) { try { return exec(hostname, service, cmd); } catch (IOException e) { retryOrThrow(retryCounter, e, hostname, cmd); } try { retryCounter.sleepUntilNextRetry(); } catch (InterruptedException ex) { // ignore LOG.warn("Sleep Interrupted:" + ex); } } }
private String createNonSequential(String path, byte[] data, List<ACL> acl, CreateMode createMode) throws KeeperException, InterruptedException { RetryCounter retryCounter = retryCounterFactory.create(); boolean isRetry = false; // False for first attempt, true for all retries. long startTime;
RetryCounter retryCounter = retryCounterFactory.create(); boolean isRetry = false; // False for first attempt, true for all retries. while (true) {
/** * getChildren is an idempotent operation. Retry before throwing exception * @return List of children znodes */ public List<String> getChildren(String path, Watcher watcher) throws KeeperException, InterruptedException { try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getChildren")) { RetryCounter retryCounter = retryCounterFactory.create(); while (true) { try { long startTime = EnvironmentEdgeManager.currentTime(); List<String> children = checkZk().getChildren(path, watcher); return children; } catch (KeeperException e) { switch (e.code()) { case CONNECTIONLOSS: retryOrThrow(retryCounter, e, "getChildren"); break; case OPERATIONTIMEOUT: retryOrThrow(retryCounter, e, "getChildren"); break; default: throw e; } } retryCounter.sleepUntilNextRetry(); } } }
/** * getChildren is an idempotent operation. Retry before throwing exception * @return List of children znodes */ public List<String> getChildren(String path, boolean watch) throws KeeperException, InterruptedException { try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getChildren")) { RetryCounter retryCounter = retryCounterFactory.create(); while (true) { try { long startTime = EnvironmentEdgeManager.currentTime(); List<String> children = checkZk().getChildren(path, watch); return children; } catch (KeeperException e) { switch (e.code()) { case CONNECTIONLOSS: retryOrThrow(retryCounter, e, "getChildren"); break; case OPERATIONTIMEOUT: retryOrThrow(retryCounter, e, "getChildren"); break; default: throw e; } } retryCounter.sleepUntilNextRetry(); } } }
/** * exists is an idempotent operation. Retry before throwing exception * @return A Stat instance */ public Stat exists(String path, Watcher watcher) throws KeeperException, InterruptedException { try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.exists")) { RetryCounter retryCounter = retryCounterFactory.create(); while (true) { try { long startTime = EnvironmentEdgeManager.currentTime(); Stat nodeStat = checkZk().exists(path, watcher); return nodeStat; } catch (KeeperException e) { switch (e.code()) { case CONNECTIONLOSS: retryOrThrow(retryCounter, e, "exists"); break; case OPERATIONTIMEOUT: retryOrThrow(retryCounter, e, "exists"); break; default: throw e; } } retryCounter.sleepUntilNextRetry(); } } }
/** * exists is an idempotent operation. Retry before throwing exception * @return A Stat instance */ public Stat exists(String path, boolean watch) throws KeeperException, InterruptedException { try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.exists")) { RetryCounter retryCounter = retryCounterFactory.create(); while (true) { try { long startTime = EnvironmentEdgeManager.currentTime(); Stat nodeStat = checkZk().exists(path, watch); return nodeStat; } catch (KeeperException e) { switch (e.code()) { case CONNECTIONLOSS: retryOrThrow(retryCounter, e, "exists"); break; case OPERATIONTIMEOUT: retryOrThrow(retryCounter, e, "exists"); break; default: throw e; } } retryCounter.sleepUntilNextRetry(); } } }
private String createSequential(String path, byte[] data, List<ACL> acl, CreateMode createMode) throws KeeperException, InterruptedException { RetryCounter retryCounter = retryCounterFactory.create(); boolean first = true; String newPath = path+this.identifier;
/** * getData is an idempotent operation. Retry before throwing exception * @return Data */ public byte[] getData(String path, boolean watch, Stat stat) throws KeeperException, InterruptedException { try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getData")) { RetryCounter retryCounter = retryCounterFactory.create(); while (true) { try { long startTime = EnvironmentEdgeManager.currentTime(); byte[] revData = checkZk().getData(path, watch, stat); return ZKMetadata.removeMetaData(revData); } catch (KeeperException e) { switch (e.code()) { case CONNECTIONLOSS: retryOrThrow(retryCounter, e, "getData"); break; case OPERATIONTIMEOUT: retryOrThrow(retryCounter, e, "getData"); break; default: throw e; } } retryCounter.sleepUntilNextRetry(); } } }
/** * getData is an idempotent operation. Retry before throwing exception * @return Data */ public byte[] getData(String path, Watcher watcher, Stat stat) throws KeeperException, InterruptedException { try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getData")) { RetryCounter retryCounter = retryCounterFactory.create(); while (true) { try { long startTime = EnvironmentEdgeManager.currentTime(); byte[] revData = checkZk().getData(path, watcher, stat); return ZKMetadata.removeMetaData(revData); } catch (KeeperException e) { switch (e.code()) { case CONNECTIONLOSS: retryOrThrow(retryCounter, e, "getData"); break; case OPERATIONTIMEOUT: retryOrThrow(retryCounter, e, "getData"); break; default: throw e; } } retryCounter.sleepUntilNextRetry(); } } }
@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); } }
throws KeeperException, InterruptedException { try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.setData")) { RetryCounter retryCounter = retryCounterFactory.create(); byte[] newData = ZKMetadata.appendMetaData(id, data); boolean isRetry = false;
/** * Get an exponential backoff retry counter. The base unit is 100 milliseconds, and the max * backoff time is 30 seconds. */ public static RetryCounter getRetryCounter() { return new RetryCounterFactory( new RetryCounter.RetryConfig().setBackoffPolicy(new RetryCounter.ExponentialBackoffPolicy()) .setSleepInterval(100).setMaxSleepTime(30000).setMaxAttempts(Integer.MAX_VALUE) .setTimeUnit(TimeUnit.MILLISECONDS).setJitter(0.01f)).create(); } }
/** * getAcl is an idempotent operation. Retry before throwing exception * @return list of ACLs */ public List<ACL> getAcl(String path, Stat stat) throws KeeperException, InterruptedException { try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getAcl")) { RetryCounter retryCounter = retryCounterFactory.create(); while (true) { try { long startTime = EnvironmentEdgeManager.currentTime(); List<ACL> nodeACL = checkZk().getACL(path, stat); return nodeACL; } catch (KeeperException e) { switch (e.code()) { case CONNECTIONLOSS: retryOrThrow(retryCounter, e, "getAcl"); break; case OPERATIONTIMEOUT: retryOrThrow(retryCounter, e, "getAcl"); break; default: throw e; } } retryCounter.sleepUntilNextRetry(); } } }
/** * setAcl is an idempotent operation. Retry before throwing exception * @return list of ACLs */ public Stat setAcl(String path, List<ACL> acls, int version) throws KeeperException, InterruptedException { try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.setAcl")) { RetryCounter retryCounter = retryCounterFactory.create(); while (true) { try { long startTime = EnvironmentEdgeManager.currentTime(); Stat nodeStat = checkZk().setACL(path, acls, version); return nodeStat; } catch (KeeperException e) { switch (e.code()) { case CONNECTIONLOSS: retryOrThrow(retryCounter, e, "setAcl"); break; case OPERATIONTIMEOUT: retryOrThrow(retryCounter, e, "setAcl"); break; default: throw e; } } retryCounter.sleepUntilNextRetry(); } } }
/** * Run multiple operations in a transactional manner. Retry before throwing exception */ public List<OpResult> multi(Iterable<Op> ops) throws KeeperException, InterruptedException { try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.multi")) { RetryCounter retryCounter = retryCounterFactory.create(); Iterable<Op> multiOps = prepareZKMulti(ops); while (true) { try { long startTime = EnvironmentEdgeManager.currentTime(); List<OpResult> opResults = checkZk().multi(multiOps); return opResults; } catch (KeeperException e) { switch (e.code()) { case CONNECTIONLOSS: retryOrThrow(retryCounter, e, "multi"); break; case OPERATIONTIMEOUT: retryOrThrow(retryCounter, e, "multi"); break; default: throw e; } } retryCounter.sleepUntilNextRetry(); } } }