Package org.apache.zookeeper.server
Class ZKDatabase
- java.lang.Object
- 
- org.apache.zookeeper.server.ZKDatabase
 
- 
 public class ZKDatabase extends Object This class maintains the in memory database of zookeeper server states that includes the sessions, datatree and the committed logs. It is booted up after reading the logs and snapshots from the disk.
- 
- 
Field SummaryFields Modifier and Type Field Description static StringCOMMIT_LOG_COUNTintcommitLogCountprotected Queue<Leader.Proposal>committedLogprotected DataTreedataTreemake sure on a clear you take care of all these members.static intDEFAULT_COMMIT_LOG_COUNTstatic doubleDEFAULT_SNAPSHOT_SIZE_FACTORprotected ReentrantReadWriteLocklogLockprotected longmaxCommittedLogprotected longminCommittedLogprotected ConcurrentHashMap<Long,Integer>sessionsWithTimeoutsprotected FileTxnSnapLogsnapLogstatic StringSNAPSHOT_SIZE_FACTORDefault value is to use snapshot if txnlog size exceeds 1/3 the size of snapshot
 - 
Constructor SummaryConstructors Constructor Description ZKDatabase(FileTxnSnapLog snapLog)the filetxnsnaplog that this zk database maps to.
 - 
Method SummaryAll Methods Instance Methods Concrete Methods Modifier and Type Method Description List<ACL>aclForNode(DataNode n)voidaddCommittedProposal(Request request)maintains a list of last committedLog or so committed requests.voidaddWatch(String basePath, Watcher watcher, int mode)Add a watchbooleanappend(Request si)append to the underlying transaction loglongcalculateTxnLogSizeLimit()voidclear()clear the zkdatabase.voidclose()close this database.voidcommit()commit to the underlying transaction logbooleancompareDigest(TxnHeader header, Record txn, TxnDigest digest)booleancontainsWatcher(String path, Watcher.WatcherType type, Watcher watcher)Check whether the given watcher exists in datatreeDataTreecreateDataTree()voiddeserializeSnapshot(InputArchive ia)deserialize a snapshot from an input archivevoiddumpEphemerals(PrintWriter pwriter)write a text dump of all the ephemerals in the datatreelongfastForwardDataBase()Fast forward the database adding transactions from the committed log into memory.List<ACL>getACL(String path, Stat stat)get acl for a pathintgetAclSize()get the acl size of the datatreeintgetAllChildrenNumber(String path)List<String>getChildren(String path, Stat stat, Watcher watcher)get children list for this pathCollection<Leader.Proposal>getCommittedLog()byte[]getData(String path, Stat stat, Watcher watcher)get data and stat for a pathDataTreegetDataTree()the datatree for this zkdatabaselonggetDataTreeLastProcessedZxid()get the last processed zxid from a datatreeMap<Long,Set<String>>getEphemerals()Set<String>getEphemerals(long sessionId)the paths for ephemeral session idReentrantReadWriteLockgetLogLock()Get the lock that controls the committedLog.longgetmaxCommittedLog()the committed log for this zk databaselonggetminCommittedLog()the minimum committed transaction log available in memoryDataNodegetNode(String path)get the datanode for this pathintgetNodeCount()the node count of the datatreeIterator<Leader.Proposal>getProposalsFromTxnLog(long startZxid, long sizeLimit)Get proposals from txnlog.longgetSessionCount()Collection<Long>getSessions()return the sessions in the datatreeConcurrentHashMap<Long,Integer>getSessionWithTimeOuts()get sessions with timeoutsintgetTxnCount()Get the number of txn since last snapshotlonggetTxnSize()Get the size of txn since last snapshotvoidinitConfigInZKDatabase(QuorumVerifier qv)booleanisInitialized()checks to see if the zk database has been initialized or not.booleanisSpecialPath(String path)check if the path is special or notbooleanisTxnLogSyncEnabled()voidkillSession(long sessionId, long zxid)kill a given session in the datatreelongloadDataBase()load the database from the disk onto memory and also add the transactions to the committedlog in memory.DataTree.ProcessTxnResultprocessTxn(TxnHeader hdr, Record txn, TxnDigest digest)the process txn on the data and perform digest comparision.voidremoveCnxn(ServerCnxn cnxn)remove a cnxn from the datatreebooleanremoveWatch(String path, Watcher.WatcherType type, Watcher watcher)Remove watch from the datatreevoidresetTxnCount()Reset the number of txn since last rollLogvoidrollLog()roll the underlying logvoidserializeSnapshot(OutputArchive oa)serialize the snapshotvoidsetlastProcessedZxid(long zxid)the last processed zxid in the datatreevoidsetSnapshotSizeFactor(double snapshotSizeFactor)Use for unit testing, so we can turn this feature on/offvoidsetWatches(long relativeZxid, List<String> dataWatches, List<String> existWatches, List<String> childWatches, List<String> persistentWatches, List<String> persistentRecursiveWatches, Watcher watcher)set watches on the datatreeStatstatNode(String path, ServerCnxn serverCnxn)stat the pathbooleantruncateLog(long zxid)Truncate the ZKDatabase to the specified zxid
 
