HBASE-8918 Removes redundant identifiers from interfaces

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1502128 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2013-07-11 07:01:27 +00:00
parent afca885131
commit e11ff8a5fd
122 changed files with 986 additions and 903 deletions

View File

@ -35,11 +35,11 @@ public interface Abortable {
* @param why Why we're aborting.
* @param e Throwable that caused abort. Can be null.
*/
public void abort(String why, Throwable e);
void abort(String why, Throwable e);
/**
* Check if the server or client was aborted.
* @return true if the server or client was aborted, false otherwise
*/
public boolean isAborted();
boolean isAborted();
}

View File

@ -26,21 +26,21 @@ import java.io.IOException;
@InterfaceAudience.Public
@InterfaceStability.Evolving
public interface Coprocessor {
static final int VERSION = 1;
int VERSION = 1;
/** Highest installation priority */
static final int PRIORITY_HIGHEST = 0;
int PRIORITY_HIGHEST = 0;
/** High (system) installation priority */
static final int PRIORITY_SYSTEM = Integer.MAX_VALUE / 4;
int PRIORITY_SYSTEM = Integer.MAX_VALUE / 4;
/** Default installation priority for user coprocessors */
static final int PRIORITY_USER = Integer.MAX_VALUE / 2;
int PRIORITY_USER = Integer.MAX_VALUE / 2;
/** Lowest installation priority */
static final int PRIORITY_LOWEST = Integer.MAX_VALUE;
int PRIORITY_LOWEST = Integer.MAX_VALUE;
/**
* Lifecycle state of a given coprocessor instance.
*/
public enum State {
enum State {
UNINSTALLED,
INSTALLED,
STARTING,

View File

@ -30,26 +30,26 @@ import java.io.IOException;
public interface CoprocessorEnvironment {
/** @return the Coprocessor interface version */
public int getVersion();
int getVersion();
/** @return the HBase version as a string (e.g. "0.21.0") */
public String getHBaseVersion();
String getHBaseVersion();
/** @return the loaded coprocessor instance */
public Coprocessor getInstance();
Coprocessor getInstance();
/** @return the priority assigned to the loaded coprocessor */
public int getPriority();
int getPriority();
/** @return the load sequence number */
public int getLoadSequence();
int getLoadSequence();
/** @return the configuration */
public Configuration getConfiguration();
Configuration getConfiguration();
/**
* @return an interface for accessing the given table
* @throws IOException
*/
public HTableInterface getTable(byte[] tableName) throws IOException;
HTableInterface getTable(byte[] tableName) throws IOException;
}

View File

@ -32,20 +32,20 @@ public interface Server extends Abortable, Stoppable {
/**
* Gets the configuration object for this server.
*/
public Configuration getConfiguration();
Configuration getConfiguration();
/**
* Gets the ZooKeeper instance for this server.
*/
public ZooKeeperWatcher getZooKeeper();
ZooKeeperWatcher getZooKeeper();
/**
* @return Master's instance of {@link CatalogTracker}
*/
public CatalogTracker getCatalogTracker();
CatalogTracker getCatalogTracker();
/**
* @return The unique server name for this server.
*/
public ServerName getServerName();
ServerName getServerName();
}

View File

@ -29,10 +29,10 @@ public interface Stoppable {
* Stop this service.
* @param why Why we're stopping.
*/
public void stop(String why);
void stop(String why);
/**
* @return True if {@link #stop(String)} has been closed.
*/
public boolean isStopped();
boolean isStopped();
}

View File

@ -558,7 +558,7 @@ public class MetaReader {
* @return True if we are to proceed scanning the table, else false if
* we are to stop now.
*/
public boolean visit(final Result r) throws IOException;
boolean visit(final Result r) throws IOException;
}
/**

View File

@ -117,7 +117,7 @@ class AsyncProcess<CResult> {
* </li>
* </list>
*/
static interface AsyncProcessCallback<CResult> {
interface AsyncProcessCallback<CResult> {
/**
* Called on success. originalIndex holds the index in the action list.

View File

@ -34,18 +34,18 @@ public interface Attributes {
* @param name attribute name
* @param value attribute value
*/
public void setAttribute(String name, byte[] value);
void setAttribute(String name, byte[] value);
/**
* Gets an attribute
* @param name attribute name
* @return attribute value if attribute is set, <tt>null</tt> otherwise
*/
public byte[] getAttribute(String name);
byte[] getAttribute(String name);
/**
* Gets all attributes
* @return unmodifiable map of all attributes
*/
public Map<String, byte[]> getAttributesMap();
Map<String, byte[]> getAttributesMap();
}

View File

@ -82,19 +82,19 @@ class ClusterStatusListener implements Closeable {
*
* @param sn - the server name
*/
public void newDead(ServerName sn);
void newDead(ServerName sn);
}
/**
* The interface to be implented by a listener of a cluster status event.
*/
static interface Listener extends Closeable {
interface Listener extends Closeable {
/**
* Called to close the resources, if any. Cannot throw an exception.
*/
@Override
public void close();
void close();
/**
* Called to connect.
@ -102,7 +102,7 @@ class ClusterStatusListener implements Closeable {
* @param conf Configuration to use.
* @throws IOException
*/
public void connect(Configuration conf) throws IOException;
void connect(Configuration conf) throws IOException;
}
public ClusterStatusListener(DeadServerHandler dsh, Configuration conf,

View File

@ -62,10 +62,10 @@ public interface HConnection extends Abortable, Closeable {
/**
* @return Configuration instance being used by this HConnection instance.
*/
public Configuration getConfiguration();
Configuration getConfiguration();
/** @return - true if the master server is running */
public boolean isMasterRunning()
boolean isMasterRunning()
throws MasterNotRunningException, ZooKeeperConnectionException;
/**
@ -76,21 +76,21 @@ public interface HConnection extends Abortable, Closeable {
* @return true if the table is enabled, false otherwise
* @throws IOException if a remote or network exception occurs
*/
public boolean isTableEnabled(byte[] tableName) throws IOException;
boolean isTableEnabled(byte[] tableName) throws IOException;
/**
* @param tableName table name
* @return true if the table is disabled, false otherwise
* @throws IOException if a remote or network exception occurs
*/
public boolean isTableDisabled(byte[] tableName) throws IOException;
boolean isTableDisabled(byte[] tableName) throws IOException;
/**
* @param tableName table name
* @return true if all regions of the table are available, false otherwise
* @throws IOException if a remote or network exception occurs
*/
public boolean isTableAvailable(byte[] tableName) throws IOException;
boolean isTableAvailable(byte[] tableName) throws IOException;
/**
* Use this api to check if the table has been created with the specified number of
@ -104,7 +104,7 @@ public interface HConnection extends Abortable, Closeable {
* @throws IOException
* if a remote or network exception occurs
*/
public boolean isTableAvailable(byte[] tableName, byte[][] splitKeys) throws IOException;
boolean isTableAvailable(byte[] tableName, byte[][] splitKeys) throws IOException;
/**
* List all the userspace tables. In other words, scan the META table.
@ -116,14 +116,14 @@ public interface HConnection extends Abortable, Closeable {
* @return - returns an array of HTableDescriptors
* @throws IOException if a remote or network exception occurs
*/
public HTableDescriptor[] listTables() throws IOException;
HTableDescriptor[] listTables() throws IOException;
/**
* @param tableName table name
* @return table metadata
* @throws IOException if a remote or network exception occurs
*/
public HTableDescriptor getHTableDescriptor(byte[] tableName)
HTableDescriptor getHTableDescriptor(byte[] tableName)
throws IOException;
/**
@ -135,14 +135,15 @@ public interface HConnection extends Abortable, Closeable {
* question
* @throws IOException if a remote or network exception occurs
*/
public HRegionLocation locateRegion(final byte [] tableName,
final byte [] row)
HRegionLocation locateRegion(
final byte[] tableName, final byte[] row
)
throws IOException;
/**
* Allows flushing the region cache.
*/
public void clearRegionCache();
void clearRegionCache();
/**
* Allows flushing the region cache of all locations that pertain to
@ -150,13 +151,13 @@ public interface HConnection extends Abortable, Closeable {
* @param tableName Name of the table whose regions we are to remove from
* cache.
*/
public void clearRegionCache(final byte [] tableName);
void clearRegionCache(final byte[] tableName);
/**
* Deletes cached locations for the specific region.
* @param location The location object for the region, to be purged from cache.
*/
public void deleteCachedRegionLocation(final HRegionLocation location);
void deleteCachedRegionLocation(final HRegionLocation location);
/**
* Find the location of the region of <i>tableName</i> that <i>row</i>
@ -167,8 +168,9 @@ public interface HConnection extends Abortable, Closeable {
* question
* @throws IOException if a remote or network exception occurs
*/
public HRegionLocation relocateRegion(final byte [] tableName,
final byte [] row)
HRegionLocation relocateRegion(
final byte[] tableName, final byte[] row
)
throws IOException;
/**
@ -179,8 +181,9 @@ public interface HConnection extends Abortable, Closeable {
* @param exception the exception if any. Can be null.
* @param source the previous location
*/
public void updateCachedLocations(byte[] tableName, byte[] rowkey,
Object exception, HRegionLocation source);
void updateCachedLocations(
byte[] tableName, byte[] rowkey, Object exception, HRegionLocation source
);
/**
* Gets the location of the region of <i>regionName</i>.
@ -189,7 +192,7 @@ public interface HConnection extends Abortable, Closeable {
* question
* @throws IOException if a remote or network exception occurs
*/
public HRegionLocation locateRegion(final byte [] regionName)
HRegionLocation locateRegion(final byte[] regionName)
throws IOException;
/**
@ -198,7 +201,7 @@ public interface HConnection extends Abortable, Closeable {
* @return list of region locations for all regions of table
* @throws IOException
*/
public List<HRegionLocation> locateRegions(final byte[] tableName)
List<HRegionLocation> locateRegions(final byte[] tableName)
throws IOException;
/**
@ -210,18 +213,19 @@ public interface HConnection extends Abortable, Closeable {
* @return list of region locations for all regions of table
* @throws IOException
*/
public List<HRegionLocation> locateRegions(final byte[] tableName, final boolean useCache,
final boolean offlined) throws IOException;
List<HRegionLocation> locateRegions(
final byte[] tableName, final boolean useCache, final boolean offlined
) throws IOException;
/**
* Returns a {@link MasterAdminKeepAliveConnection} to the active master
*/
public MasterAdminService.BlockingInterface getMasterAdmin() throws IOException;
MasterAdminService.BlockingInterface getMasterAdmin() throws IOException;
/**
* Returns an {@link MasterMonitorKeepAliveConnection} to the active master
*/
public MasterMonitorService.BlockingInterface getMasterMonitor() throws IOException;
MasterMonitorService.BlockingInterface getMasterMonitor() throws IOException;
/**
* Establishes a connection to the region server at the specified address.
@ -229,7 +233,7 @@ public interface HConnection extends Abortable, Closeable {
* @return proxy for HRegionServer
* @throws IOException if a remote or network exception occurs
*/
public AdminService.BlockingInterface getAdmin(final ServerName serverName) throws IOException;
AdminService.BlockingInterface getAdmin(final ServerName serverName) throws IOException;
/**
* Establishes a connection to the region server at the specified address, and returns
@ -240,7 +244,7 @@ public interface HConnection extends Abortable, Closeable {
* @throws IOException if a remote or network exception occurs
*
*/
public ClientService.BlockingInterface getClient(final ServerName serverName) throws IOException;
ClientService.BlockingInterface getClient(final ServerName serverName) throws IOException;
/**
* Establishes a connection to the region server at the specified address.
@ -250,7 +254,7 @@ public interface HConnection extends Abortable, Closeable {
* @throws IOException if a remote or network exception occurs
* @deprecated You can pass master flag but nothing special is done.
*/
public AdminService.BlockingInterface getAdmin(final ServerName serverName, boolean getMaster)
AdminService.BlockingInterface getAdmin(final ServerName serverName, boolean getMaster)
throws IOException;
/**
@ -277,7 +281,7 @@ public interface HConnection extends Abortable, Closeable {
* @throws RuntimeException other unspecified error
*/
@Deprecated
public <T> T getRegionServerWithRetries(ServerCallable<T> callable)
<T> T getRegionServerWithRetries(ServerCallable<T> callable)
throws IOException, RuntimeException;
/**
@ -290,7 +294,7 @@ public interface HConnection extends Abortable, Closeable {
* @throws RuntimeException other unspecified error
*/
@Deprecated
public <T> T getRegionServerWithoutRetries(ServerCallable<T> callable)
<T> T getRegionServerWithoutRetries(ServerCallable<T> callable)
throws IOException, RuntimeException;
/**
@ -309,8 +313,9 @@ public interface HConnection extends Abortable, Closeable {
* @deprecated since 0.96 - Use {@link HTableInterface#batch} instead
*/
@Deprecated
public void processBatch(List<? extends Row> actions, final byte[] tableName,
ExecutorService pool, Object[] results)
void processBatch(
List<? extends Row> actions, final byte[] tableName, ExecutorService pool, Object[] results
)
throws IOException, InterruptedException;
/**
@ -319,11 +324,13 @@ public interface HConnection extends Abortable, Closeable {
* @deprecated since 0.96 - Use {@link HTableInterface#batchCallback} instead
*/
@Deprecated
public <R> void processBatchCallback(List<? extends Row> list,
byte[] tableName,
ExecutorService pool,
Object[] results,
Batch.Callback<R> callback) throws IOException, InterruptedException;
<R> void processBatchCallback(
List<? extends Row> list,
byte[] tableName,
ExecutorService pool,
Object[] results,
Batch.Callback<R> callback
) throws IOException, InterruptedException;
/**
* Enable or disable region cache prefetch for the table. It will be
@ -332,8 +339,9 @@ public interface HConnection extends Abortable, Closeable {
* @param tableName name of table to configure.
* @param enable Set to true to enable region cache prefetch.
*/
public void setRegionCachePrefetch(final byte[] tableName,
final boolean enable);
void setRegionCachePrefetch(
final byte[] tableName, final boolean enable
);
/**
* Check whether region cache prefetch is enabled or not.
@ -341,34 +349,34 @@ public interface HConnection extends Abortable, Closeable {
* @return true if table's region cache prefetch is enabled. Otherwise
* it is disabled.
*/
public boolean getRegionCachePrefetch(final byte[] tableName);
boolean getRegionCachePrefetch(final byte[] tableName);
/**
* @return the number of region servers that are currently running
* @throws IOException if a remote or network exception occurs
* @deprecated This method will be changed from public to package protected.
*/
public int getCurrentNrHRS() throws IOException;
int getCurrentNrHRS() throws IOException;
/**
* @param tableNames List of table names
* @return HTD[] table metadata
* @throws IOException if a remote or network exception occurs
*/
public HTableDescriptor[] getHTableDescriptors(List<String> tableNames)
HTableDescriptor[] getHTableDescriptors(List<String> tableNames)
throws IOException;
/**
* @return true if this connection is closed
*/
public boolean isClosed();
boolean isClosed();
/**
* Clear any caches that pertain to server name <code>sn</code>.
* @param sn A server name
*/
public void clearCaches(final ServerName sn);
void clearCaches(final ServerName sn);
/**
* This function allows HBaseAdmin and potentially others to get a shared MasterMonitor
@ -377,7 +385,7 @@ public interface HConnection extends Abortable, Closeable {
* @throws MasterNotRunningException
*/
// TODO: Why is this in the public interface when the returned type is shutdown package access?
public MasterMonitorKeepAliveConnection getKeepAliveMasterMonitorService()
MasterMonitorKeepAliveConnection getKeepAliveMasterMonitorService()
throws MasterNotRunningException;
/**
@ -387,11 +395,11 @@ public interface HConnection extends Abortable, Closeable {
* @throws MasterNotRunningException
*/
// TODO: Why is this in the public interface when the returned type is shutdown package access?
public MasterAdminKeepAliveConnection getKeepAliveMasterAdminService() throws MasterNotRunningException;
MasterAdminKeepAliveConnection getKeepAliveMasterAdminService() throws MasterNotRunningException;
/**
* @param serverName
* @return true if the server is known as dead, false otherwise.
*/
public boolean isDeadServer(ServerName serverName);
boolean isDeadServer(ServerName serverName);
}

View File

@ -127,8 +127,9 @@ public interface HTableInterface extends Closeable {
* Same as {@link #batch(List, Object[])}, but with a callback.
* @since 0.96.0
*/
public <R> void batchCallback(
final List<? extends Row> actions, final Object[] results, final Batch.Callback<R> callback)
<R> void batchCallback(
final List<? extends Row> actions, final Object[] results, final Batch.Callback<R> callback
)
throws IOException, InterruptedException;
@ -136,8 +137,9 @@ public interface HTableInterface extends Closeable {
* Same as {@link #batch(List)}, but with a callback.
* @since 0.96.0
*/
public <R> Object[] batchCallback(
List<? extends Row> actions, Batch.Callback<R> callback) throws IOException,
<R> Object[] batchCallback(
List<? extends Row> actions, Batch.Callback<R> callback
) throws IOException,
InterruptedException;
/**
@ -309,7 +311,7 @@ public interface HTableInterface extends Closeable {
* @param rm object that specifies the set of mutations to perform atomically
* @throws IOException
*/
public void mutateRow(final RowMutations rm) throws IOException;
void mutateRow(final RowMutations rm) throws IOException;
/**
* Appends values to one or more columns within a single row.
@ -324,7 +326,7 @@ public interface HTableInterface extends Closeable {
* @throws IOException e
* @return values of columns after the append operation (maybe null)
*/
public Result append(final Append append) throws IOException;
Result append(final Append append) throws IOException;
/**
* Increments one or more columns within a single row.
@ -339,7 +341,7 @@ public interface HTableInterface extends Closeable {
* @throws IOException e
* @return values of columns after the increment
*/
public Result increment(final Increment increment) throws IOException;
Result increment(final Increment increment) throws IOException;
/**
* See {@link #incrementColumnValue(byte[], byte[], byte[], long, Durability)}
@ -493,7 +495,7 @@ public interface HTableInterface extends Closeable {
* @param autoFlush
* Whether or not to enable 'auto-flush'.
*/
public void setAutoFlush(boolean autoFlush);
void setAutoFlush(boolean autoFlush);
/**
* Turns 'auto-flush' on or off.
@ -522,7 +524,7 @@ public interface HTableInterface extends Closeable {
* Whether to keep Put failures in the writeBuffer
* @see #flushCommits
*/
public void setAutoFlush(boolean autoFlush, boolean clearBufferOnFail);
void setAutoFlush(boolean autoFlush, boolean clearBufferOnFail);
/**
* Returns the maximum size in bytes of the write buffer for this HTable.
@ -531,7 +533,7 @@ public interface HTableInterface extends Closeable {
* {@code hbase.client.write.buffer}.
* @return The size of the write buffer in bytes.
*/
public long getWriteBufferSize();
long getWriteBufferSize();
/**
* Sets the size of the buffer in bytes.
@ -541,5 +543,5 @@ public interface HTableInterface extends Closeable {
* @param writeBufferSize The new write buffer size, in bytes.
* @throws IOException if a remote or network exception occurs.
*/
public void setWriteBufferSize(long writeBufferSize) throws IOException;
void setWriteBufferSize(long writeBufferSize) throws IOException;
}

View File

@ -40,5 +40,5 @@ extends MasterAdminProtos.MasterAdminService.BlockingInterface {
*/
// The Closeable Interface wants to throw an IOE out of a close.
// Thats a PITA. Do this below instead of Closeable.
public void close();
}
void close();
}

View File

@ -297,7 +297,7 @@ public class MetaScanner {
* @return A boolean to know if it should continue to loop in the region
* @throws IOException e
*/
public boolean processRow(Result rowResult) throws IOException;
boolean processRow(Result rowResult) throws IOException;
}
public static abstract class MetaScannerVisitorBase implements MetaScannerVisitor {

View File

@ -38,17 +38,17 @@ public interface ResultScanner extends Closeable, Iterable<Result> {
* exhausted.
* @throws IOException e
*/
public Result next() throws IOException;
Result next() throws IOException;
/**
* @param nbRows number of rows to return
* @return Between zero and <param>nbRows</param> Results
* @throws IOException e
*/
public Result [] next(int nbRows) throws IOException;
Result [] next(int nbRows) throws IOException;
/**
* Closes the scanner and releases any resources it has allocated
*/
public void close();
void close();
}

View File

@ -30,5 +30,5 @@ public interface Row extends Comparable<Row> {
/**
* @return The row.
*/
public byte [] getRow();
}
byte [] getRow();
}

View File

@ -50,8 +50,8 @@ public abstract class Batch {
* {@link Batch.Call#call(Object)}
* @param <R> the return type from {@link Batch.Call#call(Object)}
*/
public static interface Call<T,R> {
public R call(T instance) throws IOException;
public interface Call<T,R> {
R call(T instance) throws IOException;
}
/**
@ -68,7 +68,7 @@ public abstract class Batch {
* @param <R> the return type from the associated {@link Batch.Call#call(Object)}
* @see org.apache.hadoop.hbase.client.HTable#coprocessorService(Class, byte[], byte[], org.apache.hadoop.hbase.client.coprocessor.Batch.Call)
*/
public static interface Callback<R> {
public void update(byte[] region, byte[] row, R result);
public interface Callback<R> {
void update(byte[] region, byte[] row, R result);
}
}
}

View File

@ -46,7 +46,7 @@ public interface ReplicationPeers {
* Initialize the ReplicationPeers interface.
* @throws KeeperException
*/
public void init() throws IOException, KeeperException;
void init() throws IOException, KeeperException;
/**
* Add a new remote slave cluster for replication.
@ -54,65 +54,65 @@ public interface ReplicationPeers {
* @param clusterKey the concatenation of the slave cluster's:
* hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
*/
public void addPeer(String peerId, String clusterKey) throws IOException;
void addPeer(String peerId, String clusterKey) throws IOException;
/**
* Removes a remote slave cluster and stops the replication to it.
* @param peerId a short that identifies the cluster
*/
public void removePeer(String peerId) throws IOException;
void removePeer(String peerId) throws IOException;
/**
* Restart the replication to the specified remote slave cluster.
* @param peerId a short that identifies the cluster
*/
public void enablePeer(String peerId) throws IOException;
void enablePeer(String peerId) throws IOException;
/**
* Stop the replication to the specified remote slave cluster.
* @param peerId a short that identifies the cluster
*/
public void disablePeer(String peerId) throws IOException;
void disablePeer(String peerId) throws IOException;
/**
* Get the replication status for the specified connected remote slave cluster.
* @param peerId a short that identifies the cluster
* @return true if replication is enabled, false otherwise.
*/
public boolean getStatusOfConnectedPeer(String peerId);
boolean getStatusOfConnectedPeer(String peerId);
/**
* Get a set of all connected remote slave clusters.
* @return set of peer ids
*/
public Set<String> getConnectedPeers();
Set<String> getConnectedPeers();
/**
* List the cluster keys of all remote slave clusters (whether they are enabled/disabled or
* connected/disconnected).
* @return A map of peer ids to peer cluster keys
*/
public Map<String, String> getAllPeerClusterKeys();
Map<String, String> getAllPeerClusterKeys();
/**
* List the peer ids of all remote slave clusters (whether they are enabled/disabled or
* connected/disconnected).
* @return A list of peer ids
*/
public List<String> getAllPeerIds();
List<String> getAllPeerIds();
/**
* Attempt to connect to a new remote slave cluster.
* @param peerId a short that identifies the cluster
* @return true if a new connection was made, false if no new connection was made.
*/
public boolean connectToPeer(String peerId) throws IOException, KeeperException;
boolean connectToPeer(String peerId) throws IOException, KeeperException;
/**
* Disconnect from a remote slave cluster.
* @param peerId a short that identifies the cluster
*/
public void disconnectFromPeer(String peerId);
void disconnectFromPeer(String peerId);
/**
* Returns all region servers from given connected remote slave cluster.
@ -120,19 +120,19 @@ public interface ReplicationPeers {
* @return addresses of all region servers in the peer cluster. Returns an empty list if the peer
* cluster is unavailable or there are no region servers in the cluster.
*/
public List<ServerName> getRegionServersOfConnectedPeer(String peerId);
List<ServerName> getRegionServersOfConnectedPeer(String peerId);
/**
* Returns the UUID of the provided peer id.
* @param peerId the peer's ID that will be converted into a UUID
* @return a UUID or null if the peer cluster does not exist or is not connected.
*/
public UUID getPeerUUID(String peerId);
UUID getPeerUUID(String peerId);
/**
* Returns the configuration needed to talk to the remote slave cluster.
* @param peerId a short that identifies the cluster
* @return the configuration for the peer cluster, null if it was unable to get the configuration
*/
public Configuration getPeerConf(String peerId) throws KeeperException;
}
Configuration getPeerConf(String peerId) throws KeeperException;
}

View File

@ -37,13 +37,13 @@ public interface ReplicationQueues {
* @param serverName The server name of the region server that owns the replication queues this
* interface manages.
*/
public void init(String serverName) throws KeeperException;
void init(String serverName) throws KeeperException;
/**
* Remove a replication queue.
* @param queueId a String that identifies the queue.
*/
public void removeQueue(String queueId);
void removeQueue(String queueId);
/**
* Add a new HLog file to the given queue. If the queue does not exist it is created.
@ -51,14 +51,14 @@ public interface ReplicationQueues {
* @param filename name of the HLog
* @throws KeeperException
*/
public void addLog(String queueId, String filename) throws KeeperException;
void addLog(String queueId, String filename) throws KeeperException;
/**
* Remove an HLog file from the given queue.
* @param queueId a String that identifies the queue.
* @param filename name of the HLog
*/
public void removeLog(String queueId, String filename);
void removeLog(String queueId, String filename);
/**
* Set the current position for a specific HLog in a given queue.
@ -66,7 +66,7 @@ public interface ReplicationQueues {
* @param filename name of the HLog
* @param position the current position in the file
*/
public void setLogPosition(String queueId, String filename, long position);
void setLogPosition(String queueId, String filename, long position);
/**
* Get the current position for a specific HLog in a given queue.
@ -74,25 +74,25 @@ public interface ReplicationQueues {
* @param filename name of the HLog
* @return the current position in the file
*/
public long getLogPosition(String queueId, String filename) throws KeeperException;
long getLogPosition(String queueId, String filename) throws KeeperException;
/**
* Remove all replication queues for this region server.
*/
public void removeAllQueues();
void removeAllQueues();
/**
* Get a list of all HLogs in the given queue.
* @param queueId a String that identifies the queue
* @return a list of HLogs, null if this region server is dead and has no outstanding queues
*/
public List<String> getLogsInQueue(String queueId);
List<String> getLogsInQueue(String queueId);
/**
* Get a list of all queues for this region server.
* @return a list of queueIds, null if this region server is dead and has no outstanding queues
*/
public List<String> getAllQueues();
List<String> getAllQueues();
/**
* Take ownership for the set of queues belonging to a dead region server.
@ -100,12 +100,12 @@ public interface ReplicationQueues {
* @return A SortedMap of the queues that have been claimed, including a SortedSet of HLogs in
* each queue. Returns an empty map if no queues were failed-over.
*/
public SortedMap<String, SortedSet<String>> claimQueues(String regionserver);
SortedMap<String, SortedSet<String>> claimQueues(String regionserver);
/**
* Get a list of all region servers that have outstanding replication queues. These servers could
* be alive, dead or from a previous run of the cluster.
* @return a list of server names
*/
public List<String> getListOfReplicators();
}
List<String> getListOfReplicators();
}

View File

@ -31,7 +31,7 @@ public interface ReplicationQueuesClient {
* be alive, dead or from a previous run of the cluster.
* @return a list of server names
*/
public List<String> getListOfReplicators();
List<String> getListOfReplicators();
/**
* Get a list of all HLogs in the given queue on the given region server.
@ -39,12 +39,12 @@ public interface ReplicationQueuesClient {
* @param queueId a String that identifies the queue
* @return a list of HLogs, null if this region server is dead and has no outstanding queues
*/
public List<String> getLogsInQueue(String serverName, String queueId);
List<String> getLogsInQueue(String serverName, String queueId);
/**
* Get a list of all queues for the specified region server.
* @param serverName the server name of the region server that owns the set of queues
* @return a list of queueIds, null if this region server is not a replicator.
*/
public List<String> getAllQueues(String serverName);
}
List<String> getAllQueues(String serverName);
}

View File

@ -216,17 +216,17 @@ public class PoolMap<K, V> implements Map<K, V> {
}
protected interface Pool<R> {
public R get();
R get();
public R put(R resource);
R put(R resource);
public boolean remove(R resource);
boolean remove(R resource);
public void clear();
void clear();
public Collection<R> values();
Collection<R> values();
public int size();
int size();
}
public enum PoolType {

View File

@ -66,7 +66,7 @@ public class CompoundConfiguration extends Configuration {
// Devs: these APIs are the same contract as their counterparts in
// Configuration.java
private static interface ImmutableConfigMap extends Iterable<Map.Entry<String,String>> {
private interface ImmutableConfigMap extends Iterable<Map.Entry<String,String>> {
String get(String key);
String getRaw(String key);
Class<?> getClassByName(String name) throws ClassNotFoundException;

View File

@ -2514,14 +2514,14 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
/**
* Avoids redundant comparisons for better performance.
*/
public static interface SamePrefixComparator<T> {
public interface SamePrefixComparator<T> {
/**
* Compare two keys assuming that the first n bytes are the same.
* @param commonPrefix How many bytes are the same.
*/
public int compareIgnoringPrefix(int commonPrefix,
T left, int loffset, int llength,
T right, int roffset, int rlength);
int compareIgnoringPrefix(
int commonPrefix, T left, int loffset, int llength, T right, int roffset, int rlength
);
}
/**

View File

@ -37,14 +37,14 @@ public interface Codec {
* Call flush when done. Some encoders may not put anything on the stream until flush is called.
* On flush, let go of any resources used by the encoder.
*/
public interface Encoder extends CellOutputStream {}
interface Encoder extends CellOutputStream {}
/**
* Implementations should implicitly clean up any resources allocated when the
* Decoder/CellScanner runs off the end of the cell block. Do this rather than require the user
* call close explicitly.
*/
public interface Decoder extends CellScanner {};
interface Decoder extends CellScanner {};
Decoder getDecoder(InputStream is);
Encoder getEncoder(OutputStream os);

View File

@ -45,5 +45,5 @@ public interface HeapSize {
* @return Approximate 'exclusive deep size' of implementing object. Includes
* count of payload and hosting object sizings.
*/
public long heapSize();
}
long heapSize();
}

View File

@ -57,9 +57,9 @@ public interface DataBlockEncoder {
* @throws IOException
* If there is an error writing to output stream.
*/
public void encodeKeyValues(
ByteBuffer in, boolean includesMemstoreTS,
HFileBlockEncodingContext encodingContext) throws IOException;
void encodeKeyValues(
ByteBuffer in, boolean includesMemstoreTS, HFileBlockEncodingContext encodingContext
) throws IOException;
/**
* Decode.
@ -69,8 +69,9 @@ public interface DataBlockEncoder {
* @return Uncompressed block of KeyValues.
* @throws IOException If there is an error in source.
*/
public ByteBuffer decodeKeyValues(DataInputStream source,
boolean includesMemstoreTS) throws IOException;
ByteBuffer decodeKeyValues(
DataInputStream source, boolean includesMemstoreTS
) throws IOException;
/**
* Uncompress.
@ -82,8 +83,9 @@ public interface DataBlockEncoder {
* @return Uncompressed block of KeyValues.
* @throws IOException If there is an error in source.
*/
public ByteBuffer decodeKeyValues(DataInputStream source,
int allocateHeaderLength, int skipLastBytes, boolean includesMemstoreTS)
ByteBuffer decodeKeyValues(
DataInputStream source, int allocateHeaderLength, int skipLastBytes, boolean includesMemstoreTS
)
throws IOException;
/**
@ -94,7 +96,7 @@ public interface DataBlockEncoder {
* @param block encoded block we want index, the position will not change
* @return First key in block.
*/
public ByteBuffer getFirstKeyInBlock(ByteBuffer block);
ByteBuffer getFirstKeyInBlock(ByteBuffer block);
/**
* Create a HFileBlock seeker which find KeyValues within a block.
@ -103,8 +105,9 @@ public interface DataBlockEncoder {
* key-value pair
* @return A newly created seeker.
*/
public EncodedSeeker createSeeker(RawComparator<byte[]> comparator,
boolean includesMemstoreTS);
EncodedSeeker createSeeker(
RawComparator<byte[]> comparator, boolean includesMemstoreTS
);
/**
* Creates a encoder specific encoding context
@ -119,9 +122,9 @@ public interface DataBlockEncoder {
* is unknown
* @return a newly created encoding context
*/
public HFileBlockEncodingContext newDataBlockEncodingContext(
Algorithm compressionAlgorithm, DataBlockEncoding encoding,
byte[] headerBytes);
HFileBlockEncodingContext newDataBlockEncodingContext(
Algorithm compressionAlgorithm, DataBlockEncoding encoding, byte[] headerBytes
);
/**
* Creates an encoder specific decoding context, which will prepare the data
@ -131,8 +134,9 @@ public interface DataBlockEncoder {
* compression algorithm used if the data needs to be decompressed
* @return a newly created decoding context
*/
public HFileBlockDecodingContext newDataBlockDecodingContext(
Algorithm compressionAlgorithm);
HFileBlockDecodingContext newDataBlockDecodingContext(
Algorithm compressionAlgorithm
);
/**
* An interface which enable to seek while underlying data is encoded.
@ -140,19 +144,19 @@ public interface DataBlockEncoder {
* It works on one HFileBlock, but it is reusable. See
* {@link #setCurrentBuffer(ByteBuffer)}.
*/
public static interface EncodedSeeker {
interface EncodedSeeker {
/**
* Set on which buffer there will be done seeking.
* @param buffer Used for seeking.
*/
public void setCurrentBuffer(ByteBuffer buffer);
void setCurrentBuffer(ByteBuffer buffer);
/**
* Does a deep copy of the key at the current position. A deep copy is
* necessary because buffers are reused in the decoder.
* @return key at current position
*/
public ByteBuffer getKeyDeepCopy();
ByteBuffer getKeyDeepCopy();
/**
* Does a shallow copy of the value at the current position. A shallow
@ -160,25 +164,25 @@ public interface DataBlockEncoder {
* of the original encoded buffer.
* @return value at current position
*/
public ByteBuffer getValueShallowCopy();
ByteBuffer getValueShallowCopy();
/** @return key value at current position with position set to limit */
public ByteBuffer getKeyValueBuffer();
ByteBuffer getKeyValueBuffer();
/**
* @return the KeyValue object at the current position. Includes memstore
* timestamp.
*/
public KeyValue getKeyValue();
KeyValue getKeyValue();
/** Set position to beginning of given block */
public void rewind();
void rewind();
/**
* Move to next position
* @return true on success, false if there is no more positions.
*/
public boolean next();
boolean next();
/**
* Moves the seeker position within the current block to:
@ -197,7 +201,8 @@ public interface DataBlockEncoder {
* of an exact match. Does not matter in case of an inexact match.
* @return 0 on exact match, 1 on inexact match.
*/
public int seekToKeyInBlock(byte[] key, int offset, int length,
boolean seekBefore);
int seekToKeyInBlock(
byte[] key, int offset, int length, boolean seekBefore
);
}
}

View File

@ -32,7 +32,7 @@ public interface HFileBlockDecodingContext {
/**
* @return the compression algorithm used by this decoding context
*/
public Compression.Algorithm getCompression();
Compression.Algorithm getCompression();
/**
* Perform all actions that need to be done before the encoder's real decoding process.
@ -47,7 +47,12 @@ public interface HFileBlockDecodingContext {
* @param offset data start offset in onDiskBlock
* @throws IOException
*/
public void prepareDecoding(int onDiskSizeWithoutHeader, int uncompressedSizeWithoutHeader,
ByteBuffer blockBufferWithoutHeader, byte[] onDiskBlock, int offset) throws IOException;
void prepareDecoding(
int onDiskSizeWithoutHeader,
int uncompressedSizeWithoutHeader,
ByteBuffer blockBufferWithoutHeader,
byte[] onDiskBlock,
int offset
) throws IOException;
}

View File

@ -34,39 +34,39 @@ public interface HFileBlockEncodingContext {
/**
* @return OutputStream to which encoded data is written
*/
public OutputStream getOutputStreamForEncoder();
OutputStream getOutputStreamForEncoder();
/**
* @return encoded and compressed bytes with header which are ready to write
* out to disk
*/
public byte[] getOnDiskBytesWithHeader();
byte[] getOnDiskBytesWithHeader();
/**
* @return encoded but not heavily compressed bytes with header which can be
* cached in block cache
*/
public byte[] getUncompressedBytesWithHeader();
byte[] getUncompressedBytesWithHeader();
/**
* @return the block type after encoding
*/
public BlockType getBlockType();
BlockType getBlockType();
/**
* @return the compression algorithm used by this encoding context
*/
public Compression.Algorithm getCompression();
Compression.Algorithm getCompression();
/**
* sets the dummy header bytes
*/
public void setDummyHeader(byte[] headerBytes);
void setDummyHeader(byte[] headerBytes);
/**
* @return the {@link DataBlockEncoding} encoding used
*/
public DataBlockEncoding getDataBlockEncoding();
DataBlockEncoding getDataBlockEncoding();
/**
* Do any action that needs to be performed after the encoding.
@ -76,11 +76,11 @@ public interface HFileBlockEncodingContext {
* @param blockType
* @throws IOException
*/
public void postEncoding(BlockType blockType) throws IOException;
void postEncoding(BlockType blockType) throws IOException;
/**
* Releases the resources used.
*/
public void close();
void close();
}

View File

@ -1000,8 +1000,9 @@ public class Bytes {
}
interface Comparer<T> {
abstract public int compareTo(T buffer1, int offset1, int length1,
T buffer2, int offset2, int length2);
int compareTo(
T buffer1, int offset1, int length1, T buffer2, int offset2, int length2
);
}
@VisibleForTesting

View File

@ -50,16 +50,16 @@ public class ClassFinder {
private ClassFilter classFilter;
private FileFilter fileFilter;
public static interface ResourcePathFilter {
public boolean isCandidatePath(String resourcePath, boolean isJar);
public interface ResourcePathFilter {
boolean isCandidatePath(String resourcePath, boolean isJar);
};
public static interface FileNameFilter {
public boolean isCandidateFile(String fileName, String absFilePath);
public interface FileNameFilter {
boolean isCandidateFile(String fileName, String absFilePath);
};
public static interface ClassFilter {
public boolean isCandidateClass(Class<?> c);
public interface ClassFilter {
boolean isCandidateClass(Class<?> c);
};
public ClassFinder() {

View File

@ -86,14 +86,14 @@ public final class Waiter {
* {@link Waiter#waitFor(Configuration, long, long, boolean, Predicate) methods.
*/
@InterfaceAudience.Private
public static interface Predicate<E extends Exception> {
public interface Predicate<E extends Exception> {
/**
* Perform a predicate evaluation.
* @return the boolean result of the evaluation.
* @throws Exception thrown if the predicate evaluation could not evaluate.
*/
public boolean evaluate() throws E;
boolean evaluate() throws E;
}

View File

@ -22,37 +22,37 @@ package org.apache.hadoop.hbase.ipc;
import org.apache.hadoop.hbase.metrics.BaseSource;
public interface MetricsHBaseServerSource extends BaseSource {
public static final String AUTHORIZATION_SUCCESSES_NAME = "authorizationSuccesses";
public static final String AUTHORIZATION_SUCCESSES_DESC =
String AUTHORIZATION_SUCCESSES_NAME = "authorizationSuccesses";
String AUTHORIZATION_SUCCESSES_DESC =
"Number of authorization successes.";
public static final String AUTHORIZATION_FAILURES_NAME = "authorizationFailures";
public static final String AUTHORIZATION_FAILURES_DESC =
String AUTHORIZATION_FAILURES_NAME = "authorizationFailures";
String AUTHORIZATION_FAILURES_DESC =
"Number of authorization failures.";
public static final String AUTHENTICATION_SUCCESSES_NAME = "authenticationSuccesses";
public static final String AUTHENTICATION_SUCCESSES_DESC =
String AUTHENTICATION_SUCCESSES_NAME = "authenticationSuccesses";
String AUTHENTICATION_SUCCESSES_DESC =
"Number of authentication successes.";
public static final String AUTHENTICATION_FAILURES_NAME = "authenticationFailures";
public static final String AUTHENTICATION_FAILURES_DESC =
String AUTHENTICATION_FAILURES_NAME = "authenticationFailures";
String AUTHENTICATION_FAILURES_DESC =
"Number of authentication failures.";
public static final String SENT_BYTES_NAME = "sentBytes";
public static final String SENT_BYTES_DESC = "Number of bytes sent.";
public static final String RECEIVED_BYTES_NAME = "receivedBytes";
public static final String RECEIVED_BYTES_DESC = "Number of bytes received.";
public static final String QUEUE_CALL_TIME_NAME = "queueCallTime";
public static final String QUEUE_CALL_TIME_DESC = "Queue Call Time.";
public static final String PROCESS_CALL_TIME_NAME = "processCallTime";
public static final String PROCESS_CALL_TIME_DESC = "Processing call time.";
public static final String QUEUE_SIZE_NAME = "queueSize";
public static final String QUEUE_SIZE_DESC = "Number of bytes in the call queues.";
public static final String GENERAL_QUEUE_NAME = "numCallsInGeneralQueue";
public static final String GENERAL_QUEUE_DESC = "Number of calls in the general call queue.";
public static final String PRIORITY_QUEUE_NAME = "numCallsInPriorityQueue";
public static final String REPLICATION_QUEUE_NAME = "numCallsInReplicationQueue";
public static final String REPLICATION_QUEUE_DESC =
String SENT_BYTES_NAME = "sentBytes";
String SENT_BYTES_DESC = "Number of bytes sent.";
String RECEIVED_BYTES_NAME = "receivedBytes";
String RECEIVED_BYTES_DESC = "Number of bytes received.";
String QUEUE_CALL_TIME_NAME = "queueCallTime";
String QUEUE_CALL_TIME_DESC = "Queue Call Time.";
String PROCESS_CALL_TIME_NAME = "processCallTime";
String PROCESS_CALL_TIME_DESC = "Processing call time.";
String QUEUE_SIZE_NAME = "queueSize";
String QUEUE_SIZE_DESC = "Number of bytes in the call queues.";
String GENERAL_QUEUE_NAME = "numCallsInGeneralQueue";
String GENERAL_QUEUE_DESC = "Number of calls in the general call queue.";
String PRIORITY_QUEUE_NAME = "numCallsInPriorityQueue";
String REPLICATION_QUEUE_NAME = "numCallsInReplicationQueue";
String REPLICATION_QUEUE_DESC =
"Number of calls in the replication call queue.";
public static final String PRIORITY_QUEUE_DESC = "Number of calls in the priority call queue.";
public static final String NUM_OPEN_CONNECTIONS_NAME = "numOpenConnections";
public static final String NUM_OPEN_CONNECTIONS_DESC = "Number of open connections.";
String PRIORITY_QUEUE_DESC = "Number of calls in the priority call queue.";
String NUM_OPEN_CONNECTIONS_NAME = "numOpenConnections";
String NUM_OPEN_CONNECTIONS_DESC = "Number of open connections.";
void authorizationSuccess();

View File

@ -28,60 +28,60 @@ public interface MetricsMasterSource extends BaseSource {
/**
* The name of the metrics
*/
static final String METRICS_NAME = "Server";
String METRICS_NAME = "Server";
/**
* The context metrics will be under.
*/
static final String METRICS_CONTEXT = "master";
String METRICS_CONTEXT = "master";
/**
* The name of the metrics context that metrics will be under in jmx
*/
static final String METRICS_JMX_CONTEXT = "Master,sub=" + METRICS_NAME;
String METRICS_JMX_CONTEXT = "Master,sub=" + METRICS_NAME;
/**
* Description
*/
static final String METRICS_DESCRIPTION = "Metrics about HBase master server";
String METRICS_DESCRIPTION = "Metrics about HBase master server";
// Strings used for exporting to metrics system.
static final String MASTER_ACTIVE_TIME_NAME = "masterActiveTime";
static final String MASTER_START_TIME_NAME = "masterStartTime";
static final String AVERAGE_LOAD_NAME = "averageLoad";
static final String NUM_REGION_SERVERS_NAME = "numRegionServers";
static final String NUM_DEAD_REGION_SERVERS_NAME = "numDeadRegionServers";
static final String ZOOKEEPER_QUORUM_NAME = "zookeeperQuorum";
static final String SERVER_NAME_NAME = "serverName";
static final String CLUSTER_ID_NAME = "clusterId";
static final String IS_ACTIVE_MASTER_NAME = "isActiveMaster";
static final String SPLIT_TIME_NAME = "hlogSplitTime";
static final String SPLIT_SIZE_NAME = "hlogSplitSize";
static final String SNAPSHOT_TIME_NAME = "snapshotTime";
static final String SNAPSHOT_RESTORE_TIME_NAME = "snapshotRestoreTime";
static final String SNAPSHOT_CLONE_TIME_NAME = "snapshotCloneTime";
static final String META_SPLIT_TIME_NAME = "metaHlogSplitTime";
static final String META_SPLIT_SIZE_NAME = "metaHlogSplitSize";
static final String CLUSTER_REQUESTS_NAME = "clusterRequests";
static final String RIT_COUNT_NAME = "ritCount";
static final String RIT_COUNT_OVER_THRESHOLD_NAME = "ritCountOverThreshold";
static final String RIT_OLDEST_AGE_NAME = "ritOldestAge";
static final String MASTER_ACTIVE_TIME_DESC = "Master Active Time";
static final String MASTER_START_TIME_DESC = "Master Start Time";
static final String AVERAGE_LOAD_DESC = "AverageLoad";
static final String NUMBER_OF_REGION_SERVERS_DESC = "Number of RegionServers";
static final String NUMBER_OF_DEAD_REGION_SERVERS_DESC = "Number of dead RegionServers";
static final String ZOOKEEPER_QUORUM_DESC = "Zookeeper Quorum";
static final String SERVER_NAME_DESC = "Server Name";
static final String CLUSTER_ID_DESC = "Cluster Id";
static final String IS_ACTIVE_MASTER_DESC = "Is Active Master";
static final String SPLIT_TIME_DESC = "Time it takes to finish HLog.splitLog()";
static final String SPLIT_SIZE_DESC = "Size of HLog files being split";
static final String SNAPSHOT_TIME_DESC = "Time it takes to finish snapshot()";
static final String SNAPSHOT_RESTORE_TIME_DESC = "Time it takes to finish restoreSnapshot()";
static final String SNAPSHOT_CLONE_TIME_DESC = "Time it takes to finish cloneSnapshot()";
static final String META_SPLIT_TIME_DESC = "Time it takes to finish splitMetaLog()";
static final String META_SPLIT_SIZE_DESC = "Size of META HLog files being split";
String MASTER_ACTIVE_TIME_NAME = "masterActiveTime";
String MASTER_START_TIME_NAME = "masterStartTime";
String AVERAGE_LOAD_NAME = "averageLoad";
String NUM_REGION_SERVERS_NAME = "numRegionServers";
String NUM_DEAD_REGION_SERVERS_NAME = "numDeadRegionServers";
String ZOOKEEPER_QUORUM_NAME = "zookeeperQuorum";
String SERVER_NAME_NAME = "serverName";
String CLUSTER_ID_NAME = "clusterId";
String IS_ACTIVE_MASTER_NAME = "isActiveMaster";
String SPLIT_TIME_NAME = "hlogSplitTime";
String SPLIT_SIZE_NAME = "hlogSplitSize";
String SNAPSHOT_TIME_NAME = "snapshotTime";
String SNAPSHOT_RESTORE_TIME_NAME = "snapshotRestoreTime";
String SNAPSHOT_CLONE_TIME_NAME = "snapshotCloneTime";
String META_SPLIT_TIME_NAME = "metaHlogSplitTime";
String META_SPLIT_SIZE_NAME = "metaHlogSplitSize";
String CLUSTER_REQUESTS_NAME = "clusterRequests";
String RIT_COUNT_NAME = "ritCount";
String RIT_COUNT_OVER_THRESHOLD_NAME = "ritCountOverThreshold";
String RIT_OLDEST_AGE_NAME = "ritOldestAge";
String MASTER_ACTIVE_TIME_DESC = "Master Active Time";
String MASTER_START_TIME_DESC = "Master Start Time";
String AVERAGE_LOAD_DESC = "AverageLoad";
String NUMBER_OF_REGION_SERVERS_DESC = "Number of RegionServers";
String NUMBER_OF_DEAD_REGION_SERVERS_DESC = "Number of dead RegionServers";
String ZOOKEEPER_QUORUM_DESC = "Zookeeper Quorum";
String SERVER_NAME_DESC = "Server Name";
String CLUSTER_ID_DESC = "Cluster Id";
String IS_ACTIVE_MASTER_DESC = "Is Active Master";
String SPLIT_TIME_DESC = "Time it takes to finish HLog.splitLog()";
String SPLIT_SIZE_DESC = "Size of HLog files being split";
String SNAPSHOT_TIME_DESC = "Time it takes to finish snapshot()";
String SNAPSHOT_RESTORE_TIME_DESC = "Time it takes to finish restoreSnapshot()";
String SNAPSHOT_CLONE_TIME_DESC = "Time it takes to finish cloneSnapshot()";
String META_SPLIT_TIME_DESC = "Time it takes to finish splitMetaLog()";
String META_SPLIT_SIZE_DESC = "Size of META HLog files being split";
/**
* Increment the number of requests the cluster has seen.

View File

@ -24,7 +24,7 @@ package org.apache.hadoop.hbase.metrics;
*/
public interface BaseSource {
public static final String HBASE_METRICS_SYSTEM_NAME = "HBase";
String HBASE_METRICS_SYSTEM_NAME = "HBase";
/**
* Clear out the metrics and re-prepare the source.

View File

@ -29,22 +29,22 @@ public interface MetricsRegionAggregateSource extends BaseSource {
/**
* The name of the metrics
*/
static final String METRICS_NAME = "Regions";
String METRICS_NAME = "Regions";
/**
* The name of the metrics context that metrics will be under.
*/
static final String METRICS_CONTEXT = "regionserver";
String METRICS_CONTEXT = "regionserver";
/**
* Description
*/
static final String METRICS_DESCRIPTION = "Metrics about HBase RegionServer regions and tables";
String METRICS_DESCRIPTION = "Metrics about HBase RegionServer regions and tables";
/**
* The name of the metrics context that metrics will be under in jmx
*/
static final String METRICS_JMX_CONTEXT = "RegionServer,sub=" + METRICS_NAME;
String METRICS_JMX_CONTEXT = "RegionServer,sub=" + METRICS_NAME;
/**
* Register a MetricsRegionSource as being open.

View File

@ -28,22 +28,22 @@ public interface MetricsRegionServerSource extends BaseSource {
/**
* The name of the metrics
*/
static final String METRICS_NAME = "Server";
String METRICS_NAME = "Server";
/**
* The name of the metrics context that metrics will be under.
*/
static final String METRICS_CONTEXT = "regionserver";
String METRICS_CONTEXT = "regionserver";
/**
* Description
*/
static final String METRICS_DESCRIPTION = "Metrics about HBase RegionServer";
String METRICS_DESCRIPTION = "Metrics about HBase RegionServer";
/**
* The name of the metrics context that metrics will be under in jmx
*/
static final String METRICS_JMX_CONTEXT = "RegionServer,sub=" + METRICS_NAME;
String METRICS_JMX_CONTEXT = "RegionServer,sub=" + METRICS_NAME;
/**
* Update the Put time histogram
@ -113,103 +113,103 @@ public interface MetricsRegionServerSource extends BaseSource {
void incrSlowAppend();
// Strings used for exporting to metrics system.
static final String REGION_COUNT = "regionCount";
static final String REGION_COUNT_DESC = "Number of regions";
static final String STORE_COUNT = "storeCount";
static final String STORE_COUNT_DESC = "Number of Stores";
static final String STOREFILE_COUNT = "storeFileCount";
static final String STOREFILE_COUNT_DESC = "Number of Store Files";
static final String MEMSTORE_SIZE = "memStoreSize";
static final String MEMSTORE_SIZE_DESC = "Size of the memstore";
static final String STOREFILE_SIZE = "storeFileSize";
static final String STOREFILE_SIZE_DESC = "Size of storefiles being served.";
static final String TOTAL_REQUEST_COUNT = "totalRequestCount";
static final String TOTAL_REQUEST_COUNT_DESC =
String REGION_COUNT = "regionCount";
String REGION_COUNT_DESC = "Number of regions";
String STORE_COUNT = "storeCount";
String STORE_COUNT_DESC = "Number of Stores";
String STOREFILE_COUNT = "storeFileCount";
String STOREFILE_COUNT_DESC = "Number of Store Files";
String MEMSTORE_SIZE = "memStoreSize";
String MEMSTORE_SIZE_DESC = "Size of the memstore";
String STOREFILE_SIZE = "storeFileSize";
String STOREFILE_SIZE_DESC = "Size of storefiles being served.";
String TOTAL_REQUEST_COUNT = "totalRequestCount";
String TOTAL_REQUEST_COUNT_DESC =
"Total number of requests this RegionServer has answered.";
static final String READ_REQUEST_COUNT = "readRequestCount";
static final String READ_REQUEST_COUNT_DESC =
String READ_REQUEST_COUNT = "readRequestCount";
String READ_REQUEST_COUNT_DESC =
"Number of read requests this region server has answered.";
static final String WRITE_REQUEST_COUNT = "writeRequestCount";
static final String WRITE_REQUEST_COUNT_DESC =
String WRITE_REQUEST_COUNT = "writeRequestCount";
String WRITE_REQUEST_COUNT_DESC =
"Number of mutation requests this region server has answered.";
static final String CHECK_MUTATE_FAILED_COUNT = "checkMutateFailedCount";
static final String CHECK_MUTATE_FAILED_COUNT_DESC =
String CHECK_MUTATE_FAILED_COUNT = "checkMutateFailedCount";
String CHECK_MUTATE_FAILED_COUNT_DESC =
"Number of Check and Mutate calls that failed the checks.";
static final String CHECK_MUTATE_PASSED_COUNT = "checkMutatePassedCount";
static final String CHECK_MUTATE_PASSED_COUNT_DESC =
String CHECK_MUTATE_PASSED_COUNT = "checkMutatePassedCount";
String CHECK_MUTATE_PASSED_COUNT_DESC =
"Number of Check and Mutate calls that passed the checks.";
static final String STOREFILE_INDEX_SIZE = "storeFileIndexSize";
static final String STOREFILE_INDEX_SIZE_DESC = "Size of indexes in storefiles on disk.";
static final String STATIC_INDEX_SIZE = "staticIndexSize";
static final String STATIC_INDEX_SIZE_DESC = "Uncompressed size of the static indexes.";
static final String STATIC_BLOOM_SIZE = "staticBloomSize";
static final String STATIC_BLOOM_SIZE_DESC =
String STOREFILE_INDEX_SIZE = "storeFileIndexSize";
String STOREFILE_INDEX_SIZE_DESC = "Size of indexes in storefiles on disk.";
String STATIC_INDEX_SIZE = "staticIndexSize";
String STATIC_INDEX_SIZE_DESC = "Uncompressed size of the static indexes.";
String STATIC_BLOOM_SIZE = "staticBloomSize";
String STATIC_BLOOM_SIZE_DESC =
"Uncompressed size of the static bloom filters.";
static final String NUMBER_OF_MUTATIONS_WITHOUT_WAL = "mutationsWithoutWALCount";
static final String NUMBER_OF_MUTATIONS_WITHOUT_WAL_DESC =
String NUMBER_OF_MUTATIONS_WITHOUT_WAL = "mutationsWithoutWALCount";
String NUMBER_OF_MUTATIONS_WITHOUT_WAL_DESC =
"Number of mutations that have been sent by clients with the write ahead logging turned off.";
static final String DATA_SIZE_WITHOUT_WAL = "mutationsWithoutWALSize";
static final String DATA_SIZE_WITHOUT_WAL_DESC =
String DATA_SIZE_WITHOUT_WAL = "mutationsWithoutWALSize";
String DATA_SIZE_WITHOUT_WAL_DESC =
"Size of data that has been sent by clients with the write ahead logging turned off.";
static final String PERCENT_FILES_LOCAL = "percentFilesLocal";
static final String PERCENT_FILES_LOCAL_DESC =
String PERCENT_FILES_LOCAL = "percentFilesLocal";
String PERCENT_FILES_LOCAL_DESC =
"The percent of HFiles that are stored on the local hdfs data node.";
static final String COMPACTION_QUEUE_LENGTH = "compactionQueueLength";
static final String COMPACTION_QUEUE_LENGTH_DESC = "Length of the queue for compactions.";
static final String FLUSH_QUEUE_LENGTH = "flushQueueLength";
static final String FLUSH_QUEUE_LENGTH_DESC = "Length of the queue for region flushes";
static final String BLOCK_CACHE_FREE_SIZE = "blockCacheFreeSize";
static final String BLOCK_CACHE_FREE_DESC =
String COMPACTION_QUEUE_LENGTH = "compactionQueueLength";
String COMPACTION_QUEUE_LENGTH_DESC = "Length of the queue for compactions.";
String FLUSH_QUEUE_LENGTH = "flushQueueLength";
String FLUSH_QUEUE_LENGTH_DESC = "Length of the queue for region flushes";
String BLOCK_CACHE_FREE_SIZE = "blockCacheFreeSize";
String BLOCK_CACHE_FREE_DESC =
"Size of the block cache that is not occupied.";
static final String BLOCK_CACHE_COUNT = "blockCacheCount";
static final String BLOCK_CACHE_COUNT_DESC = "Number of block in the block cache.";
static final String BLOCK_CACHE_SIZE = "blockCacheSize";
static final String BLOCK_CACHE_SIZE_DESC = "Size of the block cache.";
static final String BLOCK_CACHE_HIT_COUNT = "blockCacheHitCount";
static final String BLOCK_CACHE_HIT_COUNT_DESC = "Count of the hit on the block cache.";
static final String BLOCK_CACHE_MISS_COUNT = "blockCacheMissCount";
static final String BLOCK_COUNT_MISS_COUNT_DESC =
String BLOCK_CACHE_COUNT = "blockCacheCount";
String BLOCK_CACHE_COUNT_DESC = "Number of block in the block cache.";
String BLOCK_CACHE_SIZE = "blockCacheSize";
String BLOCK_CACHE_SIZE_DESC = "Size of the block cache.";
String BLOCK_CACHE_HIT_COUNT = "blockCacheHitCount";
String BLOCK_CACHE_HIT_COUNT_DESC = "Count of the hit on the block cache.";
String BLOCK_CACHE_MISS_COUNT = "blockCacheMissCount";
String BLOCK_COUNT_MISS_COUNT_DESC =
"Number of requests for a block that missed the block cache.";
static final String BLOCK_CACHE_EVICTION_COUNT = "blockCacheEvictionCount";
static final String BLOCK_CACHE_EVICTION_COUNT_DESC =
String BLOCK_CACHE_EVICTION_COUNT = "blockCacheEvictionCount";
String BLOCK_CACHE_EVICTION_COUNT_DESC =
"Count of the number of blocks evicted from the block cache.";
static final String BLOCK_CACHE_HIT_PERCENT = "blockCountHitPercent";
static final String BLOCK_CACHE_HIT_PERCENT_DESC =
String BLOCK_CACHE_HIT_PERCENT = "blockCountHitPercent";
String BLOCK_CACHE_HIT_PERCENT_DESC =
"Percent of block cache requests that are hits";
static final String BLOCK_CACHE_EXPRESS_HIT_PERCENT = "blockCacheExpressHitPercent";
static final String BLOCK_CACHE_EXPRESS_HIT_PERCENT_DESC =
String BLOCK_CACHE_EXPRESS_HIT_PERCENT = "blockCacheExpressHitPercent";
String BLOCK_CACHE_EXPRESS_HIT_PERCENT_DESC =
"The percent of the time that requests with the cache turned on hit the cache.";
static final String RS_START_TIME_NAME = "regionServerStartTime";
static final String ZOOKEEPER_QUORUM_NAME = "zookeeperQuorum";
static final String SERVER_NAME_NAME = "serverName";
static final String CLUSTER_ID_NAME = "clusterId";
static final String RS_START_TIME_DESC = "RegionServer Start Time";
static final String ZOOKEEPER_QUORUM_DESC = "Zookeeper Quorum";
static final String SERVER_NAME_DESC = "Server Name";
static final String CLUSTER_ID_DESC = "Cluster Id";
static final String UPDATES_BLOCKED_TIME = "updatesBlockedTime";
static final String UPDATES_BLOCKED_DESC =
String RS_START_TIME_NAME = "regionServerStartTime";
String ZOOKEEPER_QUORUM_NAME = "zookeeperQuorum";
String SERVER_NAME_NAME = "serverName";
String CLUSTER_ID_NAME = "clusterId";
String RS_START_TIME_DESC = "RegionServer Start Time";
String ZOOKEEPER_QUORUM_DESC = "Zookeeper Quorum";
String SERVER_NAME_DESC = "Server Name";
String CLUSTER_ID_DESC = "Cluster Id";
String UPDATES_BLOCKED_TIME = "updatesBlockedTime";
String UPDATES_BLOCKED_DESC =
"Number of MS updates have been blocked so that the memstore can be flushed.";
static final String DELETE_KEY = "delete";
static final String GET_KEY = "get";
static final String INCREMENT_KEY = "increment";
static final String MUTATE_KEY = "mutate";
static final String APPEND_KEY = "append";
static final String REPLAY_KEY = "replay";
static final String SCAN_NEXT_KEY = "scanNext";
static final String SLOW_MUTATE_KEY = "slowPutCount";
static final String SLOW_GET_KEY = "slowGetCount";
static final String SLOW_DELETE_KEY = "slowDeleteCount";
static final String SLOW_INCREMENT_KEY = "slowIncrementCount";
static final String SLOW_APPEND_KEY = "slowAppendCount";
static final String SLOW_MUTATE_DESC =
String DELETE_KEY = "delete";
String GET_KEY = "get";
String INCREMENT_KEY = "increment";
String MUTATE_KEY = "mutate";
String APPEND_KEY = "append";
String REPLAY_KEY = "replay";
String SCAN_NEXT_KEY = "scanNext";
String SLOW_MUTATE_KEY = "slowPutCount";
String SLOW_GET_KEY = "slowGetCount";
String SLOW_DELETE_KEY = "slowDeleteCount";
String SLOW_INCREMENT_KEY = "slowIncrementCount";
String SLOW_APPEND_KEY = "slowAppendCount";
String SLOW_MUTATE_DESC =
"The number of Multis that took over 1000ms to complete";
static final String SLOW_DELETE_DESC =
String SLOW_DELETE_DESC =
"The number of Deletes that took over 1000ms to complete";
static final String SLOW_GET_DESC = "The number of Gets that took over 1000ms to complete";
static final String SLOW_INCREMENT_DESC =
String SLOW_GET_DESC = "The number of Gets that took over 1000ms to complete";
String SLOW_INCREMENT_DESC =
"The number of Increments that took over 1000ms to complete";
static final String SLOW_APPEND_DESC =
String SLOW_APPEND_DESC =
"The number of Appends that took over 1000ms to complete";

View File

@ -27,35 +27,35 @@ public interface MetricsRegionServerWrapper {
/**
* Get ServerName
*/
public String getServerName();
String getServerName();
/**
* Get the Cluster ID
*
* @return Cluster ID
*/
public String getClusterId();
String getClusterId();
/**
* Get the Zookeeper Quorum Info
*
* @return Zookeeper Quorum Info
*/
public String getZookeeperQuorum();
String getZookeeperQuorum();
/**
* Get the co-processors
*
* @return Co-processors
*/
public String getCoprocessors();
String getCoprocessors();
/**
* Get HRegionServer start time
*
* @return Start time of RegionServer in milliseconds
*/
public long getStartCode();
long getStartCode();
/**
* The number of online regions

View File

@ -25,8 +25,8 @@ package org.apache.hadoop.hbase.regionserver;
*/
public interface MetricsRegionSource extends Comparable<MetricsRegionSource> {
public static final String OPS_SAMPLE_NAME = "ops";
public static final String SIZE_VALUE_NAME = "size";
String OPS_SAMPLE_NAME = "ops";
String SIZE_VALUE_NAME = "size";
/**
* Close the region's metrics as this region is closing.

View File

@ -29,30 +29,30 @@ public interface MetricsEditsReplaySource extends BaseSource {
/**
* The name of the metrics
*/
static final String METRICS_NAME = "replay";
String METRICS_NAME = "replay";
/**
* The name of the metrics context that metrics will be under.
*/
static final String METRICS_CONTEXT = "regionserver";
String METRICS_CONTEXT = "regionserver";
/**
* Description
*/
static final String METRICS_DESCRIPTION = "Metrics about HBase RegionServer HLog Edits Replay";
String METRICS_DESCRIPTION = "Metrics about HBase RegionServer HLog Edits Replay";
/**
* The name of the metrics context that metrics will be under in jmx
*/
static final String METRICS_JMX_CONTEXT = "RegionServer,sub=" + METRICS_NAME;
String METRICS_JMX_CONTEXT = "RegionServer,sub=" + METRICS_NAME;
static final String REPLAY_TIME_NAME = "replayTime";
static final String REPLAY_TIME_DESC = "Time an replay operation took.";
static final String REPLAY_BATCH_SIZE_NAME = "replayBatchSize";
static final String REPLAY_BATCH_SIZE_DESC = "Number of changes in each replay batch.";
static final String REPLAY_DATA_SIZE_NAME = "replayDataSize";
static final String REPLAY_DATA_SIZE_DESC = "Size (in bytes) of the data of each replay.";
String REPLAY_TIME_NAME = "replayTime";
String REPLAY_TIME_DESC = "Time an replay operation took.";
String REPLAY_BATCH_SIZE_NAME = "replayBatchSize";
String REPLAY_BATCH_SIZE_DESC = "Number of changes in each replay batch.";
String REPLAY_DATA_SIZE_NAME = "replayDataSize";
String REPLAY_DATA_SIZE_DESC = "Size (in bytes) of the data of each replay.";
/**
* Add the time a replay command took

View File

@ -29,34 +29,34 @@ public interface MetricsWALSource extends BaseSource {
/**
* The name of the metrics
*/
static final String METRICS_NAME = "WAL";
String METRICS_NAME = "WAL";
/**
* The name of the metrics context that metrics will be under.
*/
static final String METRICS_CONTEXT = "regionserver";
String METRICS_CONTEXT = "regionserver";
/**
* Description
*/
static final String METRICS_DESCRIPTION = "Metrics about HBase RegionServer HLog";
String METRICS_DESCRIPTION = "Metrics about HBase RegionServer HLog";
/**
* The name of the metrics context that metrics will be under in jmx
*/
static final String METRICS_JMX_CONTEXT = "RegionServer,sub=" + METRICS_NAME;
String METRICS_JMX_CONTEXT = "RegionServer,sub=" + METRICS_NAME;
static final String APPEND_TIME = "appendTime";
static final String APPEND_TIME_DESC = "Time an append to the log took.";
static final String APPEND_COUNT = "appendCount";
static final String APPEND_COUNT_DESC = "Number of appends to the write ahead log.";
static final String APPEND_SIZE = "appendSize";
static final String APPEND_SIZE_DESC = "Size (in bytes) of the data appended to the HLog.";
static final String SLOW_APPEND_COUNT = "slowAppendCount";
static final String SLOW_APPEND_COUNT_DESC = "Number of appends that were slow.";
static final String SYNC_TIME = "syncTime";
static final String SYNC_TIME_DESC = "The time it took to sync the HLog to HDFS.";
String APPEND_TIME = "appendTime";
String APPEND_TIME_DESC = "Time an append to the log took.";
String APPEND_COUNT = "appendCount";
String APPEND_COUNT_DESC = "Number of appends to the write ahead log.";
String APPEND_SIZE = "appendSize";
String APPEND_SIZE_DESC = "Size (in bytes) of the data appended to the HLog.";
String SLOW_APPEND_COUNT = "slowAppendCount";
String SLOW_APPEND_COUNT_DESC = "Number of appends that were slow.";
String SYNC_TIME = "syncTime";
String SYNC_TIME_DESC = "The time it took to sync the HLog to HDFS.";
/**
* Add the append size.

View File

@ -28,21 +28,21 @@ public interface MetricsReplicationSource extends BaseSource {
/**
* The name of the metrics
*/
static final String METRICS_NAME = "Replication";
String METRICS_NAME = "Replication";
/**
* The name of the metrics context that metrics will be under.
*/
static final String METRICS_CONTEXT = "regionserver";
String METRICS_CONTEXT = "regionserver";
/**
* The name of the metrics context that metrics will be under.
*/
static final String METRICS_JMX_CONTEXT = "RegionServer,sub=" + METRICS_NAME;
String METRICS_JMX_CONTEXT = "RegionServer,sub=" + METRICS_NAME;
/**
* A description.
*/
static final String METRICS_DESCRIPTION = "Metrics about HBase replication";
String METRICS_DESCRIPTION = "Metrics about HBase replication";
}

View File

@ -25,27 +25,27 @@ import org.apache.hadoop.hbase.metrics.BaseSource;
*/
public interface MetricsRESTSource extends BaseSource {
public static String METRICS_NAME = "REST";
String METRICS_NAME = "REST";
public static String CONTEXT = "rest";
String CONTEXT = "rest";
public static String JMX_CONTEXT = "REST";
String JMX_CONTEXT = "REST";
public static String METRICS_DESCRIPTION = "Metrics about the HBase REST server";
String METRICS_DESCRIPTION = "Metrics about the HBase REST server";
static String REQUEST_KEY = "requests";
String REQUEST_KEY = "requests";
static String SUCCESSFUL_GET_KEY = "successfulGet";
String SUCCESSFUL_GET_KEY = "successfulGet";
static String SUCCESSFUL_PUT_KEY = "successfulPut";
String SUCCESSFUL_PUT_KEY = "successfulPut";
static String SUCCESSFUL_DELETE_KEY = "successfulDelete";
String SUCCESSFUL_DELETE_KEY = "successfulDelete";
static String FAILED_GET_KEY = "failedGet";
String FAILED_GET_KEY = "failedGet";
static String FAILED_PUT_KEY = "failedPut";
String FAILED_PUT_KEY = "failedPut";
static String FAILED_DELETE_KEY = "failedDelete";
String FAILED_DELETE_KEY = "failedDelete";
/**
* Increment the number of requests

View File

@ -25,12 +25,12 @@ import org.apache.hadoop.hbase.metrics.BaseSource;
*/
public interface MetricsThriftServerSource extends BaseSource {
static final String BATCH_GET_KEY = "batchGet";
static final String BATCH_MUTATE_KEY = "batchMutate";
static final String TIME_IN_QUEUE_KEY = "timeInQueue";
static final String THRIFT_CALL_KEY = "thriftCall";
static final String SLOW_THRIFT_CALL_KEY = "slowThriftCall";
static final String CALL_QUEUE_LEN_KEY = "callQueueLen";
String BATCH_GET_KEY = "batchGet";
String BATCH_MUTATE_KEY = "batchMutate";
String TIME_IN_QUEUE_KEY = "timeInQueue";
String THRIFT_CALL_KEY = "thriftCall";
String SLOW_THRIFT_CALL_KEY = "slowThriftCall";
String CALL_QUEUE_LEN_KEY = "callQueueLen";
/**
* Add how long an operation was in the queue.

View File

@ -21,12 +21,12 @@ package org.apache.hadoop.hbase.thrift;
/** Factory that will be used to create metrics sources for the two diffent types of thrift servers. */
public interface MetricsThriftServerSourceFactory {
static final String METRICS_NAME = "Thrift";
static final String METRICS_DESCRIPTION = "Thrift Server Metrics";
static final String THRIFT_ONE_METRICS_CONTEXT = "thrift-one";
static final String THRIFT_ONE_JMX_CONTEXT = "Thrift,sub=ThriftOne";
static final String THRIFT_TWO_METRICS_CONTEXT = "thrift-two";
static final String THRIFT_TWO_JMX_CONTEXT = "Thrift,sub=ThriftTwo";
String METRICS_NAME = "Thrift";
String METRICS_DESCRIPTION = "Thrift Server Metrics";
String THRIFT_ONE_METRICS_CONTEXT = "thrift-one";
String THRIFT_ONE_JMX_CONTEXT = "Thrift,sub=ThriftOne";
String THRIFT_TWO_METRICS_CONTEXT = "thrift-two";
String THRIFT_TWO_JMX_CONTEXT = "Thrift,sub=ThriftTwo";
/** Create a Source for a thrift one server */
MetricsThriftServerSource createThriftOneSource();

View File

@ -25,14 +25,14 @@ package org.apache.hadoop.metrics2;
public interface MetricHistogram {
//Strings used to create metrics names.
static final String NUM_OPS_METRIC_NAME = "_num_ops";
static final String MIN_METRIC_NAME = "_min";
static final String MAX_METRIC_NAME = "_max";
static final String MEAN_METRIC_NAME = "_mean";
static final String MEDIAN_METRIC_NAME = "_median";
static final String SEVENTY_FIFTH_PERCENTILE_METRIC_NAME = "_75th_percentile";
static final String NINETY_FIFTH_PERCENTILE_METRIC_NAME = "_95th_percentile";
static final String NINETY_NINETH_PERCENTILE_METRIC_NAME = "_99th_percentile";
String NUM_OPS_METRIC_NAME = "_num_ops";
String MIN_METRIC_NAME = "_min";
String MAX_METRIC_NAME = "_max";
String MEAN_METRIC_NAME = "_mean";
String MEDIAN_METRIC_NAME = "_median";
String SEVENTY_FIFTH_PERCENTILE_METRIC_NAME = "_75th_percentile";
String NINETY_FIFTH_PERCENTILE_METRIC_NAME = "_95th_percentile";
String NINETY_NINETH_PERCENTILE_METRIC_NAME = "_99th_percentile";
/**
* Add a single value to a histogram's stream of values.

View File

@ -32,6 +32,6 @@ public interface HadoopShims {
* TaskAttemptId.forName()
* @return a concrete TaskAttemptContext instance of o.a.h.mapreduce.TaskAttemptContext
*/
public <T,J> T createTestTaskAttemptContext(final J job, final String taskId);
<T,J> T createTestTaskAttemptContext(final J job, final String taskId);
}

View File

@ -27,7 +27,7 @@ public interface MetricsAssertHelper {
* Init helper. This method will make sure that the metrics system is set
* up for tests.
*/
public void init();
void init();
/**
* Assert that a tag exists and has a given value.
@ -37,7 +37,7 @@ public interface MetricsAssertHelper {
* @param source The BaseSource{@link BaseSource} that will provide the tags,
* gauges, and counters.
*/
public void assertTag(String name, String expected, BaseSource source);
void assertTag(String name, String expected, BaseSource source);
/**
* Assert that a gauge exists and that it's value is equal to the expected value.
@ -47,7 +47,7 @@ public interface MetricsAssertHelper {
* @param source The BaseSource{@link BaseSource} that will provide the tags,
* gauges, and counters.
*/
public void assertGauge(String name, long expected, BaseSource source);
void assertGauge(String name, long expected, BaseSource source);
/**
* Assert that a gauge exists and it's value is greater than a given value
@ -57,7 +57,7 @@ public interface MetricsAssertHelper {
* @param source The BaseSource{@link BaseSource} that will provide the tags,
* gauges, and counters.
*/
public void assertGaugeGt(String name, long expected, BaseSource source);
void assertGaugeGt(String name, long expected, BaseSource source);
/**
* Assert that a gauge exists and it's value is less than a given value
@ -67,7 +67,7 @@ public interface MetricsAssertHelper {
* @param source The BaseSource{@link BaseSource} that will provide the tags,
* gauges, and counters.
*/
public void assertGaugeLt(String name, long expected, BaseSource source);
void assertGaugeLt(String name, long expected, BaseSource source);
/**
* Assert that a gauge exists and that it's value is equal to the expected value.
@ -77,7 +77,7 @@ public interface MetricsAssertHelper {
* @param source The BaseSource{@link BaseSource} that will provide the tags,
* gauges, and counters.
*/
public void assertGauge(String name, double expected, BaseSource source);
void assertGauge(String name, double expected, BaseSource source);
/**
* Assert that a gauge exists and it's value is greater than a given value
@ -87,7 +87,7 @@ public interface MetricsAssertHelper {
* @param source The BaseSource{@link BaseSource} that will provide the tags,
* gauges, and counters.
*/
public void assertGaugeGt(String name, double expected, BaseSource source);
void assertGaugeGt(String name, double expected, BaseSource source);
/**
* Assert that a gauge exists and it's value is less than a given value
@ -97,7 +97,7 @@ public interface MetricsAssertHelper {
* @param source The BaseSource{@link BaseSource} that will provide the tags,
* gauges, and counters.
*/
public void assertGaugeLt(String name, double expected, BaseSource source);
void assertGaugeLt(String name, double expected, BaseSource source);
/**
* Assert that a counter exists and that it's value is equal to the expected value.
@ -107,7 +107,7 @@ public interface MetricsAssertHelper {
* @param source The BaseSource{@link BaseSource} that will provide the tags,
* gauges, and counters.
*/
public void assertCounter(String name, long expected, BaseSource source);
void assertCounter(String name, long expected, BaseSource source);
/**
* Assert that a counter exists and that it's value is greater than the given value.
@ -117,7 +117,7 @@ public interface MetricsAssertHelper {
* @param source The BaseSource{@link BaseSource} that will provide the tags,
* gauges, and counters.
*/
public void assertCounterGt(String name, long expected, BaseSource source);
void assertCounterGt(String name, long expected, BaseSource source);
/**
* Assert that a counter exists and that it's value is less than the given value.
@ -127,7 +127,7 @@ public interface MetricsAssertHelper {
* @param source The BaseSource{@link BaseSource} that will provide the tags,
* gauges, and counters.
*/
public void assertCounterLt(String name, long expected, BaseSource source);
void assertCounterLt(String name, long expected, BaseSource source);
/**
* Get the value of a counter.
@ -137,7 +137,7 @@ public interface MetricsAssertHelper {
* gauges, and counters.
* @return long value of the counter.
*/
public long getCounter(String name, BaseSource source);
long getCounter(String name, BaseSource source);
/**
* Get the value of a gauge as a double.
@ -147,7 +147,7 @@ public interface MetricsAssertHelper {
* gauges, and counters.
* @return double value of the gauge.
*/
public double getGaugeDouble(String name, BaseSource source);
double getGaugeDouble(String name, BaseSource source);
/**
* Get the value of a gauge as a long.
@ -157,5 +157,5 @@ public interface MetricsAssertHelper {
* gauges, and counters.
* @return long value of the gauge.
*/
public long getGaugeLong(String name, BaseSource source);
long getGaugeLong(String name, BaseSource source);
}

View File

@ -31,7 +31,7 @@ public interface TestTokenizerData {
List<byte[]> getInputs();
List<byte[]> getOutputs();
public static class InMemory {
class InMemory {
public Collection<Object[]> getAllAsObjectArray() {
List<Object[]> all = Lists.newArrayList();
all.add(new Object[] { new TestTokenizerDataBasic() });

View File

@ -32,7 +32,7 @@ public interface TestColumnData {
List<ByteRange> getInputs();
List<ByteRange> getOutputs();
public static class InMemory {
class InMemory {
public Collection<Object[]> getAllAsObjectArray() {
List<Object[]> all = Lists.newArrayList();
all.add(new Object[] { new TestColumnDataSimple() });

View File

@ -54,7 +54,7 @@ public interface TestRowData {
void individualSearcherAssertions(CellSearcher searcher);
public static class InMemory {
class InMemory {
/*
* The following are different styles of data that the codec may encounter. Having these small

View File

@ -33,7 +33,7 @@ public interface TestTimestampData {
long getMinimum();
List<Long> getOutputs();
public static class InMemory {
class InMemory {
public Collection<Object[]> getAllAsObjectArray() {
List<Object[]> all = Lists.newArrayList();
all.add(new Object[] { new TestTimestampDataBasic() });

View File

@ -36,7 +36,7 @@ public interface InterProcessLock {
* @throws InterruptedException If current thread is interrupted while
* waiting for the lock
*/
public void acquire() throws IOException, InterruptedException;
void acquire() throws IOException, InterruptedException;
/**
* Acquire the lock within a wait time.
@ -50,7 +50,7 @@ public interface InterProcessLock {
* @throws InterruptedException If the thread is interrupted while waiting to
* acquire the lock
*/
public boolean tryAcquire(long timeoutMs)
boolean tryAcquire(long timeoutMs)
throws IOException, InterruptedException;
/**
@ -59,7 +59,7 @@ public interface InterProcessLock {
* @throws InterruptedException If the thread is interrupted while releasing
* the lock
*/
public void release() throws IOException, InterruptedException;
void release() throws IOException, InterruptedException;
/**
* If supported, attempts to reap all the locks of this type by forcefully
@ -69,7 +69,7 @@ public interface InterProcessLock {
* lock holder is still alive.
* @throws IOException If there is an unrecoverable error reaping the locks
*/
public void reapExpiredLocks(long expireTimeoutMs) throws IOException;
void reapExpiredLocks(long expireTimeoutMs) throws IOException;
/**
* If supported, attempts to reap all the locks of this type by forcefully
@ -80,12 +80,12 @@ public interface InterProcessLock {
* with timeout=0.
* @throws IOException If there is an unrecoverable error reaping the locks
*/
public void reapAllLocks() throws IOException;
void reapAllLocks() throws IOException;
/**
* An interface for objects that process lock metadata.
*/
public static interface MetadataHandler {
interface MetadataHandler {
/**
* Called after lock metadata is successfully read from a distributed
@ -93,7 +93,7 @@ public interface InterProcessLock {
* printing the metadata in a humanly-readable format.
* @param metadata The metadata
*/
public void handleMetadata(byte[] metadata);
void handleMetadata(byte[] metadata);
}
/**
@ -101,5 +101,5 @@ public interface InterProcessLock {
* {@link MetadataHandler}.
* @throws InterruptedException If there is an unrecoverable error
*/
public void visitLocks(MetadataHandler handler) throws IOException;
void visitLocks(MetadataHandler handler) throws IOException;
}

View File

@ -34,7 +34,7 @@ public interface InterProcessReadWriteLock {
* which the lock was acquired).
* @return An instantiated InterProcessLock instance
*/
public InterProcessLock readLock(byte[] metadata);
InterProcessLock readLock(byte[] metadata);
/**
* Obtain a write lock containing given metadata.
@ -43,5 +43,5 @@ public interface InterProcessReadWriteLock {
* which the lock was acquired).
* @return An instantiated InterProcessLock instance
*/
public InterProcessLock writeLock(byte[] metadata);
InterProcessLock writeLock(byte[] metadata);
}

View File

@ -35,7 +35,7 @@ public interface TableDescriptors {
* @return HTableDescriptor for tablename
* @throws IOException
*/
public HTableDescriptor get(final String tablename)
HTableDescriptor get(final String tablename)
throws IOException;
/**
@ -43,7 +43,7 @@ public interface TableDescriptors {
* @return HTableDescriptor for tablename
* @throws IOException
*/
public HTableDescriptor get(final byte[] tablename)
HTableDescriptor get(final byte[] tablename)
throws IOException;
/**
@ -52,7 +52,7 @@ public interface TableDescriptors {
* @return Map of all descriptors.
* @throws IOException
*/
public Map<String, HTableDescriptor> getAll()
Map<String, HTableDescriptor> getAll()
throws IOException;
/**
@ -60,7 +60,7 @@ public interface TableDescriptors {
* @param htd Descriptor to set into TableDescriptors
* @throws IOException
*/
public void add(final HTableDescriptor htd)
void add(final HTableDescriptor htd)
throws IOException;
/**
@ -68,6 +68,6 @@ public interface TableDescriptors {
* @return Instance of table descriptor or null if none found.
* @throws IOException
*/
public HTableDescriptor remove(final String tablename)
HTableDescriptor remove(final String tablename)
throws IOException;
}

View File

@ -76,6 +76,6 @@ public interface Constraint extends Configurable {
* @throws org.apache.hadoop.hbase.exceptions.ConstraintException when the {@link Put} does not match the
* constraint.
*/
public void check(Put p) throws ConstraintException;
void check(Put p) throws ConstraintException;
}

View File

@ -29,5 +29,5 @@ import org.apache.hadoop.classification.InterfaceStability;
@InterfaceAudience.Public
@InterfaceStability.Evolving
public interface CoprocessorService {
public Service getService();
Service getService();
}

View File

@ -31,12 +31,12 @@ import org.apache.hadoop.hbase.regionserver.RegionServerServices;
@InterfaceStability.Evolving
public interface RegionCoprocessorEnvironment extends CoprocessorEnvironment {
/** @return the region associated with this coprocessor */
public HRegion getRegion();
HRegion getRegion();
/** @return reference to the region server services */
public RegionServerServices getRegionServerServices();
RegionServerServices getRegionServerServices();
/** @return shared data between all instances of this coprocessor */
public ConcurrentMap<String, Object> getSharedData();
ConcurrentMap<String, Object> getSharedData();
}

View File

@ -28,5 +28,5 @@ import org.apache.hadoop.hbase.regionserver.wal.HLog;
@InterfaceStability.Evolving
public interface WALCoprocessorEnvironment extends CoprocessorEnvironment {
/** @return reference to the region server services */
public HLog getWAL();
HLog getWAL();
}

View File

@ -36,5 +36,5 @@ public interface ForeignExceptionListener {
* Implementers must ensure that this method is thread-safe.
* @param e exception causing the error. Implementations must accept and handle null here.
*/
public void receive(ForeignException e);
}
void receive(ForeignException e);
}

View File

@ -47,7 +47,7 @@ public interface ForeignExceptionSnare {
* @throws ForeignException
* all exceptions from remote sources are procedure exceptions
*/
public void rethrowException() throws ForeignException;
void rethrowException() throws ForeignException;
/**
* Non-exceptional form of {@link #rethrowException()}. Checks to see if any
@ -56,12 +56,12 @@ public interface ForeignExceptionSnare {
*
* @return <tt>true</tt> if there has been an error,<tt>false</tt> otherwise
*/
public boolean hasException();
boolean hasException();
/**
* Get the value of the captured exception.
*
* @return the captured foreign exception or null if no exception captured.
*/
public ForeignException getException();
ForeignException getException();
}

View File

@ -87,12 +87,12 @@ public abstract class EventHandler implements Runnable, Comparable<Runnable> {
* Called before any event is processed
* @param event The event handler whose process method is about to be called.
*/
public void beforeProcess(EventHandler event);
void beforeProcess(EventHandler event);
/**
* Called after any event is processed
* @param event The event handler whose process method is about to be called.
*/
public void afterProcess(EventHandler event);
void afterProcess(EventHandler event);
}
/**

View File

@ -296,7 +296,7 @@ public class HFileSystem extends FilterFileSystem {
/**
* Interface to implement to add a specific reordering logic in hdfs.
*/
static interface ReorderBlocks {
interface ReorderBlocks {
/**
*
* @param conf - the conf to use
@ -304,7 +304,7 @@ public class HFileSystem extends FilterFileSystem {
* @param src - the file name currently read
* @throws IOException - if something went wrong
*/
public void reorderBlocks(Configuration conf, LocatedBlocks lbs, String src) throws IOException;
void reorderBlocks(Configuration conf, LocatedBlocks lbs, String src) throws IOException;
}
/**

View File

@ -34,5 +34,5 @@ public interface WritableWithSize {
*
* @return the size of the writable
*/
public long getWritableSize();
long getWritableSize();
}

View File

@ -36,14 +36,14 @@ public interface BlockCache {
* @param buf The block contents wrapped in a ByteBuffer.
* @param inMemory Whether block should be treated as in-memory
*/
public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf, boolean inMemory);
void cacheBlock(BlockCacheKey cacheKey, Cacheable buf, boolean inMemory);
/**
* Add block to cache (defaults to not in-memory).
* @param cacheKey The block's cache key.
* @param buf The object to cache.
*/
public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf);
void cacheBlock(BlockCacheKey cacheKey, Cacheable buf);
/**
* Fetch block from cache.
@ -54,62 +54,62 @@ public interface BlockCache {
* @return Block or null if block is not in 2 cache.
* @see HFileReaderV2#readBlock(long, long, boolean, boolean, boolean, BlockType)
*/
public Cacheable getBlock(BlockCacheKey cacheKey, boolean caching, boolean repeat);
Cacheable getBlock(BlockCacheKey cacheKey, boolean caching, boolean repeat);
/**
* Evict block from cache.
* @param cacheKey Block to evict
* @return true if block existed and was evicted, false if not
*/
public boolean evictBlock(BlockCacheKey cacheKey);
boolean evictBlock(BlockCacheKey cacheKey);
/**
* Evicts all blocks for the given HFile.
*
* @return the number of blocks evicted
*/
public int evictBlocksByHfileName(String hfileName);
int evictBlocksByHfileName(String hfileName);
/**
* Get the statistics for this block cache.
* @return Stats
*/
public CacheStats getStats();
CacheStats getStats();
/**
* Shutdown the cache.
*/
public void shutdown();
void shutdown();
/**
* Returns the total size of the block cache, in bytes.
* @return size of cache, in bytes
*/
public long size();
long size();
/**
* Returns the free size of the block cache, in bytes.
* @return free space in cache, in bytes
*/
public long getFreeSize();
long getFreeSize();
/**
* Returns the occupied size of the block cache, in bytes.
* @return occupied space in cache, in bytes
*/
public long getCurrentSize();
long getCurrentSize();
/**
* Returns the number of evictions that have occurred.
* @return number of evictions
*/
public long getEvictedCount();
long getEvictedCount();
/**
* Returns the number of blocks currently cached in the block cache.
* @return number of blocks in the cache
*/
public long getBlockCount();
long getBlockCount();
/**
* Performs a BlockCache summary and returns a List of BlockCacheColumnFamilySummary objects.
@ -123,5 +123,5 @@ public interface BlockCache {
* @return List of BlockCacheColumnFamilySummary
* @throws IOException exception
*/
public List<BlockCacheColumnFamilySummary> getBlockCacheColumnFamilySummaries(Configuration conf) throws IOException;
List<BlockCacheColumnFamilySummary> getBlockCacheColumnFamilySummaries(Configuration conf) throws IOException;
}

View File

@ -42,23 +42,23 @@ public interface Cacheable extends HeapSize {
* @return int length in bytes of the serialized form.
*/
public int getSerializedLength();
int getSerializedLength();
/**
* Serializes its data into destination.
*/
public void serialize(ByteBuffer destination);
void serialize(ByteBuffer destination);
/**
* Returns CacheableDeserializer instance which reconstructs original object from ByteBuffer.
*
* @return CacheableDeserialzer instance.
*/
public CacheableDeserializer<Cacheable> getDeserializer();
CacheableDeserializer<Cacheable> getDeserializer();
/**
* @return the block type of this cached HFile block
*/
public BlockType getBlockType();
BlockType getBlockType();
}

View File

@ -33,7 +33,7 @@ public interface CacheableDeserializer<T extends Cacheable> {
*
* @return T the deserialized object.
*/
public T deserialize(ByteBuffer b) throws IOException;
T deserialize(ByteBuffer b) throws IOException;
/**
*
@ -43,12 +43,12 @@ public interface CacheableDeserializer<T extends Cacheable> {
* @return T the deserialized object.
* @throws IOException
*/
public T deserialize(ByteBuffer b, boolean reuse) throws IOException;
T deserialize(ByteBuffer b, boolean reuse) throws IOException;
/**
* Get the identifier of this deserialiser. Identifier is unique for each
* deserializer and generated by {@link CacheableDeserializerIdManager}
* @return identifier number of this cacheable deserializer
*/
public int getDeserialiserIdentifier();
int getDeserialiserIdentifier();
}

View File

@ -49,8 +49,9 @@ public interface HFileDataBlockEncoder {
* generated).
* @return non null block which is coded according to the settings.
*/
public HFileBlock diskToCacheFormat(HFileBlock block,
boolean isCompaction);
HFileBlock diskToCacheFormat(
HFileBlock block, boolean isCompaction
);
/**
* Should be called before an encoded or unencoded data block is written to
@ -60,37 +61,39 @@ public interface HFileDataBlockEncoder {
* @param blockType block type
* @throws IOException
*/
public void beforeWriteToDisk(
ByteBuffer in, boolean includesMemstoreTS,
HFileBlockEncodingContext encodingResult,
BlockType blockType) throws IOException;
void beforeWriteToDisk(
ByteBuffer in,
boolean includesMemstoreTS,
HFileBlockEncodingContext encodingResult,
BlockType blockType
) throws IOException;
/**
* Decides whether we should use a scanner over encoded blocks.
* @param isCompaction whether we are in a compaction.
* @return Whether to use encoded scanner.
*/
public boolean useEncodedScanner(boolean isCompaction);
boolean useEncodedScanner(boolean isCompaction);
/**
* Save metadata in HFile which will be written to disk
* @param writer writer for a given HFile
* @exception IOException on disk problems
*/
public void saveMetadata(HFile.Writer writer)
void saveMetadata(HFile.Writer writer)
throws IOException;
/** @return the on-disk data block encoding */
public DataBlockEncoding getEncodingOnDisk();
DataBlockEncoding getEncodingOnDisk();
/** @return the preferred in-cache data block encoding for normal reads */
public DataBlockEncoding getEncodingInCache();
DataBlockEncoding getEncodingInCache();
/**
* @return the effective in-cache data block encoding, taking into account
* whether we are doing a compaction.
*/
public DataBlockEncoding getEffectiveEncodingInCache(boolean isCompaction);
DataBlockEncoding getEffectiveEncodingInCache(boolean isCompaction);
/**
* Create an encoder specific encoding context object for writing. And the
@ -101,8 +104,9 @@ public interface HFileDataBlockEncoder {
* @param headerBytes header bytes
* @return a new {@link HFileBlockEncodingContext} object
*/
public HFileBlockEncodingContext newOnDiskDataBlockEncodingContext(
Algorithm compressionAlgorithm, byte[] headerBytes);
HFileBlockEncodingContext newOnDiskDataBlockEncodingContext(
Algorithm compressionAlgorithm, byte[] headerBytes
);
/**
* create a encoder specific decoding context for reading. And the
@ -112,7 +116,8 @@ public interface HFileDataBlockEncoder {
* @param compressionAlgorithm
* @return a new {@link HFileBlockDecodingContext} object
*/
public HFileBlockDecodingContext newOnDiskDataBlockDecodingContext(
Algorithm compressionAlgorithm);
HFileBlockDecodingContext newOnDiskDataBlockDecodingContext(
Algorithm compressionAlgorithm
);
}

View File

@ -54,8 +54,8 @@ public interface HFileScanner {
* false when it is called.
* @throws IOException
*/
public int seekTo(byte[] key) throws IOException;
public int seekTo(byte[] key, int offset, int length) throws IOException;
int seekTo(byte[] key) throws IOException;
int seekTo(byte[] key, int offset, int length) throws IOException;
/**
* Reseek to or just before the passed <code>key</code>. Similar to seekTo
* except that this can be called even if the scanner is not at the beginning
@ -76,8 +76,8 @@ public interface HFileScanner {
* 1, such that k[i] < key, and scanner is left in position i.
* @throws IOException
*/
public int reseekTo(byte[] key) throws IOException;
public int reseekTo(byte[] key, int offset, int length) throws IOException;
int reseekTo(byte[] key) throws IOException;
int reseekTo(byte[] key, int offset, int length) throws IOException;
/**
* Consider the key stream of all the keys in the file,
* <code>k[0] .. k[n]</code>, where there are n keys in the file.
@ -88,28 +88,28 @@ public interface HFileScanner {
* return false (EOF).
* @throws IOException
*/
public boolean seekBefore(byte [] key) throws IOException;
public boolean seekBefore(byte []key, int offset, int length) throws IOException;
boolean seekBefore(byte[] key) throws IOException;
boolean seekBefore(byte[] key, int offset, int length) throws IOException;
/**
* Positions this scanner at the start of the file.
* @return False if empty file; i.e. a call to next would return false and
* the current key and value are undefined.
* @throws IOException
*/
public boolean seekTo() throws IOException;
boolean seekTo() throws IOException;
/**
* Scans to the next entry in the file.
* @return Returns false if you are at the end otherwise true if more in file.
* @throws IOException
*/
public boolean next() throws IOException;
boolean next() throws IOException;
/**
* Gets a buffer view to the current key. You must call
* {@link #seekTo(byte[])} before this method.
* @return byte buffer for the key. The limit is set to the key size, and the
* position is 0, the start of the buffer view.
*/
public ByteBuffer getKey();
ByteBuffer getKey();
/**
* Gets a buffer view to the current value. You must call
* {@link #seekTo(byte[])} before this method.
@ -117,31 +117,31 @@ public interface HFileScanner {
* @return byte buffer for the value. The limit is set to the value size, and
* the position is 0, the start of the buffer view.
*/
public ByteBuffer getValue();
ByteBuffer getValue();
/**
* @return Instance of {@link KeyValue}.
*/
public KeyValue getKeyValue();
KeyValue getKeyValue();
/**
* Convenience method to get a copy of the key as a string - interpreting the
* bytes as UTF8. You must call {@link #seekTo(byte[])} before this method.
* @return key as a string
*/
public String getKeyString();
String getKeyString();
/**
* Convenience method to get a copy of the value as a string - interpreting
* the bytes as UTF8. You must call {@link #seekTo(byte[])} before this method.
* @return value as a string
*/
public String getValueString();
String getValueString();
/**
* @return Reader that underlies this Scanner instance.
*/
public HFile.Reader getReader();
HFile.Reader getReader();
/**
* @return True is scanner has had one of the seek calls invoked; i.e.
* {@link #seekBefore(byte[])} or {@link #seekTo()} or {@link #seekTo(byte[])}.
* Otherwise returns false.
*/
public boolean isSeeked();
boolean isSeeked();
}

View File

@ -35,17 +35,17 @@ public interface Delayable {
* should be set when ending the delay or right away. There are cases when
* the return value can be set right away, even if the call is delayed.
*/
public void startDelay(boolean delayReturnValue);
void startDelay(boolean delayReturnValue);
/**
* @return is the call delayed?
*/
public boolean isDelayed();
boolean isDelayed();
/**
* @return is the return value delayed?
*/
public boolean isReturnValueDelayed();
boolean isReturnValueDelayed();
/**
* Signal that the RPC server is now allowed to send the response.
@ -54,14 +54,14 @@ public interface Delayable {
* not be delayed, this parameter must be null.
* @throws IOException
*/
public void endDelay(Object result) throws IOException;
void endDelay(Object result) throws IOException;
/**
* Signal the end of a delayed RPC, without specifying the return value. Use
* this only if the return value was not delayed
* @throws IOException
*/
public void endDelay() throws IOException;
void endDelay() throws IOException;
/**
* End the call, throwing and exception to the caller. This works regardless
@ -69,5 +69,5 @@ public interface Delayable {
* @param t Object to throw to the client.
* @throws IOException
*/
public void endDelayThrowing(Throwable t) throws IOException;
}
void endDelayThrowing(Throwable t) throws IOException;
}

View File

@ -31,5 +31,5 @@ public interface HBaseRPCErrorHandler {
* @param e the throwable
* @return if the server should be shut down
*/
public boolean checkOOME(final Throwable e) ;
boolean checkOOME(final Throwable e) ;
}

View File

@ -65,7 +65,7 @@ public interface RpcServerInterface {
*/
MetricsHBaseServer getMetrics();
public void setQosFunction(Function<Pair<RequestHeader, Message>, Integer> newFunc);
void setQosFunction(Function<Pair<RequestHeader, Message>, Integer> newFunc);
/**
* Refresh autentication manager policy.
@ -73,4 +73,4 @@ public interface RpcServerInterface {
*/
@VisibleForTesting
void refreshAuthManager(PolicyProvider pp);
}
}

View File

@ -1145,11 +1145,11 @@ public class AssignmentManager extends ZooKeeperListener {
/**
* A specific runnable that works only on a region.
*/
private static interface RegionRunnable extends Runnable{
private interface RegionRunnable extends Runnable{
/**
* @return - the name of the region it works on.
*/
public String getRegionName();
String getRegionName();
}
/**

View File

@ -220,14 +220,14 @@ public class ClusterStatusPublisher extends Chore {
}
public static interface Publisher extends Closeable {
public interface Publisher extends Closeable {
public void connect(Configuration conf) throws IOException;
void connect(Configuration conf) throws IOException;
public void publish(ClusterStatus cs);
void publish(ClusterStatus cs);
@Override
public void close();
void close();
}
public static class MulticastPublisher implements Publisher {

View File

@ -50,21 +50,21 @@ public interface LoadBalancer extends Configurable {
* Set the current cluster status. This allows a LoadBalancer to map host name to a server
* @param st
*/
public void setClusterStatus(ClusterStatus st);
void setClusterStatus(ClusterStatus st);
/**
* Set the master service.
* @param masterServices
*/
public void setMasterServices(MasterServices masterServices);
void setMasterServices(MasterServices masterServices);
/**
* Perform the major balance operation
* @param clusterState
* @return List of plans
*/
public List<RegionPlan> balanceCluster(Map<ServerName, List<HRegionInfo>> clusterState);
List<RegionPlan> balanceCluster(Map<ServerName, List<HRegionInfo>> clusterState);
/**
* Perform a Round Robin assignment of regions.
@ -72,7 +72,10 @@ public interface LoadBalancer extends Configurable {
* @param servers
* @return Map of servername to regioninfos
*/
public Map<ServerName, List<HRegionInfo>> roundRobinAssignment(List<HRegionInfo> regions, List<ServerName> servers);
Map<ServerName, List<HRegionInfo>> roundRobinAssignment(
List<HRegionInfo> regions,
List<ServerName> servers
);
/**
* Assign regions to the previously hosting region server
@ -80,7 +83,10 @@ public interface LoadBalancer extends Configurable {
* @param servers
* @return List of plans
*/
public Map<ServerName, List<HRegionInfo>> retainAssignment(Map<HRegionInfo, ServerName> regions, List<ServerName> servers);
Map<ServerName, List<HRegionInfo>> retainAssignment(
Map<HRegionInfo, ServerName> regions,
List<ServerName> servers
);
/**
* Sync assign a region
@ -88,7 +94,10 @@ public interface LoadBalancer extends Configurable {
* @param servers
* @return Map regioninfos to servernames
*/
public Map<HRegionInfo, ServerName> immediateAssignment(List<HRegionInfo> regions, List<ServerName> servers);
Map<HRegionInfo, ServerName> immediateAssignment(
List<HRegionInfo> regions,
List<ServerName> servers
);
/**
* Get a random region server from the list
@ -96,6 +105,7 @@ public interface LoadBalancer extends Configurable {
* @param servers
* @return Servername
*/
public ServerName randomAssignment(HRegionInfo regionInfo,
List<ServerName> servers);
ServerName randomAssignment(
HRegionInfo regionInfo, List<ServerName> servers
);
}

View File

@ -40,32 +40,32 @@ public interface MasterServices extends Server {
/**
* @return Master's instance of the {@link AssignmentManager}
*/
public AssignmentManager getAssignmentManager();
AssignmentManager getAssignmentManager();
/**
* @return Master's filesystem {@link MasterFileSystem} utility class.
*/
public MasterFileSystem getMasterFileSystem();
MasterFileSystem getMasterFileSystem();
/**
* @return Master's {@link ServerManager} instance.
*/
public ServerManager getServerManager();
ServerManager getServerManager();
/**
* @return Master's instance of {@link ExecutorService}
*/
public ExecutorService getExecutorService();
ExecutorService getExecutorService();
/**
* @return Master's instance of {@link TableLockManager}
*/
public TableLockManager getTableLockManager();
TableLockManager getTableLockManager();
/**
* @return Master's instance of {@link MasterCoprocessorHost}
*/
public MasterCoprocessorHost getCoprocessorHost();
MasterCoprocessorHost getCoprocessorHost();
/**
* Check table is modifiable; i.e. exists and is offline.
@ -75,7 +75,7 @@ public interface MasterServices extends Server {
* @throws IOException
*/
// We actually throw the exceptions mentioned in the
public void checkTableModifiable(final byte [] tableName)
void checkTableModifiable(final byte[] tableName)
throws IOException, TableNotFoundException, TableNotDisabledException;
/**
@ -84,7 +84,7 @@ public interface MasterServices extends Server {
* @param splitKeys Starting row keys for the initial table regions. If null
* a single region is created.
*/
public void createTable(HTableDescriptor desc, byte [][] splitKeys)
void createTable(HTableDescriptor desc, byte[][] splitKeys)
throws IOException;
/**
@ -92,7 +92,7 @@ public interface MasterServices extends Server {
* @param tableName The table name
* @throws IOException
*/
public void deleteTable(final byte[] tableName) throws IOException;
void deleteTable(final byte[] tableName) throws IOException;
/**
* Modify the descriptor of an existing table
@ -100,7 +100,7 @@ public interface MasterServices extends Server {
* @param descriptor The updated table descriptor
* @throws IOException
*/
public void modifyTable(final byte[] tableName, final HTableDescriptor descriptor)
void modifyTable(final byte[] tableName, final HTableDescriptor descriptor)
throws IOException;
/**
@ -108,14 +108,14 @@ public interface MasterServices extends Server {
* @param tableName The table name
* @throws IOException
*/
public void enableTable(final byte[] tableName) throws IOException;
void enableTable(final byte[] tableName) throws IOException;
/**
* Disable an existing table
* @param tableName The table name
* @throws IOException
*/
public void disableTable(final byte[] tableName) throws IOException;
void disableTable(final byte[] tableName) throws IOException;
/**
* Add a new column to an existing table
@ -123,7 +123,7 @@ public interface MasterServices extends Server {
* @param column The column definition
* @throws IOException
*/
public void addColumn(final byte[] tableName, final HColumnDescriptor column)
void addColumn(final byte[] tableName, final HColumnDescriptor column)
throws IOException;
/**
@ -132,7 +132,7 @@ public interface MasterServices extends Server {
* @param descriptor The updated column definition
* @throws IOException
*/
public void modifyColumn(byte[] tableName, HColumnDescriptor descriptor)
void modifyColumn(byte[] tableName, HColumnDescriptor descriptor)
throws IOException;
/**
@ -141,18 +141,18 @@ public interface MasterServices extends Server {
* @param columnName The column name
* @throws IOException
*/
public void deleteColumn(final byte[] tableName, final byte[] columnName)
void deleteColumn(final byte[] tableName, final byte[] columnName)
throws IOException;
/**
* @return Return table descriptors implementation.
*/
public TableDescriptors getTableDescriptors();
TableDescriptors getTableDescriptors();
/**
* @return true if master enables ServerShutdownHandler;
*/
public boolean isServerShutdownHandlerEnabled();
boolean isServerShutdownHandlerEnabled();
/**
* Registers a new protocol buffer {@link Service} subclass as a master coprocessor endpoint.
@ -167,7 +167,7 @@ public interface MasterServices extends Server {
* @return {@code true} if the registration was successful, {@code false}
* otherwise
*/
public boolean registerService(Service instance);
boolean registerService(Service instance);
/**
* Merge two regions. The real implementation is on the regionserver, master
@ -178,12 +178,13 @@ public interface MasterServices extends Server {
* two adjacent regions
* @throws IOException
*/
public void dispatchMergingRegions(final HRegionInfo region_a,
final HRegionInfo region_b, final boolean forcible) throws IOException;
void dispatchMergingRegions(
final HRegionInfo region_a, final HRegionInfo region_b, final boolean forcible
) throws IOException;
/**
* @return true if master is initialized
*/
public boolean isInitialized();
boolean isInitialized();
}

View File

@ -34,30 +34,30 @@ public interface SnapshotSentinel {
* @return <tt>false</tt> if the snapshot is still in progress, <tt>true</tt> if the snapshot has
* finished
*/
public boolean isFinished();
boolean isFinished();
/**
* @return -1 if the snapshot is in progress, otherwise the completion timestamp.
*/
public long getCompletionTimestamp();
long getCompletionTimestamp();
/**
* Actively cancel a running snapshot.
* @param why Reason for cancellation.
*/
public void cancel(String why);
void cancel(String why);
/**
* @return the description of the snapshot being run
*/
public SnapshotDescription getSnapshot();
SnapshotDescription getSnapshot();
/**
* Get the exception that caused the snapshot to fail, if the snapshot has failed.
* @return {@link ForeignException} that caused the snapshot to fail, or <tt>null</tt> if the
* snapshot is still in progress or has succeeded
*/
public ForeignException getExceptionIfFailed();
ForeignException getExceptionIfFailed();
/**
* Rethrow the exception returned by {@link SnapshotSentinel#getExceptionIfFailed}.
@ -65,5 +65,5 @@ public interface SnapshotSentinel {
*
* @throws ForeignException all exceptions from remote sources are procedure exceptions
*/
public void rethrowExceptionIfFailed() throws ForeignException;
void rethrowExceptionIfFailed() throws ForeignException;
}

View File

@ -1593,11 +1593,11 @@ public class SplitLogManager extends ZooKeeperListener {
* a serialization point at the end of the task processing. Must be
* restartable and idempotent.
*/
static public interface TaskFinisher {
public interface TaskFinisher {
/**
* status that can be returned finish()
*/
static public enum Status {
enum Status {
/**
* task completed successfully
*/
@ -1616,7 +1616,7 @@ public class SplitLogManager extends ZooKeeperListener {
* @param taskname
* @return DONE if task completed successfully, ERR otherwise
*/
public Status finish(ServerName workerName, String taskname);
Status finish(ServerName workerName, String taskname);
}
enum ResubmitDirective {

View File

@ -82,20 +82,20 @@ public abstract class TableLockManager {
* A distributed lock for a table.
*/
@InterfaceAudience.Private
public static interface TableLock {
public interface TableLock {
/**
* Acquire the lock, with the configured lock timeout.
* @throws LockTimeoutException If unable to acquire a lock within a specified
* time period (if any)
* @throws IOException If unrecoverable error occurs
*/
public void acquire() throws IOException;
void acquire() throws IOException;
/**
* Release the lock already held.
* @throws IOException If there is an unrecoverable error releasing the lock
*/
public void release() throws IOException;
void release() throws IOException;
}
/**

View File

@ -35,5 +35,5 @@ public interface FileCleanerDelegate extends Configurable, Stoppable {
* @param fStat file status of the file to check
* @return <tt>true</tt> if the file is deletable, <tt>false</tt> if not
*/
public boolean isFileDeletable(FileStatus fStat);
boolean isFileDeletable(FileStatus fStat);
}

View File

@ -33,5 +33,5 @@ public interface TotesHRegionInfo {
/**
* @return HRegionInfo instance.
*/
public HRegionInfo getHRegionInfo();
HRegionInfo getHRegionInfo();
}

View File

@ -29,16 +29,16 @@ import com.google.protobuf.Message;
*/
@InterfaceAudience.Private
public interface MonitoredRPCHandler extends MonitoredTask {
public abstract String getRPC();
public abstract String getRPC(boolean withParams);
public abstract long getRPCPacketLength();
public abstract String getClient();
public abstract long getRPCStartTime();
public abstract long getRPCQueueTime();
public abstract boolean isRPCRunning();
public abstract boolean isOperationRunning();
String getRPC();
String getRPC(boolean withParams);
long getRPCPacketLength();
String getClient();
long getRPCStartTime();
long getRPCQueueTime();
boolean isRPCRunning();
boolean isOperationRunning();
public abstract void setRPC(String methodName, Object [] params, long queueTime);
public abstract void setRPCPacket(Message param);
public abstract void setConnection(String clientAddress, int remotePort);
void setRPC(String methodName, Object[] params, long queueTime);
void setRPCPacket(Message param);
void setConnection(String clientAddress, int remotePort);
}

View File

@ -32,47 +32,47 @@ public interface MonitoredTask extends Cloneable {
ABORTED;
}
public abstract long getStartTime();
public abstract String getDescription();
public abstract String getStatus();
public abstract long getStatusTime();
public abstract State getState();
public abstract long getStateTime();
public abstract long getCompletionTimestamp();
long getStartTime();
String getDescription();
String getStatus();
long getStatusTime();
State getState();
long getStateTime();
long getCompletionTimestamp();
public abstract void markComplete(String msg);
public abstract void pause(String msg);
public abstract void resume(String msg);
public abstract void abort(String msg);
public abstract void expireNow();
void markComplete(String msg);
void pause(String msg);
void resume(String msg);
void abort(String msg);
void expireNow();
public abstract void setStatus(String status);
public abstract void setDescription(String description);
void setStatus(String status);
void setDescription(String description);
/**
* Explicitly mark this status as able to be cleaned up,
* even though it might not be complete.
*/
public abstract void cleanup();
void cleanup();
/**
* Public exposure of Object.clone() in order to allow clients to easily
* capture current state.
* @return a copy of the object whose references will not change
*/
public abstract MonitoredTask clone();
MonitoredTask clone();
/**
* Creates a string map of internal details for extensible exposure of
* monitored tasks.
* @return A Map containing information for this task.
*/
public abstract Map<String, Object> toMap() throws IOException;
Map<String, Object> toMap() throws IOException;
/**
* Creates a JSON object for parseable exposure of monitored tasks.
* @return An encoded JSON object containing information for this task.
*/
public abstract String toJSON() throws IOException;
String toJSON() throws IOException;
}

View File

@ -39,7 +39,7 @@ public interface ProcedureCoordinatorRpcs extends Closeable {
* @param listener
* @return true if succeed, false if encountered initialization errors.
*/
public boolean start(final ProcedureCoordinator listener);
boolean start(final ProcedureCoordinator listener);
/**
* Notify the members that the coordinator has aborted the procedure and that it should release
@ -50,7 +50,7 @@ public interface ProcedureCoordinatorRpcs extends Closeable {
* @throws IOException if the rpcs can't reach the other members of the procedure (and can't
* recover).
*/
public void sendAbortToMembers(Procedure procName, ForeignException cause) throws IOException;
void sendAbortToMembers(Procedure procName, ForeignException cause) throws IOException;
/**
* Notify the members to acquire barrier for the procedure
@ -61,7 +61,7 @@ public interface ProcedureCoordinatorRpcs extends Closeable {
* @throws IllegalArgumentException if the procedure was already marked as failed
* @throws IOException if we can't reach the remote notification mechanism
*/
public void sendGlobalBarrierAcquire(Procedure procName, byte[] info, List<String> members)
void sendGlobalBarrierAcquire(Procedure procName, byte[] info, List<String> members)
throws IOException, IllegalArgumentException;
/**
@ -74,12 +74,12 @@ public interface ProcedureCoordinatorRpcs extends Closeable {
* @param members members to tell we have reached in-barrier phase
* @throws IOException if we can't reach the remote notification mechanism
*/
public void sendGlobalBarrierReached(Procedure procName, List<String> members) throws IOException;
void sendGlobalBarrierReached(Procedure procName, List<String> members) throws IOException;
/**
* Notify Members to reset the distributed state for procedure
* @param procName name of the procedure to reset
* @throws IOException if the remote notification mechanism cannot be reached
*/
public void resetMembers(Procedure procName) throws IOException;
void resetMembers(Procedure procName) throws IOException;
}

View File

@ -35,13 +35,13 @@ public interface ProcedureMemberRpcs extends Closeable {
/**
* Initialize and start any threads or connections the member needs.
*/
public void start(final String memberName, final ProcedureMember member);
void start(final String memberName, final ProcedureMember member);
/**
* Each subprocedure is being executed on a member. This is the identifier for the member.
* @return the member name
*/
public String getMemberName();
String getMemberName();
/**
* Notify the coordinator that we aborted the specified {@link Subprocedure}
@ -51,7 +51,7 @@ public interface ProcedureMemberRpcs extends Closeable {
* @throws IOException thrown when the rpcs can't reach the other members of the procedure (and
* thus can't recover).
*/
public void sendMemberAborted(Subprocedure sub, ForeignException cause) throws IOException;
void sendMemberAborted(Subprocedure sub, ForeignException cause) throws IOException;
/**
* Notify the coordinator that the specified {@link Subprocedure} has acquired the locally required
@ -60,7 +60,7 @@ public interface ProcedureMemberRpcs extends Closeable {
* @param sub the specified {@link Subprocedure}
* @throws IOException if we can't reach the coordinator
*/
public void sendMemberAcquired(Subprocedure sub) throws IOException;
void sendMemberAcquired(Subprocedure sub) throws IOException;
/**
* Notify the coordinator that the specified {@link Subprocedure} has completed the work that
@ -69,5 +69,5 @@ public interface ProcedureMemberRpcs extends Closeable {
* @param sub the specified {@link Subprocedure}
* @throws IOException if we can't reach the coordinator
*/
public void sendMemberCompleted(Subprocedure sub) throws IOException;
}
void sendMemberCompleted(Subprocedure sub) throws IOException;
}

View File

@ -36,5 +36,5 @@ public interface SubprocedureFactory {
* request
* @throws IllegalStateException if the current runner cannot accept any more new requests
*/
public Subprocedure buildSubprocedure(String procName, byte[] procArgs);
Subprocedure buildSubprocedure(String procName, byte[] procArgs);
}

View File

@ -56,25 +56,26 @@ public interface ColumnTracker {
* @throws IOException in case there is an internal consistency problem
* caused by a data corruption.
*/
public ScanQueryMatcher.MatchCode checkColumn(byte[] bytes, int offset,
int length, long ttl, byte type, boolean ignoreCount)
ScanQueryMatcher.MatchCode checkColumn(
byte[] bytes, int offset, int length, long ttl, byte type, boolean ignoreCount
)
throws IOException;
/**
* Updates internal variables in between files
*/
public void update();
void update();
/**
* Resets the Matcher
*/
public void reset();
void reset();
/**
*
* @return <code>true</code> when done.
*/
public boolean done();
boolean done();
/**
* Used by matcher and scan/get to get a hint of the next column
@ -87,13 +88,14 @@ public interface ColumnTracker {
*
* @return null, or a ColumnCount that we should seek to
*/
public ColumnCount getColumnHint();
ColumnCount getColumnHint();
/**
* Retrieve the MatchCode for the next row or column
*/
public MatchCode getNextRowOrNextColumn(byte[] bytes, int offset,
int qualLength);
MatchCode getNextRowOrNextColumn(
byte[] bytes, int offset, int qualLength
);
/**
* Give the tracker a chance to declare it's done based on only the timestamp
@ -102,5 +104,5 @@ public interface ColumnTracker {
* @param timestamp
* @return <code>true</code> to early out based on timestamp.
*/
public boolean isDone(long timestamp);
boolean isDone(long timestamp);
}

View File

@ -34,7 +34,7 @@ public interface CompactionRequestor {
* compactions were started
* @throws IOException
*/
public List<CompactionRequest> requestCompaction(final HRegion r, final String why)
List<CompactionRequest> requestCompaction(final HRegion r, final String why)
throws IOException;
/**
@ -47,8 +47,9 @@ public interface CompactionRequestor {
* compactions were started
* @throws IOException
*/
public List<CompactionRequest> requestCompaction(final HRegion r, final String why,
List<Pair<CompactionRequest, Store>> requests)
List<CompactionRequest> requestCompaction(
final HRegion r, final String why, List<Pair<CompactionRequest, Store>> requests
)
throws IOException;
/**
@ -60,8 +61,9 @@ public interface CompactionRequestor {
* @return The created {@link CompactionRequest} or <tt>null</tt> if no compaction was started.
* @throws IOException
*/
public CompactionRequest requestCompaction(final HRegion r, final Store s, final String why,
CompactionRequest request) throws IOException;
CompactionRequest requestCompaction(
final HRegion r, final Store s, final String why, CompactionRequest request
) throws IOException;
/**
* @param r Region to compact
@ -74,8 +76,9 @@ public interface CompactionRequestor {
* compactions were started.
* @throws IOException
*/
public List<CompactionRequest> requestCompaction(final HRegion r, final String why, int pri,
List<Pair<CompactionRequest, Store>> requests) throws IOException;
List<CompactionRequest> requestCompaction(
final HRegion r, final String why, int pri, List<Pair<CompactionRequest, Store>> requests
) throws IOException;
/**
* @param r Region to compact
@ -87,6 +90,7 @@ public interface CompactionRequestor {
* @return The created {@link CompactionRequest} or <tt>null</tt> if no compaction was started
* @throws IOException
*/
public CompactionRequest requestCompaction(final HRegion r, final Store s, final String why,
int pri, CompactionRequest request) throws IOException;
CompactionRequest requestCompaction(
final HRegion r, final Store s, final String why, int pri, CompactionRequest request
) throws IOException;
}

View File

@ -43,8 +43,9 @@ public interface DeleteTracker {
* @param timestamp timestamp
* @param type delete type as byte
*/
public void add(byte [] buffer, int qualifierOffset, int qualifierLength,
long timestamp, byte type);
void add(
byte[] buffer, int qualifierOffset, int qualifierLength, long timestamp, byte type
);
/**
* Check if the specified KeyValue buffer has been deleted by a previously
@ -55,13 +56,14 @@ public interface DeleteTracker {
* @param timestamp timestamp
* @return deleteResult The result tells whether the KeyValue is deleted and why
*/
public DeleteResult isDeleted(byte [] buffer, int qualifierOffset,
int qualifierLength, long timestamp);
DeleteResult isDeleted(
byte[] buffer, int qualifierOffset, int qualifierLength, long timestamp
);
/**
* @return true if there are no current delete, false otherwise
*/
public boolean isEmpty();
boolean isEmpty();
/**
* Called at the end of every StoreFile.
@ -69,14 +71,14 @@ public interface DeleteTracker {
* Many optimized implementations of Trackers will require an update at
* when the end of each StoreFile is reached.
*/
public void update();
void update();
/**
* Called between rows.
* <p>
* This clears everything as if a new DeleteTracker was instantiated.
*/
public void reset();
void reset();
/**
@ -102,7 +104,7 @@ public interface DeleteTracker {
* Based on the delete result, the ScanQueryMatcher will decide the next
* operation
*/
public static enum DeleteResult {
enum DeleteResult {
FAMILY_DELETED, // The KeyValue is deleted by a delete family.
FAMILY_VERSION_DELETED, // The KeyValue is deleted by a delete family version.
COLUMN_DELETED, // The KeyValue is deleted by a delete column.

View File

@ -5630,7 +5630,7 @@ public class HRegion implements HeapSize { // , Writable{
* bulkLoadHFile() to perform any necessary
* pre/post processing of a given bulkload call
*/
public static interface BulkLoadListener {
public interface BulkLoadListener {
/**
* Called before an HFile is actually loaded

View File

@ -47,7 +47,7 @@ public interface InternalScanner extends Closeable {
* @return true if more rows exist after this one, false if scanner is done
* @throws IOException e
*/
public boolean next(List<KeyValue> results) throws IOException;
boolean next(List<KeyValue> results) throws IOException;
/**
* Grab the next row's worth of values with a limit on the number of values
@ -57,11 +57,11 @@ public interface InternalScanner extends Closeable {
* @return true if more rows exist after this one, false if scanner is done
* @throws IOException e
*/
public boolean next(List<KeyValue> result, int limit) throws IOException;
boolean next(List<KeyValue> result, int limit) throws IOException;
/**
* Closes the scanner and releases any resources it has allocated
* @throws IOException
*/
public void close() throws IOException;
void close() throws IOException;
}

View File

@ -34,20 +34,20 @@ public interface KeyValueScanner {
* Look at the next KeyValue in this scanner, but do not iterate scanner.
* @return the next KeyValue
*/
public KeyValue peek();
KeyValue peek();
/**
* Return the next KeyValue in this scanner, iterating the scanner
* @return the next KeyValue
*/
public KeyValue next() throws IOException;
KeyValue next() throws IOException;
/**
* Seek the scanner at or after the specified KeyValue.
* @param key seek value
* @return true if scanner has values left, false if end of scanner
*/
public boolean seek(KeyValue key) throws IOException;
boolean seek(KeyValue key) throws IOException;
/**
* Reseek the scanner at or after the specified KeyValue.
@ -57,7 +57,7 @@ public interface KeyValueScanner {
* @param key seek value (should be non-null)
* @return true if scanner has values left, false if end of scanner
*/
public boolean reseek(KeyValue key) throws IOException;
boolean reseek(KeyValue key) throws IOException;
/**
* Get the sequence id associated with this KeyValueScanner. This is required
@ -65,12 +65,12 @@ public interface KeyValueScanner {
* The default implementation for this would be to return 0. A file having
* lower sequence id will be considered to be the older one.
*/
public long getSequenceID();
long getSequenceID();
/**
* Close the KeyValue scanner.
*/
public void close();
void close();
/**
* Allows to filter out scanners (both StoreFile and memstore) that we don't
@ -82,8 +82,9 @@ public interface KeyValueScanner {
* this query, based on TTL
* @return true if the scanner should be included in the query
*/
public boolean shouldUseScanner(Scan scan, SortedSet<byte[]> columns,
long oldestUnexpiredTS);
boolean shouldUseScanner(
Scan scan, SortedSet<byte[]> columns, long oldestUnexpiredTS
);
// "Lazy scanner" optimizations
@ -97,7 +98,7 @@ public interface KeyValueScanner {
* @param forward do a forward-only "reseek" instead of a random-access seek
* @param useBloom whether to enable multi-column Bloom filter optimization
*/
public boolean requestSeek(KeyValue kv, boolean forward, boolean useBloom)
boolean requestSeek(KeyValue kv, boolean forward, boolean useBloom)
throws IOException;
/**
@ -106,7 +107,7 @@ public interface KeyValueScanner {
* store scanner bubbles up to the top of the key-value heap. This method is
* then used to ensure the top store file scanner has done a seek operation.
*/
public boolean realSeekDone();
boolean realSeekDone();
/**
* Does the real seek operation in case it was skipped by
@ -115,11 +116,11 @@ public interface KeyValueScanner {
* of the scanners). The easiest way to achieve this is to call
* {@link #realSeekDone()} first.
*/
public void enforceSeek() throws IOException;
void enforceSeek() throws IOException;
/**
* @return true if this is a file scanner. Otherwise a memory scanner is
* assumed.
*/
public boolean isFileScanner();
boolean isFileScanner();
}

View File

@ -29,5 +29,5 @@ public interface LastSequenceId {
* @param regionName Encoded region name
* @return Last flushed sequence Id for regionName or -1 if it can't be determined
*/
public long getLastSequenceId(byte[] regionName);
long getLastSequenceId(byte[] regionName);
}

View File

@ -32,5 +32,5 @@ import org.apache.hadoop.classification.InterfaceAudience;
@InterfaceAudience.Private
public interface LeaseListener {
/** When a lease expires, this method is called. */
public void leaseExpired();
void leaseExpired();
}

View File

@ -35,7 +35,7 @@ interface OnlineRegions extends Server {
* Add to online regions.
* @param r
*/
public void addToOnlineRegions(final HRegion r);
void addToOnlineRegions(final HRegion r);
/**
* This method removes HRegion corresponding to hri from the Map of onlineRegions.
@ -44,7 +44,7 @@ interface OnlineRegions extends Server {
* @param destination Destination, if any, null otherwise.
* @return True if we removed a region from online list.
*/
public boolean removeFromOnlineRegions(final HRegion r, ServerName destination);
boolean removeFromOnlineRegions(final HRegion r, ServerName destination);
/**
* Return {@link HRegion} instance.
@ -54,7 +54,7 @@ interface OnlineRegions extends Server {
* @return HRegion for the passed encoded <code>encodedRegionName</code> or
* null if named region is not member of the online regions.
*/
public HRegion getFromOnlineRegions(String encodedRegionName);
HRegion getFromOnlineRegions(String encodedRegionName);
/**
* Get all online regions of a table in this RS.
@ -62,5 +62,5 @@ interface OnlineRegions extends Server {
* @return List of HRegion
* @throws java.io.IOException
*/
public List<HRegion> getOnlineRegions(byte[] tableName) throws IOException;
List<HRegion> getOnlineRegions(byte[] tableName) throws IOException;
}

View File

@ -34,13 +34,13 @@ public interface RegionScanner extends InternalScanner {
/**
* @return The RegionInfo for this scanner.
*/
public HRegionInfo getRegionInfo();
HRegionInfo getRegionInfo();
/**
* @return True if a filter indicates that this scanner will return no further rows.
* @throws IOException in case of I/O failure on a filter.
*/
public boolean isFilterDone() throws IOException;
boolean isFilterDone() throws IOException;
/**
* Do a reseek to the required row. Should not be used to seek to a key which
@ -52,17 +52,17 @@ public interface RegionScanner extends InternalScanner {
* if row is null
*
*/
public boolean reseek(byte[] row) throws IOException;
boolean reseek(byte[] row) throws IOException;
/**
* @return The preferred max buffersize. See {@link Scan#setMaxResultSize(long)}
*/
public long getMaxResultSize();
long getMaxResultSize();
/**
* @return The Scanner's MVCC readPt see {@link MultiVersionConsistencyControl}
*/
public long getMvccReadPoint();
long getMvccReadPoint();
/**
* Grab the next row's worth of values with the default limit on the number of values
@ -74,7 +74,7 @@ public interface RegionScanner extends InternalScanner {
* @return true if more rows exist after this one, false if scanner is done
* @throws IOException e
*/
public boolean nextRaw(List<KeyValue> result) throws IOException;
boolean nextRaw(List<KeyValue> result) throws IOException;
/**
* Grab the next row's worth of values with a limit on the number of values
@ -102,5 +102,5 @@ public interface RegionScanner extends InternalScanner {
* @return true if more rows exist after this one, false if scanner is done
* @throws IOException e
*/
public boolean nextRaw(List<KeyValue> result, int limit) throws IOException;
boolean nextRaw(List<KeyValue> result, int limit) throws IOException;
}

View File

@ -40,31 +40,31 @@ public interface RegionServerServices extends OnlineRegions, FavoredNodesForRegi
/**
* @return True if this regionserver is stopping.
*/
public boolean isStopping();
boolean isStopping();
/** @return the HLog for a particular region. Pass null for getting the
* default (common) WAL */
public HLog getWAL(HRegionInfo regionInfo) throws IOException;
HLog getWAL(HRegionInfo regionInfo) throws IOException;
/**
* @return Implementation of {@link CompactionRequestor} or null.
*/
public CompactionRequestor getCompactionRequester();
CompactionRequestor getCompactionRequester();
/**
* @return Implementation of {@link FlushRequester} or null.
*/
public FlushRequester getFlushRequester();
FlushRequester getFlushRequester();
/**
* @return the RegionServerAccounting for this Region Server
*/
public RegionServerAccounting getRegionServerAccounting();
RegionServerAccounting getRegionServerAccounting();
/**
* @return RegionServer's instance of {@link TableLockManager}
*/
public TableLockManager getTableLockManager();
TableLockManager getTableLockManager();
/**
* Tasks to perform after region open to complete deploy of region on
@ -75,42 +75,42 @@ public interface RegionServerServices extends OnlineRegions, FavoredNodesForRegi
* @throws KeeperException
* @throws IOException
*/
public void postOpenDeployTasks(final HRegion r, final CatalogTracker ct)
void postOpenDeployTasks(final HRegion r, final CatalogTracker ct)
throws KeeperException, IOException;
/**
* Returns a reference to the region server's RPC server
*/
public RpcServerInterface getRpcServer();
RpcServerInterface getRpcServer();
/**
* Get the regions that are currently being opened or closed in the RS
* @return map of regions in transition in this RS
*/
public ConcurrentMap<byte[], Boolean> getRegionsInTransitionInRS();
ConcurrentMap<byte[], Boolean> getRegionsInTransitionInRS();
/**
* @return Return the FileSystem object used by the regionserver
*/
public FileSystem getFileSystem();
FileSystem getFileSystem();
/**
* @return The RegionServer's "Leases" service
*/
public Leases getLeases();
Leases getLeases();
/**
* @return hbase executor service
*/
public ExecutorService getExecutorService();
ExecutorService getExecutorService();
/**
* @return The RegionServer's CatalogTracker
*/
public CatalogTracker getCatalogTracker();
CatalogTracker getCatalogTracker();
/**
* @return set of recovering regions on the hosting region server
*/
public Map<String, HRegion> getRecoveringRegions();
Map<String, HRegion> getRecoveringRegions();
}

View File

@ -38,17 +38,18 @@ public interface ReplicationService {
* Initializes the replication service object.
* @throws IOException
*/
public void initialize(Server rs, FileSystem fs, Path logdir,
Path oldLogDir) throws IOException;
void initialize(
Server rs, FileSystem fs, Path logdir, Path oldLogDir
) throws IOException;
/**
* Start replication services.
* @throws IOException
*/
public void startReplicationService() throws IOException;
void startReplicationService() throws IOException;
/**
* Stops replication service.
*/
public void stopReplicationService();
void stopReplicationService();
}

View File

@ -38,5 +38,5 @@ public interface ReplicationSinkService extends ReplicationService {
* @param cells Cells that the WALEntries refer to (if cells is non-null)
* @throws IOException
*/
public void replicateLogEntries(List<WALEntry> entries, CellScanner cells) throws IOException;
}
void replicateLogEntries(List<WALEntry> entries, CellScanner cells) throws IOException;
}

View File

@ -32,5 +32,5 @@ public interface ReplicationSourceService extends ReplicationService {
* Returns a WALObserver for the service. This is needed to
* observe log rolls and log archival events.
*/
public WALActionsListener getWALActionsListener();
}
WALActionsListener getWALActionsListener();
}

View File

@ -640,13 +640,13 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
* is better to have workers prepare the task and then have the
* {@link SplitLogManager} commit the work in SplitLogManager.TaskFinisher
*/
static public interface TaskExecutor {
static public enum Status {
public interface TaskExecutor {
enum Status {
DONE(),
ERR(),
RESIGNED(),
PREEMPTED()
}
public Status exec(String name, CancelableProgressable p);
Status exec(String name, CancelableProgressable p);
}
}

View File

@ -50,14 +50,13 @@ public interface Store extends HeapSize, StoreConfigInformation {
/* The default priority for user-specified compaction requests.
* The user gets top priority unless we have blocking compactions. (Pri <= 0)
*/
public static final int PRIORITY_USER = 1;
public static final int NO_PRIORITY = Integer.MIN_VALUE;
*/ int PRIORITY_USER = 1;
int NO_PRIORITY = Integer.MIN_VALUE;
// General Accessors
public KeyValue.KVComparator getComparator();
KeyValue.KVComparator getComparator();
public Collection<StoreFile> getStorefiles();
Collection<StoreFile> getStorefiles();
/**
* Close all the readers We don't need to worry about subsequent requests because the HRegion
@ -65,7 +64,7 @@ public interface Store extends HeapSize, StoreConfigInformation {
* @return the {@link StoreFile StoreFiles} that were previously being used.
* @throws IOException on failure
*/
public Collection<StoreFile> close() throws IOException;
Collection<StoreFile> close() throws IOException;
/**
* Return a scanner for both the memstore and the HStore files. Assumes we are not in a
@ -75,7 +74,7 @@ public interface Store extends HeapSize, StoreConfigInformation {
* @return a scanner over the current key values
* @throws IOException on failure
*/
public KeyValueScanner getScanner(Scan scan, final NavigableSet<byte[]> targetCols)
KeyValueScanner getScanner(Scan scan, final NavigableSet<byte[]> targetCols)
throws IOException;
/**
@ -89,11 +88,16 @@ public interface Store extends HeapSize, StoreConfigInformation {
* @param stopRow
* @return all scanners for this store
*/
public List<KeyValueScanner> getScanners(boolean cacheBlocks,
boolean isGet, boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow,
byte[] stopRow) throws IOException;
List<KeyValueScanner> getScanners(
boolean cacheBlocks,
boolean isGet,
boolean isCompaction,
ScanQueryMatcher matcher,
byte[] startRow,
byte[] stopRow
) throws IOException;
public ScanInfo getScanInfo();
ScanInfo getScanInfo();
/**
* Adds or replaces the specified KeyValues.
@ -108,14 +112,14 @@ public interface Store extends HeapSize, StoreConfigInformation {
* @return memstore size delta
* @throws IOException
*/
public long upsert(Iterable<? extends Cell> cells, long readpoint) throws IOException;
long upsert(Iterable<? extends Cell> cells, long readpoint) throws IOException;
/**
* Adds a value to the memstore
* @param kv
* @return memstore size delta
*/
public long add(KeyValue kv);
long add(KeyValue kv);
/**
* When was the last edit done in the memstore
@ -127,7 +131,7 @@ public interface Store extends HeapSize, StoreConfigInformation {
* key & memstoreTS value of the kv parameter.
* @param kv
*/
public void rollback(final KeyValue kv);
void rollback(final KeyValue kv);
/**
* Find the key that matches <i>row</i> exactly, or the one that immediately precedes it. WARNING:
@ -141,9 +145,9 @@ public interface Store extends HeapSize, StoreConfigInformation {
* @return Found keyvalue or null if none found.
* @throws IOException
*/
public KeyValue getRowKeyAtOrBefore(final byte[] row) throws IOException;
KeyValue getRowKeyAtOrBefore(final byte[] row) throws IOException;
public FileSystem getFileSystem();
FileSystem getFileSystem();
/*
* @param maxKeyCount
@ -152,44 +156,48 @@ public interface Store extends HeapSize, StoreConfigInformation {
* @param includeMVCCReadpoint whether we should out the MVCC readpoint
* @return Writer for a new StoreFile in the tmp dir.
*/
public StoreFile.Writer createWriterInTmp(long maxKeyCount, Compression.Algorithm compression,
boolean isCompaction, boolean includeMVCCReadpoint) throws IOException;
StoreFile.Writer createWriterInTmp(
long maxKeyCount,
Compression.Algorithm compression,
boolean isCompaction,
boolean includeMVCCReadpoint
) throws IOException;
// Compaction oriented methods
public boolean throttleCompaction(long compactionSize);
boolean throttleCompaction(long compactionSize);
/**
* getter for CompactionProgress object
* @return CompactionProgress object; can be null
*/
public CompactionProgress getCompactionProgress();
CompactionProgress getCompactionProgress();
public CompactionContext requestCompaction() throws IOException;
CompactionContext requestCompaction() throws IOException;
public CompactionContext requestCompaction(int priority, CompactionRequest baseRequest)
CompactionContext requestCompaction(int priority, CompactionRequest baseRequest)
throws IOException;
public void cancelRequestedCompaction(CompactionContext compaction);
void cancelRequestedCompaction(CompactionContext compaction);
public List<StoreFile> compact(CompactionContext compaction) throws IOException;
List<StoreFile> compact(CompactionContext compaction) throws IOException;
/**
* @return true if we should run a major compaction.
*/
public boolean isMajorCompaction() throws IOException;
boolean isMajorCompaction() throws IOException;
public void triggerMajorCompaction();
void triggerMajorCompaction();
/**
* See if there's too much store files in this store
* @return true if number of store files is greater than the number defined in minFilesToCompact
*/
public boolean needsCompaction();
boolean needsCompaction();
public int getCompactPriority();
int getCompactPriority();
public StoreFlushContext createFlushContext(long cacheFlushId);
StoreFlushContext createFlushContext(long cacheFlushId);
/**
* Call to complete a compaction. Its for the case where we find in the WAL a compaction
@ -197,18 +205,18 @@ public interface Store extends HeapSize, StoreConfigInformation {
* See HBASE-2331.
* @param compaction
*/
public void completeCompactionMarker(CompactionDescriptor compaction)
void completeCompactionMarker(CompactionDescriptor compaction)
throws IOException;
// Split oriented methods
public boolean canSplit();
boolean canSplit();
/**
* Determines if Store should be split
* @return byte[] if store should be split, null otherwise.
*/
public byte[] getSplitPoint();
byte[] getSplitPoint();
// Bulk Load methods
@ -216,7 +224,7 @@ public interface Store extends HeapSize, StoreConfigInformation {
* This throws a WrongRegionException if the HFile does not fit in this region, or an
* InvalidHFileException if the HFile is not valid.
*/
public void assertBulkLoadHFileOk(Path srcPath) throws IOException;
void assertBulkLoadHFileOk(Path srcPath) throws IOException;
/**
* This method should only be called from HRegion. It is assumed that the ranges of values in the
@ -225,7 +233,7 @@ public interface Store extends HeapSize, StoreConfigInformation {
* @param srcPathStr
* @param sequenceId sequence Id associated with the HFile
*/
public void bulkLoadHFile(String srcPathStr, long sequenceId) throws IOException;
void bulkLoadHFile(String srcPathStr, long sequenceId) throws IOException;
// General accessors into the state of the store
// TODO abstract some of this out into a metrics class
@ -233,50 +241,50 @@ public interface Store extends HeapSize, StoreConfigInformation {
/**
* @return <tt>true</tt> if the store has any underlying reference files to older HFiles
*/
public boolean hasReferences();
boolean hasReferences();
/**
* @return The size of this store's memstore, in bytes
*/
public long getMemStoreSize();
long getMemStoreSize();
public HColumnDescriptor getFamily();
HColumnDescriptor getFamily();
/**
* @return The maximum memstoreTS in all store files.
*/
public long getMaxMemstoreTS();
long getMaxMemstoreTS();
/**
* @return the data block encoder
*/
public HFileDataBlockEncoder getDataBlockEncoder();
HFileDataBlockEncoder getDataBlockEncoder();
/** @return aggregate size of all HStores used in the last compaction */
public long getLastCompactSize();
long getLastCompactSize();
/** @return aggregate size of HStore */
public long getSize();
long getSize();
/**
* @return Count of store files
*/
public int getStorefilesCount();
int getStorefilesCount();
/**
* @return The size of the store files, in bytes, uncompressed.
*/
public long getStoreSizeUncompressed();
long getStoreSizeUncompressed();
/**
* @return The size of the store files, in bytes.
*/
public long getStorefilesSize();
long getStorefilesSize();
/**
* @return The size of the store file indexes, in bytes.
*/
public long getStorefilesIndexSize();
long getStorefilesIndexSize();
/**
* Returns the total size of all index blocks in the data block indexes, including the root level,
@ -284,14 +292,14 @@ public interface Store extends HeapSize, StoreConfigInformation {
* single-level indexes.
* @return the total size of block indexes in the store
*/
public long getTotalStaticIndexSize();
long getTotalStaticIndexSize();
/**
* Returns the total byte size of all Bloom filter bit arrays. For compound Bloom filters even the
* Bloom blocks currently not loaded into the block cache are counted.
* @return the total size of all Bloom filters in the store
*/
public long getTotalStaticBloomSize();
long getTotalStaticBloomSize();
// Test-helper methods
@ -299,40 +307,40 @@ public interface Store extends HeapSize, StoreConfigInformation {
* Used for tests.
* @return cache configuration for this Store.
*/
public CacheConfig getCacheConfig();
CacheConfig getCacheConfig();
/**
* @return the parent region info hosting this store
*/
public HRegionInfo getRegionInfo();
HRegionInfo getRegionInfo();
public RegionCoprocessorHost getCoprocessorHost();
RegionCoprocessorHost getCoprocessorHost();
public boolean areWritesEnabled();
boolean areWritesEnabled();
/**
* @return The smallest mvcc readPoint across all the scanners in this
* region. Writes older than this readPoint, are included in every
* read operation.
*/
public long getSmallestReadPoint();
long getSmallestReadPoint();
public String getColumnFamilyName();
String getColumnFamilyName();
public String getTableName();
String getTableName();
/*
* @param o Observer who wants to know about changes in set of Readers
*/
public void addChangedReaderObserver(ChangedReadersObserver o);
void addChangedReaderObserver(ChangedReadersObserver o);
/*
* @param o Observer no longer interested in changes in set of Readers.
*/
public void deleteChangedReaderObserver(ChangedReadersObserver o);
void deleteChangedReaderObserver(ChangedReadersObserver o);
/**
* @return Whether this store has too many store files.
*/
public boolean hasTooManyStoreFiles();
boolean hasTooManyStoreFiles();
}

Some files were not shown because too many files have changed in this diff Show More