HBASE-11858 Add InterfaceAudience for classes missing it in regionserver packages.

* RpcSchedulerFactory and PriorityFunction scoped to Coproc / Phoenix based on SimpleRpcSchedulerFactory and RpcScheduler
* RegionCoprocessorHost, RegionServerCoprocessorHost set to Coproc based on their need for the Observers that are in the book section on coprocessors
* RowProcessor and BaseRowProcessor scoped to Coproc based on RowProcessorClient
* RowTooBigException scope to Public because the book says it will be thrown by the client

Other modified classes set expressly to private.

Signed-off-by: Andrew Purtell <apurtell@apache.org>
This commit is contained in:
Sean Busbey 2014-08-28 22:36:19 -05:00 committed by Andrew Purtell
parent cf69a13f42
commit 8b5582370e
24 changed files with 69 additions and 4 deletions

View File

@ -18,11 +18,16 @@
package org.apache.hadoop.hbase.ipc;
import com.google.protobuf.Message;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
/**
* Function to figure priority of incoming request.
*/
@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
@InterfaceStability.Evolving
public interface PriorityFunction {
/**
* Returns the 'priority type' of the specified request.
@ -41,4 +46,4 @@ public interface PriorityFunction {
* @return Deadline of this request. 0 now, otherwise msec of 'delay'
*/
long getDeadline(RequestHeader header, Message param);
}
}

View File

