Package org.apache.zookeeper.server
Class ZooKeeperServer
- java.lang.Object
-
- org.apache.zookeeper.server.ZooKeeperServer
-
- All Implemented Interfaces:
ServerStats.Provider
,SessionTracker.SessionExpirer
- Direct Known Subclasses:
QuorumZooKeeperServer
,ReadOnlyZooKeeperServer
public class ZooKeeperServer extends Object implements SessionTracker.SessionExpirer, ServerStats.Provider
This class implements a simple standalone ZooKeeperServer. It sets up the following chain of RequestProcessors to process requests: PrepRequestProcessor -> SyncRequestProcessor -> FinalRequestProcessor
-
-
Nested Class Summary
Nested Classes Modifier and Type Class Description static class
ZooKeeperServer.MissingSessionException
protected static class
ZooKeeperServer.State
-
Field Summary
-
Constructor Summary
Constructors Constructor Description ZooKeeperServer()
Creates a ZooKeeperServer instance.ZooKeeperServer(File snapDir, File logDir, int tickTime)
This constructor is for backward compatibility with the existing unit test code.ZooKeeperServer(FileTxnSnapLog txnLogFactory)
Default constructor, relies on the config for its argument valuesZooKeeperServer(FileTxnSnapLog txnLogFactory, int tickTime, int minSessionTimeout, int maxSessionTimeout, int clientPortListenBacklog, ZKDatabase zkDb, String initialConfig)
Keeping this constructor for backward compatibilityZooKeeperServer(FileTxnSnapLog txnLogFactory, int tickTime, int minSessionTimeout, int maxSessionTimeout, int clientPortListenBacklog, ZKDatabase zkDb, String initialConfig, boolean reconfigEnabled)
* Creates a ZooKeeperServer instance.ZooKeeperServer(FileTxnSnapLog txnLogFactory, int tickTime, String initialConfig)
creates a zookeeperserver instance.ZooKeeperServer(JvmPauseMonitor jvmPauseMonitor, FileTxnSnapLog txnLogFactory, int tickTime, int minSessionTimeout, int maxSessionTimeout, int clientPortListenBacklog, ZKDatabase zkDb, String initialConfig)
Adds JvmPauseMonitor and callsZooKeeperServer(FileTxnSnapLog, int, int, int, int, ZKDatabase, String)
-
Method Summary
All Methods Static Methods Instance Methods Concrete Methods Modifier and Type Method Description boolean
authWriteRequest(Request request)
Check Write Requests for Potential Access Restrictionsprotected boolean
canShutdown()
This can be used while shutting down the server to see whether the server is already shutdown or not.void
checkACL(ServerCnxn cnxn, List<ACL> acl, int perm, List<Id> ids, String path, List<ACL> setAcls)
Grant or deny authorization to an operation on a node as a function of:protected boolean
checkPasswd(long sessionId, byte[] passwd)
void
checkQuota(String path, byte[] lastData, byte[] data, int type)
check a path whether exceeded the quota.boolean
checkRequestSizeWhenReceivingMessage(int length)
void
closeSession(long sessionId)
void
closeSession(ServerCnxn cnxn, RequestHeader requestHeader)
BlueThrottle
connThrottle()
protected RequestThrottler
createRequestThrottler()
protected void
createSessionTracker()
void
decInProcess()
void
dumpConf(PrintWriter pwriter)
void
dumpEphemerals(PrintWriter pwriter)
void
dumpMonitorValues(BiConsumer<String,Object> response)
Hook into admin server, useful to expose additional data that do not represent metrics.void
enqueueRequest(Request si)
void
expire(long sessionId)
void
expire(SessionTracker.Session session)
void
finishSessionInit(ServerCnxn cnxn, boolean valid)
int
getClientPort()
int
getClientPortListenBacklog()
ZooKeeperServerConf
getConf()
double
getConnectionDropChance()
long
getDataDirSize()
Map<Long,Set<String>>
getEphemerals()
ResponseCache
getGetChildrenResponseCache()
int
getGlobalOutstandingLimit()
int
getInflight()
String
getInitialConfig()
int
getInProcess()
int
getLargeRequestBytes()
int
getLargeRequestMaxBytes()
int
getLargeRequestThreshold()
long
getLastProcessedZxid()
return the last processed id from the datatreelong
getLogDirSize()
int
getMaxClientCnxnsPerHost()
Maximum number of connections allowed from particular host (ip)int
getMaxSessionTimeout()
int
getMinSessionTimeout()
int
getNumAliveConnections()
return the total number of client connections that are alive to this serverint
getOutstandingHandshakeNum()
long
getOutstandingRequests()
return the outstanding requests in the queue, which haven't been processed yetResponseCache
getReadResponseCache()
RequestPathMetricsCollector
getRequestPathMetricsCollector()
int
getSecureClientPort()
ServerCnxnFactory
getSecureServerCnxnFactory()
ServerCnxnFactory
getServerCnxnFactory()
long
getServerId()
Map<Long,Set<Long>>
getSessionExpiryMap()
SessionTracker
getSessionTracker()
static int
getSnapCount()
static long
getSnapSizeInBytes()
String
getState()
static int
getThrottledOpWaitTime()
int
getTickTime()
long
getTxnLogElapsedSyncTime()
Returns the elapsed sync of time of transaction log in milliseconds.FileTxnSnapLog
getTxnLogFactory()
ZKDatabase
getZKDatabase()
get the zookeeper database for this serverZooKeeperServerShutdownHandler
getZkShutdownHandler()
ZooKeeperServerListener
getZooKeeperServerListener()
long
getZxid()
void
incInProcess()
static boolean
isCloseSessionTxnEnabled()
static boolean
isDigestEnabled()
static boolean
isEnableEagerACLCheck()
boolean
isReconfigEnabled()
boolean
isResponseCachingEnabled()
boolean
isRunning()
static boolean
isSerializeLastProcessedZxidEnabled()
protected void
killSession(long sessionId, long zxid)
void
loadData()
Restore sessions and datavoid
processConnectRequest(ServerCnxn cnxn, ConnectRequest request)
void
processPacket(ServerCnxn cnxn, RequestHeader h, RequestRecord request)
DataTree.ProcessTxnResult
processTxn(Request request)
DataTree.ProcessTxnResult
processTxn(TxnHeader hdr, Record txn)
protected void
registerJMX()
protected void
registerMetrics()
void
reopenSession(ServerCnxn cnxn, long sessionId, byte[] passwd, int sessionTimeout)
void
requestFinished(Request request)
long
restoreFromSnapshot(InputStream inputStream)
Restores database from a snapshot.protected void
revalidateSession(ServerCnxn cnxn, long sessionId, int sessionTimeout)
ServerStats
serverStats()
void
setClientPortListenBacklog(int backlog)
static void
setCloseSessionTxnEnabled(boolean enabled)
void
setCreateSessionTrackerServerId(int newId)
Change the server ID used bycreateSessionTracker()
.static void
setDigestEnabled(boolean digestEnabled)
static void
setEnableEagerACLCheck(boolean enabled)
void
setLargeRequestMaxBytes(int bytes)
void
setLargeRequestThreshold(int threshold)
protected void
setLocalSessionFlag(Request si)
If the underlying Zookeeper server support local session, this method will set a isLocalSession to true if a request is associated with a local session.void
setMaxSessionTimeout(int max)
void
setMinSessionTimeout(int min)
void
setOwner(long id, Object owner)
set the owner of this session as ownervoid
setResponseCachingEnabled(boolean isEnabled)
void
setSecureServerCnxnFactory(ServerCnxnFactory factory)
static void
setSerializeLastProcessedZxidEnabled(boolean serializeLastZxidEnabled)
void
setServerCnxnFactory(ServerCnxnFactory factory)
protected void
setState(ZooKeeperServer.State state)
Sets the state of ZooKeeper server.static void
setThrottledOpWaitTime(int time)
void
setTickTime(int tickTime)
void
setTxnLogFactory(FileTxnSnapLog txnLog)
protected void
setupRequestProcessors()
void
setZKDatabase(ZKDatabase zkDb)
set the zkdatabase for this zookeeper servervoid
setZxid(long zxid)
boolean
shouldForceWriteInitialSnapshotAfterLeaderElection()
boolean
shouldThrottle(long outStandingCount)
void
shutdown()
void
shutdown(boolean fullyShutDown)
Shut down the server instancevoid
startdata()
protected void
startJvmPauseMonitor()
protected void
startRequestThrottler()
void
startServing()
protected void
startSessionTracker()
void
startup()
void
startupWithoutServing()
void
submitRequest(Request si)
void
submitRequestNow(Request si)
File
takeSnapshot()
File
takeSnapshot(boolean syncSnap)
File
takeSnapshot(boolean syncSnap, boolean isSevere, boolean fastForwardFromEdits)
Takes a snapshot on the server.void
truncateLog(long zxid)
truncate the log to get in sync with others if in a quorumprotected void
unregisterJMX()
protected void
unregisterMetrics()
protected void
validateSession(ServerCnxn cnxn, long sessionId)
Validate if a particular session can be reestablished.
-
-
-
Field Detail
-
LOG
protected static final org.slf4j.Logger LOG
-
GLOBAL_OUTSTANDING_LIMIT
public static final String GLOBAL_OUTSTANDING_LIMIT
- See Also:
- Constant Field Values
-
ENABLE_EAGER_ACL_CHECK
public static final String ENABLE_EAGER_ACL_CHECK
- See Also:
- Constant Field Values
-
SKIP_ACL
public static final String SKIP_ACL
- See Also:
- Constant Field Values
-
ENFORCE_QUOTA
public static final String ENFORCE_QUOTA
- See Also:
- Constant Field Values
-
enforceQuota
public static final boolean enforceQuota
-
SASL_SUPER_USER
public static final String SASL_SUPER_USER
- See Also:
- Constant Field Values
-
ALLOW_SASL_FAILED_CLIENTS
public static final String ALLOW_SASL_FAILED_CLIENTS
- See Also:
- Constant Field Values
-
ZOOKEEPER_DIGEST_ENABLED
public static final String ZOOKEEPER_DIGEST_ENABLED
- See Also:
- Constant Field Values
-
ZOOKEEPER_SERIALIZE_LAST_PROCESSED_ZXID_ENABLED
public static final String ZOOKEEPER_SERIALIZE_LAST_PROCESSED_ZXID_ENABLED
- See Also:
- Constant Field Values
-
CLOSE_SESSION_TXN_ENABLED
public static final String CLOSE_SESSION_TXN_ENABLED
- See Also:
- Constant Field Values
-
jmxServerBean
protected ZooKeeperServerBean jmxServerBean
-
jmxDataTreeBean
protected DataTreeBean jmxDataTreeBean
-
DEFAULT_TICK_TIME
public static final int DEFAULT_TICK_TIME
- See Also:
- Constant Field Values
-
tickTime
protected int tickTime
-
DEFAULT_THROTTLED_OP_WAIT_TIME
public static final int DEFAULT_THROTTLED_OP_WAIT_TIME
- See Also:
- Constant Field Values
-
throttledOpWaitTime
protected static volatile int throttledOpWaitTime
-
minSessionTimeout
protected int minSessionTimeout
value of -1 indicates unset, use default
-
maxSessionTimeout
protected int maxSessionTimeout
value of -1 indicates unset, use default
-
listenBacklog
protected int listenBacklog
Socket listen backlog. Value of -1 indicates unset
-
sessionTracker
protected SessionTracker sessionTracker
-
ok
public static final Exception ok
-
firstProcessor
protected RequestProcessor firstProcessor
-
jvmPauseMonitor
protected JvmPauseMonitor jvmPauseMonitor
-
state
protected volatile ZooKeeperServer.State state
-
initialConfig
protected String initialConfig
-
reconfigEnabled
protected boolean reconfigEnabled
-
serverCnxnFactory
protected ServerCnxnFactory serverCnxnFactory
-
secureServerCnxnFactory
protected ServerCnxnFactory secureServerCnxnFactory
-
INT_BUFFER_STARTING_SIZE_BYTES
public static final String INT_BUFFER_STARTING_SIZE_BYTES
Starting size of read and write ByteArroyOuputBuffers. Default is 32 bytes. Flag not used for small transfers like connectResponses.- See Also:
- Constant Field Values
-
DEFAULT_STARTING_BUFFER_SIZE
public static final int DEFAULT_STARTING_BUFFER_SIZE
- See Also:
- Constant Field Values
-
intBufferStartingSizeBytes
public static final int intBufferStartingSizeBytes
-
GET_DATA_RESPONSE_CACHE_SIZE
public static final String GET_DATA_RESPONSE_CACHE_SIZE
- See Also:
- Constant Field Values
-
GET_CHILDREN_RESPONSE_CACHE_SIZE
public static final String GET_CHILDREN_RESPONSE_CACHE_SIZE
- See Also:
- Constant Field Values
-
SNAP_COUNT
public static final String SNAP_COUNT
- See Also:
- Constant Field Values
-
-
Constructor Detail
-
ZooKeeperServer
public ZooKeeperServer()
Creates a ZooKeeperServer instance. Nothing is setup, use the setX methods to prepare the instance (eg datadir, datalogdir, ticktime, builder, etc...)
-
ZooKeeperServer
public ZooKeeperServer(FileTxnSnapLog txnLogFactory, int tickTime, int minSessionTimeout, int maxSessionTimeout, int clientPortListenBacklog, ZKDatabase zkDb, String initialConfig)
Keeping this constructor for backward compatibility
-
ZooKeeperServer
public ZooKeeperServer(FileTxnSnapLog txnLogFactory, int tickTime, int minSessionTimeout, int maxSessionTimeout, int clientPortListenBacklog, ZKDatabase zkDb, String initialConfig, boolean reconfigEnabled)
* Creates a ZooKeeperServer instance. It sets everything up, but doesn't actually start listening for clients until run() is invoked.
-
ZooKeeperServer
public ZooKeeperServer(JvmPauseMonitor jvmPauseMonitor, FileTxnSnapLog txnLogFactory, int tickTime, int minSessionTimeout, int maxSessionTimeout, int clientPortListenBacklog, ZKDatabase zkDb, String initialConfig)
Adds JvmPauseMonitor and callsZooKeeperServer(FileTxnSnapLog, int, int, int, int, ZKDatabase, String)
-
ZooKeeperServer
public ZooKeeperServer(FileTxnSnapLog txnLogFactory, int tickTime, String initialConfig)
creates a zookeeperserver instance.- Parameters:
txnLogFactory
- the file transaction snapshot logging classtickTime
- the ticktime for the server- Throws:
IOException
-
ZooKeeperServer
public ZooKeeperServer(File snapDir, File logDir, int tickTime) throws IOException
This constructor is for backward compatibility with the existing unit test code. It defaults to FileLogProvider persistence provider.- Throws:
IOException
-
ZooKeeperServer
public ZooKeeperServer(FileTxnSnapLog txnLogFactory) throws IOException
Default constructor, relies on the config for its argument values- Throws:
IOException
-
-
Method Detail
-
isEnableEagerACLCheck
public static boolean isEnableEagerACLCheck()
-
setEnableEagerACLCheck
public static void setEnableEagerACLCheck(boolean enabled)
-
isCloseSessionTxnEnabled
public static boolean isCloseSessionTxnEnabled()
-
setCloseSessionTxnEnabled
public static void setCloseSessionTxnEnabled(boolean enabled)
-
getInitialConfig
public String getInitialConfig()
-
serverStats
public ServerStats serverStats()
-
getRequestPathMetricsCollector
public RequestPathMetricsCollector getRequestPathMetricsCollector()
-
connThrottle
public BlueThrottle connThrottle()
-
dumpConf
public void dumpConf(PrintWriter pwriter)
-
getConf
public ZooKeeperServerConf getConf()
-
getZKDatabase
public ZKDatabase getZKDatabase()
get the zookeeper database for this server- Returns:
- the zookeeper database for this server
-
setZKDatabase
public void setZKDatabase(ZKDatabase zkDb)
set the zkdatabase for this zookeeper server- Parameters:
zkDb
-
-
loadData
public void loadData() throws IOException, InterruptedException
Restore sessions and data- Throws:
IOException
InterruptedException
-
takeSnapshot
public File takeSnapshot() throws IOException
- Throws:
IOException
-
takeSnapshot
public File takeSnapshot(boolean syncSnap) throws IOException
- Throws:
IOException
-
takeSnapshot
public File takeSnapshot(boolean syncSnap, boolean isSevere, boolean fastForwardFromEdits) throws IOException
Takes a snapshot on the server.- Parameters:
syncSnap
- syncSnap sync the snapshot immediately after writeisSevere
- if true system exist, otherwise throw IOExceptionfastForwardFromEdits
- whether fast forward database to the latest recorded transactions- Returns:
- file snapshot file object
- Throws:
IOException
-
restoreFromSnapshot
public long restoreFromSnapshot(InputStream inputStream) throws IOException
Restores database from a snapshot. It is used by the restore admin server command.- Parameters:
inputStream
- input stream of snapshot- Throws:
IOException
-
shouldForceWriteInitialSnapshotAfterLeaderElection
public boolean shouldForceWriteInitialSnapshotAfterLeaderElection()
-
getDataDirSize
public long getDataDirSize()
- Specified by:
getDataDirSize
in interfaceServerStats.Provider
-
getLogDirSize
public long getLogDirSize()
- Specified by:
getLogDirSize
in interfaceServerStats.Provider
-
getZxid
public long getZxid()
-
getSessionTracker
public SessionTracker getSessionTracker()
-
setZxid
public void setZxid(long zxid)
-
closeSession
public void closeSession(long sessionId)
-
killSession
protected void killSession(long sessionId, long zxid)
-
expire
public void expire(SessionTracker.Session session)
- Specified by:
expire
in interfaceSessionTracker.SessionExpirer
-
expire
public void expire(long sessionId)
-
registerJMX
protected void registerJMX()
-
startdata
public void startdata() throws IOException, InterruptedException
- Throws:
IOException
InterruptedException
-
startup
public void startup()
-
startupWithoutServing
public void startupWithoutServing()
-
startServing
public void startServing()
-
startJvmPauseMonitor
protected void startJvmPauseMonitor()
-
startRequestThrottler
protected void startRequestThrottler()
-
createRequestThrottler
protected RequestThrottler createRequestThrottler()
-
setupRequestProcessors
protected void setupRequestProcessors()
-
getZooKeeperServerListener
public ZooKeeperServerListener getZooKeeperServerListener()
-
setCreateSessionTrackerServerId
public void setCreateSessionTrackerServerId(int newId)
Change the server ID used bycreateSessionTracker()
. Must be called prior tostartup()
being called- Parameters:
newId
- ID to use
-
createSessionTracker
protected void createSessionTracker()
-
startSessionTracker
protected void startSessionTracker()
-
setState
protected void setState(ZooKeeperServer.State state)
Sets the state of ZooKeeper server. After changing the state, it notifies the server state change to a registered shutdown handler, if any.The following are the server state transitions:
- During startup the server will be in the INITIAL state.
- After successfully starting, the server sets the state to RUNNING.
- The server transitions to the ERROR state if it hits an internal
error.
ZooKeeperServerListenerImpl
notifies any critical resource error events, e.g., SyncRequestProcessor not being able to write a txn to disk. - During shutdown the server sets the state to SHUTDOWN, which corresponds to the server not running.
- During maintenance (e.g. restore) the server sets the state to MAINTENANCE
- Parameters:
state
- new server state.
-
canShutdown
protected boolean canShutdown()
This can be used while shutting down the server to see whether the server is already shutdown or not.- Returns:
- true if the server is running or server hits an error, false otherwise.
-
isRunning
public boolean isRunning()
- Returns:
- true if the server is running, false otherwise.
-
shutdown
public void shutdown()
-
shutdown
public void shutdown(boolean fullyShutDown)
Shut down the server instance- Parameters:
fullyShutDown
- true if another server using the same database will not replace this one in the same process
-
unregisterJMX
protected void unregisterJMX()
-
incInProcess
public void incInProcess()
-
decInProcess
public void decInProcess()
-
getInProcess
public int getInProcess()
-
getInflight
public int getInflight()
-
checkPasswd
protected boolean checkPasswd(long sessionId, byte[] passwd)
-
setOwner
public void setOwner(long id, Object owner) throws KeeperException.SessionExpiredException
set the owner of this session as owner- Parameters:
id
- the session idowner
- the owner of the session- Throws:
KeeperException.SessionExpiredException
-
revalidateSession
protected void revalidateSession(ServerCnxn cnxn, long sessionId, int sessionTimeout) throws IOException
- Throws:
IOException
-
reopenSession
public void reopenSession(ServerCnxn cnxn, long sessionId, byte[] passwd, int sessionTimeout) throws IOException
- Throws:
IOException
-
finishSessionInit
public void finishSessionInit(ServerCnxn cnxn, boolean valid)
-
closeSession
public void closeSession(ServerCnxn cnxn, RequestHeader requestHeader)
-
getServerId
public long getServerId()
- Specified by:
getServerId
in interfaceSessionTracker.SessionExpirer
-
setLocalSessionFlag
protected void setLocalSessionFlag(Request si)
If the underlying Zookeeper server support local session, this method will set a isLocalSession to true if a request is associated with a local session.- Parameters:
si
-
-
submitRequest
public void submitRequest(Request si)
-
enqueueRequest
public void enqueueRequest(Request si)
-
submitRequestNow
public void submitRequestNow(Request si)
-
getSnapCount
public static int getSnapCount()
-
getGlobalOutstandingLimit
public int getGlobalOutstandingLimit()
-
getSnapSizeInBytes
public static long getSnapSizeInBytes()
-
setServerCnxnFactory
public void setServerCnxnFactory(ServerCnxnFactory factory)
-
getServerCnxnFactory
public ServerCnxnFactory getServerCnxnFactory()
-
getSecureServerCnxnFactory
public ServerCnxnFactory getSecureServerCnxnFactory()
-
setSecureServerCnxnFactory
public void setSecureServerCnxnFactory(ServerCnxnFactory factory)
-
getLastProcessedZxid
public long getLastProcessedZxid()
return the last processed id from the datatree- Specified by:
getLastProcessedZxid
in interfaceServerStats.Provider
-
getOutstandingRequests
public long getOutstandingRequests()
return the outstanding requests in the queue, which haven't been processed yet- Specified by:
getOutstandingRequests
in interfaceServerStats.Provider
-
getNumAliveConnections
public int getNumAliveConnections()
return the total number of client connections that are alive to this server- Specified by:
getNumAliveConnections
in interfaceServerStats.Provider
-
truncateLog
public void truncateLog(long zxid) throws IOException
truncate the log to get in sync with others if in a quorum- Parameters:
zxid
- the zxid that it needs to get in sync with others- Throws:
IOException
-
getTickTime
public int getTickTime()
-
setTickTime
public void setTickTime(int tickTime)
-
getThrottledOpWaitTime
public static int getThrottledOpWaitTime()
-
setThrottledOpWaitTime
public static void setThrottledOpWaitTime(int time)
-
getMinSessionTimeout
public int getMinSessionTimeout()
-
setMinSessionTimeout
public void setMinSessionTimeout(int min)
-
getMaxSessionTimeout
public int getMaxSessionTimeout()
-
setMaxSessionTimeout
public void setMaxSessionTimeout(int max)
-
getClientPortListenBacklog
public int getClientPortListenBacklog()
-
setClientPortListenBacklog
public void setClientPortListenBacklog(int backlog)
-
getClientPort
public int getClientPort()
-
getSecureClientPort
public int getSecureClientPort()
-
getMaxClientCnxnsPerHost
public int getMaxClientCnxnsPerHost()
Maximum number of connections allowed from particular host (ip)
-
setTxnLogFactory
public void setTxnLogFactory(FileTxnSnapLog txnLog)
-
getTxnLogFactory
public FileTxnSnapLog getTxnLogFactory()
-
getTxnLogElapsedSyncTime
public long getTxnLogElapsedSyncTime()
Returns the elapsed sync of time of transaction log in milliseconds.
-
getState
public String getState()
- Specified by:
getState
in interfaceServerStats.Provider
-
dumpEphemerals
public void dumpEphemerals(PrintWriter pwriter)
-
getConnectionDropChance
public double getConnectionDropChance()
-
processConnectRequest
public void processConnectRequest(ServerCnxn cnxn, ConnectRequest request) throws IOException, ClientCnxnLimitException
- Throws:
IOException
ClientCnxnLimitException
-
validateSession
protected void validateSession(ServerCnxn cnxn, long sessionId) throws IOException
Validate if a particular session can be reestablished.- Parameters:
cnxn
-sessionId
-- Throws:
IOException
-
shouldThrottle
public boolean shouldThrottle(long outStandingCount)
-
getLargeRequestMaxBytes
public int getLargeRequestMaxBytes()
-
setLargeRequestMaxBytes
public void setLargeRequestMaxBytes(int bytes)
-
getLargeRequestThreshold
public int getLargeRequestThreshold()
-
setLargeRequestThreshold
public void setLargeRequestThreshold(int threshold)
-
getLargeRequestBytes
public int getLargeRequestBytes()
-
checkRequestSizeWhenReceivingMessage
public boolean checkRequestSizeWhenReceivingMessage(int length) throws IOException
- Throws:
IOException
-
requestFinished
public void requestFinished(Request request)
-
processPacket
public void processPacket(ServerCnxn cnxn, RequestHeader h, RequestRecord request) throws IOException
- Throws:
IOException
-
processTxn
public DataTree.ProcessTxnResult processTxn(TxnHeader hdr, Record txn)
-
processTxn
public DataTree.ProcessTxnResult processTxn(Request request)
-
isResponseCachingEnabled
public boolean isResponseCachingEnabled()
-
setResponseCachingEnabled
public void setResponseCachingEnabled(boolean isEnabled)
-
getReadResponseCache
public ResponseCache getReadResponseCache()
-
getGetChildrenResponseCache
public ResponseCache getGetChildrenResponseCache()
-
registerMetrics
protected void registerMetrics()
-
unregisterMetrics
protected void unregisterMetrics()
-
dumpMonitorValues
public void dumpMonitorValues(BiConsumer<String,Object> response)
Hook into admin server, useful to expose additional data that do not represent metrics.- Parameters:
response
- a sink which collects the data.
-
checkACL
public void checkACL(ServerCnxn cnxn, List<ACL> acl, int perm, List<Id> ids, String path, List<ACL> setAcls) throws KeeperException.NoAuthException
Grant or deny authorization to an operation on a node as a function of:- Parameters:
cnxn
- : the server connection or null for admin server commandsacl
- : set of ACLs for the nodeperm
- : the permission that the client is requestingids
- : the credentials supplied by the clientpath
- : the ZNode pathsetAcls
- : for set ACL operations, the list of ACLs being set. Otherwise null.- Throws:
KeeperException.NoAuthException
-
checkQuota
public void checkQuota(String path, byte[] lastData, byte[] data, int type) throws KeeperException.QuotaExceededException
check a path whether exceeded the quota.- Parameters:
path
- the path of the node, used for the quota prefix checklastData
- the current node data,null
for nonedata
- the data to be set, ornull
for nonetype
- currently, create and setData need to check quota- Throws:
KeeperException.QuotaExceededException
-
isDigestEnabled
public static boolean isDigestEnabled()
-
setDigestEnabled
public static void setDigestEnabled(boolean digestEnabled)
-
isSerializeLastProcessedZxidEnabled
public static boolean isSerializeLastProcessedZxidEnabled()
-
setSerializeLastProcessedZxidEnabled
public static void setSerializeLastProcessedZxidEnabled(boolean serializeLastZxidEnabled)
-
authWriteRequest
public boolean authWriteRequest(Request request)
Check Write Requests for Potential Access Restrictions Before a request is being proposed to the quorum, lets check it against local ACLs. Non-write requests (read, session, etc.) are passed along. Invalid requests are sent a response. While we are at it, if the request will set an ACL: make sure it's a valid one.- Parameters:
request
-- Returns:
- true if request is permitted, false if not.
- Throws:
IOException
-
getOutstandingHandshakeNum
public int getOutstandingHandshakeNum()
-
isReconfigEnabled
public boolean isReconfigEnabled()
-
getZkShutdownHandler
public ZooKeeperServerShutdownHandler getZkShutdownHandler()
-
-