- 
- 
- 
Field Detail- 
dataTreeprotected DataTree dataTree make sure on a clear you take care of all these members.
 - 
sessionsWithTimeoutsprotected ConcurrentHashMap<Long,Integer> sessionsWithTimeouts 
 - 
snapLogprotected FileTxnSnapLog snapLog 
 - 
minCommittedLogprotected long minCommittedLog 
 - 
maxCommittedLogprotected long maxCommittedLog 
 - 
SNAPSHOT_SIZE_FACTORpublic static final String SNAPSHOT_SIZE_FACTOR Default value is to use snapshot if txnlog size exceeds 1/3 the size of snapshot- See Also:
- Constant Field Values
 
 - 
DEFAULT_SNAPSHOT_SIZE_FACTORpublic static final double DEFAULT_SNAPSHOT_SIZE_FACTOR - See Also:
- Constant Field Values
 
 - 
COMMIT_LOG_COUNTpublic static final String COMMIT_LOG_COUNT - See Also:
- Constant Field Values
 
 - 
DEFAULT_COMMIT_LOG_COUNTpublic static final int DEFAULT_COMMIT_LOG_COUNT - See Also:
- Constant Field Values
 
 - 
commitLogCountpublic int commitLogCount 
 - 
committedLogprotected Queue<Leader.Proposal> committedLog 
 - 
logLockprotected ReentrantReadWriteLock logLock 
 
- 
 - 
Constructor Detail- 
ZKDatabasepublic ZKDatabase(FileTxnSnapLog snapLog) the filetxnsnaplog that this zk database maps to. There is a one to one relationship between a filetxnsnaplog and zkdatabase.- Parameters:
- snapLog- the FileTxnSnapLog mapping this zkdatabase
 
 
- 
 - 
Method Detail- 
isInitializedpublic boolean isInitialized() checks to see if the zk database has been initialized or not.- Returns:
- true if zk database is initialized and false if not
 
 - 
clearpublic void clear() clear the zkdatabase. Note to developers - be careful to see that the clear method does clear out all the data structures in zkdatabase.
 - 
getDataTreepublic DataTree getDataTree() the datatree for this zkdatabase- Returns:
- the datatree for this zkdatabase
 
 - 
getmaxCommittedLogpublic long getmaxCommittedLog() the committed log for this zk database- Returns:
- the committed log for this zkdatabase
 
 - 
getminCommittedLogpublic long getminCommittedLog() the minimum committed transaction log available in memory- Returns:
- the minimum committed transaction log available in memory
 
 - 
getLogLockpublic ReentrantReadWriteLock getLogLock() Get the lock that controls the committedLog. If you want to get the pointer to the committedLog, you need to use this lock to acquire a read lock before calling getCommittedLog()- Returns:
- the lock that controls the committed log
 
 - 
getCommittedLogpublic Collection<Leader.Proposal> getCommittedLog() 
 - 
getDataTreeLastProcessedZxidpublic long getDataTreeLastProcessedZxid() get the last processed zxid from a datatree- Returns:
- the last processed zxid of a datatree
 
 - 
getSessionspublic Collection<Long> getSessions() return the sessions in the datatree- Returns:
- the data tree sessions
 
 - 
