├── .asf.yaml ├── .git-blame-ignore-revs ├── .github └── workflows │ └── ci.yml ├── .gitignore ├── .mvn ├── develocity-custom-user-data.groovy ├── develocity.xml ├── extensions.xml └── wrapper │ └── maven-wrapper.properties ├── DEPENDENCIES ├── LICENSE ├── NOTICE ├── README.md ├── curator-client ├── LICENSE ├── NOTICE ├── pom.xml └── src │ ├── main │ └── java │ │ └── org │ │ └── apache │ │ └── curator │ │ ├── ConnectionState.java │ │ ├── CuratorConnectionLossException.java │ │ ├── CuratorZookeeperClient.java │ │ ├── HandleHolder.java │ │ ├── Helper.java │ │ ├── RetryLoop.java │ │ ├── RetryLoopImpl.java │ │ ├── RetryPolicy.java │ │ ├── RetrySleeper.java │ │ ├── SessionFailRetryLoop.java │ │ ├── SessionFailedRetryPolicy.java │ │ ├── TimeTrace.java │ │ ├── connection │ │ └── ThreadLocalRetryLoop.java │ │ ├── drivers │ │ ├── AdvancedTracerDriver.java │ │ ├── EventTrace.java │ │ ├── OperationTrace.java │ │ └── TracerDriver.java │ │ ├── ensemble │ │ ├── EnsembleProvider.java │ │ └── fixed │ │ │ └── FixedEnsembleProvider.java │ │ ├── retry │ │ ├── BoundedExponentialBackoffRetry.java │ │ ├── ExponentialBackoffRetry.java │ │ ├── RetryForever.java │ │ ├── RetryNTimes.java │ │ ├── RetryOneTime.java │ │ ├── RetryUntilElapsed.java │ │ └── SleepingRetry.java │ │ └── utils │ │ ├── CloseableExecutorService.java │ │ ├── CloseableScheduledExecutorService.java │ │ ├── CloseableUtils.java │ │ ├── Compatibility.java │ │ ├── ConfigurableZookeeperFactory.java │ │ ├── DebugUtils.java │ │ ├── DefaultTracerDriver.java │ │ ├── DefaultZookeeperFactory.java │ │ ├── EnsurePath.java │ │ ├── ExceptionAccumulator.java │ │ ├── InternalACLProvider.java │ │ ├── NonAdminZookeeperFactory.java │ │ ├── PathUtils.java │ │ ├── ThreadUtils.java │ │ ├── ZKPaths.java │ │ ├── ZookeeperCompatibility.java │ │ └── ZookeeperFactory.java │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── curator │ │ ├── BasicTests.java │ │ ├── TestRetryLoop.java │ │ ├── TestSessionFailRetryLoop.java │ │ └── utils │ │ ├── TestCloseableExecutorService.java │ │ ├── TestCloseableScheduledExecutorService.java │ │ └── TestZKPaths.java │ └── resources │ └── log4j.properties ├── curator-examples ├── LICENSE ├── NOTICE ├── pom.xml └── src │ └── main │ ├── java │ ├── async │ │ └── AsyncExamples.java │ ├── cache │ │ ├── CuratorCacheExample.java │ │ ├── PathCacheExample.java │ │ └── TreeCacheExample.java │ ├── discovery │ │ ├── DiscoveryExample.java │ │ ├── ExampleServer.java │ │ └── InstanceDetails.java │ ├── framework │ │ ├── CreateClientExamples.java │ │ ├── CrudExamples.java │ │ └── TransactionExamples.java │ ├── leader │ │ ├── ExampleClient.java │ │ └── LeaderSelectorExample.java │ ├── locking │ │ ├── ExampleClientThatLocks.java │ │ ├── FakeLimitedResource.java │ │ └── LockingExample.java │ ├── modeled │ │ ├── ContainerType.java │ │ ├── ModeledCuratorExamples.java │ │ ├── ModeledCuratorExamplesAlt.java │ │ ├── PersonId.java │ │ ├── PersonModel.java │ │ └── PersonModelSpec.java │ └── pubsub │ │ ├── Clients.java │ │ ├── Publisher.java │ │ ├── README.md │ │ ├── SubPubTest.java │ │ ├── Subscriber.java │ │ ├── messages │ │ ├── LocationAvailable.java │ │ └── UserCreated.java │ │ └── models │ │ ├── Group.java │ │ ├── Instance.java │ │ ├── InstanceType.java │ │ ├── Message.java │ │ └── Priority.java │ └── resources │ └── log4j.properties ├── curator-framework ├── LICENSE ├── NOTICE ├── pom.xml └── src │ ├── main │ └── java │ │ └── org │ │ └── apache │ │ └── curator │ │ └── framework │ │ ├── AuthInfo.java │ │ ├── CuratorFramework.java │ │ ├── CuratorFrameworkFactory.java │ │ ├── CuratorTempFramework.java │ │ ├── EnsureContainers.java │ │ ├── WatcherRemoveCuratorFramework.java │ │ ├── api │ │ ├── ACLBackgroundPathAndBytesable.java │ │ ├── ACLCreateModeBackgroundPathAndBytesable.java │ │ ├── ACLCreateModePathAndBytesable.java │ │ ├── ACLCreateModeStatBackgroundPathAndBytesable.java │ │ ├── ACLPathAndBytesable.java │ │ ├── ACLProvider.java │ │ ├── ACLable.java │ │ ├── ACLableExistBuilderMain.java │ │ ├── AddStatConfigEnsembleable.java │ │ ├── AddWatchBuilder.java │ │ ├── AddWatchBuilder2.java │ │ ├── Addable.java │ │ ├── AsyncReconfigurable.java │ │ ├── BackgroundCallback.java │ │ ├── BackgroundEnsembleable.java │ │ ├── BackgroundPathAndBytesable.java │ │ ├── BackgroundPathable.java │ │ ├── BackgroundPathableQuietlyable.java │ │ ├── BackgroundVersionable.java │ │ ├── Backgroundable.java │ │ ├── ChildrenDeletable.java │ │ ├── Compressible.java │ │ ├── CompressionProvider.java │ │ ├── ConfigureEnsembleable.java │ │ ├── CreateBackgroundModeACLable.java │ │ ├── CreateBackgroundModeStatACLable.java │ │ ├── CreateBuilder.java │ │ ├── CreateBuilder2.java │ │ ├── CreateBuilderMain.java │ │ ├── CreateModable.java │ │ ├── CreateProtectACLCreateModePathAndBytesable.java │ │ ├── CuratorClosedException.java │ │ ├── CuratorEvent.java │ │ ├── CuratorEventType.java │ │ ├── CuratorListener.java │ │ ├── CuratorWatcher.java │ │ ├── DataCallbackable.java │ │ ├── Decompressible.java │ │ ├── DeleteBuilder.java │ │ ├── DeleteBuilderMain.java │ │ ├── Ensembleable.java │ │ ├── ErrorListenerEnsembleable.java │ │ ├── ErrorListenerMultiTransactionMain.java │ │ ├── ErrorListenerPathAndBytesable.java │ │ ├── ErrorListenerPathable.java │ │ ├── ErrorListenerReconfigBuilderMain.java │ │ ├── ExistsBuilder.java │ │ ├── ExistsBuilderMain.java │ │ ├── GetACLBuilder.java │ │ ├── GetChildrenBuilder.java │ │ ├── GetConfigBuilder.java │ │ ├── GetDataBuilder.java │ │ ├── GetDataWatchBackgroundStatable.java │ │ ├── Guaranteeable.java │ │ ├── GuaranteeableDeletable.java │ │ ├── Idempotentable.java │ │ ├── JoinStatConfigEnsembleable.java │ │ ├── Joinable.java │ │ ├── LeaveStatConfigEnsembleable.java │ │ ├── Leaveable.java │ │ ├── Membersable.java │ │ ├── ParentACLable.java │ │ ├── PathAndBytesable.java │ │ ├── Pathable.java │ │ ├── ProtectACLCreateModePathAndBytesable.java │ │ ├── ProtectACLCreateModeStatPathAndBytesable.java │ │ ├── Quietly.java │ │ ├── ReconfigBuilder.java │ │ ├── ReconfigBuilderMain.java │ │ ├── RemoveWatchesBuilder.java │ │ ├── RemoveWatchesLocal.java │ │ ├── RemoveWatchesType.java │ │ ├── SetACLBuilder.java │ │ ├── SetDataBackgroundVersionable.java │ │ ├── SetDataBuilder.java │ │ ├── StatConfigureEnsembleable.java │ │ ├── StatPathable.java │ │ ├── Statable.java │ │ ├── SyncBuilder.java │ │ ├── TempGetDataBuilder.java │ │ ├── UnhandledErrorListener.java │ │ ├── VersionPathAndBytesable.java │ │ ├── Versionable.java │ │ ├── WatchBackgroundEnsembleable.java │ │ ├── WatchPathable.java │ │ ├── Watchable.java │ │ ├── WatchableBase.java │ │ ├── WatchesBuilder.java │ │ └── transaction │ │ │ ├── CuratorMultiTransaction.java │ │ │ ├── CuratorMultiTransactionMain.java │ │ │ ├── CuratorOp.java │ │ │ ├── CuratorTransaction.java │ │ │ ├── CuratorTransactionBridge.java │ │ │ ├── CuratorTransactionFinal.java │ │ │ ├── CuratorTransactionResult.java │ │ │ ├── OperationType.java │ │ │ ├── TransactionCheckBuilder.java │ │ │ ├── TransactionCreateBuilder.java │ │ │ ├── TransactionCreateBuilder2.java │ │ │ ├── TransactionDeleteBuilder.java │ │ │ ├── TransactionOp.java │ │ │ ├── TransactionSetDataBuilder.java │ │ │ └── TypeAndPath.java │ │ ├── imps │ │ ├── ACLing.java │ │ ├── AddWatchBuilderImpl.java │ │ ├── BackgroundOperation.java │ │ ├── BackgroundSyncImpl.java │ │ ├── Backgrounding.java │ │ ├── CreateBuilderImpl.java │ │ ├── CuratorEventImpl.java │ │ ├── CuratorFrameworkBase.java │ │ ├── CuratorFrameworkImpl.java │ │ ├── CuratorFrameworkState.java │ │ ├── CuratorMultiTransactionImpl.java │ │ ├── CuratorMultiTransactionRecord.java │ │ ├── CuratorTempFrameworkImpl.java │ │ ├── CuratorTransactionImpl.java │ │ ├── DefaultACLProvider.java │ │ ├── DelegatingCuratorFramework.java │ │ ├── DeleteBuilderImpl.java │ │ ├── EnsembleTracker.java │ │ ├── ExistsBuilderImpl.java │ │ ├── ExtractingCuratorOp.java │ │ ├── FailedDeleteManager.java │ │ ├── FailedOperationManager.java │ │ ├── FailedRemoveWatchManager.java │ │ ├── FindAndDeleteProtectedNodeInBackground.java │ │ ├── FrameworkUtils.java │ │ ├── GetACLBuilderImpl.java │ │ ├── GetChildrenBuilderImpl.java │ │ ├── GetConfigBuilderImpl.java │ │ ├── GetDataBuilderImpl.java │ │ ├── GzipCompressionProvider.java │ │ ├── IdempotentUtils.java │ │ ├── InternalConnectionHandler.java │ │ ├── NamespaceFacade.java │ │ ├── NamespaceFacadeCache.java │ │ ├── NamespaceImpl.java │ │ ├── NamespaceWatchedEvent.java │ │ ├── NamespaceWatcher.java │ │ ├── OperationAndData.java │ │ ├── PathAndBytes.java │ │ ├── ProtectedMode.java │ │ ├── ProtectedUtils.java │ │ ├── ReconfigBuilderImpl.java │ │ ├── RemoveWatchesBuilderImpl.java │ │ ├── SetACLBuilderImpl.java │ │ ├── SetDataBuilderImpl.java │ │ ├── StandardInternalConnectionHandler.java │ │ ├── SyncBuilderImpl.java │ │ ├── TempGetDataBuilderImpl.java │ │ ├── TransactionOpImpl.java │ │ ├── WatcherRemovalFacade.java │ │ ├── WatcherRemovalManager.java │ │ ├── WatchesBuilderImpl.java │ │ └── Watching.java │ │ ├── listen │ │ ├── Listenable.java │ │ ├── ListenerEntry.java │ │ ├── ListenerManager.java │ │ ├── MappingListenerManager.java │ │ ├── StandardListenerManager.java │ │ └── UnaryListenerManager.java │ │ ├── schema │ │ ├── DefaultSchemaValidator.java │ │ ├── Schema.java │ │ ├── SchemaBuilder.java │ │ ├── SchemaSet.java │ │ ├── SchemaSetLoader.java │ │ ├── SchemaValidator.java │ │ └── SchemaViolation.java │ │ └── state │ │ ├── CircuitBreaker.java │ │ ├── CircuitBreakingConnectionStateListener.java │ │ ├── CircuitBreakingManager.java │ │ ├── ConnectionState.java │ │ ├── ConnectionStateErrorPolicy.java │ │ ├── ConnectionStateListener.java │ │ ├── ConnectionStateListenerManagerFactory.java │ │ ├── ConnectionStateManager.java │ │ ├── SessionConnectionStateErrorPolicy.java │ │ └── StandardConnectionStateErrorPolicy.java │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── curator │ │ └── framework │ │ ├── ensemble │ │ └── TestEnsembleProvider.java │ │ ├── imps │ │ ├── TestBlockUntilConnected.java │ │ ├── TestCleanState.java │ │ ├── TestCompression.java │ │ ├── TestCompressionInTransactionNew.java │ │ ├── TestCompressionInTransactionOld.java │ │ ├── TestCreate.java │ │ ├── TestCreateReturningStat.java │ │ ├── TestDelete.java │ │ ├── TestEnabledSessionExpiredState.java │ │ ├── TestEnsureContainers.java │ │ ├── TestExistsBuilder.java │ │ ├── TestFailedDeleteManager.java │ │ ├── TestFramework.java │ │ ├── TestFrameworkBackground.java │ │ ├── TestFrameworkEdges.java │ │ ├── TestGzipCompressionProvider.java │ │ ├── TestMultiClient.java │ │ ├── TestNamespaceFacade.java │ │ ├── TestNeverConnected.java │ │ ├── TestReadOnly.java │ │ ├── TestReconfiguration.java │ │ ├── TestSetData.java │ │ ├── TestTempFramework.java │ │ ├── TestTransactionsNew.java │ │ ├── TestTransactionsOld.java │ │ ├── TestTtlNodes.java │ │ ├── TestWatcherIdentity.java │ │ ├── TestWatcherRemovalManager.java │ │ ├── TestWatchesBuilder.java │ │ ├── TestWithCluster.java │ │ └── TransactionsHelper.java │ │ ├── schema │ │ └── TestSchema.java │ │ └── state │ │ ├── TestCircuitBreaker.java │ │ ├── TestCircuitBreakingConnectionStateListener.java │ │ └── TestConnectionStateManager.java │ └── resources │ ├── log4j.properties │ ├── schema.yaml │ ├── schema1.json │ ├── schema2.json │ ├── schema3.json │ ├── schema4.json │ └── schema5.json ├── curator-recipes ├── LICENSE ├── NOTICE ├── pom.xml └── src │ ├── main │ └── java │ │ └── org │ │ └── apache │ │ └── curator │ │ └── framework │ │ └── recipes │ │ ├── AfterConnectionEstablished.java │ │ ├── atomic │ │ ├── AtomicStats.java │ │ ├── AtomicValue.java │ │ ├── CachedAtomicInteger.java │ │ ├── CachedAtomicLong.java │ │ ├── DistributedAtomicInteger.java │ │ ├── DistributedAtomicLong.java │ │ ├── DistributedAtomicNumber.java │ │ ├── DistributedAtomicValue.java │ │ ├── MakeValue.java │ │ ├── MutableAtomicValue.java │ │ └── PromotedToLock.java │ │ ├── barriers │ │ ├── DistributedBarrier.java │ │ └── DistributedDoubleBarrier.java │ │ ├── cache │ │ ├── ChildData.java │ │ ├── CompatibleCuratorCacheBridge.java │ │ ├── CuratorCache.java │ │ ├── CuratorCacheAccessor.java │ │ ├── CuratorCacheBridge.java │ │ ├── CuratorCacheBridgeBuilder.java │ │ ├── CuratorCacheBridgeBuilderImpl.java │ │ ├── CuratorCacheBuilder.java │ │ ├── CuratorCacheBuilderImpl.java │ │ ├── CuratorCacheImpl.java │ │ ├── CuratorCacheListener.java │ │ ├── CuratorCacheListenerBuilder.java │ │ ├── CuratorCacheListenerBuilderImpl.java │ │ ├── CuratorCacheStorage.java │ │ ├── DefaultTreeCacheSelector.java │ │ ├── EventOperation.java │ │ ├── GetDataOperation.java │ │ ├── NodeCache.java │ │ ├── NodeCacheListener.java │ │ ├── NodeCacheListenerWrapper.java │ │ ├── Operation.java │ │ ├── OutstandingOps.java │ │ ├── PathChildrenCache.java │ │ ├── PathChildrenCacheEvent.java │ │ ├── PathChildrenCacheListener.java │ │ ├── PathChildrenCacheListenerWrapper.java │ │ ├── PathChildrenCacheMode.java │ │ ├── RefreshOperation.java │ │ ├── StandardCuratorCacheStorage.java │ │ ├── TreeCache.java │ │ ├── TreeCacheEvent.java │ │ ├── TreeCacheIterator.java │ │ ├── TreeCacheListener.java │ │ ├── TreeCacheListenerWrapper.java │ │ └── TreeCacheSelector.java │ │ ├── leader │ │ ├── CancelLeadershipException.java │ │ ├── LeaderLatch.java │ │ ├── LeaderLatchListener.java │ │ ├── LeaderSelector.java │ │ ├── LeaderSelectorListener.java │ │ ├── LeaderSelectorListenerAdapter.java │ │ └── Participant.java │ │ ├── locks │ │ ├── InterProcessLock.java │ │ ├── InterProcessMultiLock.java │ │ ├── InterProcessMutex.java │ │ ├── InterProcessReadWriteLock.java │ │ ├── InterProcessSemaphore.java │ │ ├── InterProcessSemaphoreMutex.java │ │ ├── InterProcessSemaphoreV2.java │ │ ├── Lease.java │ │ ├── LockInternals.java │ │ ├── LockInternalsDriver.java │ │ ├── LockInternalsSorter.java │ │ ├── Locker.java │ │ ├── PredicateResults.java │ │ ├── Revocable.java │ │ ├── RevocationListener.java │ │ ├── RevocationSpec.java │ │ ├── Revoker.java │ │ └── StandardLockInternalsDriver.java │ │ ├── nodes │ │ ├── GroupMember.java │ │ ├── PersistentEphemeralNode.java │ │ ├── PersistentNode.java │ │ ├── PersistentNodeListener.java │ │ └── PersistentTtlNode.java │ │ ├── queue │ │ ├── BlockingQueueConsumer.java │ │ ├── ChildrenCache.java │ │ ├── DistributedDelayQueue.java │ │ ├── DistributedIdQueue.java │ │ ├── DistributedPriorityQueue.java │ │ ├── DistributedQueue.java │ │ ├── ErrorMode.java │ │ ├── ItemSerializer.java │ │ ├── MultiItem.java │ │ ├── QueueAllocator.java │ │ ├── QueueBase.java │ │ ├── QueueBuilder.java │ │ ├── QueueConsumer.java │ │ ├── QueuePutListener.java │ │ ├── QueueSafety.java │ │ ├── QueueSerializer.java │ │ ├── QueueSharder.java │ │ ├── QueueSharderPolicies.java │ │ └── SimpleDistributedQueue.java │ │ ├── shared │ │ ├── IllegalTrySetVersionException.java │ │ ├── SharedCount.java │ │ ├── SharedCountListener.java │ │ ├── SharedCountReader.java │ │ ├── SharedValue.java │ │ ├── SharedValueListener.java │ │ ├── SharedValueReader.java │ │ └── VersionedValue.java │ │ └── watch │ │ └── PersistentWatcher.java │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── curator │ │ ├── connection │ │ └── TestThreadLocalRetryLoop.java │ │ └── framework │ │ ├── client │ │ ├── TestBackgroundStates.java │ │ └── TestResetConnectionWithBackgroundFailure.java │ │ ├── recipes │ │ ├── atomic │ │ │ ├── TestCachedAtomicCounter.java │ │ │ └── TestDistributedAtomicLong.java │ │ ├── barriers │ │ │ ├── TestDistributedBarrier.java │ │ │ └── TestDistributedDoubleBarrier.java │ │ ├── cache │ │ │ ├── BaseTestTreeCache.java │ │ │ ├── TestCuratorCache.java │ │ │ ├── TestCuratorCacheBridge.java │ │ │ ├── TestCuratorCacheConsistency.java │ │ │ ├── TestCuratorCacheEdges.java │ │ │ ├── TestCuratorCacheEventOrdering.java │ │ │ ├── TestCuratorCacheWrappers.java │ │ │ ├── TestEventOrdering.java │ │ │ ├── TestNodeCache.java │ │ │ ├── TestPathChildrenCache.java │ │ │ ├── TestPathChildrenCacheEventOrdering.java │ │ │ ├── TestPathChildrenCacheInCluster.java │ │ │ ├── TestTreeCache.java │ │ │ ├── TestTreeCacheEventOrdering.java │ │ │ ├── TestTreeCacheIteratorAndSize.java │ │ │ ├── TestTreeCacheRandomTree.java │ │ │ └── TestWrappedNodeCache.java │ │ ├── leader │ │ │ ├── ChaosMonkeyCnxnFactory.java │ │ │ ├── TestLeaderAcls.java │ │ │ ├── TestLeaderLatch.java │ │ │ ├── TestLeaderLatchCluster.java │ │ │ ├── TestLeaderSelector.java │ │ │ ├── TestLeaderSelectorCluster.java │ │ │ ├── TestLeaderSelectorEdges.java │ │ │ ├── TestLeaderSelectorParticipants.java │ │ │ └── TestLeaderSelectorWithExecutor.java │ │ ├── locks │ │ │ ├── Counter.java │ │ │ ├── SemaphoreClient.java │ │ │ ├── Stepper.java │ │ │ ├── TestInterProcessMultiMutex.java │ │ │ ├── TestInterProcessMutex.java │ │ │ ├── TestInterProcessMutexBase.java │ │ │ ├── TestInterProcessReadWriteLock.java │ │ │ ├── TestInterProcessSemaphore.java │ │ │ ├── TestInterProcessSemaphoreCluster.java │ │ │ ├── TestInterProcessSemaphoreMutex.java │ │ │ ├── TestLockACLs.java │ │ │ └── TestLockInternals.java │ │ ├── nodes │ │ │ ├── TestGroupMember.java │ │ │ ├── TestPersistentEphemeralNode.java │ │ │ ├── TestPersistentEphemeralNodeListener.java │ │ │ ├── TestPersistentNode.java │ │ │ └── TestPersistentTtlNode.java │ │ ├── queue │ │ │ ├── QueueItemSerializer.java │ │ │ ├── QueueTestProducer.java │ │ │ ├── TestBoundedDistributedQueue.java │ │ │ ├── TestDistributedDelayQueue.java │ │ │ ├── TestDistributedIdQueue.java │ │ │ ├── TestDistributedPriorityQueue.java │ │ │ ├── TestDistributedQueue.java │ │ │ ├── TestLongNetworkPartition.java │ │ │ ├── TestQueueItem.java │ │ │ ├── TestQueueSharder.java │ │ │ └── TestSimpleDistributedQueue.java │ │ ├── shared │ │ │ └── TestSharedCount.java │ │ └── watch │ │ │ └── TestPersistentWatcher.java │ │ └── state │ │ └── DummyConnectionStateListener.java │ └── resources │ └── log4j.properties ├── curator-test-zk35 ├── pom.xml └── src │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── curator │ │ ├── framework │ │ └── TestCompatibility.java │ │ └── zk35 │ │ └── TestIs35.java │ └── resources │ └── log4j.properties ├── curator-test-zk36 ├── pom.xml └── src │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── curator │ │ └── zk36 │ │ └── TestIs36.java │ └── resources │ └── log4j.properties ├── curator-test-zk37 ├── pom.xml └── src │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── curator │ │ └── zk37 │ │ └── TestIs37.java │ └── resources │ └── log4j.properties ├── curator-test-zk38 ├── pom.xml └── src │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── curator │ │ └── zk38 │ │ └── TestIs38.java │ └── resources │ └── log4j.properties ├── curator-test ├── LICENSE ├── NOTICE ├── pom.xml └── src │ ├── main │ └── java │ │ └── org │ │ └── apache │ │ └── curator │ │ └── test │ │ ├── BaseClassForTests.java │ │ ├── Compatibility.java │ │ ├── DelegatingExecutorService.java │ │ ├── DirectoryUtils.java │ │ ├── ExecuteCalledWatchingExecutorService.java │ │ ├── FailedServerStartException.java │ │ ├── InstanceSpec.java │ │ ├── KillSession.java │ │ ├── QuorumConfigBuilder.java │ │ ├── QuorumPeerConfigBuilder.java │ │ ├── ServerHelper.java │ │ ├── TestingCluster.java │ │ ├── TestingQuorumPeerMain.java │ │ ├── TestingServer.java │ │ ├── TestingZooKeeperMain.java │ │ ├── TestingZooKeeperServer.java │ │ ├── Timing.java │ │ ├── WatchersDebug.java │ │ ├── ZooKeeperMainFace.java │ │ ├── ZooKeeperServerEmbeddedAdapter.java │ │ └── compatibility │ │ ├── CuratorTestBase.java │ │ └── Timing2.java │ └── test │ └── java │ └── org │ └── apache │ └── curator │ └── test │ ├── TestQuorumConfigBuilder.java │ └── TestTestingServer.java ├── curator-x-async ├── pom.xml └── src │ ├── main │ └── java │ │ └── org │ │ └── apache │ │ └── curator │ │ └── x │ │ └── async │ │ ├── AsyncCuratorFramework.java │ │ ├── AsyncEventException.java │ │ ├── AsyncResult.java │ │ ├── AsyncStage.java │ │ ├── AsyncWrappers.java │ │ ├── WatchMode.java │ │ ├── api │ │ ├── AsyncCreateBuilder.java │ │ ├── AsyncCuratorFrameworkDsl.java │ │ ├── AsyncDeleteBuilder.java │ │ ├── AsyncEnsemblable.java │ │ ├── AsyncExistsBuilder.java │ │ ├── AsyncGetACLBuilder.java │ │ ├── AsyncGetChildrenBuilder.java │ │ ├── AsyncGetConfigBuilder.java │ │ ├── AsyncGetDataBuilder.java │ │ ├── AsyncMultiTransaction.java │ │ ├── AsyncPathAndBytesable.java │ │ ├── AsyncPathable.java │ │ ├── AsyncReconfigBuilder.java │ │ ├── AsyncRemoveWatchesBuilder.java │ │ ├── AsyncSetACLBuilder.java │ │ ├── AsyncSetDataBuilder.java │ │ ├── AsyncSyncBuilder.java │ │ ├── AsyncTransactionCheckBuilder.java │ │ ├── AsyncTransactionCreateBuilder.java │ │ ├── AsyncTransactionDeleteBuilder.java │ │ ├── AsyncTransactionOp.java │ │ ├── AsyncTransactionSetDataBuilder.java │ │ ├── AsyncWatchBuilder.java │ │ ├── AsyncWatchBuilder2.java │ │ ├── CreateOption.java │ │ ├── DeleteOption.java │ │ ├── ExistsOption.java │ │ ├── RemoveWatcherOption.java │ │ └── WatchableAsyncCuratorFramework.java │ │ ├── details │ │ ├── AsyncCreateBuilderImpl.java │ │ ├── AsyncCuratorFrameworkImpl.java │ │ ├── AsyncDeleteBuilderImpl.java │ │ ├── AsyncExistsBuilderImpl.java │ │ ├── AsyncGetChildrenBuilderImpl.java │ │ ├── AsyncGetConfigBuilderImpl.java │ │ ├── AsyncGetDataBuilderImpl.java │ │ ├── AsyncReconfigBuilderImpl.java │ │ ├── AsyncRemoveWatchesBuilderImpl.java │ │ ├── AsyncResultImpl.java │ │ ├── AsyncSetACLBuilderImpl.java │ │ ├── AsyncSetDataBuilderImpl.java │ │ ├── AsyncTransactionOpImpl.java │ │ ├── AsyncWatchBuilderImpl.java │ │ ├── BackgroundProc.java │ │ ├── BackgroundProcs.java │ │ ├── BuilderCommon.java │ │ ├── Filters.java │ │ ├── InternalCallback.java │ │ └── InternalWatcher.java │ │ ├── migrations │ │ ├── Migration.java │ │ ├── MigrationException.java │ │ ├── MigrationManager.java │ │ └── MigrationSet.java │ │ └── modeled │ │ ├── JacksonModelSerializer.java │ │ ├── ModelSerializer.java │ │ ├── ModelSpec.java │ │ ├── ModelSpecBuilder.java │ │ ├── ModeledFramework.java │ │ ├── ModeledFrameworkBuilder.java │ │ ├── ModeledOptions.java │ │ ├── NodeName.java │ │ ├── Resolvable.java │ │ ├── ZNode.java │ │ ├── ZPath.java │ │ ├── cached │ │ ├── CachedModeledFramework.java │ │ ├── ModeledCache.java │ │ └── ModeledCacheListener.java │ │ ├── details │ │ ├── CachedModeledFrameworkImpl.java │ │ ├── ModelSpecImpl.java │ │ ├── ModelStage.java │ │ ├── ModeledCacheImpl.java │ │ ├── ModeledFrameworkImpl.java │ │ ├── VersionedModeledFrameworkImpl.java │ │ ├── ZNodeImpl.java │ │ └── ZPathImpl.java │ │ ├── typed │ │ ├── TypedModelSpec.java │ │ ├── TypedModelSpec0.java │ │ ├── TypedModelSpec10.java │ │ ├── TypedModelSpec2.java │ │ ├── TypedModelSpec3.java │ │ ├── TypedModelSpec4.java │ │ ├── TypedModelSpec5.java │ │ ├── TypedModelSpec6.java │ │ ├── TypedModelSpec7.java │ │ ├── TypedModelSpec8.java │ │ ├── TypedModelSpec9.java │ │ ├── TypedModeledFramework.java │ │ ├── TypedModeledFramework0.java │ │ ├── TypedModeledFramework10.java │ │ ├── TypedModeledFramework2.java │ │ ├── TypedModeledFramework3.java │ │ ├── TypedModeledFramework4.java │ │ ├── TypedModeledFramework5.java │ │ ├── TypedModeledFramework6.java │ │ ├── TypedModeledFramework7.java │ │ ├── TypedModeledFramework8.java │ │ ├── TypedModeledFramework9.java │ │ ├── TypedZPath.java │ │ ├── TypedZPath0.java │ │ ├── TypedZPath10.java │ │ ├── TypedZPath2.java │ │ ├── TypedZPath3.java │ │ ├── TypedZPath4.java │ │ ├── TypedZPath5.java │ │ ├── TypedZPath6.java │ │ ├── TypedZPath7.java │ │ ├── TypedZPath8.java │ │ └── TypedZPath9.java │ │ └── versioned │ │ ├── Versioned.java │ │ └── VersionedModeledFramework.java │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── curator │ │ ├── framework │ │ └── imps │ │ │ ├── TestAddWatch.java │ │ │ ├── TestFramework.java │ │ │ └── TestFrameworkBackground.java │ │ └── x │ │ └── async │ │ ├── CompletableBaseClassForTests.java │ │ ├── TestAsyncWrappers.java │ │ ├── TestBasicOperations.java │ │ ├── migrations │ │ ├── TestMigrationManager.java │ │ └── models │ │ │ ├── ModelV1.java │ │ │ ├── ModelV2.java │ │ │ └── ModelV3.java │ │ └── modeled │ │ ├── TestCachedModeledFramework.java │ │ ├── TestModeledFramework.java │ │ ├── TestModeledFrameworkBase.java │ │ ├── TestZPath.java │ │ └── models │ │ ├── TestModel.java │ │ ├── TestNewerModel.java │ │ └── TestSimpleModel.java │ └── resources │ └── log4j.properties ├── curator-x-discovery-server ├── LICENSE ├── NOTICE ├── README.txt ├── pom.xml └── src │ ├── main │ └── java │ │ └── org │ │ └── apache │ │ └── curator │ │ └── x │ │ └── discovery │ │ └── server │ │ ├── contexts │ │ ├── GenericDiscoveryContext.java │ │ ├── IntegerDiscoveryContext.java │ │ ├── MapDiscoveryContext.java │ │ └── StringDiscoveryContext.java │ │ ├── entity │ │ ├── JsonServiceInstanceMarshaller.java │ │ ├── JsonServiceInstancesMarshaller.java │ │ ├── JsonServiceNamesMarshaller.java │ │ ├── ServiceInstances.java │ │ └── ServiceNames.java │ │ └── rest │ │ ├── DiscoveryContext.java │ │ ├── DiscoveryResource.java │ │ └── InstanceCleanup.java │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── curator │ │ └── x │ │ └── discovery │ │ └── server │ │ ├── jetty_jersey │ │ ├── MapDiscoveryResource.java │ │ ├── ServiceDetails.java │ │ ├── ServiceDetailsDiscoveryContext.java │ │ ├── ServiceDetailsDiscoveryResource.java │ │ ├── StringDiscoveryResource.java │ │ ├── TestMapsWithJersey.java │ │ ├── TestObjectPayloadWithJersey.java │ │ └── TestStringsWithJersey.java │ │ ├── jetty_resteasy │ │ ├── RestEasyApplication.java │ │ ├── RestEasySingletons.java │ │ ├── StringDiscoveryResource.java │ │ └── TestStringsWithRestEasy.java │ │ └── mocks │ │ └── MockServiceDiscovery.java │ └── resources │ └── log4j.properties ├── curator-x-discovery ├── LICENSE ├── NOTICE ├── pom.xml └── src │ ├── main │ └── java │ │ └── org │ │ └── apache │ │ └── curator │ │ └── x │ │ └── discovery │ │ ├── DiscoveryPathConstructor.java │ │ ├── DownInstancePolicy.java │ │ ├── InstanceFilter.java │ │ ├── LocalIpFilter.java │ │ ├── ProviderStrategy.java │ │ ├── ServiceCache.java │ │ ├── ServiceCacheBuilder.java │ │ ├── ServiceDiscovery.java │ │ ├── ServiceDiscoveryBuilder.java │ │ ├── ServiceInstance.java │ │ ├── ServiceInstanceBuilder.java │ │ ├── ServiceProvider.java │ │ ├── ServiceProviderBuilder.java │ │ ├── ServiceType.java │ │ ├── UriSpec.java │ │ ├── details │ │ ├── DiscoveryPathConstructorImpl.java │ │ ├── DownInstanceManager.java │ │ ├── FilteredInstanceProvider.java │ │ ├── InstanceProvider.java │ │ ├── InstanceSerializer.java │ │ ├── JsonInstanceSerializer.java │ │ ├── Latch.java │ │ ├── OldServiceInstance.java │ │ ├── ServiceCacheBuilderImpl.java │ │ ├── ServiceCacheImpl.java │ │ ├── ServiceCacheListener.java │ │ ├── ServiceDiscoveryImpl.java │ │ ├── ServiceProviderBuilderImpl.java │ │ └── ServiceProviderImpl.java │ │ └── strategies │ │ ├── RandomStrategy.java │ │ ├── RoundRobinStrategy.java │ │ └── StickyStrategy.java │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── curator │ │ └── x │ │ └── discovery │ │ ├── ServiceCacheLeakTester.java │ │ ├── TestJsonInstanceSerializer.java │ │ ├── TestLocalIpFilter.java │ │ ├── TestServiceCache.java │ │ ├── TestStrategies.java │ │ ├── TestUriSpec.java │ │ └── details │ │ ├── DiscoveryPathConstructorImplTest.java │ │ ├── TestDownInstanceManager.java │ │ ├── TestJsonInstanceSerializerCompatibility.java │ │ ├── TestNewServiceInstance.java │ │ ├── TestServiceCacheRace.java │ │ ├── TestServiceDiscovery.java │ │ ├── TestServiceDiscoveryBuilder.java │ │ ├── TestServiceProvider.java │ │ └── TestWatchedInstances.java │ └── resources │ └── log4j.properties ├── doap.rdf ├── licenserc.toml ├── merge-pr.py ├── mvnw ├── mvnw.cmd └── pom.xml /.git-blame-ignore-revs: -------------------------------------------------------------------------------- 1 | # You can configure git to automatically use this file with the following config: 2 | # git config blame.ignoreRevsFile .git-blame-ignore-revs 3 | 4 | # ignore spotless format 5 | 484ede9a33abb1427ba58c77e5792814aac889f0 6 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | # Compiled source # 2 | ################### 3 | *.com 4 | *.class 5 | *.dll 6 | *.exe 7 | *.o 8 | *.so 9 | 10 | # Packages # 11 | ############ 12 | # it's better to unpack these files and commit the raw source 13 | # git has its own built in compression methods 14 | *.7z 15 | *.dmg 16 | *.gz 17 | *.iso 18 | *.jar 19 | *.rar 20 | *.tar 21 | *.zip 22 | 23 | # Logs and databases # 24 | ###################### 25 | *.log 26 | 27 | # OS generated files # 28 | ###################### 29 | .DS_Store* 30 | ehthumbs.db 31 | Icon? 32 | Thumbs.db 33 | 34 | # Editor Files # 35 | ################ 36 | *~ 37 | *.swp 38 | 39 | # Gradle Files # 40 | ################ 41 | .gradle 42 | 43 | # Build output directies 44 | /target 45 | */target 46 | /build 47 | */build 48 | */bin 49 | 50 | # IntelliJ specific files/directories 51 | out 52 | .idea 53 | *.ipr 54 | *.iws 55 | *.iml 56 | atlassian-ide-plugin.xml 57 | 58 | # Eclipse specific files/directories 59 | .classpath 60 | .project 61 | .settings 62 | .metadata 63 | 64 | # NetBeans specific files/directories 65 | .nbattrs 66 | 67 | # Gradle Enterprise 68 | test-reports/ 69 | .mvn/.gradle-enterprise/ 70 | .mvn/.develocity/ 71 | -------------------------------------------------------------------------------- /.mvn/wrapper/maven-wrapper.properties: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, 12 | # software distributed under the License is distributed on an 13 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | # KIND, either express or implied. See the License for the 15 | # specific language governing permissions and limitations 16 | # under the License. 17 | distributionUrl=https://repo.maven.apache.org/maven2/org/apache/maven/apache-maven/3.9.4/apache-maven-3.9.4-bin.zip 18 | wrapperUrl=https://repo.maven.apache.org/maven2/org/apache/maven/wrapper/maven-wrapper/3.2.0/maven-wrapper-3.2.0.jar 19 | -------------------------------------------------------------------------------- /DEPENDENCIES: -------------------------------------------------------------------------------- 1 | * intentionally left blank * 2 | -------------------------------------------------------------------------------- /NOTICE: -------------------------------------------------------------------------------- 1 | Apache Curator 2 | Copyright 2013-2023 The Apache Software Foundation 3 | 4 | This product includes software developed at 5 | The Apache Software Foundation (http://www.apache.org/). 6 | -------------------------------------------------------------------------------- /curator-client/NOTICE: -------------------------------------------------------------------------------- 1 | Apache Curator 2 | Copyright 2013-2023 The Apache Software Foundation 3 | 4 | This product includes software developed at 5 | The Apache Software Foundation (http://www.apache.org/). 6 | -------------------------------------------------------------------------------- /curator-client/src/main/java/org/apache/curator/CuratorConnectionLossException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator; 21 | 22 | import org.apache.zookeeper.KeeperException; 23 | 24 | public class CuratorConnectionLossException extends KeeperException.ConnectionLossException { 25 | private static final long serialVersionUID = 1L; 26 | } 27 | -------------------------------------------------------------------------------- /curator-client/src/main/java/org/apache/curator/RetrySleeper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator; 21 | 22 | import java.util.concurrent.TimeUnit; 23 | 24 | /** 25 | * Abstraction for retry policies to sleep 26 | */ 27 | public interface RetrySleeper { 28 | /** 29 | * Sleep for the given time 30 | * 31 | * @param time time 32 | * @param unit time unit 33 | * @throws InterruptedException if the sleep is interrupted 34 | */ 35 | public void sleepFor(long time, TimeUnit unit) throws InterruptedException; 36 | } 37 | -------------------------------------------------------------------------------- /curator-client/src/main/java/org/apache/curator/retry/RetryNTimes.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.retry; 21 | 22 | /** 23 | * Retry policy that retries a max number of times 24 | */ 25 | public class RetryNTimes extends SleepingRetry { 26 | private final int sleepMsBetweenRetries; 27 | 28 | public RetryNTimes(int n, int sleepMsBetweenRetries) { 29 | super(n); 30 | this.sleepMsBetweenRetries = sleepMsBetweenRetries; 31 | } 32 | 33 | @Override 34 | protected long getSleepTimeMs(int retryCount, long elapsedTimeMs) { 35 | return sleepMsBetweenRetries; 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /curator-client/src/main/java/org/apache/curator/retry/RetryOneTime.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.retry; 21 | 22 | /** 23 | * A retry policy that retries only once 24 | */ 25 | public class RetryOneTime extends RetryNTimes { 26 | public RetryOneTime(int sleepMsBetweenRetry) { 27 | super(1, sleepMsBetweenRetry); 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /curator-client/src/main/java/org/apache/curator/utils/DefaultZookeeperFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.utils; 21 | 22 | import org.apache.zookeeper.Watcher; 23 | import org.apache.zookeeper.ZooKeeper; 24 | import org.apache.zookeeper.admin.ZooKeeperAdmin; 25 | 26 | public class DefaultZookeeperFactory implements ZookeeperFactory { 27 | @Override 28 | public ZooKeeper newZooKeeper(String connectString, int sessionTimeout, Watcher watcher, boolean canBeReadOnly) 29 | throws Exception { 30 | return new ZooKeeperAdmin(connectString, sessionTimeout, watcher, canBeReadOnly); 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /curator-client/src/main/java/org/apache/curator/utils/NonAdminZookeeperFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.utils; 21 | 22 | import org.apache.zookeeper.Watcher; 23 | import org.apache.zookeeper.ZooKeeper; 24 | 25 | public class NonAdminZookeeperFactory implements ZookeeperFactory { 26 | @Override 27 | public ZooKeeper newZooKeeper(String connectString, int sessionTimeout, Watcher watcher, boolean canBeReadOnly) 28 | throws Exception { 29 | return new ZooKeeper(connectString, sessionTimeout, watcher, canBeReadOnly); 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /curator-client/src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, 12 | # software distributed under the License is distributed on an 13 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | # KIND, either express or implied. See the License for the 15 | # specific language governing permissions and limitations 16 | # under the License. 17 | 18 | log4j.rootLogger=ERROR, console 19 | 20 | log4j.logger.org.apache.curator=DEBUG, console 21 | log4j.additivity.org.apache.curator=false 22 | 23 | log4j.appender.console=org.apache.log4j.ConsoleAppender 24 | log4j.appender.console.layout=org.apache.log4j.PatternLayout 25 | log4j.appender.console.layout.ConversionPattern=%-5p %c %x %m [%t]%n 26 | -------------------------------------------------------------------------------- /curator-examples/NOTICE: -------------------------------------------------------------------------------- 1 | Apache Curator 2 | Copyright 2013-2023 The Apache Software Foundation 3 | 4 | This product includes software developed at 5 | The Apache Software Foundation (http://www.apache.org/). 6 | -------------------------------------------------------------------------------- /curator-examples/src/main/java/pubsub/models/Group.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package pubsub.models; 21 | 22 | import org.apache.curator.x.async.modeled.NodeName; 23 | 24 | public class Group implements NodeName { 25 | private final String groupName; 26 | 27 | public Group() { 28 | this(""); 29 | } 30 | 31 | public Group(String groupName) { 32 | this.groupName = groupName; 33 | } 34 | 35 | public String getGroupName() { 36 | return groupName; 37 | } 38 | 39 | @Override 40 | public String nodeName() { 41 | return groupName; 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /curator-examples/src/main/java/pubsub/models/InstanceType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package pubsub.models; 21 | 22 | public enum InstanceType { 23 | database, 24 | cache, 25 | web, 26 | proxy 27 | } 28 | -------------------------------------------------------------------------------- /curator-examples/src/main/java/pubsub/models/Priority.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package pubsub.models; 21 | 22 | public enum Priority { 23 | low, 24 | medium, 25 | high 26 | } 27 | -------------------------------------------------------------------------------- /curator-examples/src/main/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, 12 | # software distributed under the License is distributed on an 13 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | # KIND, either express or implied. See the License for the 15 | # specific language governing permissions and limitations 16 | # under the License. 17 | 18 | log4j.rootLogger=ERROR, console 19 | 20 | log4j.appender.console=org.apache.log4j.ConsoleAppender 21 | log4j.appender.console.layout=org.apache.log4j.PatternLayout 22 | log4j.appender.console.layout.ConversionPattern=%-5p %c %x %m [%t]%n 23 | -------------------------------------------------------------------------------- /curator-framework/NOTICE: -------------------------------------------------------------------------------- 1 | Apache Curator 2 | Copyright 2013-2023 The Apache Software Foundation 3 | 4 | This product includes software developed at 5 | The Apache Software Foundation (http://www.apache.org/). 6 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/WatcherRemoveCuratorFramework.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework; 21 | 22 | /** 23 | * A CuratorFramework facade that tracks watchers created and allows a one-shot removal of all watchers 24 | */ 25 | public interface WatcherRemoveCuratorFramework extends CuratorFramework { 26 | /** 27 | * Remove all outstanding watchers that have been set 28 | */ 29 | void removeWatchers(); 30 | } 31 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/ACLBackgroundPathAndBytesable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface ACLBackgroundPathAndBytesable 23 | extends ParentACLable>, BackgroundPathAndBytesable {} 24 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/ACLCreateModeBackgroundPathAndBytesable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface ACLCreateModeBackgroundPathAndBytesable 23 | extends ACLBackgroundPathAndBytesable, 24 | BackgroundPathAndBytesable, 25 | CreateModable> {} 26 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/ACLCreateModePathAndBytesable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface ACLCreateModePathAndBytesable 23 | extends ACLPathAndBytesable, CreateModable> {} 24 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/ACLCreateModeStatBackgroundPathAndBytesable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface ACLCreateModeStatBackgroundPathAndBytesable 23 | extends ACLCreateModeBackgroundPathAndBytesable, Statable> {} 24 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/ACLPathAndBytesable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface ACLPathAndBytesable extends ParentACLable>, PathAndBytesable {} 23 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/ACLable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | import java.util.List; 23 | import org.apache.zookeeper.ZooDefs; 24 | import org.apache.zookeeper.data.ACL; 25 | 26 | public interface ACLable { 27 | /** 28 | * Set an ACL list (default is {@link ZooDefs.Ids#OPEN_ACL_UNSAFE}) 29 | * 30 | * @param aclList the ACL list to use 31 | * @return this 32 | */ 33 | T withACL(List aclList); 34 | } 35 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/ACLableExistBuilderMain.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface ACLableExistBuilderMain extends ExistsBuilderMain, ACLable {} 23 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/AddStatConfigEnsembleable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | /** 23 | * An non-incremental reconfiguration builder. 24 | * This builder has access only to the non-incremental reconfiguration methods withMembers, so that we prevent 25 | * mixing concepts that can't be used together. 26 | */ 27 | public interface AddStatConfigEnsembleable 28 | extends Addable>, ConfigureEnsembleable, Statable {} 29 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/AddWatchBuilder.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | import org.apache.zookeeper.AddWatchMode; 23 | 24 | public interface AddWatchBuilder extends AddWatchBuilder2 { 25 | /** 26 | * The mode to use. By default, {@link org.apache.zookeeper.AddWatchMode#PERSISTENT_RECURSIVE} is used 27 | * 28 | * @param mode mode to use 29 | * @return this 30 | */ 31 | AddWatchBuilder2 withMode(AddWatchMode mode); 32 | } 33 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/AddWatchBuilder2.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface AddWatchBuilder2 23 | extends Backgroundable>>, WatchableBase>, Pathable {} 24 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/AsyncReconfigurable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface AsyncReconfigurable { 23 | 24 | /** 25 | * Sets the configuration version to use. 26 | * @param config The version of the configuration. 27 | * @throws Exception 28 | */ 29 | void fromConfig(long config) throws Exception; 30 | } 31 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundCallback.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | import org.apache.curator.framework.CuratorFramework; 23 | 24 | /** 25 | * Functor for an async background operation 26 | */ 27 | public interface BackgroundCallback { 28 | /** 29 | * Called when the async background operation completes 30 | * 31 | * @param client the client 32 | * @param event operation result details 33 | * @throws Exception errors 34 | */ 35 | public void processResult(CuratorFramework client, CuratorEvent event) throws Exception; 36 | } 37 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundEnsembleable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface BackgroundEnsembleable extends Backgroundable>, Ensembleable {} 23 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundPathAndBytesable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface BackgroundPathAndBytesable 23 | extends Backgroundable>, PathAndBytesable {} 24 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundPathable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface BackgroundPathable extends Backgroundable>, Pathable {} 23 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundPathableQuietlyable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface BackgroundPathableQuietlyable extends BackgroundPathable, Quietly> {} 23 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/BackgroundVersionable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface BackgroundVersionable extends BackgroundPathable, Versionable> {} 23 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/ChildrenDeletable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface ChildrenDeletable extends BackgroundVersionable { 23 | 24 | /** 25 | *

