HBASE-9529 Audit of hbase-client @InterfaceAudience.Public apis

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1523542 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Jonathan Hsieh 2013-09-16 05:18:27 +00:00
parent 704a00b544
commit cd63f2055e
67 changed files with 266 additions and 223 deletions

View File

@ -254,6 +254,7 @@ public class ClusterStatus extends VersionedWritable {
return this.liveServers.get(sn);
}
@InterfaceAudience.Private
public Map<String, RegionState> getRegionsInTransition() {
return this.intransition;
}

View File

@ -18,13 +18,11 @@ package org.apache.hadoop.hbase;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Coprocess interface.
* Coprocessor interface.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
@InterfaceAudience.Private
public interface Coprocessor {
int VERSION = 1;

View File

@ -15,18 +15,16 @@
package org.apache.hadoop.hbase;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.client.HTableInterface;
import java.io.IOException;
/**
* Coprocessor environment state.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
@InterfaceAudience.Private
public interface CoprocessorEnvironment {
/** @return the Coprocessor interface version */

View File

@ -21,6 +21,7 @@
package org.apache.hadoop.hbase;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Strings;
@ -28,7 +29,8 @@ import org.apache.hadoop.hbase.util.Strings;
/**
* Encapsulates per-region load metrics.
*/
@InterfaceAudience.Private
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class RegionLoad {
protected ClusterStatusProtos.RegionLoad regionLoadPB;

View File

@ -18,14 +18,15 @@
package org.apache.hadoop.hbase;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.classification.InterfaceStability;
/**
*
* Failed to find .tableinfo file under table dir
*
*/
@InterfaceAudience.Private
@InterfaceAudience.Public
@InterfaceStability.Evolving
@SuppressWarnings("serial")
public class TableInfoMissingException extends HBaseIOException {

View File

@ -17,17 +17,15 @@
*/
package org.apache.hadoop.hbase.client;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import java.io.IOException;
import java.util.Iterator;
import org.apache.hadoop.classification.InterfaceAudience;
/**
* Helper class for custom client scanners.
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
@InterfaceAudience.Private
public abstract class AbstractClientScanner implements ResultScanner {
@Override

View File

@ -276,6 +276,7 @@ public class ClientScanner extends AbstractClientScanner {
return true;
}
@InterfaceAudience.Private
protected ScannerCallable getScannerCallable(byte [] localStartKey,
int nbRows) {
scan.setStartRow(localStartKey);

View File

@ -17,17 +17,15 @@
*/
package org.apache.hadoop.hbase.client;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.HConstants;
import java.util.Random;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
/**
* Utility used by client connections.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
@InterfaceAudience.Private
public class ConnectionUtils {
private static final Random RANDOM = new Random();

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.client;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
/**
@ -37,6 +38,7 @@ import org.apache.hadoop.conf.Configuration;
* the return type of the {@link HConnectable#connect(HConnection)}
* method.
*/
@InterfaceAudience.Private
public abstract class HConnectable<T> {
public Configuration conf;

View File

@ -549,6 +549,7 @@ public class HConnectionManager {
* @return the value returned by the connect method
* @throws IOException
*/
@InterfaceAudience.Private
public static <T> T execute(HConnectable<T> connectable) throws IOException {
if (connectable == null || connectable.conf == null) {
return null;

View File

@ -446,6 +446,7 @@ public interface HTableInterface extends Closeable {
* @param row The row key used to identify the remote region location
* @return A CoprocessorRpcChannel instance
*/
@InterfaceAudience.Private // TODO add coproc audience level
CoprocessorRpcChannel coprocessorService(byte[] row);
/**
@ -469,6 +470,7 @@ public interface HTableInterface extends Closeable {
* {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} method
* @return a map of result values keyed by region name
*/
@InterfaceAudience.Private // TODO add coproc audience level
<T extends Service, R> Map<byte[],R> coprocessorService(final Class<T> service,
byte[] startKey, byte[] endKey, final Batch.Call<T,R> callable)
throws ServiceException, Throwable;
@ -500,6 +502,7 @@ public interface HTableInterface extends Closeable {
* @param <R> Return type for the {@code callable} parameter's
* {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} method
*/
@InterfaceAudience.Private // TODO add coproc audience level
<T extends Service, R> void coprocessorService(final Class<T> service,
byte[] startKey, byte[] endKey, final Batch.Call<T,R> callable,
final Batch.Callback<R> callback) throws ServiceException, Throwable;

View File

@ -18,14 +18,17 @@
*/
package org.apache.hadoop.hbase.client;
import com.google.protobuf.Service;
import com.google.protobuf.ServiceException;
import java.io.Closeable;
import java.io.IOException;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
@ -33,11 +36,8 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.PoolMap;
import org.apache.hadoop.hbase.util.PoolMap.PoolType;
import java.io.Closeable;
import java.io.IOException;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import com.google.protobuf.Service;
import com.google.protobuf.ServiceException;
/**
* A simple pool of HTable instances.
@ -62,8 +62,7 @@ import java.util.Map;
* {@link HConnectionManager}.
* @deprecated Use {@link HConnection#getTable(String)} instead.
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
@InterfaceAudience.Private
public class HTablePool implements Closeable {
private final PoolMap<String, HTableInterface> tables;
private final int maxSize;

View File

@ -33,8 +33,7 @@ import java.util.Map;
*
*
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
@InterfaceAudience.Private
public class HTableUtil {
private static final int INITIAL_LIST_SIZE = 250;

View File

@ -18,22 +18,20 @@
*/
package org.apache.hadoop.hbase.client;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.util.Bytes;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.util.Bytes;
/**
* Container for Actions (i.e. Get, Delete, or Put), which are grouped by
* regionName. Intended to be used with HConnectionManager.processBatch()
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
@InterfaceAudience.Private
public final class MultiAction<R> {
// TODO: This class should not be visible outside of the client package.

View File

@ -19,22 +19,20 @@
package org.apache.hadoop.hbase.client;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
/**
* A container for Result objects, grouped by regionName.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
@InterfaceAudience.Private
public class MultiResponse {
// map of regionName to list of (Results paired to the original index for that

View File

@ -26,7 +26,6 @@ import java.net.SocketTimeoutException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.TableName;
@ -39,8 +38,7 @@ import org.apache.hadoop.hbase.util.Bytes;
* Passed to a {@link RpcRetryingCaller} so we retry on fail.
* @param <T> the class that the ServerCallable handles
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
@InterfaceAudience.Private
public abstract class RegionServerCallable<T> implements RetryingCallable<T> {
// Public because used outside of this package over in ipc.
static final Log LOG = LogFactory.getLog(RegionServerCallable.class);

View File

@ -23,15 +23,13 @@ import java.io.IOException;
import java.util.concurrent.Callable;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* A Callable<T> that will be retried. If {@link #call()} invocation throws exceptions,
* we will call {@link #throwable(Throwable, boolean)} with whatever the exception was.
* @param <T>
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
@InterfaceAudience.Private
public interface RetryingCallable<T> extends Callable<T> {
/**
* Prepare by setting up any connections to servers, etc., ahead of {@link #call()} invocation.

View File

@ -29,10 +29,9 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ipc.RpcClient;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.ipc.RemoteException;
@ -44,8 +43,7 @@ import com.google.protobuf.ServiceException;
* threadlocal outstanding timeouts as so we don't persist too much.
* Dynamic rather than static so can set the generic appropriately.
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
@InterfaceAudience.Private
public class RpcRetryingCaller<T> {
static final Log LOG = LogFactory.getLog(RpcRetryingCaller.class);
/**

View File

@ -24,7 +24,6 @@ import java.net.UnknownHostException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellScanner;
@ -55,8 +54,7 @@ import com.google.protobuf.TextFormat;
* Used by {@link ResultScanner}s made by {@link HTable}. Passed to a retrying caller such as
* {@link RpcRetryingCaller} so fails are retried.
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
@InterfaceAudience.Private
public class ScannerCallable extends RegionServerCallable<Result[]> {
public static final String LOG_SCANNER_LATENCY_CUTOFF
= "hbase.client.log.scanner.latency.cutoff";

View File

@ -17,8 +17,12 @@
*/
package org.apache.hadoop.hbase.client;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.HBaseIOException;
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class WrongRowIOException extends HBaseIOException {
private static final long serialVersionUID = -5849522209440123059L;

View File

@ -19,17 +19,23 @@
package org.apache.hadoop.hbase.client.coprocessor;
import com.google.protobuf.ByteString;
import com.google.protobuf.Message;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.NavigableMap;
import java.util.NavigableSet;
import java.util.TreeMap;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
@ -44,15 +50,8 @@ import org.apache.hadoop.hbase.protobuf.generated.AggregateProtos.AggregateServi
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.NavigableMap;
import java.util.NavigableSet;
import java.util.TreeMap;
import java.util.concurrent.atomic.AtomicLong;
import com.google.protobuf.ByteString;
import com.google.protobuf.Message;
/**
* This client class is for invoking the aggregate functions deployed on the
@ -73,8 +72,7 @@ import java.util.concurrent.atomic.AtomicLong;
* parameter type. For average and std, it returns a double value. For row
* count, it returns a long value.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
@InterfaceAudience.Private
public class AggregationClient {
private static final Log log = LogFactory.getLog(AggregationClient.class);

View File

@ -19,18 +19,16 @@
package org.apache.hadoop.hbase.client.coprocessor;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
/**
* A collection of interfaces and utilities used for interacting with custom RPC
* interfaces exposed by Coprocessors.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
@InterfaceAudience.Private
public abstract class Batch {
/**
* Defines a unit of work to be executed.

View File

@ -23,7 +23,6 @@ import java.math.BigDecimal;
import java.math.RoundingMode;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.coprocessor.ColumnInterpreter;
@ -38,8 +37,7 @@ import com.google.protobuf.ByteString;
* is required at the RegionServer also.
*
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
@InterfaceAudience.Private
public class BigDecimalColumnInterpreter extends ColumnInterpreter<BigDecimal, BigDecimal,
EmptyMsg, BigDecimalMsg, BigDecimalMsg> {

View File

@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.client.coprocessor;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.coprocessor.ColumnInterpreter;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.EmptyMsg;
@ -36,8 +35,7 @@ import org.apache.hadoop.hbase.util.Bytes;
* TestAggregateProtocol methods for its sample usage.
* Its methods handle null arguments gracefully.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
@InterfaceAudience.Private
public class LongColumnInterpreter extends ColumnInterpreter<Long, Long,
EmptyMsg, LongMsg, LongMsg> {

View File

@ -19,6 +19,8 @@
package org.apache.hadoop.hbase.client.coprocessor;
import com.google.protobuf.ByteString;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HConstants;
@ -40,6 +42,7 @@ import java.util.List;
* Client proxy for SecureBulkLoadProtocol
* used in conjunction with SecureBulkLoadEndpoint
*/
@InterfaceAudience.Private
public class SecureBulkLoadClient {
private HTable table;

View File

@ -18,16 +18,16 @@
package org.apache.hadoop.hbase.client.metrics;
import com.google.common.collect.ImmutableMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import com.google.common.collect.ImmutableMap;
/**
* Provides client-side metrics related to scan operations
@ -40,8 +40,7 @@ import java.util.concurrent.atomic.AtomicLong;
* However, there is no need for this. So they are defined under scan operation
* for now.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
@InterfaceAudience.Private
public class ScanMetrics {

View File

@ -18,11 +18,22 @@
*/
package org.apache.hadoop.hbase.client.replication;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.replication.ReplicationException;
@ -30,17 +41,6 @@ import org.apache.hadoop.hbase.replication.ReplicationFactory;
import org.apache.hadoop.hbase.replication.ReplicationPeers;
import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.HColumnDescriptor;
import java.io.Closeable;
import java.io.IOException;
import java.util.Map;
import java.util.List;
import java.util.ArrayList;
import java.util.HashMap;
import java.lang.Integer;
/**
* <p>
@ -65,6 +65,8 @@ import java.lang.Integer;
* <code>replication</code>.
* </p>
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class ReplicationAdmin implements Closeable {
private static final Log LOG = LogFactory.getLog(ReplicationAdmin.class);

View File

@ -22,7 +22,6 @@ package org.apache.hadoop.hbase.coprocessor;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
@ -56,8 +55,7 @@ import com.google.protobuf.Message;
* @param <Q> PB message that is used to transport Cell (<T>) instance
* @param <R> PB message that is used to transport Promoted (<S>) instance
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
@InterfaceAudience.Private
public abstract class ColumnInterpreter<T, S, P extends Message,
Q extends Message, R extends Message> {

View File

@ -34,6 +34,8 @@ import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
public class BitComparator extends ByteArrayComparable {
/** Bit operators. */
@InterfaceAudience.Public
@InterfaceStability.Stable
public enum BitwiseOp {
/** and */
AND,

View File

@ -49,6 +49,8 @@ import java.util.ArrayList;
public abstract class CompareFilter extends FilterBase {
/** Comparison operators. */
@InterfaceAudience.Public
@InterfaceStability.Stable
public enum CompareOp {
/** less than */
LESS,

View File

@ -141,6 +141,8 @@ public abstract class Filter {
/**
* Return codes for filterValue().
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
public enum ReturnCode {
/**
* Include the Cell

View File

@ -37,8 +37,7 @@ import org.apache.hadoop.hbase.KeyValueUtil;
* If you could instantiate FilterBase, it would end up being a "null" filter -
* that is one that never filters anything.
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
@InterfaceAudience.Private // TODO add filter limited private level
public abstract class FilterBase extends Filter {
/**

View File

@ -55,6 +55,8 @@ import com.google.protobuf.InvalidProtocolBufferException;
@InterfaceStability.Stable
final public class FilterList extends Filter {
/** set operator */
@InterfaceAudience.Public
@InterfaceStability.Stable
public static enum Operator {
/** !AND */
MUST_PASS_ALL,

View File

@ -43,7 +43,6 @@ import com.google.protobuf.InvalidProtocolBufferException;
*
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
final public class FilterWrapper extends Filter {
Filter filter = null;

View File

@ -17,6 +17,11 @@
*/
package org.apache.hadoop.hbase.ipc;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class BadAuthException extends FatalConnectionException {
public BadAuthException() {
super();

View File

@ -18,21 +18,20 @@
package org.apache.hadoop.hbase.ipc;
import com.google.protobuf.RpcCallback;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import java.io.IOException;
import java.io.InterruptedIOException;
import org.apache.hadoop.classification.InterfaceAudience;
import com.google.protobuf.RpcCallback;
/**
* Simple {@link RpcCallback} implementation providing a
* {@link java.util.concurrent.Future}-like {@link BlockingRpcCallback#get()} method, which
* will block util the instance's {@link BlockingRpcCallback#run(Object)} method has been called.
* {@code R} is the RPC response type that will be passed to the {@link #run(Object)} method.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
@InterfaceAudience.Private
public class BlockingRpcCallback<R> implements RpcCallback<R> {
private R result;
private boolean resultSet = false;

View File

@ -19,11 +19,16 @@ package org.apache.hadoop.hbase.ipc;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Exception indicating that the remote host making this IPC lost its
* IPC connection. This will never be returned back to a client,
* but is only used for logging on the server side, etc.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class CallerDisconnectedException extends IOException {
private static final long serialVersionUID = 1L;
public CallerDisconnectedException(String msg) {

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.hbase.ipc;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.DoNotRetryIOException;
/**
@ -26,7 +27,8 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
* Closes connection after throwing this exception with message on why the failure.
*/
@SuppressWarnings("serial")
@InterfaceAudience.Private
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class FatalConnectionException extends DoNotRetryIOException {
public FatalConnectionException() {
super();

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.hbase.ipc;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.ipc.RemoteException;
@ -27,7 +28,8 @@ import org.apache.hadoop.ipc.RemoteException;
* <p>A {@link RemoteException} hosts exceptions we got from the server.
*/
@SuppressWarnings("serial")
@InterfaceAudience.Private
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class RemoteWithExtrasException extends RemoteException {
private final String hostname;
private final int port;

View File

@ -52,6 +52,7 @@ import javax.security.sasl.SaslException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.HConstants;
@ -208,6 +209,8 @@ public class RpcClient {
}
@SuppressWarnings("serial")
@InterfaceAudience.Public
@InterfaceStability.Evolving
public static class FailedServerException extends IOException {
public FailedServerException(String s) {
super(s);
@ -1202,6 +1205,8 @@ public class RpcClient {
* Client-side call timeout
*/
@SuppressWarnings("serial")
@InterfaceAudience.Public
@InterfaceStability.Evolving
public static class CallTimeoutException extends IOException {
public CallTimeoutException(final String msg) {
super(msg);

View File

@ -20,11 +20,13 @@
package org.apache.hadoop.hbase.ipc;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import java.io.IOException;
@SuppressWarnings("serial")
@InterfaceAudience.Private
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class ServerNotRunningYetException extends IOException {
public ServerNotRunningYetException(String s) {
super(s);

View File

@ -17,8 +17,12 @@
*/
package org.apache.hadoop.hbase.ipc;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.HBaseIOException;
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class StoppedRpcClientException extends HBaseIOException {
public StoppedRpcClientException() {
super();

View File

@ -17,6 +17,11 @@
*/
package org.apache.hadoop.hbase.ipc;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class UnsupportedCellCodecException extends FatalConnectionException {
public UnsupportedCellCodecException() {
super();

View File

@ -17,6 +17,11 @@
*/
package org.apache.hadoop.hbase.ipc;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class UnsupportedCompressionCodecException extends FatalConnectionException {
public UnsupportedCompressionCodecException() {
super();

View File

@ -17,6 +17,11 @@
*/
package org.apache.hadoop.hbase.ipc;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class WrongVersionException extends FatalConnectionException {
public WrongVersionException() {
super();

View File

@ -17,17 +17,18 @@
*/
package org.apache.hadoop.hbase.master;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.util.Date;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
/**
* State of a Region while undergoing transitions.
* Region state cannot be modified except the stamp field.
@ -35,6 +36,9 @@ import java.util.concurrent.atomic.AtomicLong;
*/
@InterfaceAudience.Private
public class RegionState implements org.apache.hadoop.io.Writable {
@InterfaceAudience.Public
@InterfaceStability.Evolving
public enum State {
OFFLINE, // region is in an offline state
PENDING_OPEN, // sent rpc to server to open but has not begun

View File

@ -19,6 +19,11 @@
package org.apache.hadoop.hbase.regionserver;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@InterfaceAudience.Public
@InterfaceStability.Evolving
public enum BloomType {
/**
* Bloomfilters disabled

View File

@ -19,12 +19,14 @@
package org.apache.hadoop.hbase.regionserver;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.DoNotRetryIOException;
/**
* Reports a problem with a lease
*/
@InterfaceAudience.Private
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class LeaseException extends DoNotRetryIOException {
private static final long serialVersionUID = 8179703995292418650L;

View File

@ -19,12 +19,14 @@
package org.apache.hadoop.hbase.regionserver;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.DoNotRetryIOException;
/**
* Thrown if request for nonexistent column family.
*/
@InterfaceAudience.Private
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class NoSuchColumnFamilyException extends DoNotRetryIOException {
private static final long serialVersionUID = -6569952730832331274L;

View File

@ -18,15 +18,17 @@
*/
package org.apache.hadoop.hbase.regionserver;
import org.apache.hadoop.classification.InterfaceAudience;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Thrown if the region server log directory exists (which indicates another
* region server is running at the same address)
*/
@InterfaceAudience.Private
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class RegionServerRunningException extends IOException {
private static final long serialVersionUID = 1L << 31 - 1L;

View File

@ -21,12 +21,14 @@ package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Thrown by the region server when it is in shutting down state.
*/
@SuppressWarnings("serial")
@InterfaceAudience.Private
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class RegionServerStoppedException extends IOException {
public RegionServerStoppedException(String s) {
super(s);

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.hbase.regionserver.wal;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import java.io.IOException;
@ -26,7 +27,8 @@ import java.io.IOException;
* Thrown when we fail close of the write-ahead-log file.
* Package private. Only used inside this package.
*/
@InterfaceAudience.Private
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class FailedLogCloseException extends IOException {
private static final long serialVersionUID = 1759152841462990925L;

View File

@ -17,12 +17,16 @@
*/
package org.apache.hadoop.hbase.security;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.DoNotRetryIOException;
/**
* Exception thrown by access-related methods.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class AccessDeniedException extends DoNotRetryIOException {
private static final long serialVersionUID = 1913879564363001780L;

View File

@ -19,22 +19,20 @@
package org.apache.hadoop.hbase.security;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.hbase.util.Methods;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.security.UserGroupInformation;
import java.io.IOException;
import java.lang.reflect.UndeclaredThrowableException;
import java.security.PrivilegedAction;
import java.security.PrivilegedExceptionAction;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.util.Methods;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.security.UserGroupInformation;
/**
* Wrapper to abstract out usage of user and group information in HBase.
*
@ -46,8 +44,7 @@ import java.security.PrivilegedExceptionAction;
* HBase, but can be extended as needs change.
* </p>
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
@InterfaceAudience.Private
public abstract class User {
public static final String HBASE_SECURITY_CONF_KEY =
"hbase.security.authentication";

View File

@ -18,12 +18,16 @@
package org.apache.hadoop.hbase.snapshot;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
/**
* Thrown when a snapshot could not be restored due to a server-side error when restoring it.
*/
@SuppressWarnings("serial")
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class RestoreSnapshotException extends HBaseSnapshotException {
public RestoreSnapshotException(String msg, SnapshotDescription desc) {
super(msg, desc);

View File

@ -17,6 +17,8 @@
*/
package org.apache.hadoop.hbase.snapshot;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
/**
@ -24,6 +26,8 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescriptio
* taking the snapshot.
*/
@SuppressWarnings("serial")
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class SnapshotCreationException extends HBaseSnapshotException {
/**

View File

@ -18,9 +18,6 @@
*/
package org.apache.hadoop.hbase.util;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
@ -34,6 +31,8 @@ import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.hadoop.classification.InterfaceAudience;
/**
*
* The <code>PoolMap</code> maps a key to a collection of values, the elements
@ -52,8 +51,7 @@ import java.util.concurrent.atomic.AtomicInteger;
* @param <V>
* the type of the resource being pooled
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
@InterfaceAudience.Private
public class PoolMap<K, V> implements Map<K, V> {
private PoolType poolType;

View File

@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.util;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Stoppable;
/**
@ -30,8 +29,7 @@ import org.apache.hadoop.hbase.Stoppable;
* will return if the flag is set (rather than go back to sleep until its
* sleep time is up).
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
@InterfaceAudience.Private
public class Sleeper {
private final Log LOG = LogFactory.getLog(this.getClass().getName());
private final int period;

View File

@ -18,10 +18,17 @@
*/
package org.apache.hadoop.hbase.zookeeper;
import java.io.IOException;
import java.lang.management.ManagementFactory;
import java.security.SecureRandom;
import java.util.ArrayList;
import java.util.LinkedList;
import java.util.List;
import java.util.Random;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.RetryCounter;
import org.apache.hadoop.hbase.util.RetryCounterFactory;
@ -41,14 +48,6 @@ import org.apache.zookeeper.proto.SetDataRequest;
import org.cloudera.htrace.Trace;
import org.cloudera.htrace.TraceScope;
import java.io.IOException;
import java.lang.management.ManagementFactory;
import java.security.SecureRandom;
import java.util.ArrayList;
import java.util.LinkedList;
import java.util.List;
import java.util.Random;
/**
* A zookeeper that can handle 'recoverable' errors.
* To handle recoverable errors, developers need to realize that there are two
@ -72,8 +71,7 @@ import java.util.Random;
* succeeded an the znode it created is "x-352-109".
* @see "http://wiki.apache.org/hadoop/ZooKeeper/ErrorHandling"
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
@InterfaceAudience.Private
public class RecoverableZooKeeper {
private static final Log LOG = LogFactory.getLog(RecoverableZooKeeper.class);
// the actual ZooKeeper client instance

View File

@ -18,10 +18,11 @@
*/
package org.apache.hadoop.hbase.zookeeper;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.RegionTransition;
@ -35,8 +36,6 @@ import org.apache.zookeeper.KeeperException.NoNodeException;
import org.apache.zookeeper.KeeperException.NodeExistsException;
import org.apache.zookeeper.data.Stat;
import java.util.List;
// We should not be importing this Type here, nor a RegionTransition, etc. This class should be
// about zk and bytes only.
@ -94,8 +93,7 @@ import java.util.List;
* </li>
* </ol>
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
@InterfaceAudience.Private
public class ZKAssign {
private static final Log LOG = LogFactory.getLog(ZKAssign.class);

View File

@ -18,14 +18,6 @@
*/
package org.apache.hadoop.hbase.zookeeper;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.util.StringUtils;
import java.io.IOException;
import java.io.InputStream;
import java.net.InetAddress;
@ -35,11 +27,17 @@ import java.util.List;
import java.util.Map.Entry;
import java.util.Properties;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.util.StringUtils;
/**
* Utility methods for reading, and building the ZooKeeper configuration.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
@InterfaceAudience.Private
public class ZKConfig {
private static final Log LOG = LogFactory.getLog(ZKConfig.class);

View File

@ -18,16 +18,15 @@
package org.apache.hadoop.hbase.zookeeper;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.zookeeper.KeeperException;
import java.util.concurrent.atomic.AtomicBoolean;
/**
* Handles coordination of a single "leader" instance among many possible
* candidates. The first {@link ZKLeaderManager} to successfully create
@ -40,8 +39,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
* @deprecated Not used
*/
@Deprecated
@InterfaceAudience.Public
@InterfaceStability.Evolving
@InterfaceAudience.Private
public class ZKLeaderManager extends ZooKeeperListener {
private static Log LOG = LogFactory.getLog(ZKLeaderManager.class);

View File

@ -20,6 +20,8 @@
package org.apache.hadoop.hbase.zookeeper;
import com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@ -37,6 +39,7 @@ import java.util.Set;
*
* <p>Does not cache state like {@link ZKTable}, actually reads from ZK each call.
*/
@InterfaceAudience.Private
public class ZKTableReadOnly {
private ZKTableReadOnly() {}

View File

@ -18,11 +18,27 @@
*/
package org.apache.hadoop.hbase.zookeeper;
import java.io.BufferedReader;
import java.io.IOException;
import java.io.InputStreamReader;
import java.io.PrintWriter;
import java.net.InetSocketAddress;
import java.net.Socket;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import javax.security.auth.login.AppConfigurationEntry;
import javax.security.auth.login.AppConfigurationEntry.LoginModuleControlFlag;
import org.apache.commons.lang.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ServerName;
@ -30,7 +46,6 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionStoreSequenceIds;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.StoreSequenceId;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.CreateAndFailSilent;
@ -57,23 +72,6 @@ import org.apache.zookeeper.server.ZooKeeperSaslServer;
import com.google.protobuf.ByteString;
import com.google.protobuf.InvalidProtocolBufferException;
import javax.security.auth.login.AppConfigurationEntry;
import javax.security.auth.login.AppConfigurationEntry.LoginModuleControlFlag;
import java.io.BufferedReader;
import java.io.IOException;
import java.io.InputStreamReader;
import java.io.PrintWriter;
import java.net.InetSocketAddress;
import java.net.Socket;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.TreeMap;
/**
* Internal HBase utility class for ZooKeeper.
*
@ -83,8 +81,7 @@ import java.util.TreeMap;
* zookeeper exception, so callers of these methods must handle appropriately.
* If ZK is required for the operation, the server will need to be aborted.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
@InterfaceAudience.Private
public class ZKUtil {
private static final Log LOG = LogFactory.getLog(ZKUtil.class);

View File

@ -34,8 +34,7 @@ import org.apache.hadoop.classification.InterfaceStability;
* Note that the watcher will be blocked when invoking methods in listeners so
* they must not be long-running.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
@InterfaceAudience.Private
public abstract class ZooKeeperListener {
// Reference to the zk watcher which also contains configuration and constants

View File

@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.zookeeper;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Abortable;
import org.apache.zookeeper.KeeperException;
@ -34,8 +33,7 @@ import org.apache.zookeeper.KeeperException;
* <p>This is the base class used by trackers in both the Master and
* RegionServers.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
@InterfaceAudience.Private
public abstract class ZooKeeperNodeTracker extends ZooKeeperListener {
static final Log LOG = LogFactory.getLog(ZooKeeperNodeTracker.class);

View File

@ -18,10 +18,16 @@
*/
package org.apache.hadoop.hbase.zookeeper;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.CountDownLatch;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HConstants;
@ -33,13 +39,6 @@ import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.ZooDefs;
import org.apache.zookeeper.data.ACL;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.CountDownLatch;
/**
* Acts as the single ZooKeeper Watcher. One instance of this is instantiated
* for each Master, RegionServer, and client process.
@ -51,8 +50,7 @@ import java.util.concurrent.CountDownLatch;
* <p>This class also holds and manages the connection to ZooKeeper. Code to
* deal with connection related events and exceptions are handled here.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
@InterfaceAudience.Private
public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
private static final Log LOG = LogFactory.getLog(ZooKeeperWatcher.class);