@ -23,6 +23,7 @@ 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.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ipc.PriorityFunction;
@ -66,6 +67,7 @@ import com.google.protobuf.TextFormat;
//All the argument classes declare a 'getRegion' method that returns a
//RegionSpecifier object. Methods can be invoked on the returned object
//to figure out whether it is a meta region or not.
@InterfaceAudience.Private
class AnnotationReadingPriorityFunction implements PriorityFunction {
public static final Log LOG =
LogFactory.getLog(AnnotationReadingPriorityFunction.class.getName());

View File

@ -22,6 +22,9 @@ import java.util.ArrayList;
import java.util.List;
import java.util.UUID;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
@ -30,6 +33,8 @@ import com.google.protobuf.Message;
/**
* Base class for RowProcessor with some default implementations.
*/
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
@InterfaceStability.Evolving
public abstract class BaseRowProcessor<S extends Message,T extends Message>
implements RowProcessor<S,T> {

View File

@ -24,6 +24,7 @@ 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.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
@ -32,6 +33,7 @@ import org.apache.hadoop.util.StringUtils;
/**
* Default implementation of StoreFlusher.
*/
@InterfaceAudience.Private
public class DefaultStoreFlusher extends StoreFlusher {
private static final Log LOG = LogFactory.getLog(DefaultStoreFlusher.class);
private final Object flushLock = new Object();

View File

@ -18,12 +18,15 @@
package org.apache.hadoop.hbase.regionserver;
import org.apache.hadoop.classification.InterfaceAudience;
/**
* A {@link RegionSplitPolicy} that disables region splits.
* This should be used with care, since it will disable automatic sharding.
* Most of the time, using {@link ConstantSizeRegionSplitPolicy} with a
* large region size (10GB, etc) is safer.
*/
@InterfaceAudience.Private
public class DisabledRegionSplitPolicy extends RegionSplitPolicy {
@Override
protected boolean shouldSplit() {

View File

@ -22,6 +22,7 @@ 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.conf.Configuration;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HConstants;
@ -37,6 +38,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
* 3^3 * 128M*2 = 6912M, and so on until we reach the configured
* maximum filesize and then from there on out, we'll use that.
*/
@InterfaceAudience.Private
public class IncreasingToUpperBoundRegionSplitPolicy
extends ConstantSizeRegionSplitPolicy {
static final Log LOG =

View File

@ -27,11 +27,13 @@ import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import org.apache.commons.math.stat.descriptive.DescriptiveStatistics;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.metrics2.MetricsExecutor;
@InterfaceAudience.Private
public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable {
public static final int PERIOD = 45;

View File

@ -23,6 +23,7 @@ import java.util.Collection;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.KeyValue;
@ -39,6 +40,7 @@ import org.apache.hadoop.hbase.util.Bytes;
/**
* A <code>MultiRowProcessor</code> that performs multiple puts and deletes.
*/
@InterfaceAudience.Private
class MultiRowMutationProcessor extends BaseRowProcessor<MultiRowMutationProcessorRequest,
MultiRowMutationProcessorResponse> {
Collection<byte[]> rowsToLock;

View File

@ -37,6 +37,8 @@ import org.apache.commons.collections.map.ReferenceMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.math.stat.descriptive.DescriptiveStatistics;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -44,6 +46,7 @@ import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.client.Append;
@ -84,6 +87,8 @@ import com.google.protobuf.Service;
* Implements the coprocessor environment and runtime support for coprocessors
* loaded within a {@link HRegion}.
*/
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
@InterfaceStability.Evolving
public class RegionCoprocessorHost
extends CoprocessorHost<RegionCoprocessorHost.RegionEnvironment> {

View File

@ -22,9 +22,12 @@ import java.io.IOException;
import java.util.Comparator;
import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.MetaMutationAnnotation;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
@ -32,6 +35,8 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment;
import org.apache.hadoop.hbase.coprocessor.RegionServerObserver;
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
@InterfaceStability.Evolving
public class RegionServerCoprocessorHost extends
CoprocessorHost<RegionServerCoprocessorHost.RegionServerEnvironment> {

View File

@ -23,14 +23,14 @@ import java.util.List;
import java.util.UUID;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import com.google.protobuf.Message;
@InterfaceAudience.Private
/**
* Defines the procedure to atomically perform multiple scans and mutations
* on a HRegion.
@ -44,6 +44,8 @@ import com.google.protobuf.Message;
* parameter S is the type of the request data sent to the server.
* The generic type parameter T is the return type of RowProcessor.getResult().
*/
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
@InterfaceStability.Evolving
public interface RowProcessor<S extends Message, T extends Message> {
/**
@ -154,4 +156,4 @@ public interface RowProcessor<S extends Message, T extends Message> {
* @return The {@link Durability} to use
*/
Durability useDurability();
}
}

View File

@ -18,6 +18,7 @@
*/
package org.apache.hadoop.hbase.regionserver;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.RegionException;
/**
@ -25,6 +26,7 @@ import org.apache.hadoop.hbase.RegionException;
* set and row size appears to exceed max configured size (configurable via
* hbase.table.max.rowsize).
*/
@InterfaceAudience.Public
public class RowTooBigException extends RegionException {
public RowTooBigException(String message) {

View File

@ -18,12 +18,17 @@
package org.apache.hadoop.hbase.regionserver;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.ipc.PriorityFunction;
import org.apache.hadoop.hbase.ipc.RpcScheduler;
/**
* A factory class that constructs an {@link org.apache.hadoop.hbase.ipc.RpcScheduler}.
*/
@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
@InterfaceStability.Evolving
public interface RpcSchedulerFactory {
/**

View File

@ -22,9 +22,12 @@ import java.io.IOException;
import java.util.Collection;
import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
/**
* Utility functions for region server storage layer.
*/
@InterfaceAudience.Private
public class StoreUtils {
/**
* Creates a deterministic hash code for store file collection.

View File

@ -25,6 +25,7 @@ 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.hbase.Chore;
import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner;
@ -40,6 +41,7 @@ import org.apache.hadoop.util.StringUtils;
* will stop serving read requests because the referenced files might have been deleted (by the
* primary region).
*/
@InterfaceAudience.Private
public class StorefileRefresherChore extends Chore {
private static final Log LOG = LogFactory.getLog(StorefileRefresherChore.class);

View File

@ -24,6 +24,7 @@ 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.fs.Path;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
@ -33,6 +34,7 @@ import org.apache.hadoop.hbase.util.Bytes;
/**
* Base class for cell sink that separates the provided cells into multiple files.
*/
@InterfaceAudience.Private
public abstract class StripeMultiFileWriter implements Compactor.CellSink {
private static final Log LOG = LogFactory.getLog(StripeMultiFileWriter.class);

View File

@ -26,6 +26,7 @@ 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.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
@ -39,6 +40,7 @@ import com.google.common.annotations.VisibleForTesting;
* Stripe implementation of StoreFlusher. Flushes files either into L0 file w/o metadata, or
* into separate striped files, avoiding L0.
*/
@InterfaceAudience.Private
public class StripeStoreFlusher extends StoreFlusher {
private static final Log LOG = LogFactory.getLog(StripeStoreFlusher.class);
private final Object flushLock = new Object();

View File

@ -26,6 +26,7 @@ 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.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.KeyValue;
@ -45,6 +46,7 @@ import org.apache.hadoop.hbase.util.Bytes;
* This is the placeholder for stripe compactor. The implementation,
* as well as the proper javadoc, will be added in HBASE-7967.
*/
@InterfaceAudience.Private
public class StripeCompactor extends Compactor {
private static final Log LOG = LogFactory.getLog(StripeCompactor.class);
public StripeCompactor(Configuration conf, Store store) {

View File

@ -27,6 +27,7 @@ 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.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
@ -37,6 +38,7 @@ import org.apache.hadoop.hbase.regionserver.wal.HLog.Writer;
import org.apache.hadoop.hbase.util.CancelableProgressable;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@InterfaceAudience.Private
public class HLogFactory {
private static final Log LOG = LogFactory.getLog(HLogFactory.class);

View File

@ -28,6 +28,7 @@ import java.util.regex.Pattern;
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.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@ -45,6 +46,7 @@ import org.apache.hadoop.hbase.util.FSUtils;
import com.google.protobuf.TextFormat;
@InterfaceAudience.Private
public class HLogUtil {
static final Log LOG = LogFactory.getLog(HLogUtil.class);

View File

@ -22,6 +22,7 @@ import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.WritableUtils;
@ -36,6 +37,7 @@ import org.apache.hadoop.io.WritableUtils;
* qualifier. More may be added depending on use patterns.
*/
@Deprecated
@InterfaceAudience.Private
class KeyValueCompression {
/**
* Uncompresses a KeyValue from a DataInput and returns it.

View File

@ -26,6 +26,7 @@ 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.fs.FSDataInputStream;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.io.crypto.Cipher;
@ -36,6 +37,7 @@ import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.WALHdrResult;
import org.apache.hadoop.hbase.security.EncryptionUtil;
import org.apache.hadoop.hbase.security.User;
@InterfaceAudience.Private
public class SecureProtobufLogReader extends ProtobufLogReader {
private static final Log LOG = LogFactory.getLog(SecureProtobufLogReader.class);

View File

@ -28,6 +28,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.io.crypto.Cipher;
import org.apache.hadoop.hbase.io.crypto.Encryption;
@ -36,6 +37,7 @@ import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALHeader;
import org.apache.hadoop.hbase.security.EncryptionUtil;
import org.apache.hadoop.hbase.security.User;
@InterfaceAudience.Private
public class SecureProtobufLogWriter extends ProtobufLogWriter {
private static final Log LOG = LogFactory.getLog(SecureProtobufLogWriter.class);

View File

@ -26,6 +26,7 @@ import java.io.OutputStream;
import java.security.SecureRandom;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.KeyValue;
@ -39,6 +40,7 @@ import org.apache.hadoop.hbase.util.Bytes;
/**
* A WALCellCodec that encrypts the WALedits.
*/
@InterfaceAudience.Private
public class SecureWALCellCodec extends WALCellCodec {
private Encryptor encryptor;