getSessionCountpublic long getSessionCount() - Returns:
- number of (global) sessions
 
 - 
getSessionWithTimeOutspublic ConcurrentHashMap<Long,Integer> getSessionWithTimeOuts() get sessions with timeouts- Returns:
- the hashmap of sessions with timeouts
 
 - 
loadDataBasepublic long loadDataBase() throws IOExceptionload the database from the disk onto memory and also add the transactions to the committedlog in memory.- Returns:
- the last valid zxid on disk
- Throws:
- IOException
 
 - 
fastForwardDataBasepublic long fastForwardDataBase() throws IOExceptionFast forward the database adding transactions from the committed log into memory.- Returns:
- the last valid zxid.
- Throws:
- IOException
 
 - 
addCommittedProposalpublic void addCommittedProposal(Request request) maintains a list of last committedLog or so committed requests. This is used for fast follower synchronization.- Parameters:
- request- committed request
 
 - 
isTxnLogSyncEnabledpublic boolean isTxnLogSyncEnabled() 
 - 
calculateTxnLogSizeLimitpublic long calculateTxnLogSizeLimit() 
 - 
getProposalsFromTxnLogpublic Iterator<Leader.Proposal> getProposalsFromTxnLog(long startZxid, long sizeLimit) Get proposals from txnlog. Only packet part of proposal is populated.- Parameters:
- startZxid- the starting zxid of the proposal
- sizeLimit- maximum on-disk size of txnlog to fetch 0 is unlimited, negative value means disable.
- Returns:
- list of proposal (request part of each proposal is null)
 
 - 
removeCnxnpublic void removeCnxn(ServerCnxn cnxn) remove a cnxn from the datatree- Parameters:
- cnxn- the cnxn to remove from the datatree
 
 - 
killSessionpublic void killSession(long sessionId, long zxid)kill a given session in the datatree- Parameters:
- sessionId- the session id to be killed
- zxid- the zxid of kill session transaction
 
 - 
dumpEphemeralspublic void dumpEphemerals(PrintWriter pwriter) write a text dump of all the ephemerals in the datatree- Parameters:
- pwriter- the output to write to
 
 - 
getNodeCountpublic int getNodeCount() the node count of the datatree- Returns:
- the node count of datatree
 
 - 
getEphemeralspublic Set<String> getEphemerals(long sessionId) the paths for ephemeral session id- Parameters:
- sessionId- the session id for which paths match to
- Returns:
- the paths for a session id
 
 - 
setlastProcessedZxidpublic void setlastProcessedZxid(long zxid) the last processed zxid in the datatree- Parameters:
- zxid- the last processed zxid in the datatree
 
 - 
processTxnpublic DataTree.ProcessTxnResult processTxn(TxnHeader hdr, Record txn, TxnDigest digest) the process txn on the data and perform digest comparision.- Parameters:
- hdr- the txnheader for the txn
- txn- the transaction that needs to be processed
- digest- the expected digest. A null value would skip the check
- Returns:
- the result of processing the transaction on this datatree/zkdatabase
 
 - 
statNodepublic Stat statNode(String path, ServerCnxn serverCnxn) throws KeeperException.NoNodeException stat the path- Parameters:
- path- the path for which stat is to be done
- serverCnxn- the servercnxn attached to this request
- Returns:
- the stat of this node
- Throws:
- KeeperException.NoNodeException
 
 - 
getNodepublic DataNode getNode(String path) get the datanode for this path- Parameters:
- path- the path to lookup
- Returns:
- the datanode for getting the path
 
 - 
getDatapublic byte[] getData(String path, Stat stat, Watcher watcher) throws KeeperException.NoNodeException get data and stat for a path- Parameters:
- path- the path being queried
- stat- the stat for this path
- watcher- the watcher function
- Throws:
- KeeperException.NoNodeException
 
 - 