26 | * Will also delete children if they exist. 27 | *

28 | * @return 29 | */ 30 | public BackgroundVersionable deletingChildrenIfNeeded(); 31 | } 32 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/Compressible.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface Compressible { 23 | /** 24 | * Cause the data to be compressed using the configured compression provider 25 | * 26 | * @return this 27 | */ 28 | public T compressed(); 29 | 30 | /** 31 | * Cause the data to be uncompressed, even if the {@link org.apache.curator.framework.CuratorFramework} 32 | * has compressionEnabled 33 | * 34 | * @return this 35 | */ 36 | public T uncompressed(); 37 | } 38 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/CompressionProvider.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface CompressionProvider { 23 | public byte[] compress(String path, byte[] data) throws Exception; 24 | 25 | public byte[] decompress(String path, byte[] compressedData) throws Exception; 26 | } 27 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/ConfigureEnsembleable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface ConfigureEnsembleable extends Ensembleable { 23 | 24 | /** 25 | * Sets the configuration version to use. 26 | * @param config The version of the configuration. 27 | * @throws Exception 28 | */ 29 | Ensembleable fromConfig(long config) throws Exception; 30 | } 31 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/CreateModable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | import org.apache.zookeeper.CreateMode; 23 | 24 | public interface CreateModable { 25 | /** 26 | * Set a create mode - the default is {@link CreateMode#PERSISTENT} 27 | * 28 | * @param mode new create mode 29 | * @return this 30 | */ 31 | public T withMode(CreateMode mode); 32 | } 33 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/CuratorClosedException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public class CuratorClosedException extends IllegalStateException { 23 | public CuratorClosedException() { 24 | super("Expected state [STARTED] was [STOPPED]"); 25 | } 26 | } 27 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/CuratorListener.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | import org.apache.curator.framework.CuratorFramework; 23 | 24 | /** 25 | * Receives notifications about errors and background events 26 | */ 27 | public interface CuratorListener { 28 | /** 29 | * Called when a background task has completed or a watch has triggered 30 | * 31 | * @param client client 32 | * @param event the event 33 | * @throws Exception any errors 34 | */ 35 | public void eventReceived(CuratorFramework client, CuratorEvent event) throws Exception; 36 | } 37 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/CuratorWatcher.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | import org.apache.zookeeper.WatchedEvent; 23 | import org.apache.zookeeper.Watcher; 24 | 25 | /** 26 | * A version of {@link Watcher} that can throw an exception 27 | */ 28 | public interface CuratorWatcher { 29 | /** 30 | * Same as {@link Watcher#process(WatchedEvent)}. If an exception 31 | * is thrown, Curator will log it 32 | * 33 | * @param event the event 34 | * @throws Exception any exceptions to log 35 | */ 36 | void process(WatchedEvent event) throws Exception; 37 | } 38 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/DataCallbackable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | import org.apache.zookeeper.AsyncCallback.DataCallback; 23 | 24 | public interface DataCallbackable { 25 | 26 | /** 27 | * Passes a callback and a context object to the config/reconfig command. 28 | * @param callback The async callback to use. 29 | * @param ctx An object that will be passed to the callback. 30 | * @return this 31 | */ 32 | T usingDataCallback(DataCallback callback, Object ctx); 33 | } 34 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/Decompressible.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface Decompressible { 23 | /** 24 | * Cause the data to be de-compressed using the configured compression provider 25 | * 26 | * @return this 27 | */ 28 | public T decompressed(); 29 | 30 | /** 31 | * Cause the data to not be de-compressed, even if the {@link org.apache.curator.framework.CuratorFramework} 32 | * has compressionEnabled 33 | * 34 | * @return this 35 | */ 36 | public T undecompressed(); 37 | } 38 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/DeleteBuilder.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface DeleteBuilder 23 | extends Quietly, DeleteBuilderMain, Idempotentable {} 24 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/DeleteBuilderMain.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface DeleteBuilderMain extends GuaranteeableDeletable, ChildrenDeletable {} 23 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/Ensembleable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface Ensembleable { 23 | 24 | T forEnsemble() throws Exception; 25 | } 26 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/ExistsBuilderMain.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | import org.apache.zookeeper.data.Stat; 23 | 24 | public interface ExistsBuilderMain extends Watchable>, BackgroundPathable {} 25 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/GetACLBuilder.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | import java.util.List; 23 | import org.apache.zookeeper.data.ACL; 24 | 25 | public interface GetACLBuilder extends BackgroundPathable>, Statable>> {} 26 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/GetChildrenBuilder.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | import java.util.List; 23 | 24 | public interface GetChildrenBuilder 25 | extends Watchable>>, 26 | BackgroundPathable>, 27 | Statable>> {} 28 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/GetConfigBuilder.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface GetConfigBuilder 23 | extends Ensembleable, 24 | Backgroundable>, 25 | Watchable>, 26 | Statable> {} 27 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/GetDataBuilder.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface GetDataBuilder 23 | extends Watchable>, 24 | BackgroundPathable, 25 | Statable>, 26 | Decompressible {} 27 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/GetDataWatchBackgroundStatable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface GetDataWatchBackgroundStatable 23 | extends Watchable>, BackgroundPathable, Statable> {} 24 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/Guaranteeable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface Guaranteeable { 23 | /** 24 | * Solves edge cases where an operation may succeed on the server but connection failure occurs before a 25 | * response can be successfully returned to the client. 26 | * 27 | * @see org.apache.curator.framework.api.GuaranteeableDeletable 28 | * 29 | * @return this 30 | */ 31 | public T guaranteed(); 32 | } 33 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/Idempotentable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface Idempotentable { 23 | /** 24 | * If the first try of this operation gets a transient error, curator will retry 25 | * the operation, and treat it as successful so long as the end state of the znode 26 | * is the same as if the operation had completed without error on the first try. 27 | * @return this: 28 | */ 29 | public T idempotent(); 30 | } 31 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/JoinStatConfigEnsembleable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | /** 23 | * An incremental reconfiguration builder. 24 | * This builder has access only to the incremental reconfiguration methods joining and leaving, so that we prevent 25 | * mixing concepts that can't be used together. 26 | */ 27 | public interface JoinStatConfigEnsembleable 28 | extends Joinable, ConfigureEnsembleable, Statable {} 29 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/LeaveStatConfigEnsembleable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | /** 23 | * An incremental reconfiguration builder. 24 | * This builder has access only to the incremental reconfiguration methods joining and leaving, so that we prevent 25 | * mixing concepts that can't be used together. 26 | */ 27 | public interface LeaveStatConfigEnsembleable 28 | extends Leaveable, ConfigureEnsembleable, Statable {} 29 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/Leaveable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | import java.util.List; 23 | 24 | public interface Leaveable { 25 | /** 26 | * Sets one or more servers to leaving the ensemble. 27 | * 28 | * @param server The server ids 29 | * @return this 30 | */ 31 | T leaving(String... server); 32 | 33 | /** 34 | * Sets one or more servers to leaving the ensemble. 35 | * 36 | * @param servers The server ids 37 | * @return this 38 | */ 39 | T leaving(List servers); 40 | } 41 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/Pathable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface Pathable { 23 | /** 24 | * Commit the currently building operation using the given path 25 | * 26 | * @param path the path 27 | * @return operation result if any 28 | * @throws Exception errors 29 | */ 30 | public T forPath(String path) throws Exception; 31 | } 32 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/ProtectACLCreateModeStatPathAndBytesable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface ProtectACLCreateModeStatPathAndBytesable 23 | extends ProtectACLCreateModePathAndBytesable, Statable> {} 24 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/Quietly.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface Quietly { 23 | public T quietly(); 24 | } 25 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/ReconfigBuilder.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface ReconfigBuilder extends ReconfigBuilderMain, Backgroundable {} 23 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/ReconfigBuilderMain.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface ReconfigBuilderMain 23 | extends Joinable, 24 | Leaveable, 25 | Membersable {} 26 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/SetACLBuilder.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | import org.apache.zookeeper.data.Stat; 23 | 24 | public interface SetACLBuilder 25 | extends ACLable>, Versionable>> {} 26 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/SetDataBackgroundVersionable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | import org.apache.zookeeper.data.Stat; 23 | 24 | public interface SetDataBackgroundVersionable 25 | extends BackgroundPathAndBytesable, Versionable> {} 26 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/SetDataBuilder.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | import org.apache.zookeeper.data.Stat; 23 | 24 | public interface SetDataBuilder 25 | extends BackgroundPathAndBytesable, 26 | Versionable>, 27 | Compressible, 28 | Idempotentable {} 29 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/StatConfigureEnsembleable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface StatConfigureEnsembleable extends Statable, ConfigureEnsembleable {} 23 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/StatPathable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface StatPathable extends Pathable, Statable> {} 23 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/Statable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | import org.apache.zookeeper.data.Stat; 23 | 24 | public interface Statable { 25 | /** 26 | * Have the operation fill the provided stat object 27 | * 28 | * @param stat the stat to have filled in 29 | * @return this 30 | */ 31 | public T storingStatIn(Stat stat); 32 | } 33 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/SyncBuilder.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface SyncBuilder extends BackgroundPathable {} 23 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/TempGetDataBuilder.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface TempGetDataBuilder 23 | extends StatPathable, Decompressible>, Pathable {} 24 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/UnhandledErrorListener.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface UnhandledErrorListener { 23 | /** 24 | * Called when an exception is caught in a background thread, handler, etc. 25 | * 26 | * @param message Source message 27 | * @param e exception 28 | */ 29 | public void unhandledError(String message, Throwable e); 30 | } 31 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/VersionPathAndBytesable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface VersionPathAndBytesable extends Versionable>, PathAndBytesable {} 23 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/Versionable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface Versionable { 23 | /** 24 | * Use the given version (the default is -1) 25 | * 26 | * @param version version to use 27 | * @return this 28 | */ 29 | public T withVersion(int version); 30 | } 31 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/WatchBackgroundEnsembleable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface WatchBackgroundEnsembleable 23 | extends Watchable>, BackgroundEnsembleable {} 24 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/WatchPathable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface WatchPathable extends Watchable>, Pathable {} 23 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/Watchable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | public interface Watchable extends WatchableBase { 23 | /** 24 | * Have the operation set a watch 25 | * 26 | * @return this 27 | */ 28 | T watched(); 29 | } 30 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/WatchableBase.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | import org.apache.zookeeper.Watcher; 23 | 24 | public interface WatchableBase { 25 | /** 26 | * Set a watcher for the operation 27 | * 28 | * @param watcher the watcher 29 | * @return this 30 | */ 31 | T usingWatcher(Watcher watcher); 32 | 33 | /** 34 | * Set a watcher for the operation 35 | * 36 | * @param watcher the watcher 37 | * @return this 38 | */ 39 | T usingWatcher(CuratorWatcher watcher); 40 | } 41 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/WatchesBuilder.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api; 21 | 22 | /** 23 | * Builder to allow watches to be removed 24 | */ 25 | public interface WatchesBuilder extends RemoveWatchesBuilder { 26 | /** 27 | * Start an add watch operation 28 | * 29 | * @return builder 30 | */ 31 | AddWatchBuilder add(); 32 | } 33 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/transaction/CuratorMultiTransaction.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api.transaction; 21 | 22 | import org.apache.curator.framework.api.Backgroundable; 23 | import org.apache.curator.framework.api.ErrorListenerMultiTransactionMain; 24 | 25 | public interface CuratorMultiTransaction 26 | extends Backgroundable, CuratorMultiTransactionMain {} 27 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/transaction/CuratorOp.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api.transaction; 21 | 22 | import org.apache.zookeeper.Op; 23 | 24 | /** 25 | * Internal representation of a transaction operation 26 | */ 27 | public interface CuratorOp { 28 | Op get(); 29 | 30 | TypeAndPath getTypeAndPath(); 31 | } 32 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/transaction/CuratorTransactionBridge.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api.transaction; 21 | 22 | public interface CuratorTransactionBridge { 23 | /** 24 | * Syntactic sugar to make the fluent interface more readable 25 | * 26 | * @return transaction continuation 27 | */ 28 | public CuratorTransactionFinal and(); 29 | } 30 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/transaction/OperationType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api.transaction; 21 | 22 | /** 23 | * Transaction operation types 24 | */ 25 | public enum OperationType { 26 | /** 27 | * {@link TransactionOp#create()} 28 | */ 29 | CREATE, 30 | 31 | /** 32 | * {@link TransactionOp#delete()} 33 | */ 34 | DELETE, 35 | 36 | /** 37 | * {@link TransactionOp#setData()} 38 | */ 39 | SET_DATA, 40 | 41 | /** 42 | * {@link TransactionOp#check()} 43 | */ 44 | CHECK 45 | } 46 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionCheckBuilder.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api.transaction; 21 | 22 | import org.apache.curator.framework.api.Pathable; 23 | import org.apache.curator.framework.api.Versionable; 24 | 25 | public interface TransactionCheckBuilder extends Pathable, Versionable> {} 26 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TransactionDeleteBuilder.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api.transaction; 21 | 22 | import org.apache.curator.framework.api.Pathable; 23 | import org.apache.curator.framework.api.Versionable; 24 | 25 | public interface TransactionDeleteBuilder extends Pathable, Versionable> {} 26 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/api/transaction/TypeAndPath.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.api.transaction; 21 | 22 | public class TypeAndPath { 23 | private final OperationType type; 24 | private final String forPath; 25 | 26 | public TypeAndPath(OperationType type, String forPath) { 27 | this.type = type; 28 | this.forPath = forPath; 29 | } 30 | 31 | public OperationType getType() { 32 | return type; 33 | } 34 | 35 | public String getForPath() { 36 | return forPath; 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/imps/BackgroundOperation.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.imps; 21 | 22 | import org.apache.curator.framework.api.CuratorEventType; 23 | 24 | interface BackgroundOperation { 25 | public void performBackgroundOperation(OperationAndData data) throws Exception; 26 | 27 | /** 28 | * Most events are delivered by operations themselves, so they know their event types. 29 | * But in occasional cases(says, closing or background exception), events are delivered 30 | * by curator framework. 31 | */ 32 | CuratorEventType getBackgroundEventType(); 33 | } 34 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkState.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.imps; 21 | 22 | import org.apache.curator.framework.CuratorFramework; 23 | 24 | /** 25 | * @see CuratorFramework#getState() 26 | */ 27 | public enum CuratorFrameworkState { 28 | /** 29 | * {@link CuratorFramework#start()} has not yet been called 30 | */ 31 | LATENT, 32 | 33 | /** 34 | * {@link CuratorFramework#start()} has been called 35 | */ 36 | STARTED, 37 | 38 | /** 39 | * {@link CuratorFramework#close()} has been called 40 | */ 41 | STOPPED 42 | } 43 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/imps/DefaultACLProvider.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.imps; 21 | 22 | import java.util.List; 23 | import org.apache.curator.framework.api.ACLProvider; 24 | import org.apache.zookeeper.ZooDefs; 25 | import org.apache.zookeeper.data.ACL; 26 | 27 | public class DefaultACLProvider implements ACLProvider { 28 | @Override 29 | public List getDefaultAcl() { 30 | return ZooDefs.Ids.OPEN_ACL_UNSAFE; 31 | } 32 | 33 | @Override 34 | public List getAclForPath(String path) { 35 | return ZooDefs.Ids.OPEN_ACL_UNSAFE; 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/imps/FailedDeleteManager.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.imps; 21 | 22 | import org.apache.curator.framework.CuratorFramework; 23 | 24 | class FailedDeleteManager extends FailedOperationManager { 25 | FailedDeleteManager(CuratorFramework client) { 26 | super(client); 27 | } 28 | 29 | @Override 30 | protected void executeGuaranteedOperationInBackground(String path) throws Exception { 31 | client.delete().guaranteed().inBackground().forPath(path); 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/imps/IdempotentUtils.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.imps; 21 | 22 | import java.util.Arrays; 23 | 24 | /* 25 | * Utilties Class for idempotent operations. 26 | */ 27 | class IdempotentUtils { 28 | 29 | /** 30 | * Returns whether the version and data currently in the node match what would be expected in the idempotent retry case. 31 | */ 32 | static boolean matches(int expectedVersion, byte[] expectedData, int actualVersion, byte[] actualData) { 33 | return expectedVersion == actualVersion && Arrays.equals(expectedData, actualData); 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/imps/InternalConnectionHandler.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.imps; 21 | 22 | interface InternalConnectionHandler { 23 | void checkNewConnection(CuratorFrameworkImpl client); 24 | 25 | void suspendConnection(CuratorFrameworkImpl client); 26 | } 27 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/imps/NamespaceWatchedEvent.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.imps; 21 | 22 | import org.apache.zookeeper.WatchedEvent; 23 | 24 | class NamespaceWatchedEvent extends WatchedEvent { 25 | NamespaceWatchedEvent(CuratorFrameworkBase client, WatchedEvent event) { 26 | super(event.getType(), event.getState(), client.unfixForNamespace(event.getPath())); 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/imps/PathAndBytes.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.imps; 21 | 22 | class PathAndBytes { 23 | private final String path; 24 | private final byte[] data; 25 | 26 | PathAndBytes(String path, byte[] data) { 27 | this.path = path; 28 | this.data = data; 29 | } 30 | 31 | String getPath() { 32 | return path; 33 | } 34 | 35 | byte[] getData() { 36 | return data; 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/imps/StandardInternalConnectionHandler.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.imps; 21 | 22 | class StandardInternalConnectionHandler implements InternalConnectionHandler { 23 | @Override 24 | public void suspendConnection(CuratorFrameworkImpl client) { 25 | client.setToSuspended(); 26 | } 27 | 28 | @Override 29 | public void checkNewConnection(CuratorFrameworkImpl client) { 30 | client.checkInstanceIndex(); 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/listen/ListenerEntry.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.listen; 21 | 22 | import java.util.concurrent.Executor; 23 | 24 | /** 25 | * Generic holder POJO for a listener and its executor 26 | * @param the listener type 27 | */ 28 | public class ListenerEntry { 29 | public final T listener; 30 | public final Executor executor; 31 | 32 | public ListenerEntry(T listener, Executor executor) { 33 | this.listener = listener; 34 | this.executor = executor; 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/listen/UnaryListenerManager.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.listen; 21 | 22 | /** 23 | * A {@link ListenerManager} that doesn't do any mapping 24 | */ 25 | public interface UnaryListenerManager extends ListenerManager {} 26 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/schema/DefaultSchemaValidator.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.schema; 21 | 22 | import java.util.List; 23 | import org.apache.zookeeper.data.ACL; 24 | 25 | /** 26 | * The default data validator - always returns true 27 | */ 28 | public class DefaultSchemaValidator implements SchemaValidator { 29 | @Override 30 | public boolean isValid(Schema schema, String path, byte[] data, List acl) { 31 | return true; 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/schema/SchemaValidator.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.schema; 21 | 22 | import java.util.List; 23 | import org.apache.zookeeper.data.ACL; 24 | 25 | public interface SchemaValidator { 26 | /** 27 | * @param schema the schema being validated 28 | * @param path the path being operated on 29 | * @param data data or null 30 | * @param acl acls or null 31 | * @return true if valid 32 | */ 33 | boolean isValid(Schema schema, String path, byte[] data, List acl); 34 | } 35 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/state/SessionConnectionStateErrorPolicy.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.state; 21 | 22 | /** 23 | * This policy treats only {@link ConnectionState#LOST} as an error 24 | */ 25 | public class SessionConnectionStateErrorPolicy implements ConnectionStateErrorPolicy { 26 | @Override 27 | public boolean isErrorState(ConnectionState state) { 28 | return state == ConnectionState.LOST; 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /curator-framework/src/main/java/org/apache/curator/framework/state/StandardConnectionStateErrorPolicy.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.state; 21 | 22 | /** 23 | * This policy treats {@link ConnectionState#SUSPENDED} and {@link ConnectionState#LOST} 24 | * as errors 25 | */ 26 | public class StandardConnectionStateErrorPolicy implements ConnectionStateErrorPolicy { 27 | @Override 28 | public boolean isErrorState(ConnectionState state) { 29 | return ((state == ConnectionState.SUSPENDED) || (state == ConnectionState.LOST)); 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /curator-framework/src/test/java/org/apache/curator/framework/imps/TransactionsHelper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.imps; 21 | 22 | import com.google.common.base.Predicate; 23 | import org.apache.curator.framework.api.transaction.CuratorTransactionResult; 24 | import org.apache.curator.framework.api.transaction.OperationType; 25 | 26 | public class TransactionsHelper { 27 | public static Predicate ofTypeAndPath(OperationType type, String forPath) { 28 | return result -> (result.getType() == type) && result.getForPath().equals(forPath); 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /curator-framework/src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, 12 | # software distributed under the License is distributed on an 13 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | # KIND, either express or implied. See the License for the 15 | # specific language governing permissions and limitations 16 | # under the License. 17 | 18 | log4j.rootLogger=ERROR, console 19 | 20 | log4j.logger.org.apache.curator=DEBUG, console 21 | log4j.additivity.org.apache.curator=false 22 | 23 | log4j.appender.console=org.apache.log4j.ConsoleAppender 24 | log4j.appender.console.layout=org.apache.log4j.PatternLayout 25 | log4j.appender.console.layout.ConversionPattern=%-5p %c %x %m [%t]%n 26 | -------------------------------------------------------------------------------- /curator-framework/src/test/resources/schema.yaml: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, 12 | # software distributed under the License is distributed on an 13 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | # KIND, either express or implied. See the License for the 15 | # specific language governing permissions and limitations 16 | # under the License. 17 | 18 | - name: test 19 | path: /a/b/c 20 | documentation: This is a schema 21 | ephemeral: must 22 | sequential: cannot 23 | 24 | - name: test2 25 | path: /a/.* 26 | isRegex: true 27 | ephemeral: cannot 28 | canBeDeleted: false 29 | metadata: 30 | one: 1 31 | two: 2 32 | -------------------------------------------------------------------------------- /curator-framework/src/test/resources/schema1.json: -------------------------------------------------------------------------------- 1 | [ 2 | { 3 | "name": "test", 4 | "path": "/a/b/c", 5 | "documentation": "This is a schema", 6 | "ephemeral": "must", 7 | "sequential": "cannot", 8 | "metadata": { 9 | "one": 1, 10 | "two": "2" 11 | } 12 | } 13 | ] -------------------------------------------------------------------------------- /curator-framework/src/test/resources/schema2.json: -------------------------------------------------------------------------------- 1 | [ 2 | { 3 | "name": "test", 4 | "path": "/a/b/c", 5 | "documentation": "This is a schema", 6 | "ephemeral": "must", 7 | "sequential": "cannot" 8 | }, 9 | 10 | { 11 | "name": "test2", 12 | "path": "/a/.*", 13 | "isRegex": true, 14 | "ephemeral": "cannot", 15 | "canBeDeleted": false 16 | } 17 | ] -------------------------------------------------------------------------------- /curator-framework/src/test/resources/schema3.json: -------------------------------------------------------------------------------- 1 | [ 2 | { 3 | "name": "test", 4 | "path": "/test", 5 | "schemaValidator": "test" 6 | } 7 | ] -------------------------------------------------------------------------------- /curator-framework/src/test/resources/schema4.json: -------------------------------------------------------------------------------- 1 | [ 2 | { 3 | "name": "testa", 4 | "path": "/a", 5 | "ephemeral": "must", 6 | "canBeDeleted": false 7 | }, 8 | 9 | { 10 | "name": "testb", 11 | "path": "/b", 12 | "schemaValidator": "test" 13 | } 14 | ] -------------------------------------------------------------------------------- /curator-framework/src/test/resources/schema5.json: -------------------------------------------------------------------------------- 1 | [ 2 | { 3 | "name": "1", 4 | "path": "/exact/match", 5 | "isRegex": false, 6 | "ephemeral": "cannot", 7 | "sequential": "must" 8 | }, 9 | 10 | { 11 | "name": "2", 12 | "path": "/exact/other/.*", 13 | "isRegex": true, 14 | "ephemeral": "must", 15 | "sequential": "can" 16 | }, 17 | 18 | { 19 | "name": "3", 20 | "path": "/exact/.*", 21 | "isRegex": true, 22 | "ephemeral": "can", 23 | "sequential": "cannot" 24 | } 25 | ] -------------------------------------------------------------------------------- /curator-recipes/NOTICE: -------------------------------------------------------------------------------- 1 | Apache Curator 2 | Copyright 2013-2023 The Apache Software Foundation 3 | 4 | This product includes software developed at 5 | The Apache Software Foundation (http://www.apache.org/). 6 | -------------------------------------------------------------------------------- /curator-recipes/src/main/java/org/apache/curator/framework/recipes/atomic/MakeValue.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.recipes.atomic; 21 | 22 | interface MakeValue { 23 | public byte[] makeFrom(byte[] previous); 24 | } 25 | -------------------------------------------------------------------------------- /curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/DefaultTreeCacheSelector.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.recipes.cache; 21 | 22 | /** 23 | * Default TreeCache selector - returns true for all methods 24 | */ 25 | public class DefaultTreeCacheSelector implements TreeCacheSelector { 26 | @Override 27 | public boolean traverseChildren(String fullPath) { 28 | return true; 29 | } 30 | 31 | @Override 32 | public boolean acceptChild(String fullPath) { 33 | return true; 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/NodeCacheListener.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.recipes.cache; 21 | 22 | public interface NodeCacheListener { 23 | /** 24 | * Called when a change has occurred 25 | */ 26 | public void nodeChanged() throws Exception; 27 | } 28 | -------------------------------------------------------------------------------- /curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/NodeCacheListenerWrapper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.recipes.cache; 21 | 22 | class NodeCacheListenerWrapper implements CuratorCacheListener { 23 | private final NodeCacheListener listener; 24 | 25 | NodeCacheListenerWrapper(NodeCacheListener listener) { 26 | this.listener = listener; 27 | } 28 | 29 | @Override 30 | public void event(Type type, ChildData oldData, ChildData data) { 31 | try { 32 | listener.nodeChanged(); 33 | } catch (Exception e) { 34 | throw new RuntimeException(e); 35 | } 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/Operation.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.recipes.cache; 21 | 22 | interface Operation { 23 | public void invoke() throws Exception; 24 | } 25 | -------------------------------------------------------------------------------- /curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/PathChildrenCacheListener.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.recipes.cache; 21 | 22 | import org.apache.curator.framework.CuratorFramework; 23 | 24 | /** 25 | * Listener for PathChildrenCache changes 26 | */ 27 | public interface PathChildrenCacheListener { 28 | /** 29 | * Called when a change has occurred 30 | * 31 | * @param client the client 32 | * @param event describes the change 33 | * @throws Exception errors 34 | */ 35 | public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception; 36 | } 37 | -------------------------------------------------------------------------------- /curator-recipes/src/main/java/org/apache/curator/framework/recipes/cache/TreeCacheListener.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.recipes.cache; 21 | 22 | import org.apache.curator.framework.CuratorFramework; 23 | 24 | /** 25 | * Listener for {@link TreeCache} changes 26 | */ 27 | public interface TreeCacheListener { 28 | /** 29 | * Called when a change has occurred 30 | * 31 | * @param client the client 32 | * @param event describes the change 33 | * @throws Exception errors 34 | */ 35 | public void childEvent(CuratorFramework client, TreeCacheEvent event) throws Exception; 36 | } 37 | -------------------------------------------------------------------------------- /curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/LockInternalsDriver.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.recipes.locks; 21 | 22 | import java.util.List; 23 | import org.apache.curator.framework.CuratorFramework; 24 | 25 | public interface LockInternalsDriver extends LockInternalsSorter { 26 | public PredicateResults getsTheLock( 27 | CuratorFramework client, List children, String sequenceNodeName, int maxLeases) throws Exception; 28 | 29 | public String createsTheLock(CuratorFramework client, String path, byte[] lockNodeBytes) throws Exception; 30 | } 31 | -------------------------------------------------------------------------------- /curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/LockInternalsSorter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.recipes.locks; 21 | 22 | public interface LockInternalsSorter { 23 | public String fixForSorting(String str, String lockName); 24 | } 25 | -------------------------------------------------------------------------------- /curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/PredicateResults.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.recipes.locks; 21 | 22 | public class PredicateResults { 23 | private final boolean getsTheLock; 24 | private final String pathToWatch; 25 | 26 | public PredicateResults(String pathToWatch, boolean getsTheLock) { 27 | this.pathToWatch = pathToWatch; 28 | this.getsTheLock = getsTheLock; 29 | } 30 | 31 | public String getPathToWatch() { 32 | return pathToWatch; 33 | } 34 | 35 | public boolean getsTheLock() { 36 | return getsTheLock; 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/RevocationListener.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.recipes.locks; 21 | 22 | public interface RevocationListener { 23 | /** 24 | * Called when a revocation request has been received. You should release the lock as soon 25 | * as possible. Revocation is cooperative. 26 | * 27 | * @param forLock the lock that should release 28 | */ 29 | public void revocationRequested(T forLock); 30 | } 31 | -------------------------------------------------------------------------------- /curator-recipes/src/main/java/org/apache/curator/framework/recipes/locks/RevocationSpec.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.recipes.locks; 21 | 22 | import java.util.concurrent.Executor; 23 | 24 | class RevocationSpec { 25 | private final Runnable runnable; 26 | private final Executor executor; 27 | 28 | RevocationSpec(Executor executor, Runnable runnable) { 29 | this.runnable = runnable; 30 | this.executor = executor; 31 | } 32 | 33 | Runnable getRunnable() { 34 | return runnable; 35 | } 36 | 37 | Executor getExecutor() { 38 | return executor; 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /curator-recipes/src/main/java/org/apache/curator/framework/recipes/nodes/PersistentNodeListener.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.recipes.nodes; 21 | 22 | /** 23 | * Listener for changes to a PersistentNode 24 | */ 25 | public interface PersistentNodeListener { 26 | /** 27 | * Called on a persistentNode event when node is created 28 | * 29 | * @param path Path of the znode 30 | * @throws Exception errors 31 | */ 32 | void nodeCreated(String path) throws Exception; 33 | } 34 | -------------------------------------------------------------------------------- /curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/ErrorMode.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.recipes.queue; 21 | 22 | /** 23 | * Used when the queue is created with a {@link QueueBuilder#lockPath(String)}. Determines 24 | * the behavior when the queue consumer throws an exception 25 | */ 26 | public enum ErrorMode { 27 | /** 28 | * If the consumer throws an exception, requeue the message. This is the default. 29 | */ 30 | REQUEUE, 31 | 32 | /** 33 | * If the consumer throws an exception, delete the message 34 | */ 35 | DELETE 36 | } 37 | -------------------------------------------------------------------------------- /curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/MultiItem.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.recipes.queue; 21 | 22 | /** 23 | * Abstraction for multiple items. 24 | * @see DistributedQueue#putMulti(MultiItem) 25 | * @see DistributedPriorityQueue#putMulti(MultiItem, int) 26 | */ 27 | public interface MultiItem { 28 | /** 29 | * Called repeatedly to get the items to add to the queue. This method 30 | * should return null when there are no more items to add. 31 | * 32 | * @return item or null 33 | * @throws Exception any errors 34 | */ 35 | public T nextItem() throws Exception; 36 | } 37 | -------------------------------------------------------------------------------- /curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/QueueAllocator.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.recipes.queue; 21 | 22 | import org.apache.curator.framework.CuratorFramework; 23 | 24 | public interface QueueAllocator> { 25 | public T allocateQueue(CuratorFramework client, String queuePath); 26 | } 27 | -------------------------------------------------------------------------------- /curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/QueueConsumer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.recipes.queue; 21 | 22 | import org.apache.curator.framework.state.ConnectionStateListener; 23 | 24 | /** 25 | * Message Consumer 26 | */ 27 | public interface QueueConsumer extends ConnectionStateListener { 28 | /** 29 | * Process a message from the queue 30 | * 31 | * @param message message to process 32 | * @throws Exception any errors 33 | */ 34 | public void consumeMessage(T message) throws Exception; 35 | } 36 | -------------------------------------------------------------------------------- /curator-recipes/src/main/java/org/apache/curator/framework/recipes/queue/QueueSerializer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.recipes.queue; 21 | 22 | /** 23 | * Helper to serialize/deserialize queue items 24 | */ 25 | public interface QueueSerializer { 26 | /** 27 | * Turn a queue item into bytes 28 | * 29 | * @param item the item 30 | * @return byte representation 31 | */ 32 | public byte[] serialize(T item); 33 | 34 | /** 35 | * Deserialize bytes into a queue item 36 | * 37 | * @param bytes byte representation 38 | * @return item 39 | */ 40 | public T deserialize(byte[] bytes); 41 | } 42 | -------------------------------------------------------------------------------- /curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/Counter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.recipes.locks; 21 | 22 | class Counter { 23 | int currentCount = 0; 24 | int maxCount = 0; 25 | } 26 | -------------------------------------------------------------------------------- /curator-recipes/src/test/java/org/apache/curator/framework/recipes/locks/Stepper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.recipes.locks; 21 | 22 | class Stepper { 23 | private int available = 0; 24 | 25 | synchronized void await() throws InterruptedException { 26 | while (available == 0) { 27 | wait(); 28 | } 29 | --available; 30 | notifyAll(); 31 | } 32 | 33 | synchronized void countDown(int qty) { 34 | available += qty; 35 | notifyAll(); 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /curator-recipes/src/test/java/org/apache/curator/framework/recipes/queue/QueueItemSerializer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.recipes.queue; 21 | 22 | class QueueItemSerializer implements QueueSerializer { 23 | @Override 24 | public byte[] serialize(TestQueueItem item) { 25 | return item.str.getBytes(); 26 | } 27 | 28 | @Override 29 | public TestQueueItem deserialize(byte[] bytes) { 30 | return new TestQueueItem(new String(bytes)); 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /curator-recipes/src/test/java/org/apache/curator/framework/state/DummyConnectionStateListener.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.framework.state; 21 | 22 | import org.apache.curator.framework.CuratorFramework; 23 | 24 | public class DummyConnectionStateListener implements ConnectionStateListener { 25 | @Override 26 | public void stateChanged(CuratorFramework curatorFramework, ConnectionState connectionState) { 27 | // do nothing 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /curator-recipes/src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, 12 | # software distributed under the License is distributed on an 13 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | # KIND, either express or implied. See the License for the 15 | # specific language governing permissions and limitations 16 | # under the License. 17 | 18 | log4j.rootLogger=ERROR, console 19 | 20 | log4j.logger.org.apache.curator=DEBUG, console 21 | log4j.additivity.org.apache.curator=false 22 | 23 | log4j.appender.console=org.apache.log4j.ConsoleAppender 24 | log4j.appender.console.layout=org.apache.log4j.PatternLayout 25 | log4j.appender.console.layout.ConversionPattern=%d{ISO8601} %-5p %c %x %m [%t]%n 26 | -------------------------------------------------------------------------------- /curator-test-zk35/src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, 12 | # software distributed under the License is distributed on an 13 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | # KIND, either express or implied. See the License for the 15 | # specific language governing permissions and limitations 16 | # under the License. 17 | 18 | log4j.rootLogger=ERROR, console 19 | 20 | log4j.logger.org.apache.curator=DEBUG, console 21 | log4j.additivity.org.apache.curator=false 22 | 23 | log4j.appender.console=org.apache.log4j.ConsoleAppender 24 | log4j.appender.console.layout=org.apache.log4j.PatternLayout 25 | log4j.appender.console.layout.ConversionPattern=%-5p %c %x %m [%t]%n 26 | -------------------------------------------------------------------------------- /curator-test-zk36/src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, 12 | # software distributed under the License is distributed on an 13 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | # KIND, either express or implied. See the License for the 15 | # specific language governing permissions and limitations 16 | # under the License. 17 | 18 | log4j.rootLogger=ERROR, console 19 | 20 | log4j.logger.org.apache.curator=DEBUG, console 21 | log4j.additivity.org.apache.curator=false 22 | 23 | log4j.appender.console=org.apache.log4j.ConsoleAppender 24 | log4j.appender.console.layout=org.apache.log4j.PatternLayout 25 | log4j.appender.console.layout.ConversionPattern=%-5p %c %x %m [%t]%n 26 | -------------------------------------------------------------------------------- /curator-test-zk37/src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, 12 | # software distributed under the License is distributed on an 13 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | # KIND, either express or implied. See the License for the 15 | # specific language governing permissions and limitations 16 | # under the License. 17 | 18 | log4j.rootLogger=ERROR, console 19 | 20 | log4j.logger.org.apache.curator=DEBUG, console 21 | log4j.additivity.org.apache.curator=false 22 | 23 | log4j.appender.console=org.apache.log4j.ConsoleAppender 24 | log4j.appender.console.layout=org.apache.log4j.PatternLayout 25 | log4j.appender.console.layout.ConversionPattern=%-5p %c %x %m [%t]%n 26 | -------------------------------------------------------------------------------- /curator-test-zk38/src/test/java/org/apache/curator/zk38/TestIs38.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.zk38; 21 | 22 | import static org.assertj.core.api.Assertions.assertThat; 23 | import org.apache.curator.test.compatibility.CuratorTestBase; 24 | import org.apache.zookeeper.Version; 25 | import org.junit.jupiter.api.Test; 26 | 27 | public class TestIs38 extends CuratorTestBase { 28 | @Test 29 | public void testIsZk38() { 30 | assertThat(Version.getVersion()).startsWith("3.8"); 31 | } 32 | 33 | @Override 34 | protected void createServer() { 35 | // NOP 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /curator-test-zk38/src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, 12 | # software distributed under the License is distributed on an 13 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | # KIND, either express or implied. See the License for the 15 | # specific language governing permissions and limitations 16 | # under the License. 17 | 18 | log4j.rootLogger=ERROR, console 19 | 20 | log4j.logger.org.apache.curator=DEBUG, console 21 | log4j.additivity.org.apache.curator=false 22 | 23 | log4j.appender.console=org.apache.log4j.ConsoleAppender 24 | log4j.appender.console.layout=org.apache.log4j.PatternLayout 25 | log4j.appender.console.layout.ConversionPattern=%-5p %c %x %m [%t]%n 26 | -------------------------------------------------------------------------------- /curator-test/NOTICE: -------------------------------------------------------------------------------- 1 | Apache Curator 2 | Copyright 2013-2023 The Apache Software Foundation 3 | 4 | This product includes software developed at 5 | The Apache Software Foundation (http://www.apache.org/). 6 | -------------------------------------------------------------------------------- /curator-test/src/main/java/org/apache/curator/test/FailedServerStartException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.test; 21 | 22 | public class FailedServerStartException extends IllegalStateException { 23 | public FailedServerStartException(Throwable cause) { 24 | super(cause); 25 | } 26 | 27 | public FailedServerStartException(String s) { 28 | super(s); 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /curator-test/src/main/java/org/apache/curator/test/ZooKeeperMainFace.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.test; 21 | 22 | import java.io.Closeable; 23 | 24 | interface ZooKeeperMainFace extends Closeable { 25 | void start(QuorumPeerConfigBuilder configBuilder); 26 | 27 | void kill(); 28 | 29 | int getClientPort() throws Exception; 30 | } 31 | -------------------------------------------------------------------------------- /curator-test/src/main/java/org/apache/curator/test/compatibility/CuratorTestBase.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.test.compatibility; 21 | 22 | import org.apache.curator.test.BaseClassForTests; 23 | 24 | public class CuratorTestBase extends BaseClassForTests { 25 | public static final String zk36Group = "zk36"; 26 | public static final String zk35TestCompatibilityGroup = "zk35TestCompatibility"; 27 | 28 | protected final Timing2 timing = new Timing2(); 29 | } 30 | -------------------------------------------------------------------------------- /curator-x-async/src/main/java/org/apache/curator/x/async/api/AsyncEnsemblable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.x.async.api; 21 | 22 | /** 23 | * Terminal operation for config/reconfig builders 24 | */ 25 | public interface AsyncEnsemblable { 26 | /** 27 | * Commit the currently building operation and invoke ZooKeeper 28 | * 29 | * @return async stage 30 | */ 31 | T forEnsemble(); 32 | } 33 | -------------------------------------------------------------------------------- /curator-x-async/src/main/java/org/apache/curator/x/async/api/AsyncExistsBuilder.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.x.async.api; 21 | 22 | import java.util.Set; 23 | import org.apache.curator.x.async.AsyncStage; 24 | import org.apache.zookeeper.data.Stat; 25 | 26 | /** 27 | * Builder for ZNode exists 28 | */ 29 | public interface AsyncExistsBuilder extends AsyncPathable> { 30 | /** 31 | * Use the given options. The default is to use no options 32 | * 33 | * @param options options to use 34 | * @return this 35 | */ 36 | AsyncPathable> withOptions(Set options); 37 | } 38 | -------------------------------------------------------------------------------- /curator-x-async/src/main/java/org/apache/curator/x/async/api/AsyncGetChildrenBuilder.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.x.async.api; 21 | 22 | import java.util.List; 23 | import org.apache.curator.x.async.AsyncStage; 24 | import org.apache.zookeeper.data.Stat; 25 | 26 | /** 27 | * Builder for getChildren() 28 | */ 29 | public interface AsyncGetChildrenBuilder extends AsyncPathable>> { 30 | /** 31 | * Have the operation fill the provided stat object 32 | * 33 | * @param stat the stat to have filled in 34 | * @return this 35 | */ 36 | AsyncPathable>> storingStatIn(Stat stat); 37 | } 38 | -------------------------------------------------------------------------------- /curator-x-async/src/main/java/org/apache/curator/x/async/api/AsyncGetConfigBuilder.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.x.async.api; 21 | 22 | import org.apache.curator.x.async.AsyncStage; 23 | import org.apache.zookeeper.data.Stat; 24 | 25 | /** 26 | * Builder for getConfig() 27 | */ 28 | public interface AsyncGetConfigBuilder extends AsyncEnsemblable> { 29 | /** 30 | * Have the operation fill the provided stat object 31 | * 32 | * @param stat the stat to have filled in 33 | * @return this 34 | */ 35 | AsyncEnsemblable> storingStatIn(Stat stat); 36 | } 37 | -------------------------------------------------------------------------------- /curator-x-async/src/main/java/org/apache/curator/x/async/api/AsyncPathAndBytesable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.x.async.api; 21 | 22 | /** 23 | * Terminal operation for various builders 24 | */ 25 | public interface AsyncPathAndBytesable extends AsyncPathable { 26 | /** 27 | * Commit the currently building operation using the given path and data 28 | * and invoke ZooKeeper 29 | * 30 | * @param path the path 31 | * @param data the data 32 | * @return usually an async stage 33 | */ 34 | T forPath(String path, byte[] data); 35 | } 36 | -------------------------------------------------------------------------------- /curator-x-async/src/main/java/org/apache/curator/x/async/api/AsyncPathable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.x.async.api; 21 | 22 | /** 23 | * Terminal operation for various builders 24 | */ 25 | public interface AsyncPathable { 26 | /** 27 | * Commit the currently building operation using the given path 28 | * and invoke ZooKeeper 29 | * 30 | * @param path the path 31 | * @return usually an async stage 32 | */ 33 | T forPath(String path); 34 | } 35 | -------------------------------------------------------------------------------- /curator-x-async/src/main/java/org/apache/curator/x/async/api/AsyncSyncBuilder.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.x.async.api; 21 | 22 | import org.apache.curator.x.async.AsyncStage; 23 | 24 | /** 25 | * Builder for syncs 26 | */ 27 | public interface AsyncSyncBuilder extends AsyncPathable> {} 28 | -------------------------------------------------------------------------------- /curator-x-async/src/main/java/org/apache/curator/x/async/api/AsyncTransactionCheckBuilder.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.x.async.api; 21 | 22 | import org.apache.curator.framework.api.transaction.CuratorOp; 23 | 24 | /** 25 | * @see AsyncTransactionOp#check() 26 | */ 27 | public interface AsyncTransactionCheckBuilder extends AsyncPathable { 28 | /** 29 | * Use the given version (the default is -1) 30 | * 31 | * @param version version to use 32 | * @return this 33 | */ 34 | AsyncPathable withVersion(int version); 35 | } 36 | -------------------------------------------------------------------------------- /curator-x-async/src/main/java/org/apache/curator/x/async/api/AsyncTransactionDeleteBuilder.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.x.async.api; 21 | 22 | import org.apache.curator.framework.api.transaction.CuratorOp; 23 | 24 | /** 25 | * @see AsyncTransactionOp#delete() 26 | */ 27 | public interface AsyncTransactionDeleteBuilder extends AsyncPathable { 28 | /** 29 | * Changes the version number used. By default, -1 is used 30 | * 31 | * @param version version to use 32 | * @return this 33 | */ 34 | AsyncPathable withVersion(int version); 35 | } 36 | -------------------------------------------------------------------------------- /curator-x-async/src/main/java/org/apache/curator/x/async/api/AsyncWatchBuilder2.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.x.async.api; 21 | 22 | import org.apache.curator.framework.api.WatchableBase; 23 | import org.apache.curator.x.async.AsyncStage; 24 | 25 | public interface AsyncWatchBuilder2 26 | extends WatchableBase>>, AsyncPathable> {} 27 | -------------------------------------------------------------------------------- /curator-x-async/src/main/java/org/apache/curator/x/async/api/ExistsOption.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.x.async.api; 21 | 22 | /** 23 | * Options to use when checking for ZNode existence 24 | */ 25 | public enum ExistsOption { 26 | /** 27 | * see {@link CreateOption#createParentsIfNeeded} 28 | */ 29 | createParentsIfNeeded, 30 | 31 | /** 32 | * see {@link CreateOption#createParentsAsContainers} 33 | */ 34 | createParentsAsContainers 35 | } 36 | -------------------------------------------------------------------------------- /curator-x-async/src/main/java/org/apache/curator/x/async/details/BackgroundProc.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.x.async.details; 21 | 22 | import java.util.concurrent.CompletableFuture; 23 | import java.util.function.BiFunction; 24 | import org.apache.curator.framework.api.CuratorEvent; 25 | 26 | interface BackgroundProc extends BiFunction, Void> {} 27 | -------------------------------------------------------------------------------- /curator-x-async/src/main/java/org/apache/curator/x/async/migrations/Migration.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.x.async.migrations; 21 | 22 | import java.util.List; 23 | import org.apache.curator.framework.api.transaction.CuratorOp; 24 | 25 | /** 26 | * Models a single migration/transition 27 | */ 28 | @FunctionalInterface 29 | public interface Migration { 30 | /** 31 | * Return the operations to execute in a transaction. IMPORTANT: during a migration 32 | * this method may be called multiple times. 33 | * 34 | * @return operations 35 | */ 36 | List operations(); 37 | } 38 | -------------------------------------------------------------------------------- /curator-x-async/src/main/java/org/apache/curator/x/async/migrations/MigrationException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.x.async.migrations; 21 | 22 | import java.util.Objects; 23 | 24 | public class MigrationException extends RuntimeException { 25 | private final String migrationId; 26 | 27 | public MigrationException(String migrationId, String message) { 28 | super(message); 29 | this.migrationId = Objects.requireNonNull(migrationId, "migrationId cannot be null"); 30 | } 31 | 32 | public String getMigrationId() { 33 | return migrationId; 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /curator-x-async/src/main/java/org/apache/curator/x/async/modeled/ModeledOptions.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.x.async.modeled; 21 | 22 | public enum ModeledOptions { 23 | /** 24 | * Causes {@link ModeledFramework#children()} and {@link ModeledFramework#childrenAsZNodes()} 25 | * to ignore {@link org.apache.zookeeper.KeeperException.NoNodeException} and merely return 26 | * an empty list 27 | */ 28 | ignoreMissingNodesForChildren 29 | } 30 | -------------------------------------------------------------------------------- /curator-x-async/src/test/java/org/apache/curator/x/async/migrations/models/ModelV1.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.x.async.migrations.models; 21 | 22 | public class ModelV1 { 23 | private final String name; 24 | 25 | public ModelV1() { 26 | this(""); 27 | } 28 | 29 | public ModelV1(String name) { 30 | this.name = name; 31 | } 32 | 33 | public String getName() { 34 | return name; 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /curator-x-async/src/test/java/org/apache/curator/x/async/migrations/models/ModelV2.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.x.async.migrations.models; 21 | 22 | public class ModelV2 { 23 | private final String name; 24 | private final int age; 25 | 26 | public ModelV2() { 27 | this("", 0); 28 | } 29 | 30 | public ModelV2(String name, int age) { 31 | this.name = name; 32 | this.age = age; 33 | } 34 | 35 | public String getName() { 36 | return name; 37 | } 38 | 39 | public int getAge() { 40 | return age; 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /curator-x-async/src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, 12 | # software distributed under the License is distributed on an 13 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | # KIND, either express or implied. See the License for the 15 | # specific language governing permissions and limitations 16 | # under the License. 17 | 18 | log4j.rootLogger=ERROR, console 19 | 20 | log4j.logger.org.apache.curator=DEBUG, console 21 | log4j.additivity.org.apache.curator=false 22 | 23 | log4j.appender.console=org.apache.log4j.ConsoleAppender 24 | log4j.appender.console.layout=org.apache.log4j.PatternLayout 25 | log4j.appender.console.layout.ConversionPattern=%-5p %c %x %m [%t]%n 26 | -------------------------------------------------------------------------------- /curator-x-discovery-server/NOTICE: -------------------------------------------------------------------------------- 1 | Apache Curator 2 | Copyright 2013-2023 The Apache Software Foundation 3 | 4 | This product includes software developed at 5 | The Apache Software Foundation (http://www.apache.org/). 6 | -------------------------------------------------------------------------------- /curator-x-discovery-server/src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, 12 | # software distributed under the License is distributed on an 13 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | # KIND, either express or implied. See the License for the 15 | # specific language governing permissions and limitations 16 | # under the License. 17 | 18 | log4j.rootLogger=ERROR, console 19 | 20 | log4j.logger.org.apache.curator=DEBUG, console 21 | log4j.additivity.org.apache.curator=false 22 | 23 | log4j.appender.console=org.apache.log4j.ConsoleAppender 24 | log4j.appender.console.layout=org.apache.log4j.PatternLayout 25 | log4j.appender.console.layout.ConversionPattern=%-5p %c %x %m [%t]%n 26 | -------------------------------------------------------------------------------- /curator-x-discovery/NOTICE: -------------------------------------------------------------------------------- 1 | Apache Curator 2 | Copyright 2013-2023 The Apache Software Foundation 3 | 4 | This product includes software developed at 5 | The Apache Software Foundation (http://www.apache.org/). 6 | -------------------------------------------------------------------------------- /curator-x-discovery/src/main/java/org/apache/curator/x/discovery/InstanceFilter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.x.discovery; 21 | 22 | import com.google.common.base.Predicate; 23 | 24 | /** 25 | * Typedef for an Instance predicate 26 | */ 27 | public interface InstanceFilter extends Predicate> {} 28 | -------------------------------------------------------------------------------- /curator-x-discovery/src/main/java/org/apache/curator/x/discovery/LocalIpFilter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.x.discovery; 21 | 22 | import java.net.InetAddress; 23 | import java.net.NetworkInterface; 24 | import java.net.SocketException; 25 | 26 | public interface LocalIpFilter { 27 | public boolean use(NetworkInterface networkInterface, InetAddress address) throws SocketException; 28 | } 29 | -------------------------------------------------------------------------------- /curator-x-discovery/src/main/java/org/apache/curator/x/discovery/ServiceType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.x.discovery; 21 | 22 | public enum ServiceType { 23 | DYNAMIC, 24 | STATIC, 25 | PERMANENT, 26 | DYNAMIC_SEQUENTIAL; 27 | 28 | public boolean isDynamic() { 29 | return this == DYNAMIC || this == DYNAMIC_SEQUENTIAL; 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /curator-x-discovery/src/main/java/org/apache/curator/x/discovery/details/InstanceProvider.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.x.discovery.details; 21 | 22 | import java.util.List; 23 | import org.apache.curator.x.discovery.ServiceInstance; 24 | 25 | /** 26 | * Provides a set of available instances for a service so that a strategy can pick one of them 27 | */ 28 | public interface InstanceProvider { 29 | /** 30 | * Return the current available set of instances 31 | * @return instances 32 | * @throws Exception any errors 33 | */ 34 | public List> getInstances() throws Exception; 35 | } 36 | -------------------------------------------------------------------------------- /curator-x-discovery/src/main/java/org/apache/curator/x/discovery/details/Latch.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.x.discovery.details; 21 | 22 | class Latch { 23 | private volatile boolean laden = false; 24 | 25 | synchronized void set() { 26 | laden = true; 27 | notifyAll(); 28 | } 29 | 30 | synchronized void await() throws InterruptedException { 31 | while (!laden) { 32 | wait(); 33 | } 34 | laden = false; 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /curator-x-discovery/src/main/java/org/apache/curator/x/discovery/details/ServiceCacheListener.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.curator.x.discovery.details; 21 | 22 | import org.apache.curator.framework.state.ConnectionStateListener; 23 | 24 | /** 25 | * Listener for changes to a service cache 26 | */ 27 | public interface ServiceCacheListener extends ConnectionStateListener { 28 | /** 29 | * Called when the cache has changed (instances added/deleted, etc.) 30 | */ 31 | public void cacheChanged(); 32 | } 33 | -------------------------------------------------------------------------------- /curator-x-discovery/src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, 12 | # software distributed under the License is distributed on an 13 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | # KIND, either express or implied. See the License for the 15 | # specific language governing permissions and limitations 16 | # under the License. 17 | 18 | log4j.rootLogger=ERROR, console 19 | 20 | log4j.logger.org.apache.curator=DEBUG, console 21 | log4j.additivity.org.apache.curator=false 22 | 23 | log4j.appender.console=org.apache.log4j.ConsoleAppender 24 | log4j.appender.console.layout=org.apache.log4j.PatternLayout 25 | log4j.appender.console.layout.ConversionPattern=%-5p %c %x %m [%t]%n 26 | -------------------------------------------------------------------------------- /licenserc.toml: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, 12 | # software distributed under the License is distributed on an 13 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | # KIND, either express or implied. See the License for the 15 | # specific language governing permissions and limitations 16 | # under the License. 17 | 18 | headerPath = "Apache-2.0-ASF.txt" 19 | 20 | excludes = [ 21 | "DEPENDENCIES", 22 | "doap.rdf", 23 | "**/NOTICE", 24 | "**/*.confluence", 25 | ] 26 | --------------------------------------------------------------------------------