/** * Returns the current value of the counter. NOTE: if the value has never been set, <code>0</code> * is returned. * * @return value info * @throws Exception ZooKeeper errors */ public AtomicValue<byte[]> get() throws Exception { MutableAtomicValue<byte[]> result = new MutableAtomicValue<byte[]>(null, null, false); getCurrentValue(result, new Stat()); result.postValue = result.preValue; result.succeeded = true; return result; }
private void tryWithMutex(MutableAtomicValue<byte[]> result, MakeValue makeValue) throws Exception { long startMs = System.currentTimeMillis(); int retryCount = 0; if (mutex.acquire(promotedToLock.getMaxLockTime(), promotedToLock.getMaxLockTimeUnit())) { try { boolean done = false; while (!done) { result.stats.incrementPromotedTries(); if (tryOnce(result, makeValue)) { result.succeeded = true; done = true; } else { if (!promotedToLock .getRetryPolicy() .allowRetry( retryCount++, System.currentTimeMillis() - startMs, RetryLoop.getDefaultRetrySleeper())) { done = true; } } } } finally { mutex.release(); } } result.stats.setPromotedTimeMs(System.currentTimeMillis() - startMs); }
AtomicValue<byte[]> trySet(MakeValue makeValue) throws Exception { MutableAtomicValue<byte[]> result = new MutableAtomicValue<byte[]>(null, null, false); tryOptimistic(result, makeValue); if (!result.succeeded() && (mutex != null)) { tryWithMutex(result, makeValue); } return result; }
/** * Atomically sets the value to the given updated value if the current value {@code ==} the * expected value. Remember to always check {@link AtomicValue#succeeded()}. * * @param expectedValue the expected value * @param newValue the new value * @return value info * @throws Exception ZooKeeper errors */ public AtomicValue<byte[]> compareAndSet(byte[] expectedValue, byte[] newValue) throws Exception { Stat stat = new Stat(); MutableAtomicValue<byte[]> result = new MutableAtomicValue<byte[]>(null, null, false); boolean createIt = getCurrentValue(result, stat); if (!createIt && Arrays.equals(expectedValue, result.preValue)) { try { client.setData().withVersion(stat.getVersion()).forPath(path, newValue); result.succeeded = true; result.postValue = newValue; } catch (KeeperException.BadVersionException dummy) { result.succeeded = false; } catch (KeeperException.NoNodeException dummy) { result.succeeded = false; } } else { result.succeeded = false; } return result; }
/** * Attempt to atomically set the value to the given value. Remember to always check {@link * AtomicValue#succeeded()}. * * @param newValue the value to set * @return value info * @throws Exception ZooKeeper errors */ public AtomicValue<byte[]> trySet(final byte[] newValue) throws Exception { MutableAtomicValue<byte[]> result = new MutableAtomicValue<byte[]>(null, null, false); MakeValue makeValue = new MakeValue() { @Override public byte[] makeFrom(byte[] previous) { return newValue; } }; tryOptimistic(result, makeValue); if (!result.succeeded() && (mutex != null)) { tryWithMutex(result, makeValue); } return result; }
private void tryOptimistic(MutableAtomicValue<byte[]> result, MakeValue makeValue) throws Exception { long startMs = System.currentTimeMillis(); int retryCount = 0; boolean done = false; while (!done) { result.stats.incrementOptimisticTries(); if (tryOnce(result, makeValue)) { result.succeeded = true; done = true; } else { if (!retryPolicy.allowRetry( retryCount++, System.currentTimeMillis() - startMs, RetryLoop.getDefaultRetrySleeper())) { done = true; } } } result.stats.setOptimisticTimeMs(System.currentTimeMillis() - startMs); }