setWatchespublic void setWatches(long relativeZxid, List<String> dataWatches, List<String> existWatches, List<String> childWatches, List<String> persistentWatches, List<String> persistentRecursiveWatches, Watcher watcher)set watches on the datatree- Parameters:
- relativeZxid- the relative zxid that client has seen
- dataWatches- the data watches the client wants to reset
- existWatches- the exists watches the client wants to reset
- childWatches- the child watches the client wants to reset
- persistentWatches- the persistent watches the client wants to reset
- persistentRecursiveWatches- the persistent recursive watches the client wants to reset
- watcher- the watcher function
 
 - 
addWatchpublic void addWatch(String basePath, Watcher watcher, int mode) Add a watch- Parameters:
- basePath- watch base
- watcher- the watcher
- mode- a mode from ZooDefs.AddWatchModes
 
 - 
getACLpublic List<ACL> getACL(String path, Stat stat) throws KeeperException.NoNodeException get acl for a path- Parameters:
- path- the path to query for acl
- stat- the stat for the node
- Returns:
- the acl list for this path
- Throws:
- KeeperException.NoNodeException
 
 - 
getChildrenpublic List<String> getChildren(String path, Stat stat, Watcher watcher) throws KeeperException.NoNodeException get children list for this path- Parameters:
- path- the path of the node
- stat- the stat of the node
- watcher- the watcher function for this path
- Returns:
- the list of children for this path
- Throws:
- KeeperException.NoNodeException
 
 - 
getAllChildrenNumberpublic int getAllChildrenNumber(String path) throws KeeperException.NoNodeException - Throws:
- KeeperException.NoNodeException
 
 - 
isSpecialPathpublic boolean isSpecialPath(String path) check if the path is special or not- Parameters:
- path- the input path
- Returns:
- true if path is special and false if not
 
 - 
getAclSizepublic int getAclSize() get the acl size of the datatree- Returns:
- the acl size of the datatree
 
 - 
truncateLogpublic boolean truncateLog(long zxid) throws IOExceptionTruncate the ZKDatabase to the specified zxid- Parameters:
- zxid- the zxid to truncate zk database to
- Returns:
- true if the truncate is successful and false if not
- Throws:
- IOException
 
 - 
deserializeSnapshotpublic void deserializeSnapshot(InputArchive ia) throws IOException deserialize a snapshot from an input archive- Parameters:
- ia- the input archive you want to deserialize from
- Throws:
- IOException
 
 - 
serializeSnapshotpublic void serializeSnapshot(OutputArchive oa) throws IOException, InterruptedException serialize the snapshot- Parameters:
- oa- the output archive to which the snapshot needs to be serialized
- Throws:
- IOException
- InterruptedException
 
 - 
appendpublic boolean append(Request si) throws IOException append to the underlying transaction log- Parameters:
- si- the request to append
- Returns:
- true if the append was succesfull and false if not
- Throws:
- IOException
 
 - 
rollLogpublic void rollLog() throws IOExceptionroll the underlying log- Throws:
- IOException
 
 - 
commitpublic void commit() throws IOExceptioncommit to the underlying transaction log- Throws:
- IOException
 
 - 
closepublic void close() throws IOExceptionclose this database. free the resources- Throws:
- IOException
 
 - 
initConfigInZKDatabasepublic void initConfigInZKDatabase(QuorumVerifier qv) 
 - 
setSnapshotSizeFactorpublic void setSnapshotSizeFactor(double snapshotSizeFactor) Use for unit testing, so we can turn this feature on/off- Parameters:
- snapshotSizeFactor- Set to minus value to turn this off.
 
 - 
containsWatcherpublic boolean containsWatcher(String path, Watcher.WatcherType type, Watcher watcher) Check whether the given watcher exists in datatree- Parameters:
- path- node to check watcher existence
- type- type of watcher
- watcher- watcher function
 
 - 
removeWatchpublic boolean removeWatch(String path, Watcher.WatcherType type, Watcher watcher) Remove watch from the datatree- Parameters:
- path- node to remove watches from
- type- type of watcher to remove
- watcher- watcher function to remove
 
 - 
createDataTreepublic DataTree createDataTree() 
 - 
resetTxnCountpublic void resetTxnCount() Reset the number of txn since last rollLog
 - 
getTxnCountpublic int getTxnCount() Get the number of txn since last snapshot
 - 
getTxnSizepublic long getTxnSize() Get the size of txn since last snapshot
 
- 
 
-