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); return this.liveServers.get(sn);
} }
@InterfaceAudience.Private
public Map<String, RegionState> getRegionsInTransition() { public Map<String, RegionState> getRegionsInTransition() {
return this.intransition; return this.intransition;
} }

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -19,22 +19,20 @@
package org.apache.hadoop.hbase.client; 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.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.TreeMap; 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. * A container for Result objects, grouped by regionName.
*/ */
@InterfaceAudience.Public @InterfaceAudience.Private
@InterfaceStability.Evolving
public class MultiResponse { public class MultiResponse {
// map of regionName to list of (Results paired to the original index for that // 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.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.TableName; 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. * Passed to a {@link RpcRetryingCaller} so we retry on fail.
* @param <T> the class that the ServerCallable handles * @param <T> the class that the ServerCallable handles
*/ */
@InterfaceAudience.Public @InterfaceAudience.Private
@InterfaceStability.Stable
public abstract class RegionServerCallable<T> implements RetryingCallable<T> { public abstract class RegionServerCallable<T> implements RetryingCallable<T> {
// Public because used outside of this package over in ipc. // Public because used outside of this package over in ipc.
static final Log LOG = LogFactory.getLog(RegionServerCallable.class); static final Log LOG = LogFactory.getLog(RegionServerCallable.class);

View File

@ -23,15 +23,13 @@ import java.io.IOException;
import java.util.concurrent.Callable; import java.util.concurrent.Callable;
import org.apache.hadoop.classification.InterfaceAudience; 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, * 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. * we will call {@link #throwable(Throwable, boolean)} with whatever the exception was.
* @param <T> * @param <T>
*/ */
@InterfaceAudience.Public @InterfaceAudience.Private
@InterfaceStability.Stable
public interface RetryingCallable<T> extends Callable<T> { public interface RetryingCallable<T> extends Callable<T> {
/** /**
* Prepare by setting up any connections to servers, etc., ahead of {@link #call()} invocation. * 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.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ipc.RpcClient; import org.apache.hadoop.hbase.ipc.RpcClient;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.ipc.RemoteException; 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. * threadlocal outstanding timeouts as so we don't persist too much.
* Dynamic rather than static so can set the generic appropriately. * Dynamic rather than static so can set the generic appropriately.
*/ */
@InterfaceAudience.Public @InterfaceAudience.Private
@InterfaceStability.Stable
public class RpcRetryingCaller<T> { public class RpcRetryingCaller<T> {
static final Log LOG = LogFactory.getLog(RpcRetryingCaller.class); 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.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellScanner; 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 * Used by {@link ResultScanner}s made by {@link HTable}. Passed to a retrying caller such as
* {@link RpcRetryingCaller} so fails are retried. * {@link RpcRetryingCaller} so fails are retried.
*/ */
@InterfaceAudience.Public @InterfaceAudience.Private
@InterfaceStability.Stable
public class ScannerCallable extends RegionServerCallable<Result[]> { public class ScannerCallable extends RegionServerCallable<Result[]> {
public static final String LOG_SCANNER_LATENCY_CUTOFF public static final String LOG_SCANNER_LATENCY_CUTOFF
= "hbase.client.log.scanner.latency.cutoff"; = "hbase.client.log.scanner.latency.cutoff";

View File

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

View File

@ -19,17 +19,23 @@
package org.apache.hadoop.hbase.client.coprocessor; package org.apache.hadoop.hbase.client.coprocessor;
import com.google.protobuf.ByteString; import java.io.IOException;
import com.google.protobuf.Message; 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.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HConstants; 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.HTable;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner; 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.Bytes;
import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Pair;
import java.io.IOException; import com.google.protobuf.ByteString;
import java.nio.ByteBuffer; import com.google.protobuf.Message;
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;
/** /**
* This client class is for invoking the aggregate functions deployed on the * 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 * parameter type. For average and std, it returns a double value. For row
* count, it returns a long value. * count, it returns a long value.
*/ */
@InterfaceAudience.Public @InterfaceAudience.Private
@InterfaceStability.Evolving
public class AggregationClient { public class AggregationClient {
private static final Log log = LogFactory.getLog(AggregationClient.class); private static final Log log = LogFactory.getLog(AggregationClient.class);

View File

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

View File

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

View File

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

View File

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

View File

@ -18,16 +18,16 @@
package org.apache.hadoop.hbase.client.metrics; 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.HashMap;
import java.util.Map; import java.util.Map;
import java.util.concurrent.atomic.AtomicLong; 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 * 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 * However, there is no need for this. So they are defined under scan operation
* for now. * for now.
*/ */
@InterfaceAudience.Public @InterfaceAudience.Private
@InterfaceStability.Evolving
public class ScanMetrics { public class ScanMetrics {

View File

@ -18,11 +18,22 @@
*/ */
package org.apache.hadoop.hbase.client.replication; 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.Log;
import org.apache.commons.logging.LogFactory; 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.conf.Configuration;
import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants; 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.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.replication.ReplicationException; 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.ReplicationPeers;
import org.apache.hadoop.hbase.replication.ReplicationQueuesClient; import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; 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> * <p>
@ -65,6 +65,8 @@ import java.lang.Integer;
* <code>replication</code>. * <code>replication</code>.
* </p> * </p>
*/ */
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class ReplicationAdmin implements Closeable { public class ReplicationAdmin implements Closeable {
private static final Log LOG = LogFactory.getLog(ReplicationAdmin.class); 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 java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil; 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 <Q> PB message that is used to transport Cell (<T>) instance
* @param <R> PB message that is used to transport Promoted (<S>) instance * @param <R> PB message that is used to transport Promoted (<S>) instance
*/ */
@InterfaceAudience.Public @InterfaceAudience.Private
@InterfaceStability.Evolving
public abstract class ColumnInterpreter<T, S, P extends Message, public abstract class ColumnInterpreter<T, S, P extends Message,
Q extends Message, R 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 { public class BitComparator extends ByteArrayComparable {
/** Bit operators. */ /** Bit operators. */
@InterfaceAudience.Public
@InterfaceStability.Stable
public enum BitwiseOp { public enum BitwiseOp {
/** and */ /** and */
AND, AND,

View File

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

View File

@ -141,6 +141,8 @@ public abstract class Filter {
/** /**
* Return codes for filterValue(). * Return codes for filterValue().
*/ */
@InterfaceAudience.Public
@InterfaceStability.Stable
public enum ReturnCode { public enum ReturnCode {
/** /**
* Include the Cell * 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 - * If you could instantiate FilterBase, it would end up being a "null" filter -
* that is one that never filters anything. * that is one that never filters anything.
*/ */
@InterfaceAudience.Public @InterfaceAudience.Private // TODO add filter limited private level
@InterfaceStability.Stable
public abstract class FilterBase extends Filter { public abstract class FilterBase extends Filter {
/** /**

View File

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

View File

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

View File

@ -17,6 +17,11 @@
*/ */
package org.apache.hadoop.hbase.ipc; 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 class BadAuthException extends FatalConnectionException {
public BadAuthException() { public BadAuthException() {
super(); super();

View File

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

View File

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

View File

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

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.hbase.ipc; package org.apache.hadoop.hbase.ipc;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.ipc.RemoteException; 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. * <p>A {@link RemoteException} hosts exceptions we got from the server.
*/ */
@SuppressWarnings("serial") @SuppressWarnings("serial")
@InterfaceAudience.Private @InterfaceAudience.Public
@InterfaceStability.Evolving
public class RemoteWithExtrasException extends RemoteException { public class RemoteWithExtrasException extends RemoteException {
private final String hostname; private final String hostname;
private final int port; 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.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
@ -208,6 +209,8 @@ public class RpcClient {
} }
@SuppressWarnings("serial") @SuppressWarnings("serial")
@InterfaceAudience.Public
@InterfaceStability.Evolving
public static class FailedServerException extends IOException { public static class FailedServerException extends IOException {
public FailedServerException(String s) { public FailedServerException(String s) {
super(s); super(s);
@ -1202,6 +1205,8 @@ public class RpcClient {
* Client-side call timeout * Client-side call timeout
*/ */
@SuppressWarnings("serial") @SuppressWarnings("serial")
@InterfaceAudience.Public
@InterfaceStability.Evolving
public static class CallTimeoutException extends IOException { public static class CallTimeoutException extends IOException {
public CallTimeoutException(final String msg) { public CallTimeoutException(final String msg) {
super(msg); super(msg);

View File

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

View File

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

View File

@ -17,6 +17,11 @@
*/ */
package org.apache.hadoop.hbase.ipc; 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 class UnsupportedCellCodecException extends FatalConnectionException {
public UnsupportedCellCodecException() { public UnsupportedCellCodecException() {
super(); super();

View File

@ -17,6 +17,11 @@
*/ */
package org.apache.hadoop.hbase.ipc; 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 class UnsupportedCompressionCodecException extends FatalConnectionException {
public UnsupportedCompressionCodecException() { public UnsupportedCompressionCodecException() {
super(); super();

View File

@ -17,6 +17,11 @@
*/ */
package org.apache.hadoop.hbase.ipc; 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 class WrongVersionException extends FatalConnectionException {
public WrongVersionException() { public WrongVersionException() {
super(); super();

View File

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

View File

@ -19,6 +19,11 @@
package org.apache.hadoop.hbase.regionserver; 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 { public enum BloomType {
/** /**
* Bloomfilters disabled * Bloomfilters disabled

View File

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

View File

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

View File

@ -18,15 +18,17 @@
*/ */
package org.apache.hadoop.hbase.regionserver; package org.apache.hadoop.hbase.regionserver;
import org.apache.hadoop.classification.InterfaceAudience;
import java.io.IOException; 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 * Thrown if the region server log directory exists (which indicates another
* region server is running at the same address) * region server is running at the same address)
*/ */
@InterfaceAudience.Private @InterfaceAudience.Public
@InterfaceStability.Evolving
public class RegionServerRunningException extends IOException { public class RegionServerRunningException extends IOException {
private static final long serialVersionUID = 1L << 31 - 1L; 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 java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/** /**
* Thrown by the region server when it is in shutting down state. * Thrown by the region server when it is in shutting down state.
*/ */
@SuppressWarnings("serial") @SuppressWarnings("serial")
@InterfaceAudience.Private @InterfaceAudience.Public
@InterfaceStability.Evolving
public class RegionServerStoppedException extends IOException { public class RegionServerStoppedException extends IOException {
public RegionServerStoppedException(String s) { public RegionServerStoppedException(String s) {
super(s); super(s);

View File

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

View File

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

View File

@ -19,22 +19,20 @@
package org.apache.hadoop.hbase.security; 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.io.IOException;
import java.lang.reflect.UndeclaredThrowableException; import java.lang.reflect.UndeclaredThrowableException;
import java.security.PrivilegedAction; import java.security.PrivilegedAction;
import java.security.PrivilegedExceptionAction; 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. * 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. * HBase, but can be extended as needs change.
* </p> * </p>
*/ */
@InterfaceAudience.Public @InterfaceAudience.Private
@InterfaceStability.Evolving
public abstract class User { public abstract class User {
public static final String HBASE_SECURITY_CONF_KEY = public static final String HBASE_SECURITY_CONF_KEY =
"hbase.security.authentication"; "hbase.security.authentication";

View File

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

View File

@ -17,6 +17,8 @@
*/ */
package org.apache.hadoop.hbase.snapshot; 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; 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. * taking the snapshot.
*/ */
@SuppressWarnings("serial") @SuppressWarnings("serial")
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class SnapshotCreationException extends HBaseSnapshotException { public class SnapshotCreationException extends HBaseSnapshotException {
/** /**

View File

@ -18,9 +18,6 @@
*/ */
package org.apache.hadoop.hbase.util; 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.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.HashMap; import java.util.HashMap;
@ -34,6 +31,8 @@ import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.atomic.AtomicInteger; 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 * 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> * @param <V>
* the type of the resource being pooled * the type of the resource being pooled
*/ */
@InterfaceAudience.Public @InterfaceAudience.Private
@InterfaceStability.Evolving
public class PoolMap<K, V> implements Map<K, V> { public class PoolMap<K, V> implements Map<K, V> {
private PoolType poolType; 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.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Stoppable; 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 * will return if the flag is set (rather than go back to sleep until its
* sleep time is up). * sleep time is up).
*/ */
@InterfaceAudience.Public @InterfaceAudience.Private
@InterfaceStability.Stable
public class Sleeper { public class Sleeper {
private final Log LOG = LogFactory.getLog(this.getClass().getName()); private final Log LOG = LogFactory.getLog(this.getClass().getName());
private final int period; private final int period;

View File

@ -18,10 +18,17 @@
*/ */
package org.apache.hadoop.hbase.zookeeper; 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.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; 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.Bytes;
import org.apache.hadoop.hbase.util.RetryCounter; import org.apache.hadoop.hbase.util.RetryCounter;
import org.apache.hadoop.hbase.util.RetryCounterFactory; 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.Trace;
import org.cloudera.htrace.TraceScope; 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. * A zookeeper that can handle 'recoverable' errors.
* To handle recoverable errors, developers need to realize that there are two * 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". * succeeded an the znode it created is "x-352-109".
* @see "http://wiki.apache.org/hadoop/ZooKeeper/ErrorHandling" * @see "http://wiki.apache.org/hadoop/ZooKeeper/ErrorHandling"
*/ */
@InterfaceAudience.Public @InterfaceAudience.Private
@InterfaceStability.Evolving
public class RecoverableZooKeeper { public class RecoverableZooKeeper {
private static final Log LOG = LogFactory.getLog(RecoverableZooKeeper.class); private static final Log LOG = LogFactory.getLog(RecoverableZooKeeper.class);
// the actual ZooKeeper client instance // the actual ZooKeeper client instance

View File

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

View File

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

View File

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

View File

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

View File

@ -18,11 +18,27 @@
*/ */
package org.apache.hadoop.hbase.zookeeper; 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.lang.StringUtils;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ServerName; 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.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; 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.RegionStoreSequenceIds;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.StoreSequenceId;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.CreateAndFailSilent; 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.ByteString;
import com.google.protobuf.InvalidProtocolBufferException; 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. * Internal HBase utility class for ZooKeeper.
* *
@ -83,8 +81,7 @@ import java.util.TreeMap;
* zookeeper exception, so callers of these methods must handle appropriately. * zookeeper exception, so callers of these methods must handle appropriately.
* If ZK is required for the operation, the server will need to be aborted. * If ZK is required for the operation, the server will need to be aborted.
*/ */
@InterfaceAudience.Public @InterfaceAudience.Private
@InterfaceStability.Evolving
public class ZKUtil { public class ZKUtil {
private static final Log LOG = LogFactory.getLog(ZKUtil.class); 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 * Note that the watcher will be blocked when invoking methods in listeners so
* they must not be long-running. * they must not be long-running.
*/ */
@InterfaceAudience.Public @InterfaceAudience.Private
@InterfaceStability.Evolving
public abstract class ZooKeeperListener { public abstract class ZooKeeperListener {
// Reference to the zk watcher which also contains configuration and constants // 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.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.Abortable;
import org.apache.zookeeper.KeeperException; 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 * <p>This is the base class used by trackers in both the Master and
* RegionServers. * RegionServers.
*/ */
@InterfaceAudience.Public @InterfaceAudience.Private
@InterfaceStability.Evolving
public abstract class ZooKeeperNodeTracker extends ZooKeeperListener { public abstract class ZooKeeperNodeTracker extends ZooKeeperListener {
static final Log LOG = LogFactory.getLog(ZooKeeperNodeTracker.class); static final Log LOG = LogFactory.getLog(ZooKeeperNodeTracker.class);

View File

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