HBASE-27203 Clean up error-prone findings in hbase-client [branch-2] (#4651)
Signed-off-by: Duo Zhang <zhangduo@apache.org> Signed-off-by: Viraj Jasani <vjasani@apache.org>
This commit is contained in:
parent
5f231d3fb3
commit
29f51c05f0
|
@ -101,11 +101,7 @@ public class AsyncMetaTableAccessor {
|
|||
return future;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the HRegionLocation from meta for the given region n * @param regionName region we're
|
||||
* looking for
|
||||
* @return HRegionLocation for the given region
|
||||
*/
|
||||
/** Returns the HRegionLocation from meta for the given region */
|
||||
public static CompletableFuture<Optional<HRegionLocation>>
|
||||
getRegionLocation(AsyncTable<?> metaTable, byte[] regionName) {
|
||||
CompletableFuture<Optional<HRegionLocation>> future = new CompletableFuture<>();
|
||||
|
@ -127,11 +123,7 @@ public class AsyncMetaTableAccessor {
|
|||
return future;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the HRegionLocation from meta for the given encoded region name n * @param
|
||||
* encodedRegionName region we're looking for
|
||||
* @return HRegionLocation for the given region
|
||||
*/
|
||||
/** Returns the HRegionLocation from meta for the given encoded region name */
|
||||
public static CompletableFuture<Optional<HRegionLocation>>
|
||||
getRegionLocationWithEncodedName(AsyncTable<?> metaTable, byte[] encodedRegionName) {
|
||||
CompletableFuture<Optional<HRegionLocation>> future = new CompletableFuture<>();
|
||||
|
@ -176,8 +168,9 @@ public class AsyncMetaTableAccessor {
|
|||
}
|
||||
|
||||
/**
|
||||
* Used to get all region locations for the specific table. n * @param tableName table we're
|
||||
* looking for, can be null for getting all regions
|
||||
* Used to get all region locations for the specific table
|
||||
* @param metaTable scanner over meta table
|
||||
* @param tableName table we're looking for, can be null for getting all regions
|
||||
* @return the list of region locations. The return value will be wrapped by a
|
||||
* {@link CompletableFuture}.
|
||||
*/
|
||||
|
@ -200,8 +193,9 @@ public class AsyncMetaTableAccessor {
|
|||
}
|
||||
|
||||
/**
|
||||
* Used to get table regions' info and server. n * @param tableName table we're looking for, can
|
||||
* be null for getting all regions
|
||||
* Used to get table regions' info and server.
|
||||
* @param metaTable scanner over meta table
|
||||
* @param tableName table we're looking for, can be null for getting all regions
|
||||
* @param excludeOfflinedSplitParents don't return split parents
|
||||
* @return the list of regioninfos and server. The return value will be wrapped by a
|
||||
* {@link CompletableFuture}.
|
||||
|
@ -259,9 +253,11 @@ public class AsyncMetaTableAccessor {
|
|||
}
|
||||
|
||||
/**
|
||||
* Performs a scan of META table for given table. n * @param tableName table withing we scan
|
||||
* @param type scanned part of meta
|
||||
* @param visitor Visitor invoked against each row
|
||||
* Performs a scan of META table for given table.
|
||||
* @param metaTable scanner over meta table
|
||||
* @param tableName table within we scan
|
||||
* @param type scanned part of meta
|
||||
* @param visitor Visitor invoked against each row
|
||||
*/
|
||||
private static CompletableFuture<Void> scanMeta(AsyncTable<AdvancedScanResultConsumer> metaTable,
|
||||
TableName tableName, QueryType type, final Visitor visitor) {
|
||||
|
@ -270,11 +266,13 @@ public class AsyncMetaTableAccessor {
|
|||
}
|
||||
|
||||
/**
|
||||
* Performs a scan of META table for given table. n * @param startRow Where to start the scan
|
||||
* @param stopRow Where to stop the scan
|
||||
* @param type scanned part of meta
|
||||
* @param maxRows maximum rows to return
|
||||
* @param visitor Visitor invoked against each row
|
||||
* Performs a scan of META table for given table.
|
||||
* @param metaTable scanner over meta table
|
||||
* @param startRow Where to start the scan
|
||||
* @param stopRow Where to stop the scan
|
||||
* @param type scanned part of meta
|
||||
* @param maxRows maximum rows to return
|
||||
* @param visitor Visitor invoked against each row
|
||||
*/
|
||||
private static CompletableFuture<Void> scanMeta(AsyncTable<AdvancedScanResultConsumer> metaTable,
|
||||
byte[] startRow, byte[] stopRow, QueryType type, int maxRows, final Visitor visitor) {
|
||||
|
@ -410,9 +408,13 @@ public class AsyncMetaTableAccessor {
|
|||
* can't deserialize the result.
|
||||
*/
|
||||
private static Optional<RegionLocations> getRegionLocations(final Result r) {
|
||||
if (r == null) return Optional.empty();
|
||||
if (r == null) {
|
||||
return Optional.empty();
|
||||
}
|
||||
Optional<RegionInfo> regionInfo = getHRegionInfo(r, getRegionInfoColumn());
|
||||
if (!regionInfo.isPresent()) return Optional.empty();
|
||||
if (!regionInfo.isPresent()) {
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
List<HRegionLocation> locations = new ArrayList<HRegionLocation>(1);
|
||||
NavigableMap<byte[], NavigableMap<byte[], byte[]>> familyMap = r.getNoVersionMap();
|
||||
|
@ -420,15 +422,18 @@ public class AsyncMetaTableAccessor {
|
|||
locations.add(getRegionLocation(r, regionInfo.get(), 0));
|
||||
|
||||
NavigableMap<byte[], byte[]> infoMap = familyMap.get(getCatalogFamily());
|
||||
if (infoMap == null) return Optional.of(new RegionLocations(locations));
|
||||
if (infoMap == null) {
|
||||
return Optional.of(new RegionLocations(locations));
|
||||
}
|
||||
|
||||
// iterate until all serverName columns are seen
|
||||
int replicaId = 0;
|
||||
byte[] serverColumn = getServerColumn(replicaId);
|
||||
SortedMap<byte[], byte[]> serverMap = null;
|
||||
serverMap = infoMap.tailMap(serverColumn, false);
|
||||
SortedMap<byte[], byte[]> serverMap = infoMap.tailMap(serverColumn, false);
|
||||
|
||||
if (serverMap.isEmpty()) return Optional.of(new RegionLocations(locations));
|
||||
if (serverMap.isEmpty()) {
|
||||
return Optional.of(new RegionLocations(locations));
|
||||
}
|
||||
|
||||
for (Map.Entry<byte[], byte[]> entry : serverMap.entrySet()) {
|
||||
replicaId = parseReplicaIdFromServerColumn(entry.getKey());
|
||||
|
|
|
@ -51,6 +51,7 @@ public class ClusterId {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse the serialized representation of the {@link ClusterId}
|
||||
* @param bytes A pb serialized {@link ClusterId} instance with pb magic prefix
|
||||
* @return An instance of {@link ClusterId} made from <code>bytes</code> n * @see #toByteArray()
|
||||
*/
|
||||
|
|
|
@ -67,11 +67,11 @@ public final class ClusterMetricsBuilder {
|
|||
.collect(Collectors.toList()))
|
||||
.addAllTableRegionStatesCount(metrics.getTableRegionStatesCount().entrySet().stream()
|
||||
.map(status -> ClusterStatusProtos.TableRegionStatesCount.newBuilder()
|
||||
.setTableName(ProtobufUtil.toProtoTableName((status.getKey())))
|
||||
.setTableName(ProtobufUtil.toProtoTableName(status.getKey()))
|
||||
.setRegionStatesCount(ProtobufUtil.toTableRegionStatesCount(status.getValue())).build())
|
||||
.collect(Collectors.toList()));
|
||||
if (metrics.getMasterName() != null) {
|
||||
builder.setMaster(ProtobufUtil.toServerName((metrics.getMasterName())));
|
||||
builder.setMaster(ProtobufUtil.toServerName(metrics.getMasterName()));
|
||||
}
|
||||
if (metrics.getMasterTasks() != null) {
|
||||
builder.addAllMasterTasks(metrics.getMasterTasks().stream()
|
||||
|
|
|
@ -186,6 +186,7 @@ public class ClusterStatus implements ClusterMetrics {
|
|||
}
|
||||
|
||||
/** Returns the HBase version string as reported by the HMaster */
|
||||
@Override
|
||||
public String getHBaseVersion() {
|
||||
return metrics.getHBaseVersion();
|
||||
}
|
||||
|
@ -279,6 +280,7 @@ public class ClusterStatus implements ClusterMetrics {
|
|||
return serverMetrics == null ? null : new ServerLoad(serverMetrics);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getClusterId() {
|
||||
return metrics.getClusterId();
|
||||
}
|
||||
|
@ -289,6 +291,7 @@ public class ClusterStatus implements ClusterMetrics {
|
|||
}
|
||||
|
||||
/**
|
||||
* Get the list of master coprocessor names.
|
||||
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
|
||||
* {@link #getMasterCoprocessorNames} instead.
|
||||
*/
|
||||
|
@ -299,6 +302,7 @@ public class ClusterStatus implements ClusterMetrics {
|
|||
}
|
||||
|
||||
/**
|
||||
* Get the last major compaction time for a given table.
|
||||
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
|
||||
* {@link #getLastMajorCompactionTimestamp(TableName)} instead.
|
||||
*/
|
||||
|
@ -308,6 +312,7 @@ public class ClusterStatus implements ClusterMetrics {
|
|||
}
|
||||
|
||||
/**
|
||||
* Get the last major compaction time for a given region.
|
||||
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
|
||||
* {@link #getLastMajorCompactionTimestamp(byte[])} instead.
|
||||
*/
|
||||
|
@ -317,6 +322,7 @@ public class ClusterStatus implements ClusterMetrics {
|
|||
}
|
||||
|
||||
/**
|
||||
* Returns true if the balancer is enabled.
|
||||
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 No flag in 2.0
|
||||
*/
|
||||
@Deprecated
|
||||
|
|
|
@ -44,8 +44,8 @@ public interface CoprocessorEnvironment<C extends Coprocessor> {
|
|||
int getLoadSequence();
|
||||
|
||||
/**
|
||||
* @return a Read-only Configuration; throws {@link UnsupportedOperationException} if you try to
|
||||
* set a configuration.
|
||||
* Returns a Read-only Configuration; throws {@link UnsupportedOperationException} if you try to
|
||||
* set a configuration.
|
||||
*/
|
||||
Configuration getConfiguration();
|
||||
|
||||
|
|
|
@ -44,10 +44,7 @@ public class HBaseServerException extends HBaseIOException {
|
|||
this.serverOverloaded = serverOverloaded;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param t throwable to check for server overloaded state
|
||||
* @return True if the server was considered overloaded when the exception was thrown
|
||||
*/
|
||||
/** Returns True if the server was considered overloaded when the exception was thrown */
|
||||
public static boolean isServerOverloaded(Throwable t) {
|
||||
if (t instanceof HBaseServerException) {
|
||||
return ((HBaseServerException) t).isServerOverloaded();
|
||||
|
|
|
@ -176,6 +176,7 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
|
|||
}
|
||||
|
||||
/**
|
||||
* Check if a given family name is allowed.
|
||||
* @param b Family name.
|
||||
* @return <code>b</code>
|
||||
* @throws IllegalArgumentException If not null and not a legitimate family name: i.e. 'printable'
|
||||
|
@ -205,19 +206,12 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
|
|||
return delegatee.getNameAsString();
|
||||
}
|
||||
|
||||
/**
|
||||
* @param key The key.
|
||||
* @return The value.
|
||||
*/
|
||||
@Override
|
||||
public byte[] getValue(byte[] key) {
|
||||
return delegatee.getValue(key);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param key The key.
|
||||
* @return The value as a string.
|
||||
*/
|
||||
@Override
|
||||
public String getValue(String key) {
|
||||
byte[] value = getValue(Bytes.toBytes(key));
|
||||
return value == null ? null : Bytes.toString(value);
|
||||
|
@ -228,35 +222,22 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
|
|||
return delegatee.getValues();
|
||||
}
|
||||
|
||||
/**
|
||||
* @param key The key.
|
||||
* @param value The value.
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
public HColumnDescriptor setValue(byte[] key, byte[] value) {
|
||||
getDelegateeForModification().setValue(key, value);
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param key Key whose key and value we're to remove from HCD parameters.
|
||||
*/
|
||||
public void remove(final byte[] key) {
|
||||
getDelegateeForModification().removeValue(new Bytes(key));
|
||||
}
|
||||
|
||||
/**
|
||||
* @param key The key.
|
||||
* @param value The value.
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
public HColumnDescriptor setValue(String key, String value) {
|
||||
getDelegateeForModification().setValue(key, value);
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return compression type being used for the column family
|
||||
* Returns compression type being used for the column family
|
||||
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
|
||||
* (<a href="https://issues.apache.org/jira/browse/HBASE-13655">HBASE-13655</a>). Use
|
||||
* {@link #getCompressionType()}.
|
||||
|
@ -267,7 +248,7 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
|
|||
}
|
||||
|
||||
/**
|
||||
* @return compression type being used for the column family for major compaction
|
||||
* Returns compression type being used for the column family for major compaction
|
||||
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
|
||||
* (<a href="https://issues.apache.org/jira/browse/HBASE-13655">HBASE-13655</a>). Use
|
||||
* {@link #getCompactionCompressionType()}.
|
||||
|
@ -283,6 +264,7 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
|
|||
}
|
||||
|
||||
/**
|
||||
* Set maximum versions to keep
|
||||
* @param value maximum number of versions
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
|
@ -320,6 +302,7 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
|
|||
}
|
||||
|
||||
/**
|
||||
* Set block size to use when writing
|
||||
* @param value Blocksize to use when writing out storefiles/hfiles on this column family.
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
|
@ -423,6 +406,7 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
|
|||
}
|
||||
|
||||
/**
|
||||
* Set or clear the in memory flag.
|
||||
* @param value True if we are to favor keeping all values for this column family in the
|
||||
* HRegionServer cache
|
||||
* @return this (for chained invocation)
|
||||
|
@ -438,6 +422,7 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
|
|||
}
|
||||
|
||||
/**
|
||||
* Set the in memory compaction policy.
|
||||
* @param value the prefered in-memory compaction policy for this column family
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
|
@ -452,6 +437,7 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
|
|||
}
|
||||
|
||||
/**
|
||||
* Set the keep deleted cells policy.
|
||||
* @param value True if deleted rows should not be collected immediately.
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
|
@ -481,6 +467,7 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
|
|||
}
|
||||
|
||||
/**
|
||||
* Set the time to live of cell contents
|
||||
* @param value Time-to-live of cell contents, in seconds.
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
|
@ -490,6 +477,7 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
|
|||
}
|
||||
|
||||
/**
|
||||
* Set the time to live of cell contents
|
||||
* @param value Time to live of cell contents, in human readable format
|
||||
* @see org.apache.hadoop.hbase.util.PrettyPrinter#format(String, Unit)
|
||||
* @return this (for chained invocation)
|
||||
|
@ -505,6 +493,7 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
|
|||
}
|
||||
|
||||
/**
|
||||
* Set the minimum number of versions to keep.
|
||||
* @param value The minimum number of versions to keep. (used when timeToLive is set)
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
|
@ -519,6 +508,7 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
|
|||
}
|
||||
|
||||
/**
|
||||
* Set or clear the block cache enabled flag.
|
||||
* @param value True if hfile DATA type blocks should be cached (We always cache INDEX and BLOOM
|
||||
* blocks; you cannot turn this off).
|
||||
* @return this (for chained invocation)
|
||||
|
@ -534,6 +524,7 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
|
|||
}
|
||||
|
||||
/**
|
||||
* Set the bloom filter type.
|
||||
* @param value bloom filter type
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
|
@ -547,10 +538,6 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
|
|||
return delegatee.getScope();
|
||||
}
|
||||
|
||||
/**
|
||||
* @param value the scope tag
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
public HColumnDescriptor setScope(int value) {
|
||||
getDelegateeForModification().setScope(value);
|
||||
return this;
|
||||
|
@ -562,6 +549,7 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
|
|||
}
|
||||
|
||||
/**
|
||||
* Set or clear the cache data on write flag.
|
||||
* @param value true if we should cache data blocks on write
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
|
@ -571,7 +559,7 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
|
|||
}
|
||||
|
||||
/**
|
||||
* This is a noop call from HBase 2.0 onwards
|
||||
* Set or clear the cache in L1 flag. This is a noop call from HBase 2.0 onwards
|
||||
* @return this (for chained invocation)
|
||||
* @deprecated Since 2.0 and will be removed in 3.0 with out any replacement. Caching data in on
|
||||
* heap Cache, when there are both on heap LRU Cache and Bucket Cache will no longer
|
||||
|
@ -588,6 +576,7 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
|
|||
}
|
||||
|
||||
/**
|
||||
* Set or clear the cache indexes on write flag.
|
||||
* @param value true if we should cache index blocks on write
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
|
@ -602,6 +591,7 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
|
|||
}
|
||||
|
||||
/**
|
||||
* Set or clear the cache bloom filters on write flag.
|
||||
* @param value true if we should cache bloomfilter blocks on write
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
|
@ -616,6 +606,7 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
|
|||
}
|
||||
|
||||
/**
|
||||
* Set or clear the evict bloom filters on close flag.
|
||||
* @param value true if we should evict cached blocks from the blockcache on close
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
|
@ -630,6 +621,7 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
|
|||
}
|
||||
|
||||
/**
|
||||
* Set or clear the prefetch on open flag.
|
||||
* @param value true if we should prefetch blocks into the blockcache on open
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
|
@ -638,9 +630,6 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
|
|||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* @see java.lang.Object#toString()
|
||||
*/
|
||||
@Override
|
||||
public String toString() {
|
||||
return delegatee.toString();
|
||||
|
@ -660,9 +649,6 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
|
|||
return ColumnFamilyDescriptorBuilder.getDefaultValues();
|
||||
}
|
||||
|
||||
/**
|
||||
* @see java.lang.Object#equals(java.lang.Object)
|
||||
*/
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (this == obj) {
|
||||
|
@ -674,9 +660,6 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
|
|||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* @see java.lang.Object#hashCode()
|
||||
*/
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return delegatee.hashCode();
|
||||
|
@ -688,7 +671,7 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
|
|||
}
|
||||
|
||||
/**
|
||||
* @return This instance serialized with pb with pb magic prefix
|
||||
* Returns This instance serialized with pb with pb magic prefix
|
||||
* @see #parseFrom(byte[])
|
||||
*/
|
||||
public byte[] toByteArray() {
|
||||
|
@ -696,6 +679,7 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse a serialized representation of a {@link HColumnDescriptor}
|
||||
* @param bytes A pb serialized {@link HColumnDescriptor} instance with pb magic prefix
|
||||
* @return An instance of {@link HColumnDescriptor} made from <code>bytes</code> n * @see
|
||||
* #toByteArray()
|
||||
|
|
|
@ -221,7 +221,7 @@ public class HRegionInfo implements RegionInfo {
|
|||
|
||||
/**
|
||||
* Construct HRegionInfo with explicit parameters
|
||||
* @param tableName the table descriptor
|
||||
* @param tableName the table name
|
||||
* @param startKey first key in region
|
||||
* @param endKey end of key range
|
||||
* @param split true if this region has split and we have daughter regions regions that may or
|
||||
|
@ -234,37 +234,37 @@ public class HRegionInfo implements RegionInfo {
|
|||
|
||||
/**
|
||||
* Construct HRegionInfo with explicit parameters
|
||||
* @param tableName the table descriptor
|
||||
* @param tableName the table name
|
||||
* @param startKey first key in region
|
||||
* @param endKey end of key range
|
||||
* @param split true if this region has split and we have daughter regions regions that may or
|
||||
* may not hold references to this region.
|
||||
* @param regionid Region id to use. n
|
||||
* @param regionId Region id to use.
|
||||
*/
|
||||
public HRegionInfo(final TableName tableName, final byte[] startKey, final byte[] endKey,
|
||||
final boolean split, final long regionid) throws IllegalArgumentException {
|
||||
this(tableName, startKey, endKey, split, regionid, DEFAULT_REPLICA_ID);
|
||||
final boolean split, final long regionId) throws IllegalArgumentException {
|
||||
this(tableName, startKey, endKey, split, regionId, DEFAULT_REPLICA_ID);
|
||||
}
|
||||
|
||||
/**
|
||||
* Construct HRegionInfo with explicit parameters
|
||||
* @param tableName the table descriptor
|
||||
* @param tableName the table name
|
||||
* @param startKey first key in region
|
||||
* @param endKey end of key range
|
||||
* @param split true if this region has split and we have daughter regions regions that may or
|
||||
* may not hold references to this region.
|
||||
* @param regionid Region id to use.
|
||||
* @param replicaId the replicaId to use n
|
||||
* @param regionId Region id to use.
|
||||
* @param replicaId the replicaId to use
|
||||
*/
|
||||
public HRegionInfo(final TableName tableName, final byte[] startKey, final byte[] endKey,
|
||||
final boolean split, final long regionid, final int replicaId) throws IllegalArgumentException {
|
||||
final boolean split, final long regionId, final int replicaId) throws IllegalArgumentException {
|
||||
super();
|
||||
if (tableName == null) {
|
||||
throw new IllegalArgumentException("TableName cannot be null");
|
||||
}
|
||||
this.tableName = tableName;
|
||||
this.offLine = false;
|
||||
this.regionId = regionid;
|
||||
this.regionId = regionId;
|
||||
this.replicaId = replicaId;
|
||||
if (this.replicaId > MAX_REPLICA_ID) {
|
||||
throw new IllegalArgumentException("ReplicaId cannot be greater than" + MAX_REPLICA_ID);
|
||||
|
@ -280,7 +280,7 @@ public class HRegionInfo implements RegionInfo {
|
|||
}
|
||||
|
||||
/**
|
||||
* Costruct a copy of another HRegionInfo n
|
||||
* Construct a copy of another HRegionInfo
|
||||
*/
|
||||
public HRegionInfo(RegionInfo other) {
|
||||
super();
|
||||
|
@ -303,8 +303,10 @@ public class HRegionInfo implements RegionInfo {
|
|||
}
|
||||
|
||||
/**
|
||||
* Make a region name of passed parameters. n * @param startKey Can be null
|
||||
* @param regionid Region id (Usually timestamp from when region was created).
|
||||
* Make a region name of passed parameters.
|
||||
* @param tableName the table name
|
||||
* @param startKey Can be null
|
||||
* @param regionId Region id (Usually timestamp from when region was created).
|
||||
* @param newFormat should we create the region name in the new format (such that it contains its
|
||||
* encoded name?).
|
||||
* @return Region name made of passed tableName, startKey and id
|
||||
|
@ -314,12 +316,14 @@ public class HRegionInfo implements RegionInfo {
|
|||
@Deprecated
|
||||
@InterfaceAudience.Private
|
||||
public static byte[] createRegionName(final TableName tableName, final byte[] startKey,
|
||||
final long regionid, boolean newFormat) {
|
||||
return RegionInfo.createRegionName(tableName, startKey, Long.toString(regionid), newFormat);
|
||||
final long regionId, boolean newFormat) {
|
||||
return RegionInfo.createRegionName(tableName, startKey, Long.toString(regionId), newFormat);
|
||||
}
|
||||
|
||||
/**
|
||||
* Make a region name of passed parameters. n * @param startKey Can be null
|
||||
* Make a region name of passed parameters.
|
||||
* @param tableName the table name
|
||||
* @param startKey Can be null
|
||||
* @param id Region id (Usually timestamp from when region was created).
|
||||
* @param newFormat should we create the region name in the new format (such that it contains its
|
||||
* encoded name?).
|
||||
|
@ -335,10 +339,12 @@ public class HRegionInfo implements RegionInfo {
|
|||
}
|
||||
|
||||
/**
|
||||
* Make a region name of passed parameters. n * @param startKey Can be null
|
||||
* @param regionid Region id (Usually timestamp from when region was created). n * @param
|
||||
* newFormat should we create the region name in the new format (such that it
|
||||
* contains its encoded name?).
|
||||
* Make a region name of passed parameters.
|
||||
* @param tableName the table name
|
||||
* @param startKey Can be null
|
||||
* @param regionId Region id (Usually timestamp from when region was created).
|
||||
* @param newFormat should we create the region name in the new format (such that it contains its
|
||||
* encoded name?).
|
||||
* @return Region name made of passed tableName, startKey, id and replicaId
|
||||
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
|
||||
* {@link RegionInfo#createRegionName(TableName, byte[], long, int, boolean)}.
|
||||
|
@ -346,13 +352,15 @@ public class HRegionInfo implements RegionInfo {
|
|||
@Deprecated
|
||||
@InterfaceAudience.Private
|
||||
public static byte[] createRegionName(final TableName tableName, final byte[] startKey,
|
||||
final long regionid, int replicaId, boolean newFormat) {
|
||||
return RegionInfo.createRegionName(tableName, startKey, Bytes.toBytes(Long.toString(regionid)),
|
||||
final long regionId, int replicaId, boolean newFormat) {
|
||||
return RegionInfo.createRegionName(tableName, startKey, Bytes.toBytes(Long.toString(regionId)),
|
||||
replicaId, newFormat);
|
||||
}
|
||||
|
||||
/**
|
||||
* Make a region name of passed parameters. n * @param startKey Can be null
|
||||
* Make a region name of passed parameters.
|
||||
* @param tableName the table name
|
||||
* @param startKey Can be null
|
||||
* @param id Region id (Usually timestamp from when region was created).
|
||||
* @param newFormat should we create the region name in the new format (such that it contains its
|
||||
* encoded name?).
|
||||
|
@ -368,9 +376,11 @@ public class HRegionInfo implements RegionInfo {
|
|||
}
|
||||
|
||||
/**
|
||||
* Make a region name of passed parameters. n * @param startKey Can be null
|
||||
* @param id Region id (Usually timestamp from when region was created). n * @param newFormat
|
||||
* should we create the region name in the new format
|
||||
* Make a region name of passed parameters.
|
||||
* @param tableName the table name
|
||||
* @param startKey Can be null
|
||||
* @param id Region id (Usually timestamp from when region was created)
|
||||
* @param newFormat should we create the region name in the new format
|
||||
* @return Region name made of passed tableName, startKey, id and replicaId
|
||||
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
|
||||
* {@link RegionInfo#createRegionName(TableName, byte[], byte[], int, boolean)}.
|
||||
|
@ -546,6 +556,7 @@ public class HRegionInfo implements RegionInfo {
|
|||
}
|
||||
|
||||
/**
|
||||
* Set or clear the split status flag.
|
||||
* @param split set split status
|
||||
*/
|
||||
public void setSplit(boolean split) {
|
||||
|
@ -684,6 +695,7 @@ public class HRegionInfo implements RegionInfo {
|
|||
}
|
||||
|
||||
/**
|
||||
* Serialize a {@link HRegionInfo} into a byte array.
|
||||
* @return This instance serialized as protobuf w/ a magic pb prefix.
|
||||
* @see #parseFrom(byte[])
|
||||
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
|
||||
|
@ -695,6 +707,7 @@ public class HRegionInfo implements RegionInfo {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse a serialized representation of a {@link HRegionInfo}.
|
||||
* @return A deserialized {@link HRegionInfo} or null if we failed deserialize or passed bytes
|
||||
* null
|
||||
* @see #toByteArray()
|
||||
|
@ -708,6 +721,7 @@ public class HRegionInfo implements RegionInfo {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse a serialized representation of a {@link HRegionInfo}.
|
||||
* @return A deserialized {@link HRegionInfo} or null if we failed deserialize or passed bytes
|
||||
* null
|
||||
* @see #toByteArray()
|
||||
|
@ -725,6 +739,7 @@ public class HRegionInfo implements RegionInfo {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse a serialized representation of a {@link HRegionInfo}.
|
||||
* @param bytes A pb RegionInfo serialized with a pb magic prefix.
|
||||
* @return A deserialized {@link HRegionInfo} n * @see #toByteArray()
|
||||
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
|
||||
|
@ -736,6 +751,7 @@ public class HRegionInfo implements RegionInfo {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse a serialized representation of a {@link HRegionInfo}.
|
||||
* @param bytes A pb RegionInfo serialized with a pb magic prefix.
|
||||
* @param offset starting point in the byte array
|
||||
* @param len length to read on the byte array
|
||||
|
|
|
@ -110,8 +110,8 @@ public class HRegionLocation implements Comparable<HRegionLocation> {
|
|||
}
|
||||
|
||||
/**
|
||||
* @return String made of hostname and port formatted as per
|
||||
* {@link Addressing#createHostAndPortStr(String, int)}
|
||||
* Returns String made of hostname and port formatted as per
|
||||
* {@link Addressing#createHostAndPortStr(String, int)}
|
||||
*/
|
||||
public String getHostnamePort() {
|
||||
return Addressing.createHostAndPortStr(this.getHostname(), this.getPort());
|
||||
|
|
|
@ -801,6 +801,7 @@ public class HTableDescriptor implements TableDescriptor, Comparable<HTableDescr
|
|||
new HTableDescriptor(TableDescriptorBuilder.NAMESPACE_TABLEDESC);
|
||||
|
||||
/**
|
||||
* Set the table owner.
|
||||
* @deprecated since 0.94.1
|
||||
* @see <a href="https://issues.apache.org/jira/browse/HBASE-6188">HBASE-6188</a>
|
||||
*/
|
||||
|
@ -811,6 +812,7 @@ public class HTableDescriptor implements TableDescriptor, Comparable<HTableDescr
|
|||
}
|
||||
|
||||
/**
|
||||
* Set the table owner.
|
||||
* @deprecated since 0.94.1
|
||||
* @see <a href="https://issues.apache.org/jira/browse/HBASE-6188">HBASE-6188</a>
|
||||
*/
|
||||
|
@ -822,6 +824,7 @@ public class HTableDescriptor implements TableDescriptor, Comparable<HTableDescr
|
|||
}
|
||||
|
||||
/**
|
||||
* Get the table owner.
|
||||
* @deprecated since 0.94.1
|
||||
* @see <a href="https://issues.apache.org/jira/browse/HBASE-6188">HBASE-6188</a>
|
||||
*/
|
||||
|
@ -832,14 +835,14 @@ public class HTableDescriptor implements TableDescriptor, Comparable<HTableDescr
|
|||
}
|
||||
|
||||
/**
|
||||
* @return This instance serialized with pb with pb magic prefix
|
||||
* @see #parseFrom(byte[])
|
||||
* Returns This instance serialized with pb with pb magic prefix
|
||||
*/
|
||||
public byte[] toByteArray() {
|
||||
return TableDescriptorBuilder.toByteArray(delegatee);
|
||||
}
|
||||
|
||||
/**
|
||||
* Parse the serialized representation of a {@link HTableDescriptor}
|
||||
* @param bytes A pb serialized {@link HTableDescriptor} instance with pb magic prefix
|
||||
* @return An instance of {@link HTableDescriptor} made from <code>bytes</code> nn * @see
|
||||
* #toByteArray()
|
||||
|
|
|
@ -38,7 +38,6 @@ import java.util.regex.Matcher;
|
|||
import java.util.regex.Pattern;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Cell.Type;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.Consistency;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
|
@ -155,6 +154,7 @@ public class MetaTableAccessor {
|
|||
private static final byte SEPARATED_BYTE = 0x00;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
@SuppressWarnings("ImmutableEnumChecker")
|
||||
public enum QueryType {
|
||||
ALL(HConstants.TABLE_FAMILY, HConstants.CATALOG_FAMILY),
|
||||
REGION(HConstants.CATALOG_FAMILY),
|
||||
|
@ -364,8 +364,8 @@ public class MetaTableAccessor {
|
|||
}
|
||||
|
||||
/**
|
||||
* @return Return all regioninfos listed in the 'info:merge*' columns of the
|
||||
* <code>regionName</code> row.
|
||||
* Returns Return all regioninfos listed in the 'info:merge*' columns of the
|
||||
* <code>regionName</code> row.
|
||||
*/
|
||||
@Nullable
|
||||
public static List<RegionInfo> getMergeRegions(Connection connection, byte[] regionName)
|
||||
|
@ -381,8 +381,8 @@ public class MetaTableAccessor {
|
|||
}
|
||||
|
||||
/**
|
||||
* @return Deserialized values of <qualifier,regioninfo> pairs taken from column values that
|
||||
* match the regex 'info:merge.*' in array of <code>cells</code>.
|
||||
* Returns Deserialized values of <qualifier,regioninfo> pairs taken from column values that
|
||||
* match the regex 'info:merge.*' in array of <code>cells</code>.
|
||||
*/
|
||||
@Nullable
|
||||
public static Map<String, RegionInfo> getMergeRegionsWithName(Cell[] cells) {
|
||||
|
@ -408,8 +408,8 @@ public class MetaTableAccessor {
|
|||
}
|
||||
|
||||
/**
|
||||
* @return Deserialized regioninfo values taken from column values that match the regex
|
||||
* 'info:merge.*' in array of <code>cells</code>.
|
||||
* Returns Deserialized regioninfo values taken from column values that match the regex
|
||||
* 'info:merge.*' in array of <code>cells</code>.
|
||||
*/
|
||||
@Nullable
|
||||
public static List<RegionInfo> getMergeRegions(Cell[] cells) {
|
||||
|
@ -418,8 +418,8 @@ public class MetaTableAccessor {
|
|||
}
|
||||
|
||||
/**
|
||||
* @return True if any merge regions present in <code>cells</code>; i.e. the column in
|
||||
* <code>cell</code> matches the regex 'info:merge.*'.
|
||||
* Returns True if any merge regions present in <code>cells</code>; i.e. the column in
|
||||
* <code>cell</code> matches the regex 'info:merge.*'.
|
||||
*/
|
||||
public static boolean hasMergeRegions(Cell[] cells) {
|
||||
for (Cell cell : cells) {
|
||||
|
@ -483,6 +483,7 @@ public class MetaTableAccessor {
|
|||
return getListOfRegionInfos(result);
|
||||
}
|
||||
|
||||
@SuppressWarnings("MixedMutabilityReturnType")
|
||||
private static List<RegionInfo>
|
||||
getListOfRegionInfos(final List<Pair<RegionInfo, ServerName>> pairs) {
|
||||
if (pairs == null || pairs.isEmpty()) {
|
||||
|
@ -496,8 +497,7 @@ public class MetaTableAccessor {
|
|||
}
|
||||
|
||||
/**
|
||||
* @param tableName table we're working with
|
||||
* @return start row for scanning META according to query type
|
||||
* Returns start row for scanning META according to query type
|
||||
*/
|
||||
public static byte[] getTableStartRowForMeta(TableName tableName, QueryType type) {
|
||||
if (tableName == null) {
|
||||
|
@ -518,8 +518,7 @@ public class MetaTableAccessor {
|
|||
}
|
||||
|
||||
/**
|
||||
* @param tableName table we're working with
|
||||
* @return stop row for scanning META according to query type
|
||||
* Returns stop row for scanning META according to query type
|
||||
*/
|
||||
public static byte[] getTableStopRowForMeta(TableName tableName, QueryType type) {
|
||||
if (tableName == null) {
|
||||
|
@ -641,9 +640,10 @@ public class MetaTableAccessor {
|
|||
}
|
||||
|
||||
/**
|
||||
* Get the user regions a given server is hosting.
|
||||
* @param connection connection we're using
|
||||
* @param serverName server whose regions we're interested in
|
||||
* @return List of user regions installed on this server (does not include catalog regions). n
|
||||
* @return List of user regions installed on this server (does not include catalog regions).
|
||||
*/
|
||||
public static NavigableMap<RegionInfo, Result> getServerUserRegions(Connection connection,
|
||||
final ServerName serverName) throws IOException {
|
||||
|
@ -1284,7 +1284,7 @@ public class MetaTableAccessor {
|
|||
if (info == null) {
|
||||
return true;
|
||||
}
|
||||
if (!(info.getTable().equals(tableName))) {
|
||||
if (!info.getTable().equals(tableName)) {
|
||||
return false;
|
||||
}
|
||||
return super.visit(rowResult);
|
||||
|
@ -1321,14 +1321,14 @@ public class MetaTableAccessor {
|
|||
if (splitA != null) {
|
||||
put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(put.getRow())
|
||||
.setFamily(HConstants.CATALOG_FAMILY).setQualifier(HConstants.SPLITA_QUALIFIER)
|
||||
.setTimestamp(put.getTimestamp()).setType(Type.Put).setValue(RegionInfo.toByteArray(splitA))
|
||||
.build());
|
||||
.setTimestamp(put.getTimestamp()).setType(Cell.Type.Put)
|
||||
.setValue(RegionInfo.toByteArray(splitA)).build());
|
||||
}
|
||||
if (splitB != null) {
|
||||
put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(put.getRow())
|
||||
.setFamily(HConstants.CATALOG_FAMILY).setQualifier(HConstants.SPLITB_QUALIFIER)
|
||||
.setTimestamp(put.getTimestamp()).setType(Type.Put).setValue(RegionInfo.toByteArray(splitB))
|
||||
.build());
|
||||
.setTimestamp(put.getTimestamp()).setType(Cell.Type.Put)
|
||||
.setValue(RegionInfo.toByteArray(splitB)).build());
|
||||
}
|
||||
return put;
|
||||
}
|
||||
|
@ -1509,8 +1509,8 @@ public class MetaTableAccessor {
|
|||
String qualifier = String.format(HConstants.MERGE_QUALIFIER_PREFIX_STR + "%04d", counter++);
|
||||
put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(put.getRow())
|
||||
.setFamily(HConstants.CATALOG_FAMILY).setQualifier(Bytes.toBytes(qualifier))
|
||||
.setTimestamp(put.getTimestamp()).setType(Type.Put).setValue(RegionInfo.toByteArray(ri))
|
||||
.build());
|
||||
.setTimestamp(put.getTimestamp()).setType(Cell.Type.Put)
|
||||
.setValue(RegionInfo.toByteArray(ri)).build());
|
||||
}
|
||||
return put;
|
||||
}
|
||||
|
@ -1852,7 +1852,7 @@ public class MetaTableAccessor {
|
|||
public static Put addRegionInfo(final Put p, final RegionInfo hri) throws IOException {
|
||||
p.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(p.getRow())
|
||||
.setFamily(getCatalogFamily()).setQualifier(HConstants.REGIONINFO_QUALIFIER)
|
||||
.setTimestamp(p.getTimestamp()).setType(Type.Put)
|
||||
.setTimestamp(p.getTimestamp()).setType(Cell.Type.Put)
|
||||
// Serialize the Default Replica HRI otherwise scan of hbase:meta
|
||||
// shows an info:regioninfo value with encoded name and region
|
||||
// name that differs from that of the hbase;meta row.
|
||||
|
@ -1872,8 +1872,8 @@ public class MetaTableAccessor {
|
|||
.setQualifier(getStartCodeColumn(replicaId)).setTimestamp(p.getTimestamp())
|
||||
.setType(Cell.Type.Put).setValue(Bytes.toBytes(sn.getStartcode())).build())
|
||||
.add(builder.clear().setRow(p.getRow()).setFamily(getCatalogFamily())
|
||||
.setQualifier(getSeqNumColumn(replicaId)).setTimestamp(p.getTimestamp()).setType(Type.Put)
|
||||
.setValue(Bytes.toBytes(openSeqNum)).build());
|
||||
.setQualifier(getSeqNumColumn(replicaId)).setTimestamp(p.getTimestamp())
|
||||
.setType(Cell.Type.Put).setValue(Bytes.toBytes(openSeqNum)).build());
|
||||
}
|
||||
|
||||
private static void writeRegionName(ByteArrayOutputStream out, byte[] regionName) {
|
||||
|
@ -1922,7 +1922,7 @@ public class MetaTableAccessor {
|
|||
byte[] value = getParentsBytes(parents);
|
||||
put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(put.getRow())
|
||||
.setFamily(HConstants.REPLICATION_BARRIER_FAMILY).setQualifier(REPLICATION_PARENT_QUALIFIER)
|
||||
.setTimestamp(put.getTimestamp()).setType(Type.Put).setValue(value).build());
|
||||
.setTimestamp(put.getTimestamp()).setType(Cell.Type.Put).setValue(value).build());
|
||||
}
|
||||
|
||||
public static Put makePutForReplicationBarrier(RegionInfo regionInfo, long openSeqNum, long ts)
|
||||
|
@ -1938,7 +1938,7 @@ public class MetaTableAccessor {
|
|||
public static void addReplicationBarrier(Put put, long openSeqNum) throws IOException {
|
||||
put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(put.getRow())
|
||||
.setFamily(HConstants.REPLICATION_BARRIER_FAMILY).setQualifier(HConstants.SEQNUM_QUALIFIER)
|
||||
.setTimestamp(put.getTimestamp()).setType(Type.Put).setValue(Bytes.toBytes(openSeqNum))
|
||||
.setTimestamp(put.getTimestamp()).setType(Cell.Type.Put).setValue(Bytes.toBytes(openSeqNum))
|
||||
.build());
|
||||
}
|
||||
|
||||
|
@ -1946,8 +1946,8 @@ public class MetaTableAccessor {
|
|||
CellBuilder builder = CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY);
|
||||
return p
|
||||
.add(builder.clear().setRow(p.getRow()).setFamily(getCatalogFamily())
|
||||
.setQualifier(getServerColumn(replicaId)).setTimestamp(p.getTimestamp()).setType(Type.Put)
|
||||
.build())
|
||||
.setQualifier(getServerColumn(replicaId)).setTimestamp(p.getTimestamp())
|
||||
.setType(Cell.Type.Put).build())
|
||||
.add(builder.clear().setRow(p.getRow()).setFamily(getCatalogFamily())
|
||||
.setQualifier(getStartCodeColumn(replicaId)).setTimestamp(p.getTimestamp())
|
||||
.setType(Cell.Type.Put).build())
|
||||
|
@ -2096,7 +2096,7 @@ public class MetaTableAccessor {
|
|||
private static Put addSequenceNum(Put p, long openSeqNum, int replicaId) throws IOException {
|
||||
return p.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(p.getRow())
|
||||
.setFamily(HConstants.CATALOG_FAMILY).setQualifier(getSeqNumColumn(replicaId))
|
||||
.setTimestamp(p.getTimestamp()).setType(Type.Put).setValue(Bytes.toBytes(openSeqNum))
|
||||
.setTimestamp(p.getTimestamp()).setType(Cell.Type.Put).setValue(Bytes.toBytes(openSeqNum))
|
||||
.build());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -354,6 +354,7 @@ public class RegionLoad implements RegionMetrics {
|
|||
}
|
||||
|
||||
/** Returns the reference count for the stores of this region */
|
||||
@Override
|
||||
public int getStoreRefCount() {
|
||||
return metrics.getStoreRefCount();
|
||||
}
|
||||
|
|
|
@ -208,6 +208,7 @@ public class RegionLocations implements Iterable<HRegionLocation> {
|
|||
* @param other the locations to merge with
|
||||
* @return an RegionLocations object with merged locations or the same object if nothing is merged
|
||||
*/
|
||||
@SuppressWarnings("ReferenceEquality")
|
||||
public RegionLocations mergeLocations(RegionLocations other) {
|
||||
assert other != null;
|
||||
|
||||
|
@ -280,6 +281,7 @@ public class RegionLocations implements Iterable<HRegionLocation> {
|
|||
* @return an RegionLocations object with updated locations or the same object if nothing is
|
||||
* updated
|
||||
*/
|
||||
@SuppressWarnings("ReferenceEquality")
|
||||
public RegionLocations updateLocation(HRegionLocation location, boolean checkForEquals,
|
||||
boolean force) {
|
||||
assert location != null;
|
||||
|
|
|
@ -107,8 +107,8 @@ public interface RegionMetrics {
|
|||
int getStoreRefCount();
|
||||
|
||||
/**
|
||||
* @return the max reference count for any store file among all compacted stores files of this
|
||||
* region
|
||||
* Returns the max reference count for any store file among all compacted stores files of this
|
||||
* region
|
||||
*/
|
||||
int getMaxCompactedStoreFileRefCount();
|
||||
|
||||
|
|
|
@ -79,16 +79,17 @@ public class ServerLoad implements ServerMetrics {
|
|||
for (RegionMetrics rl : metrics.getRegionMetrics().values()) {
|
||||
stores += rl.getStoreCount();
|
||||
storefiles += rl.getStoreFileCount();
|
||||
storeUncompressedSizeMB += rl.getUncompressedStoreFileSize().get(Size.Unit.MEGABYTE);
|
||||
storefileSizeMB += rl.getStoreFileSize().get(Size.Unit.MEGABYTE);
|
||||
memstoreSizeMB += rl.getMemStoreSize().get(Size.Unit.MEGABYTE);
|
||||
storeUncompressedSizeMB += (int) rl.getUncompressedStoreFileSize().get(Size.Unit.MEGABYTE);
|
||||
storefileSizeMB += (int) rl.getStoreFileSize().get(Size.Unit.MEGABYTE);
|
||||
memstoreSizeMB += (int) rl.getMemStoreSize().get(Size.Unit.MEGABYTE);
|
||||
readRequestsCount += rl.getReadRequestCount();
|
||||
filteredReadRequestsCount += rl.getFilteredReadRequestCount();
|
||||
writeRequestsCount += rl.getWriteRequestCount();
|
||||
storefileIndexSizeKB += rl.getStoreFileIndexSize().get(Size.Unit.KILOBYTE);
|
||||
rootIndexSizeKB += rl.getStoreFileRootLevelIndexSize().get(Size.Unit.KILOBYTE);
|
||||
totalStaticIndexSizeKB += rl.getStoreFileUncompressedDataIndexSize().get(Size.Unit.KILOBYTE);
|
||||
totalStaticBloomSizeKB += rl.getBloomFilterSize().get(Size.Unit.KILOBYTE);
|
||||
storefileIndexSizeKB += (long) rl.getStoreFileIndexSize().get(Size.Unit.KILOBYTE);
|
||||
rootIndexSizeKB += (int) rl.getStoreFileRootLevelIndexSize().get(Size.Unit.KILOBYTE);
|
||||
totalStaticIndexSizeKB +=
|
||||
(int) rl.getStoreFileUncompressedDataIndexSize().get(Size.Unit.KILOBYTE);
|
||||
totalStaticBloomSizeKB += (int) rl.getBloomFilterSize().get(Size.Unit.KILOBYTE);
|
||||
totalCompactingKVs += rl.getCompactingCellCount();
|
||||
currentCompactedKVs += rl.getCompactedCellCount();
|
||||
}
|
||||
|
|
|
@ -44,10 +44,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
|
|||
@InterfaceAudience.Private
|
||||
public final class ServerMetricsBuilder {
|
||||
|
||||
/**
|
||||
* @param sn the server name
|
||||
* @return a empty metrics
|
||||
*/
|
||||
public static ServerMetrics of(ServerName sn) {
|
||||
return newBuilder(sn).build();
|
||||
}
|
||||
|
@ -280,6 +276,7 @@ public final class ServerMetricsBuilder {
|
|||
return versionNumber;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getVersion() {
|
||||
return version;
|
||||
}
|
||||
|
@ -383,15 +380,17 @@ public final class ServerMetricsBuilder {
|
|||
int currentMaxCompactedStoreFileRefCount = r.getMaxCompactedStoreFileRefCount();
|
||||
maxCompactedStoreFileRefCount =
|
||||
Math.max(maxCompactedStoreFileRefCount, currentMaxCompactedStoreFileRefCount);
|
||||
uncompressedStoreFileSizeMB += r.getUncompressedStoreFileSize().get(Size.Unit.MEGABYTE);
|
||||
storeFileSizeMB += r.getStoreFileSize().get(Size.Unit.MEGABYTE);
|
||||
memStoreSizeMB += r.getMemStoreSize().get(Size.Unit.MEGABYTE);
|
||||
storefileIndexSizeKB += r.getStoreFileUncompressedDataIndexSize().get(Size.Unit.KILOBYTE);
|
||||
uncompressedStoreFileSizeMB +=
|
||||
(long) r.getUncompressedStoreFileSize().get(Size.Unit.MEGABYTE);
|
||||
storeFileSizeMB += (long) r.getStoreFileSize().get(Size.Unit.MEGABYTE);
|
||||
memStoreSizeMB += (long) r.getMemStoreSize().get(Size.Unit.MEGABYTE);
|
||||
storefileIndexSizeKB +=
|
||||
(long) r.getStoreFileUncompressedDataIndexSize().get(Size.Unit.KILOBYTE);
|
||||
readRequestsCount += r.getReadRequestCount();
|
||||
writeRequestsCount += r.getWriteRequestCount();
|
||||
filteredReadRequestsCount += r.getFilteredReadRequestCount();
|
||||
rootLevelIndexSizeKB += r.getStoreFileRootLevelIndexSize().get(Size.Unit.KILOBYTE);
|
||||
bloomFilterSizeMB += r.getBloomFilterSize().get(Size.Unit.MEGABYTE);
|
||||
rootLevelIndexSizeKB += (long) r.getStoreFileRootLevelIndexSize().get(Size.Unit.KILOBYTE);
|
||||
bloomFilterSizeMB += (long) r.getBloomFilterSize().get(Size.Unit.MEGABYTE);
|
||||
compactedCellCount += r.getCompactedCellCount();
|
||||
compactingCellCount += r.getCompactingCellCount();
|
||||
}
|
||||
|
|
|
@ -50,8 +50,8 @@ public interface UserMetrics {
|
|||
long getWriteRequestCount();
|
||||
|
||||
/**
|
||||
* @return the number of write requests and read requests and coprocessor service requests made by
|
||||
* the user
|
||||
* Returns the number of write requests and read requests and coprocessor service requests made by
|
||||
* the user
|
||||
*/
|
||||
default long getRequestCount() {
|
||||
return getReadRequestCount() + getWriteRequestCount();
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import org.apache.hadoop.hbase.util.Strings;
|
||||
|
@ -30,7 +31,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
|
|||
public final class UserMetricsBuilder {
|
||||
|
||||
public static UserMetrics toUserMetrics(ClusterStatusProtos.UserLoad userLoad) {
|
||||
UserMetricsBuilder builder = UserMetricsBuilder.newBuilder(userLoad.getUserName().getBytes());
|
||||
UserMetricsBuilder builder =
|
||||
UserMetricsBuilder.newBuilder(userLoad.getUserName().getBytes(StandardCharsets.UTF_8));
|
||||
userLoad.getClientMetricsList().stream()
|
||||
.map(clientMetrics -> new ClientMetricsImpl(clientMetrics.getHostName(),
|
||||
clientMetrics.getReadRequestsCount(), clientMetrics.getWriteRequestsCount(),
|
||||
|
|
|
@ -27,11 +27,9 @@ abstract class AbstractResponse {
|
|||
|
||||
public enum ResponseType {
|
||||
|
||||
SINGLE(0),
|
||||
MULTI(1);
|
||||
SINGLE,
|
||||
MULTI;
|
||||
|
||||
ResponseType(int value) {
|
||||
}
|
||||
}
|
||||
|
||||
public abstract ResponseType type();
|
||||
|
|
|
@ -135,7 +135,7 @@ abstract class AbstractRpcBasedConnectionRegistry implements ConnectionRegistry
|
|||
* Typically, you can use lambda expression to implement this interface as
|
||||
*
|
||||
* <pre>
|
||||
* (c, s, d) -> s.xxx(c, your request here, d)
|
||||
* (c, s, d) -> s.xxx(c, your request here, d)
|
||||
* </pre>
|
||||
*/
|
||||
@FunctionalInterface
|
||||
|
|
|
@ -116,6 +116,7 @@ public interface Admin extends Abortable, Closeable {
|
|||
Connection getConnection();
|
||||
|
||||
/**
|
||||
* Check if a table exists.
|
||||
* @param tableName Table to check.
|
||||
* @return <code>true</code> if table exists already.
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
|
@ -267,8 +268,8 @@ public interface Admin extends Abortable, Closeable {
|
|||
* Get a table descriptor.
|
||||
* @param tableName as a {@link TableName}
|
||||
* @return the read-only tableDescriptor
|
||||
* @throws org.apache.hadoop.hbase.TableNotFoundException
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
* @throws TableNotFoundException if the table was not found
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version. Use
|
||||
* {@link #getDescriptor(TableName)}.
|
||||
*/
|
||||
|
@ -280,8 +281,8 @@ public interface Admin extends Abortable, Closeable {
|
|||
* Get a table descriptor.
|
||||
* @param tableName as a {@link TableName}
|
||||
* @return the tableDescriptor
|
||||
* @throws org.apache.hadoop.hbase.TableNotFoundException
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
* @throws TableNotFoundException if the table was not found
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
*/
|
||||
TableDescriptor getDescriptor(TableName tableName) throws TableNotFoundException, IOException;
|
||||
|
||||
|
@ -290,7 +291,7 @@ public interface Admin extends Abortable, Closeable {
|
|||
* @param desc table descriptor for table
|
||||
* @throws IllegalArgumentException if the table name is reserved
|
||||
* @throws org.apache.hadoop.hbase.MasterNotRunningException if master is not running
|
||||
* @throws org.apache.hadoop.hbase.TableExistsException if table already exists (If
|
||||
* @throws TableExistsException if table already exists (If
|
||||
* concurrent threads, the table may
|
||||
* have been created between
|
||||
* test-for-existence and
|
||||
|
@ -316,7 +317,7 @@ public interface Admin extends Abortable, Closeable {
|
|||
* @throws IOException if a remote or network exception
|
||||
* occurs
|
||||
* @throws org.apache.hadoop.hbase.MasterNotRunningException if master is not running
|
||||
* @throws org.apache.hadoop.hbase.TableExistsException if table already exists (If
|
||||
* @throws TableExistsException if table already exists (If
|
||||
* concurrent threads, the table may
|
||||
* have been created between
|
||||
* test-for-existence and
|
||||
|
@ -335,7 +336,7 @@ public interface Admin extends Abortable, Closeable {
|
|||
* split keys are repeated and if the
|
||||
* split key has empty byte array.
|
||||
* @throws org.apache.hadoop.hbase.MasterNotRunningException if master is not running
|
||||
* @throws org.apache.hadoop.hbase.TableExistsException if table already exists (If
|
||||
* @throws TableExistsException if table already exists (If
|
||||
* concurrent threads, the table may
|
||||
* have been created between
|
||||
* test-for-existence and
|
||||
|
@ -574,6 +575,7 @@ public interface Admin extends Abortable, Closeable {
|
|||
HTableDescriptor[] disableTables(Pattern pattern) throws IOException;
|
||||
|
||||
/**
|
||||
* Check if a table is enabled.
|
||||
* @param tableName name of table to check
|
||||
* @return <code>true</code> if table is on-line
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
|
@ -581,6 +583,7 @@ public interface Admin extends Abortable, Closeable {
|
|||
boolean isTableEnabled(TableName tableName) throws IOException;
|
||||
|
||||
/**
|
||||
* Check if a table is disabled.
|
||||
* @param tableName name of table to check
|
||||
* @return <code>true</code> if table is off-line
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
|
@ -588,6 +591,7 @@ public interface Admin extends Abortable, Closeable {
|
|||
boolean isTableDisabled(TableName tableName) throws IOException;
|
||||
|
||||
/**
|
||||
* Check if a table is available.
|
||||
* @param tableName name of table to check
|
||||
* @return <code>true</code> if all regions of the table are available
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
|
@ -1646,6 +1650,7 @@ public interface Admin extends Abortable, Closeable {
|
|||
ClusterMetrics getClusterMetrics(EnumSet<Option> options) throws IOException;
|
||||
|
||||
/**
|
||||
* Get the current active master.
|
||||
* @return current master server name
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
*/
|
||||
|
@ -1654,6 +1659,7 @@ public interface Admin extends Abortable, Closeable {
|
|||
}
|
||||
|
||||
/**
|
||||
* Get a list of current backup masters.
|
||||
* @return current backup master list
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
*/
|
||||
|
@ -1662,6 +1668,7 @@ public interface Admin extends Abortable, Closeable {
|
|||
}
|
||||
|
||||
/**
|
||||
* Get the live server list.
|
||||
* @return current live region servers list
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
*/
|
||||
|
@ -1772,7 +1779,7 @@ public interface Admin extends Abortable, Closeable {
|
|||
* Get a namespace descriptor by name.
|
||||
* @param name name of namespace descriptor
|
||||
* @return A descriptor
|
||||
* @throws org.apache.hadoop.hbase.NamespaceNotFoundException
|
||||
* @throws org.apache.hadoop.hbase.NamespaceNotFoundException if the namespace was not found
|
||||
* @throws IOException if a remote or network exception
|
||||
* occurs
|
||||
*/
|
||||
|
@ -1929,9 +1936,8 @@ public interface Admin extends Abortable, Closeable {
|
|||
* this method returns. As a side effect of this call, the named region server may schedule store
|
||||
* flushes at the request of the wal.
|
||||
* @param serverName The servername of the regionserver.
|
||||
* @throws IOException if a remote or network
|
||||
* exception occurs
|
||||
* @throws org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
* @throws FailedLogCloseException if we failed to close the WAL
|
||||
*/
|
||||
void rollWALWriter(ServerName serverName) throws IOException, FailedLogCloseException;
|
||||
|
||||
|
|
|
@ -47,7 +47,7 @@ import org.slf4j.LoggerFactory;
|
|||
@InterfaceAudience.Public
|
||||
public class Append extends Mutation {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(Append.class);
|
||||
private static final long HEAP_OVERHEAD = ClassSize.REFERENCE + ClassSize.TIMERANGE;
|
||||
private static final long HEAP_OVERHEAD = (long) ClassSize.REFERENCE + ClassSize.TIMERANGE;
|
||||
private TimeRange tr = TimeRange.allTime();
|
||||
|
||||
/**
|
||||
|
@ -166,6 +166,7 @@ public class Append extends Mutation {
|
|||
* Add column and value to this Append operation. n * @return This instance
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public Append add(final Cell cell) {
|
||||
try {
|
||||
super.add(cell);
|
||||
|
|
|
@ -61,6 +61,7 @@ import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
|
|||
public interface AsyncAdmin {
|
||||
|
||||
/**
|
||||
* Check if a table exists.
|
||||
* @param tableName Table to check.
|
||||
* @return True if table exists already. The return value will be wrapped by a
|
||||
* {@link CompletableFuture}.
|
||||
|
@ -210,6 +211,7 @@ public interface AsyncAdmin {
|
|||
CompletableFuture<Void> disableTable(TableName tableName);
|
||||
|
||||
/**
|
||||
* Check if a table is enabled.
|
||||
* @param tableName name of table to check
|
||||
* @return true if table is on-line. The return value will be wrapped by a
|
||||
* {@link CompletableFuture}.
|
||||
|
@ -217,6 +219,7 @@ public interface AsyncAdmin {
|
|||
CompletableFuture<Boolean> isTableEnabled(TableName tableName);
|
||||
|
||||
/**
|
||||
* Check if a table is disabled.
|
||||
* @param tableName name of table to check
|
||||
* @return true if table is off-line. The return value will be wrapped by a
|
||||
* {@link CompletableFuture}.
|
||||
|
@ -224,6 +227,7 @@ public interface AsyncAdmin {
|
|||
CompletableFuture<Boolean> isTableDisabled(TableName tableName);
|
||||
|
||||
/**
|
||||
* Check if a table is available.
|
||||
* @param tableName name of table to check
|
||||
* @return true if all regions of the table are available. The return value will be wrapped by a
|
||||
* {@link CompletableFuture}.
|
||||
|
@ -598,11 +602,15 @@ public interface AsyncAdmin {
|
|||
CompletableFuture<Void> splitRegion(byte[] regionName, byte[] splitPoint);
|
||||
|
||||
/**
|
||||
* Assign an individual region.
|
||||
* @param regionName Encoded or full name of region to assign.
|
||||
*/
|
||||
CompletableFuture<Void> assign(byte[] regionName);
|
||||
|
||||
/**
|
||||
* Unassign a region from current hosting regionserver. Region will then be assigned to a
|
||||
* regionserver chosen at random. Region could be reassigned back to the same server. Use
|
||||
* {@link #move(byte[], ServerName)} if you want to control the region movement.
|
||||
* @param regionName Encoded or full name of region to unassign.
|
||||
*/
|
||||
CompletableFuture<Void> unassign(byte[] regionName);
|
||||
|
@ -1154,19 +1162,21 @@ public interface AsyncAdmin {
|
|||
CompletableFuture<Void> rollWALWriter(ServerName serverName);
|
||||
|
||||
/**
|
||||
* Clear compacting queues on a region server. n * @param queues the set of queue name
|
||||
* Clear compacting queues on a region server.
|
||||
* @param serverName The servername of the region server.
|
||||
* @param queues the set of queue name
|
||||
*/
|
||||
CompletableFuture<Void> clearCompactionQueues(ServerName serverName, Set<String> queues);
|
||||
|
||||
/**
|
||||
* Get a list of {@link RegionMetrics} of all regions hosted on a region seerver. n * @return a
|
||||
* list of {@link RegionMetrics} wrapped by {@link CompletableFuture}
|
||||
* Get a list of {@link RegionMetrics} of all regions hosted on a region server.
|
||||
* @return list of {@link RegionMetrics} wrapped by {@link CompletableFuture}
|
||||
*/
|
||||
CompletableFuture<List<RegionMetrics>> getRegionMetrics(ServerName serverName);
|
||||
|
||||
/**
|
||||
* Get a list of {@link RegionMetrics} of all regions hosted on a region seerver for a table. nn
|
||||
* * @return a list of {@link RegionMetrics} wrapped by {@link CompletableFuture}
|
||||
* Get a list of {@link RegionMetrics} of all regions hosted on a region server for a table.
|
||||
* @return a list of {@link RegionMetrics} wrapped by {@link CompletableFuture}
|
||||
*/
|
||||
CompletableFuture<List<RegionMetrics>> getRegionMetrics(ServerName serverName,
|
||||
TableName tableName);
|
||||
|
@ -1227,8 +1237,8 @@ public interface AsyncAdmin {
|
|||
CompletableFuture<Optional<Long>> getLastMajorCompactionTimestampForRegion(byte[] regionName);
|
||||
|
||||
/**
|
||||
* @return the list of supported security capabilities. The return value will be wrapped by a
|
||||
* {@link CompletableFuture}.
|
||||
* Returns the list of supported security capabilities. The return value will be wrapped by a
|
||||
* {@link CompletableFuture}.
|
||||
*/
|
||||
CompletableFuture<List<SecurityCapability>> getSecurityCapabilities();
|
||||
|
||||
|
@ -1372,9 +1382,7 @@ public interface AsyncAdmin {
|
|||
* one line lambda expression, like:
|
||||
*
|
||||
* <pre>
|
||||
* <code>
|
||||
* channel -> xxxService.newStub(channel)
|
||||
* </code>
|
||||
* channel -> xxxService.newStub(channel)
|
||||
* </pre>
|
||||
*
|
||||
* @param stubMaker a delegation to the actual {@code newStub} call.
|
||||
|
@ -1395,9 +1403,7 @@ public interface AsyncAdmin {
|
|||
* one line lambda expression, like:
|
||||
*
|
||||
* <pre>
|
||||
* <code>
|
||||
* channel -> xxxService.newStub(channel)
|
||||
* </code>
|
||||
* channel -> xxxService.newStub(channel)
|
||||
* </pre>
|
||||
*
|
||||
* @param stubMaker a delegation to the actual {@code newStub} call.
|
||||
|
@ -1613,7 +1619,6 @@ public interface AsyncAdmin {
|
|||
* @param serverType enum for server type: HMaster or RegionServer
|
||||
* @param limit put a limit to list of records that server should send in response
|
||||
* @param filterParams additional filter params
|
||||
* @return Log entries representing online records from servers
|
||||
*/
|
||||
CompletableFuture<List<LogEntry>> getLogEntries(Set<ServerName> serverNames, String logType,
|
||||
ServerType serverType, int limit, Map<String, Object> filterParams);
|
||||
|
|
|
@ -29,9 +29,6 @@ import org.apache.hbase.thirdparty.io.netty.util.Timer;
|
|||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
|
||||
|
||||
/**
|
||||
* @since 2.0.0
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class AsyncAdminRequestRetryingCaller<T> extends AsyncRpcRetryingCaller<T> {
|
||||
|
||||
|
|
|
@ -539,7 +539,7 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
|
|||
runnable.run();
|
||||
} else {
|
||||
try {
|
||||
pool.submit(runnable);
|
||||
pool.execute(runnable);
|
||||
} catch (Throwable t) {
|
||||
if (t instanceof RejectedExecutionException) {
|
||||
// This should never happen. But as the pool is provided by the end user,
|
||||
|
@ -564,6 +564,7 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
|
|||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("MixedMutabilityReturnType")
|
||||
private Collection<? extends Runnable> getNewMultiActionRunnable(ServerName server,
|
||||
MultiAction multiAction, int numAttempt) {
|
||||
// no stats to manage, just do the standard action
|
||||
|
@ -644,7 +645,7 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
|
|||
// Start the thread that may kick off replica gets.
|
||||
// TODO: we could do it on the same thread, but it's a user thread, might be a bad idea.
|
||||
try {
|
||||
pool.submit(replicaRunnable);
|
||||
pool.execute(replicaRunnable);
|
||||
} catch (RejectedExecutionException ree) {
|
||||
LOG.warn("id=" + asyncProcess.id + " replica task rejected by pool; no replica calls", ree);
|
||||
}
|
||||
|
@ -770,6 +771,7 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
|
|||
private void logNoResubmit(ServerName oldServer, int numAttempt, int failureCount,
|
||||
Throwable throwable, int failed, int stopped) {
|
||||
if (failureCount != 0 || numAttempt > asyncProcess.startLogErrorsCnt + 1) {
|
||||
@SuppressWarnings("JavaUtilDate")
|
||||
String timeStr = new Date(errorsByServer.getStartTrackingTime()).toString();
|
||||
String logMessage = createLog(numAttempt, failureCount, 0, oldServer, throwable, -1, false,
|
||||
timeStr, failed, stopped);
|
||||
|
@ -955,14 +957,14 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
|
|||
if (result == null) {
|
||||
throw new RuntimeException("Result cannot be null");
|
||||
}
|
||||
ReplicaResultState state = null;
|
||||
boolean isStale = !RegionReplicaUtil.isDefaultReplica(action.getReplicaId());
|
||||
int index = action.getOriginalIndex();
|
||||
if (results == null) {
|
||||
decActionCounter(index);
|
||||
return; // Simple case, no replica requests.
|
||||
}
|
||||
state = trySetResultSimple(index, action.getAction(), false, result, null, isStale);
|
||||
ReplicaResultState state =
|
||||
trySetResultSimple(index, action.getAction(), false, result, null, isStale);
|
||||
if (state == null) {
|
||||
return; // Simple case, no replica requests.
|
||||
}
|
||||
|
@ -995,7 +997,6 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
|
|||
* @param server The source server.
|
||||
*/
|
||||
private void setError(int index, Row row, Throwable throwable, ServerName server) {
|
||||
ReplicaResultState state = null;
|
||||
if (results == null) {
|
||||
// Note that we currently cannot have replica requests with null results. So it shouldn't
|
||||
// happen that multiple replica calls will call dAC for same actions with results == null.
|
||||
|
@ -1004,7 +1005,7 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
|
|||
decActionCounter(index);
|
||||
return; // Simple case, no replica requests.
|
||||
}
|
||||
state = trySetResultSimple(index, row, true, throwable, server, false);
|
||||
ReplicaResultState state = trySetResultSimple(index, row, true, throwable, server, false);
|
||||
if (state == null) {
|
||||
return; // Simple case, no replica requests.
|
||||
}
|
||||
|
|
|
@ -223,16 +223,14 @@ public interface AsyncTable<C extends ScanResultConsumerBase> {
|
|||
* This is a fluent style API, the code is like:
|
||||
*
|
||||
* <pre>
|
||||
* <code>
|
||||
* table.checkAndMutate(row, family).qualifier(qualifier).ifNotExists().thenPut(put)
|
||||
* .thenAccept(succ -> {
|
||||
* if (succ) {
|
||||
* System.out.println("Check and put succeeded");
|
||||
* } else {
|
||||
* System.out.println("Check and put failed");
|
||||
* }
|
||||
* });
|
||||
* </code>
|
||||
* .thenAccept(succ -> {
|
||||
* if (succ) {
|
||||
* System.out.println("Check and put succeeded");
|
||||
* } else {
|
||||
* System.out.println("Check and put failed");
|
||||
* }
|
||||
* });
|
||||
* </pre>
|
||||
*
|
||||
* @deprecated Since 2.4.0, will be removed in 4.0.0. For internal test use only, do not use it
|
||||
|
@ -250,11 +248,13 @@ public interface AsyncTable<C extends ScanResultConsumerBase> {
|
|||
interface CheckAndMutateBuilder {
|
||||
|
||||
/**
|
||||
* Match a qualifier.
|
||||
* @param qualifier column qualifier to check.
|
||||
*/
|
||||
CheckAndMutateBuilder qualifier(byte[] qualifier);
|
||||
|
||||
/**
|
||||
* Match a timerange.
|
||||
* @param timeRange time range to check.
|
||||
*/
|
||||
CheckAndMutateBuilder timeRange(TimeRange timeRange);
|
||||
|
@ -273,12 +273,14 @@ public interface AsyncTable<C extends ScanResultConsumerBase> {
|
|||
}
|
||||
|
||||
/**
|
||||
* Compare a value
|
||||
* @param compareOp comparison operator to use
|
||||
* @param value the expected value
|
||||
*/
|
||||
CheckAndMutateBuilder ifMatches(CompareOperator compareOp, byte[] value);
|
||||
|
||||
/**
|
||||
* Specify a Put to commit if the check succeeds.
|
||||
* @param put data to put if check succeeds
|
||||
* @return {@code true} if the new put was executed, {@code false} otherwise. The return value
|
||||
* will be wrapped by a {@link CompletableFuture}.
|
||||
|
@ -286,6 +288,7 @@ public interface AsyncTable<C extends ScanResultConsumerBase> {
|
|||
CompletableFuture<Boolean> thenPut(Put put);
|
||||
|
||||
/**
|
||||
* Specify a Delete to commit if the check succeeds.
|
||||
* @param delete data to delete if check succeeds
|
||||
* @return {@code true} if the new delete was executed, {@code false} otherwise. The return
|
||||
* value will be wrapped by a {@link CompletableFuture}.
|
||||
|
@ -293,6 +296,7 @@ public interface AsyncTable<C extends ScanResultConsumerBase> {
|
|||
CompletableFuture<Boolean> thenDelete(Delete delete);
|
||||
|
||||
/**
|
||||
* Specify a RowMutations to commit if the check succeeds.
|
||||
* @param mutation mutations to perform if check succeeds
|
||||
* @return true if the new mutation was executed, false otherwise. The return value will be
|
||||
* wrapped by a {@link CompletableFuture}.
|
||||
|
@ -308,16 +312,13 @@ public interface AsyncTable<C extends ScanResultConsumerBase> {
|
|||
* execute it. This is a fluent style API, the code is like:
|
||||
*
|
||||
* <pre>
|
||||
* <code>
|
||||
* table.checkAndMutate(row, filter).thenPut(put)
|
||||
* .thenAccept(succ -> {
|
||||
* if (succ) {
|
||||
* System.out.println("Check and put succeeded");
|
||||
* } else {
|
||||
* System.out.println("Check and put failed");
|
||||
* }
|
||||
* });
|
||||
* </code>
|
||||
* table.checkAndMutate(row, filter).thenPut(put).thenAccept(succ -> {
|
||||
* if (succ) {
|
||||
* System.out.println("Check and put succeeded");
|
||||
* } else {
|
||||
* System.out.println("Check and put failed");
|
||||
* }
|
||||
* });
|
||||
* </pre>
|
||||
*
|
||||
* @deprecated Since 2.4.0, will be removed in 4.0.0. For internal test use only, do not use it
|
||||
|
@ -335,11 +336,13 @@ public interface AsyncTable<C extends ScanResultConsumerBase> {
|
|||
interface CheckAndMutateWithFilterBuilder {
|
||||
|
||||
/**
|
||||
* Match a timerange.
|
||||
* @param timeRange time range to check.
|
||||
*/
|
||||
CheckAndMutateWithFilterBuilder timeRange(TimeRange timeRange);
|
||||
|
||||
/**
|
||||
* Specify a Put to commit if the check succeeds.
|
||||
* @param put data to put if check succeeds
|
||||
* @return {@code true} if the new put was executed, {@code false} otherwise. The return value
|
||||
* will be wrapped by a {@link CompletableFuture}.
|
||||
|
@ -347,6 +350,7 @@ public interface AsyncTable<C extends ScanResultConsumerBase> {
|
|||
CompletableFuture<Boolean> thenPut(Put put);
|
||||
|
||||
/**
|
||||
* Specify a Delete to commit if the check succeeds.
|
||||
* @param delete data to delete if check succeeds
|
||||
* @return {@code true} if the new delete was executed, {@code false} otherwise. The return
|
||||
* value will be wrapped by a {@link CompletableFuture}.
|
||||
|
@ -354,6 +358,7 @@ public interface AsyncTable<C extends ScanResultConsumerBase> {
|
|||
CompletableFuture<Boolean> thenDelete(Delete delete);
|
||||
|
||||
/**
|
||||
* Specify a RowMutations to commit if the check succeeds.
|
||||
* @param mutation mutations to perform if check succeeds
|
||||
* @return true if the new mutation was executed, false otherwise. The return value will be
|
||||
* wrapped by a {@link CompletableFuture}.
|
||||
|
@ -440,16 +445,14 @@ public interface AsyncTable<C extends ScanResultConsumerBase> {
|
|||
* <p>
|
||||
*
|
||||
* <pre>
|
||||
* <code>
|
||||
* table.scanAll(new Scan().withStartRow(row, false).setLimit(1)).thenAccept(results -> {
|
||||
* table.scanAll(new Scan().withStartRow(row, false).setLimit(1)).thenAccept(results -> {
|
||||
* if (results.isEmpty()) {
|
||||
* System.out.println("No row after " + Bytes.toStringBinary(row));
|
||||
* System.out.println("No row after " + Bytes.toStringBinary(row));
|
||||
* } else {
|
||||
* System.out.println("The closest row after " + Bytes.toStringBinary(row) + " is "
|
||||
* + Bytes.toStringBinary(results.stream().findFirst().get().getRow()));
|
||||
* + Bytes.toStringBinary(results.stream().findFirst().get().getRow()));
|
||||
* }
|
||||
* });
|
||||
* </code>
|
||||
* </pre>
|
||||
* <p>
|
||||
* If your result set is very large, you should use other scan method to get a scanner or use
|
||||
|
@ -574,9 +577,7 @@ public interface AsyncTable<C extends ScanResultConsumerBase> {
|
|||
* one line lambda expression, like:
|
||||
*
|
||||
* <pre>
|
||||
* <code>
|
||||
* channel -> xxxService.newStub(channel)
|
||||
* </code>
|
||||
* channel -> xxxService.newStub(channel)
|
||||
* </pre>
|
||||
*
|
||||
* @param stubMaker a delegation to the actual {@code newStub} call.
|
||||
|
@ -609,7 +610,7 @@ public interface AsyncTable<C extends ScanResultConsumerBase> {
|
|||
*
|
||||
* <pre>
|
||||
* locateThenCall(byte[] row) {
|
||||
* locate(row).whenComplete((location, locateError) -> {
|
||||
* locate(row).whenComplete((location, locateError) -> {
|
||||
* if (locateError != null) {
|
||||
* callback.onError(locateError);
|
||||
* return;
|
||||
|
@ -621,7 +622,7 @@ public interface AsyncTable<C extends ScanResultConsumerBase> {
|
|||
* } else {
|
||||
* locateThenCall(region.getEndKey());
|
||||
* }
|
||||
* sendCall().whenComplete((resp, error) -> {
|
||||
* sendCall().whenComplete((resp, error) -> {
|
||||
* if (error != null) {
|
||||
* callback.onRegionError(region, error);
|
||||
* } else {
|
||||
|
@ -639,12 +640,14 @@ public interface AsyncTable<C extends ScanResultConsumerBase> {
|
|||
interface CoprocessorCallback<R> {
|
||||
|
||||
/**
|
||||
* Indicate that the respose of a region is available
|
||||
* @param region the region that the response belongs to
|
||||
* @param resp the response of the coprocessor call
|
||||
*/
|
||||
void onRegionComplete(RegionInfo region, R resp);
|
||||
|
||||
/**
|
||||
* Indicate that the error for a region is available
|
||||
* @param region the region that the error belongs to
|
||||
* @param error the response error of the coprocessor call
|
||||
*/
|
||||
|
@ -675,6 +678,7 @@ public interface AsyncTable<C extends ScanResultConsumerBase> {
|
|||
interface CoprocessorServiceBuilder<S, R> {
|
||||
|
||||
/**
|
||||
* Specify a start row
|
||||
* @param startKey start region selection with region containing this row, inclusive.
|
||||
*/
|
||||
default CoprocessorServiceBuilder<S, R> fromRow(byte[] startKey) {
|
||||
|
@ -682,12 +686,14 @@ public interface AsyncTable<C extends ScanResultConsumerBase> {
|
|||
}
|
||||
|
||||
/**
|
||||
* Specify a start row
|
||||
* @param startKey start region selection with region containing this row
|
||||
* @param inclusive whether to include the startKey
|
||||
*/
|
||||
CoprocessorServiceBuilder<S, R> fromRow(byte[] startKey, boolean inclusive);
|
||||
|
||||
/**
|
||||
* Specify a stop row
|
||||
* @param endKey select regions up to and including the region containing this row, exclusive.
|
||||
*/
|
||||
default CoprocessorServiceBuilder<S, R> toRow(byte[] endKey) {
|
||||
|
@ -695,6 +701,7 @@ public interface AsyncTable<C extends ScanResultConsumerBase> {
|
|||
}
|
||||
|
||||
/**
|
||||
* Specify a stop row
|
||||
* @param endKey select regions up to and including the region containing this row
|
||||
* @param inclusive whether to include the endKey
|
||||
*/
|
||||
|
@ -716,9 +723,7 @@ public interface AsyncTable<C extends ScanResultConsumerBase> {
|
|||
* is only a one line lambda expression, like:
|
||||
*
|
||||
* <pre>
|
||||
* <code>
|
||||
* channel -> xxxService.newStub(channel)
|
||||
* </code>
|
||||
* channel -> xxxService.newStub(channel)
|
||||
* </pre>
|
||||
*
|
||||
* @param stubMaker a delegation to the actual {@code newStub} call.
|
||||
|
|
|
@ -89,7 +89,6 @@ public interface AsyncTableRegionLocator {
|
|||
* Find all the replicas for the region on which the given row is being served.
|
||||
* @param row Row to find.
|
||||
* @return Locations for all the replicas of the row.
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
*/
|
||||
default CompletableFuture<List<HRegionLocation>> getRegionLocations(byte[] row) {
|
||||
return getRegionLocations(row, false);
|
||||
|
@ -100,7 +99,6 @@ public interface AsyncTableRegionLocator {
|
|||
* @param row Row to find.
|
||||
* @param reload true to reload information or false to use cached information
|
||||
* @return Locations for all the replicas of the row.
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
*/
|
||||
CompletableFuture<List<HRegionLocation>> getRegionLocations(byte[] row, boolean reload);
|
||||
|
||||
|
|
|
@ -147,8 +147,8 @@ public class BufferedMutatorParams implements Cloneable {
|
|||
}
|
||||
|
||||
/**
|
||||
* @return Name of the class we will use when we construct a {@link BufferedMutator} instance or
|
||||
* null if default implementation.
|
||||
* Returns Name of the class we will use when we construct a {@link BufferedMutator} instance or
|
||||
* null if default implementation.
|
||||
*/
|
||||
public String getImplementationClassName() {
|
||||
return this.implementationClassName;
|
||||
|
|
|
@ -129,6 +129,7 @@ class CatalogReplicaLoadBalanceSimpleSelector
|
|||
* Selector's internal state.
|
||||
* @param loc the location which causes exception.
|
||||
*/
|
||||
@Override
|
||||
public void onError(HRegionLocation loc) {
|
||||
ConcurrentNavigableMap<byte[], StaleLocationCacheEntry> tableCache = computeIfAbsent(staleCache,
|
||||
loc.getRegion().getTable(), () -> new ConcurrentSkipListMap<>(BYTES_COMPARATOR));
|
||||
|
@ -160,18 +161,19 @@ class CatalogReplicaLoadBalanceSimpleSelector
|
|||
* When it looks up a location, it will call this method to find a replica region to go. For a
|
||||
* normal case, > 99% of region locations from catalog/meta replica will be up to date. In extreme
|
||||
* cases such as region server crashes, it will depends on how fast replication catches up.
|
||||
* @param tablename table name it looks up
|
||||
* @param tableName table name it looks up
|
||||
* @param row key it looks up.
|
||||
* @param locateType locateType, Only BEFORE and CURRENT will be passed in.
|
||||
* @return catalog replica id
|
||||
*/
|
||||
public int select(final TableName tablename, final byte[] row,
|
||||
@Override
|
||||
public int select(final TableName tableName, final byte[] row,
|
||||
final RegionLocateType locateType) {
|
||||
Preconditions.checkArgument(
|
||||
locateType == RegionLocateType.BEFORE || locateType == RegionLocateType.CURRENT,
|
||||
"Expected type BEFORE or CURRENT but got: %s", locateType);
|
||||
|
||||
ConcurrentNavigableMap<byte[], StaleLocationCacheEntry> tableCache = staleCache.get(tablename);
|
||||
ConcurrentNavigableMap<byte[], StaleLocationCacheEntry> tableCache = staleCache.get(tableName);
|
||||
|
||||
// If there is no entry in StaleCache, select a random replica id.
|
||||
if (tableCache == null) {
|
||||
|
@ -200,7 +202,7 @@ class CatalogReplicaLoadBalanceSimpleSelector
|
|||
(EnvironmentEdgeManager.currentTime() - entry.getValue().getTimestamp())
|
||||
>= STALE_CACHE_TIMEOUT_IN_MILLISECONDS
|
||||
) {
|
||||
LOG.debug("Entry for table {} with startKey {}, {} times out", tablename, entry.getKey(),
|
||||
LOG.debug("Entry for table {} with startKey {}, {} times out", tableName, entry.getKey(),
|
||||
entry);
|
||||
tableCache.remove(entry.getKey());
|
||||
return getRandomReplicaId();
|
||||
|
|
|
@ -97,6 +97,7 @@ public final class CheckAndMutate implements Row {
|
|||
}
|
||||
|
||||
/**
|
||||
* Check for match
|
||||
* @param family family to check
|
||||
* @param qualifier qualifier to check
|
||||
* @param compareOp comparison operator to use
|
||||
|
@ -113,6 +114,7 @@ public final class CheckAndMutate implements Row {
|
|||
}
|
||||
|
||||
/**
|
||||
* Check for match
|
||||
* @param filter filter to check
|
||||
* @return the CheckAndMutate object
|
||||
*/
|
||||
|
@ -122,6 +124,7 @@ public final class CheckAndMutate implements Row {
|
|||
}
|
||||
|
||||
/**
|
||||
* Specify a timerange
|
||||
* @param timeRange time range to check
|
||||
* @return the CheckAndMutate object
|
||||
*/
|
||||
|
@ -144,6 +147,7 @@ public final class CheckAndMutate implements Row {
|
|||
}
|
||||
|
||||
/**
|
||||
* Build the CheckAndMutate object
|
||||
* @param put data to put if check succeeds
|
||||
* @return a CheckAndMutate object
|
||||
*/
|
||||
|
@ -157,6 +161,7 @@ public final class CheckAndMutate implements Row {
|
|||
}
|
||||
|
||||
/**
|
||||
* Build the CheckAndMutate object
|
||||
* @param delete data to delete if check succeeds
|
||||
* @return a CheckAndMutate object
|
||||
*/
|
||||
|
@ -170,6 +175,7 @@ public final class CheckAndMutate implements Row {
|
|||
}
|
||||
|
||||
/**
|
||||
* Build the CheckAndMutate object with an Increment to commit if the check succeeds.
|
||||
* @param increment data to increment if check succeeds
|
||||
* @return a CheckAndMutate object
|
||||
*/
|
||||
|
@ -183,6 +189,7 @@ public final class CheckAndMutate implements Row {
|
|||
}
|
||||
|
||||
/**
|
||||
* Build the CheckAndMutate object with an Append to commit if the check succeeds.
|
||||
* @param append data to append if check succeeds
|
||||
* @return a CheckAndMutate object
|
||||
*/
|
||||
|
@ -196,6 +203,7 @@ public final class CheckAndMutate implements Row {
|
|||
}
|
||||
|
||||
/**
|
||||
* Build the CheckAndMutate object with a RowMutations to commit if the check succeeds.
|
||||
* @param mutations mutations to perform if check succeeds
|
||||
* @return a CheckAndMutate object
|
||||
*/
|
||||
|
|
|
@ -94,34 +94,35 @@ public class ClientAsyncPrefetchScanner extends ClientSimpleScanner {
|
|||
public Result next() throws IOException {
|
||||
try (Scope ignored = span.makeCurrent()) {
|
||||
lock.lock();
|
||||
while (cache.isEmpty()) {
|
||||
try {
|
||||
while (cache.isEmpty()) {
|
||||
handleException();
|
||||
if (this.closed) {
|
||||
return null;
|
||||
}
|
||||
try {
|
||||
notEmpty.await();
|
||||
} catch (InterruptedException e) {
|
||||
span.recordException(e);
|
||||
throw new InterruptedIOException("Interrupted when wait to load cache");
|
||||
}
|
||||
}
|
||||
Result result = pollCache();
|
||||
if (prefetchCondition()) {
|
||||
notFull.signalAll();
|
||||
}
|
||||
return result;
|
||||
} finally {
|
||||
lock.unlock();
|
||||
handleException();
|
||||
if (this.closed) {
|
||||
return null;
|
||||
}
|
||||
try {
|
||||
notEmpty.await();
|
||||
} catch (InterruptedException e) {
|
||||
span.recordException(e);
|
||||
throw new InterruptedIOException("Interrupted when wait to load cache");
|
||||
}
|
||||
}
|
||||
|
||||
Result result = pollCache();
|
||||
if (prefetchCondition()) {
|
||||
notFull.signalAll();
|
||||
}
|
||||
return result;
|
||||
} finally {
|
||||
lock.unlock();
|
||||
handleException();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
lock.lock();
|
||||
try {
|
||||
lock.lock();
|
||||
super.close();
|
||||
closed = true;
|
||||
notFull.signalAll();
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.client;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.lang.management.ManagementFactory;
|
||||
import java.util.List;
|
||||
import org.apache.hadoop.hbase.util.Addressing;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
|
@ -26,6 +27,9 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.base.Splitter;
|
||||
import org.apache.hbase.thirdparty.com.google.common.collect.Iterators;
|
||||
|
||||
/**
|
||||
* The class that is able to determine some unique strings for the client, such as an IP address,
|
||||
* PID, and composite deterministic ID.
|
||||
|
@ -38,8 +42,8 @@ final class ClientIdGenerator {
|
|||
}
|
||||
|
||||
/**
|
||||
* @return a unique ID incorporating IP address, PID, TID and timer. Might be an overkill... Note
|
||||
* though that new UUID in java by default is just a random number.
|
||||
* Returns a unique ID incorporating IP address, PID, TID and timer. Might be an overkill... Note
|
||||
* though that new UUID in java by default is just a random number.
|
||||
*/
|
||||
public static byte[] generateClientId() {
|
||||
byte[] selfBytes = getIpAddressBytes();
|
||||
|
@ -61,10 +65,10 @@ final class ClientIdGenerator {
|
|||
/** Returns PID of the current process, if it can be extracted from JVM name, or null. */
|
||||
public static Long getPid() {
|
||||
String name = ManagementFactory.getRuntimeMXBean().getName();
|
||||
String[] nameParts = name.split("@");
|
||||
if (nameParts.length == 2) { // 12345@somewhere
|
||||
List<String> nameParts = Splitter.on('@').splitToList(name);
|
||||
if (nameParts.size() == 2) { // 12345@somewhere
|
||||
try {
|
||||
return Long.parseLong(nameParts[0]);
|
||||
return Long.parseLong(Iterators.get(nameParts.iterator(), 0));
|
||||
} catch (NumberFormatException ex) {
|
||||
LOG.warn("Failed to get PID from [" + name + "]", ex);
|
||||
}
|
||||
|
@ -75,8 +79,8 @@ final class ClientIdGenerator {
|
|||
}
|
||||
|
||||
/**
|
||||
* @return Some IPv4/IPv6 address available on the current machine that is up, not virtual and not
|
||||
* a loopback address. Empty array if none can be found or error occurred.
|
||||
* Returns Some IPv4/IPv6 address available on the current machine that is up, not virtual and not
|
||||
* a loopback address. Empty array if none can be found or error occurred.
|
||||
*/
|
||||
public static byte[] getIpAddressBytes() {
|
||||
try {
|
||||
|
|
|
@ -327,7 +327,7 @@ public abstract class ClientScanner extends AbstractClientScanner {
|
|||
return cache != null ? cache.size() : 0;
|
||||
}
|
||||
|
||||
private boolean scanExhausted(Result[] values) {
|
||||
private boolean scanExhausted() {
|
||||
return callable.moreResultsForScan() == MoreResults.NO;
|
||||
}
|
||||
|
||||
|
@ -484,7 +484,7 @@ public abstract class ClientScanner extends AbstractClientScanner {
|
|||
assert newLimit >= 0;
|
||||
scan.setLimit(newLimit);
|
||||
}
|
||||
if (scan.getLimit() == 0 || scanExhausted(values)) {
|
||||
if (scan.getLimit() == 0 || scanExhausted()) {
|
||||
closeScanner();
|
||||
closed = true;
|
||||
break;
|
||||
|
|
|
@ -29,7 +29,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
|
|||
/**
|
||||
* A RegionServerCallable set to use the Client protocol. Also includes some utility methods so can
|
||||
* hide protobuf references here rather than have them sprinkled about the code base.
|
||||
* @param <T>
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public abstract class ClientServiceCallable<T>
|
||||
|
|
|
@ -72,6 +72,7 @@ public interface ClusterConnection extends Connection {
|
|||
boolean isTableEnabled(TableName tableName) throws IOException;
|
||||
|
||||
/**
|
||||
* Check if a table is disabled.
|
||||
* @param tableName table name
|
||||
* @return true if the table is disabled, false otherwise
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
|
@ -95,6 +96,7 @@ public interface ClusterConnection extends Connection {
|
|||
HRegionLocation locateRegion(final TableName tableName, final byte[] row) throws IOException;
|
||||
|
||||
/**
|
||||
* Clear the region location cache.
|
||||
* @deprecated {@link #clearRegionLocationCache()} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
|
@ -179,6 +181,7 @@ public interface ClusterConnection extends Connection {
|
|||
final boolean offlined) throws IOException;
|
||||
|
||||
/**
|
||||
* Gets the locations of the region in the specified table, <i>tableName</i>, for a given row.
|
||||
* @param tableName table to get regions of
|
||||
* @param row the row
|
||||
* @param useCache Should we use the cache to retrieve the region information.
|
||||
|
@ -190,6 +193,7 @@ public interface ClusterConnection extends Connection {
|
|||
throws IOException;
|
||||
|
||||
/**
|
||||
* Gets the locations of the region in the specified table, <i>tableName</i>, for a given row.
|
||||
* @param tableName table to get regions of
|
||||
* @param row the row
|
||||
* @param useCache Should we use the cache to retrieve the region information.
|
||||
|
@ -280,8 +284,8 @@ public interface ClusterConnection extends Connection {
|
|||
MetricsConnection getConnectionMetrics();
|
||||
|
||||
/**
|
||||
* @return true when this connection uses a {@link org.apache.hadoop.hbase.codec.Codec} and so
|
||||
* supports cell blocks.
|
||||
* Returns true when this connection uses a {@link org.apache.hadoop.hbase.codec.Codec} and so
|
||||
* supports cell blocks.
|
||||
*/
|
||||
boolean hasCellBlockSupport();
|
||||
|
||||
|
|
|
@ -98,10 +98,7 @@ public interface ColumnFamilyDescriptor {
|
|||
/** Returns an unmodifiable map. */
|
||||
Map<String, String> getConfiguration();
|
||||
|
||||
/**
|
||||
* @param key the key whose associated value is to be returned
|
||||
* @return accessing the configuration value by key.
|
||||
*/
|
||||
/** Returns accessing the configuration value by key. */
|
||||
String getConfigurationValue(String key);
|
||||
|
||||
/** Returns replication factor set for this CF */
|
||||
|
@ -117,8 +114,8 @@ public interface ColumnFamilyDescriptor {
|
|||
String getEncryptionType();
|
||||
|
||||
/**
|
||||
* @return in-memory compaction policy if set for the cf. Returns null if no policy is set for for
|
||||
* this column family
|
||||
* Returns in-memory compaction policy if set for the cf. Returns null if no policy is set for for
|
||||
* this column family
|
||||
*/
|
||||
MemoryCompactionPolicy getInMemoryCompaction();
|
||||
|
||||
|
@ -163,32 +160,35 @@ public interface ColumnFamilyDescriptor {
|
|||
int getTimeToLive();
|
||||
|
||||
/**
|
||||
* Get a configuration value.
|
||||
* @param key The key.
|
||||
* @return A clone value. Null if no mapping for the key
|
||||
*/
|
||||
Bytes getValue(Bytes key);
|
||||
|
||||
/**
|
||||
* Get a configuration value.
|
||||
* @param key The key.
|
||||
* @return A clone value. Null if no mapping for the key
|
||||
*/
|
||||
String getValue(String key);
|
||||
|
||||
/**
|
||||
* Get a configuration value.
|
||||
* @param key The key.
|
||||
* @return A clone value. Null if no mapping for the key
|
||||
*/
|
||||
byte[] getValue(byte[] key);
|
||||
|
||||
/**
|
||||
* It clone all bytes of all elements.
|
||||
* Get all configuration values. It clone all bytes of all elements.
|
||||
* @return All values
|
||||
*/
|
||||
Map<Bytes, Bytes> getValues();
|
||||
|
||||
/**
|
||||
* @return True if hfile DATA type blocks should be cached (You cannot disable caching of INDEX
|
||||
* and BLOOM type blocks).
|
||||
* Returns True if hfile DATA type blocks should be cached (You cannot disable caching of INDEX
|
||||
* and BLOOM type blocks).
|
||||
*/
|
||||
boolean isBlockCacheEnabled();
|
||||
|
||||
|
@ -202,8 +202,8 @@ public interface ColumnFamilyDescriptor {
|
|||
boolean isCacheIndexesOnWrite();
|
||||
|
||||
/**
|
||||
* @return Whether KV tags should be compressed along with DataBlockEncoding. When no
|
||||
* DataBlockEncoding is been used, this is having no effect.
|
||||
* Returns Whether KV tags should be compressed along with DataBlockEncoding. When no
|
||||
* DataBlockEncoding is been used, this is having no effect.
|
||||
*/
|
||||
boolean isCompressTags();
|
||||
|
||||
|
@ -211,8 +211,8 @@ public interface ColumnFamilyDescriptor {
|
|||
boolean isEvictBlocksOnClose();
|
||||
|
||||
/**
|
||||
* @return True if we are to favor keeping all values for this column family in the HRegionServer
|
||||
* cache.
|
||||
* Returns True if we are to favor keeping all values for this column family in the HRegionServer
|
||||
* cache.
|
||||
*/
|
||||
boolean isInMemory();
|
||||
|
||||
|
|
|
@ -42,9 +42,6 @@ import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
|
|||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema;
|
||||
|
||||
/**
|
||||
* @since 2.0.0
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
public class ColumnFamilyDescriptorBuilder {
|
||||
// For future backward compatibility
|
||||
|
@ -330,6 +327,7 @@ public class ColumnFamilyDescriptorBuilder {
|
|||
}
|
||||
|
||||
/**
|
||||
* Check if the column family name is legal.
|
||||
* @param b Family name.
|
||||
* @return <code>b</code>
|
||||
* @throws IllegalArgumentException If not null and not a legitimate family name: i.e. 'printable'
|
||||
|
@ -398,6 +396,7 @@ public class ColumnFamilyDescriptorBuilder {
|
|||
}
|
||||
|
||||
/**
|
||||
* Serialize the table descriptor to a byte array.
|
||||
* @param desc The table descriptor to serialize
|
||||
* @return This instance serialized with pb with pb magic prefix
|
||||
*/
|
||||
|
@ -691,11 +690,6 @@ public class ColumnFamilyDescriptorBuilder {
|
|||
return Collections.unmodifiableMap(values);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param key The key.
|
||||
* @param value The value.
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
public ModifyableColumnFamilyDescriptor setValue(byte[] key, byte[] value) {
|
||||
return setValue(toBytesOrNull(key, Function.identity()),
|
||||
toBytesOrNull(value, Function.identity()));
|
||||
|
@ -709,11 +703,6 @@ public class ColumnFamilyDescriptorBuilder {
|
|||
return setValue(key, toBytesOrNull(value, Bytes::toBytes));
|
||||
}
|
||||
|
||||
/**
|
||||
* @param key The key.
|
||||
* @param value The value.
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
private ModifyableColumnFamilyDescriptor setValue(Bytes key, Bytes value) {
|
||||
if (value == null || value.getLength() == 0) {
|
||||
values.remove(key);
|
||||
|
@ -723,10 +712,6 @@ public class ColumnFamilyDescriptorBuilder {
|
|||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param key Key whose key and value we're to remove from HCD parameters.
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
public ModifyableColumnFamilyDescriptor removeValue(final Bytes key) {
|
||||
return setValue(key, (Bytes) null);
|
||||
}
|
||||
|
@ -758,6 +743,7 @@ public class ColumnFamilyDescriptorBuilder {
|
|||
}
|
||||
|
||||
/**
|
||||
* Set the maximum number of versions to retain.
|
||||
* @param maxVersions maximum number of versions
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
|
@ -777,7 +763,7 @@ public class ColumnFamilyDescriptorBuilder {
|
|||
}
|
||||
|
||||
/**
|
||||
* Set minimum and maximum versions to keep
|
||||
* Set minimum and maximum versions to keep.
|
||||
* @param minVersions minimal number of versions
|
||||
* @param maxVersions maximum number of versions
|
||||
* @return this (for chained invocation)
|
||||
|
@ -804,10 +790,6 @@ public class ColumnFamilyDescriptorBuilder {
|
|||
return getStringOrDefault(BLOCKSIZE_BYTES, Integer::valueOf, DEFAULT_BLOCKSIZE);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param s Blocksize to use when writing out storefiles/hfiles on this column family.
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
public ModifyableColumnFamilyDescriptor setBlocksize(int s) {
|
||||
return setValue(BLOCKSIZE_BYTES, Integer.toString(s));
|
||||
}
|
||||
|
@ -910,6 +892,7 @@ public class ColumnFamilyDescriptorBuilder {
|
|||
}
|
||||
|
||||
/**
|
||||
* Set the inMemory flag
|
||||
* @param inMemory True if we are to favor keeping all values for this column family in the
|
||||
* HRegionServer cache
|
||||
* @return this (for chained invocation)
|
||||
|
@ -924,10 +907,6 @@ public class ColumnFamilyDescriptorBuilder {
|
|||
n -> MemoryCompactionPolicy.valueOf(n.toUpperCase()), null);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param inMemoryCompaction the prefered in-memory compaction policy for this column family
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
public ModifyableColumnFamilyDescriptor
|
||||
setInMemoryCompaction(MemoryCompactionPolicy inMemoryCompaction) {
|
||||
return setValue(IN_MEMORY_COMPACTION_BYTES, inMemoryCompaction.name());
|
||||
|
@ -939,10 +918,6 @@ public class ColumnFamilyDescriptorBuilder {
|
|||
DEFAULT_KEEP_DELETED);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param keepDeletedCells True if deleted rows should not be collected immediately.
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
public ModifyableColumnFamilyDescriptor setKeepDeletedCells(KeepDeletedCells keepDeletedCells) {
|
||||
return setValue(KEEP_DELETED_CELLS_BYTES, keepDeletedCells.name());
|
||||
}
|
||||
|
@ -968,6 +943,7 @@ public class ColumnFamilyDescriptorBuilder {
|
|||
}
|
||||
|
||||
/**
|
||||
* Set the time to live
|
||||
* @param timeToLive Time-to-live of cell contents, in seconds.
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
|
@ -976,9 +952,10 @@ public class ColumnFamilyDescriptorBuilder {
|
|||
}
|
||||
|
||||
/**
|
||||
* Set the time to live
|
||||
* @param timeToLive Time-to-live of cell contents, in seconds.
|
||||
* @return this (for chained invocation)
|
||||
* @throws org.apache.hadoop.hbase.exceptions.HBaseException
|
||||
* @throws org.apache.hadoop.hbase.exceptions.HBaseException exception
|
||||
*/
|
||||
public ModifyableColumnFamilyDescriptor setTimeToLive(String timeToLive) throws HBaseException {
|
||||
return setTimeToLive(Integer.parseInt(PrettyPrinter.valueOf(timeToLive, Unit.TIME_INTERVAL)));
|
||||
|
@ -990,6 +967,7 @@ public class ColumnFamilyDescriptorBuilder {
|
|||
}
|
||||
|
||||
/**
|
||||
* Set minimum versions to retain.
|
||||
* @param minVersions The minimum number of versions to keep. (used when timeToLive is set)
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
|
@ -1019,6 +997,7 @@ public class ColumnFamilyDescriptorBuilder {
|
|||
}
|
||||
|
||||
/**
|
||||
* Set the blockCacheEnabled flag
|
||||
* @param blockCacheEnabled True if hfile DATA type blocks should be cached (We always cache
|
||||
* INDEX and BLOOM blocks; you cannot turn this off).
|
||||
* @return this (for chained invocation)
|
||||
|
@ -1043,10 +1022,6 @@ public class ColumnFamilyDescriptorBuilder {
|
|||
DEFAULT_REPLICATION_SCOPE);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param scope the scope tag
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
public ModifyableColumnFamilyDescriptor setScope(int scope) {
|
||||
return setValue(REPLICATION_SCOPE_BYTES, Integer.toString(scope));
|
||||
}
|
||||
|
@ -1058,6 +1033,7 @@ public class ColumnFamilyDescriptorBuilder {
|
|||
}
|
||||
|
||||
/**
|
||||
* Set the setCacheDataOnWrite flag
|
||||
* @param value true if we should cache data blocks on write
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
|
@ -1072,6 +1048,7 @@ public class ColumnFamilyDescriptorBuilder {
|
|||
}
|
||||
|
||||
/**
|
||||
* Set the setCacheIndexesOnWrite flag
|
||||
* @param value true if we should cache index blocks on write
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
|
@ -1086,6 +1063,7 @@ public class ColumnFamilyDescriptorBuilder {
|
|||
}
|
||||
|
||||
/**
|
||||
* Set the setCacheBloomsOnWrite flag.
|
||||
* @param value true if we should cache bloomfilter blocks on write
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
|
@ -1100,6 +1078,7 @@ public class ColumnFamilyDescriptorBuilder {
|
|||
}
|
||||
|
||||
/**
|
||||
* Set the setEvictBlocksOnClose flag.
|
||||
* @param value true if we should evict cached blocks from the blockcache on close
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
|
@ -1114,6 +1093,7 @@ public class ColumnFamilyDescriptorBuilder {
|
|||
}
|
||||
|
||||
/**
|
||||
* Set the setPrefetchBlocksOnOpen flag
|
||||
* @param value true if we should prefetch blocks into the blockcache on open
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
|
@ -1241,19 +1221,17 @@ public class ColumnFamilyDescriptorBuilder {
|
|||
return COMPARATOR.compare(this, other);
|
||||
}
|
||||
|
||||
/**
|
||||
* @return This instance serialized with pb with pb magic prefix
|
||||
* @see #parseFrom(byte[])
|
||||
*/
|
||||
/** Returns This instance serialized with pb with pb magic prefix */
|
||||
private byte[] toByteArray() {
|
||||
return ProtobufUtil.prependPBMagic(ProtobufUtil.toColumnFamilySchema(this).toByteArray());
|
||||
}
|
||||
|
||||
/**
|
||||
* Parse the serialized representation of a {@link ModifyableColumnFamilyDescriptor}
|
||||
* @param bytes A pb serialized {@link ModifyableColumnFamilyDescriptor} instance with pb magic
|
||||
* prefix
|
||||
* @return An instance of {@link ModifyableColumnFamilyDescriptor} made from <code>bytes</code>
|
||||
* n * @see #toByteArray()
|
||||
* @see #toByteArray()
|
||||
*/
|
||||
private static ColumnFamilyDescriptor parseFrom(final byte[] bytes)
|
||||
throws DeserializationException {
|
||||
|
@ -1299,8 +1277,8 @@ public class ColumnFamilyDescriptorBuilder {
|
|||
}
|
||||
|
||||
/**
|
||||
* Remove a configuration setting represented by the key from the {@link #configuration} map. n
|
||||
* * @return this (for chained invocation)
|
||||
* Remove a configuration setting represented by the key from the {@link #configuration} map.
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
public ModifyableColumnFamilyDescriptor removeConfiguration(final String key) {
|
||||
return setConfiguration(key, null);
|
||||
|
@ -1312,8 +1290,8 @@ public class ColumnFamilyDescriptorBuilder {
|
|||
}
|
||||
|
||||
/**
|
||||
* Set the encryption algorithm for use with this family n * @return this (for chained
|
||||
* invocation)
|
||||
* Set the encryption algorithm for use with this family
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
public ModifyableColumnFamilyDescriptor setEncryptionType(String algorithm) {
|
||||
return setValue(ENCRYPTION_BYTES, algorithm);
|
||||
|
@ -1325,7 +1303,8 @@ public class ColumnFamilyDescriptorBuilder {
|
|||
}
|
||||
|
||||
/**
|
||||
* Set the raw crypto key attribute for the family n * @return this (for chained invocation)
|
||||
* Set the raw crypto key attribute for the family
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
public ModifyableColumnFamilyDescriptor setEncryptionKey(byte[] keyBytes) {
|
||||
return setValue(ENCRYPTION_KEY_BYTES, new Bytes(keyBytes));
|
||||
|
|
|
@ -26,9 +26,7 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
@InterfaceAudience.Public
|
||||
public enum CompactType {
|
||||
|
||||
NORMAL(0),
|
||||
MOB(1);
|
||||
NORMAL,
|
||||
MOB
|
||||
|
||||
CompactType(int value) {
|
||||
}
|
||||
}
|
||||
|
|
|
@ -189,9 +189,13 @@ public class ConnectionImplementation implements ClusterConnection, Closeable {
|
|||
final int rpcTimeout;
|
||||
|
||||
/**
|
||||
* Global nonceGenerator shared per client.Currently there's no reason to limit its scope. Once
|
||||
* Global nonceGenerator shared per client. Currently there's no reason to limit its scope. Once
|
||||
* it's set under nonceGeneratorCreateLock, it is never unset or changed.
|
||||
*/
|
||||
// XXX: It is a bad pattern to assign a value to a static field from a constructor. However
|
||||
// it would likely change semantics if we change it because the NonceGenerator is selected
|
||||
// from configuration passed in as a parameter of the constructor. This has been cleaned up
|
||||
// in later branches.
|
||||
private static volatile NonceGenerator nonceGenerator = null;
|
||||
/** The nonce generator lock. Only taken when creating Connection, which gets a private copy. */
|
||||
private static final Object nonceGeneratorCreateLock = new Object();
|
||||
|
@ -294,7 +298,7 @@ public class ConnectionImplementation implements ClusterConnection, Closeable {
|
|||
}
|
||||
|
||||
this.stats = ServerStatisticTracker.create(conf);
|
||||
this.interceptor = (new RetryingCallerInterceptorFactory(conf)).build();
|
||||
this.interceptor = new RetryingCallerInterceptorFactory(conf).build();
|
||||
this.rpcControllerFactory = RpcControllerFactory.instantiate(conf);
|
||||
this.rpcCallerFactory = RpcRetryingCallerFactory.instantiate(conf, interceptor, this.stats);
|
||||
this.backoffPolicy = ClientBackoffPolicyFactory.create(conf);
|
||||
|
@ -1186,6 +1190,7 @@ public class ConnectionImplementation implements ClusterConnection, Closeable {
|
|||
* @param timeout how long to wait before timeout, in unit of millisecond
|
||||
* @param maxTries how many times to try
|
||||
*/
|
||||
@SuppressWarnings("JavaUtilDate")
|
||||
public ServerErrorTracker(long timeout, int maxTries) {
|
||||
this.maxTries = maxTries;
|
||||
this.canRetryUntil = EnvironmentEdgeManager.currentTime() + timeout;
|
||||
|
|
|
@ -97,6 +97,7 @@ public final class ConnectionUtils {
|
|||
}
|
||||
|
||||
/**
|
||||
* Inject a nonce generator for testing.
|
||||
* @param conn The connection for which to replace the generator.
|
||||
* @param cnm Replaces the nonce generator used, for testing.
|
||||
* @return old nonce generator.
|
||||
|
|
|
@ -109,8 +109,6 @@ public class Delete extends Mutation {
|
|||
* <p>
|
||||
* This timestamp is ONLY used for a delete row operation. If specifying families or columns, you
|
||||
* must specify each timestamp individually.
|
||||
* @param row We make a local copy of this passed in row. nn * @param timestamp maximum version
|
||||
* timestamp (only for delete row)
|
||||
*/
|
||||
public Delete(final byte[] row, final int rowOffset, final int rowLength, long timestamp) {
|
||||
checkRow(row, rowOffset, rowLength);
|
||||
|
@ -119,6 +117,7 @@ public class Delete extends Mutation {
|
|||
}
|
||||
|
||||
/**
|
||||
* Create a Delete operation using another Delete as template.
|
||||
* @param deleteToCopy delete to copy
|
||||
*/
|
||||
public Delete(final Delete deleteToCopy) {
|
||||
|
@ -154,6 +153,7 @@ public class Delete extends Mutation {
|
|||
* @param cell An existing cell of type "delete".
|
||||
* @return this for invocation chaining n
|
||||
*/
|
||||
@Override
|
||||
public Delete add(Cell cell) throws IOException {
|
||||
super.add(cell);
|
||||
return this;
|
||||
|
|
|
@ -509,7 +509,7 @@ public class Get extends Query implements Row {
|
|||
if (this == obj) {
|
||||
return true;
|
||||
}
|
||||
if (obj == null || getClass() != obj.getClass()) {
|
||||
if (!(obj instanceof Row)) {
|
||||
return false;
|
||||
}
|
||||
Row other = (Row) obj;
|
||||
|
|
|
@ -30,7 +30,6 @@ import java.util.Collections;
|
|||
import java.util.EnumSet;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
@ -46,7 +45,6 @@ import java.util.regex.Pattern;
|
|||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Abortable;
|
||||
import org.apache.hadoop.hbase.CacheEvictionStats;
|
||||
import org.apache.hadoop.hbase.CacheEvictionStatsBuilder;
|
||||
import org.apache.hadoop.hbase.ClusterMetrics;
|
||||
|
@ -782,7 +780,7 @@ public class HBaseAdmin implements Admin {
|
|||
*/
|
||||
@Override
|
||||
public HTableDescriptor[] deleteTables(Pattern pattern) throws IOException {
|
||||
List<HTableDescriptor> failed = new LinkedList<>();
|
||||
List<HTableDescriptor> failed = new ArrayList<>();
|
||||
for (HTableDescriptor table : listTables(pattern)) {
|
||||
try {
|
||||
deleteTable(table.getTableName());
|
||||
|
@ -900,7 +898,7 @@ public class HBaseAdmin implements Admin {
|
|||
|
||||
@Override
|
||||
public HTableDescriptor[] enableTables(Pattern pattern) throws IOException {
|
||||
List<HTableDescriptor> failed = new LinkedList<>();
|
||||
List<HTableDescriptor> failed = new ArrayList<>();
|
||||
for (HTableDescriptor table : listTables(pattern)) {
|
||||
if (isTableDisabled(table.getTableName())) {
|
||||
try {
|
||||
|
@ -960,7 +958,7 @@ public class HBaseAdmin implements Admin {
|
|||
|
||||
@Override
|
||||
public HTableDescriptor[] disableTables(Pattern pattern) throws IOException {
|
||||
List<HTableDescriptor> failed = new LinkedList<>();
|
||||
List<HTableDescriptor> failed = new ArrayList<>();
|
||||
for (HTableDescriptor table : listTables(pattern)) {
|
||||
if (isTableEnabled(table.getTableName())) {
|
||||
try {
|
||||
|
@ -1455,6 +1453,7 @@ public class HBaseAdmin implements Admin {
|
|||
move(encodedRegionName, (ServerName) null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void move(final byte[] encodedRegionName, ServerName destServerName) throws IOException {
|
||||
executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
|
||||
@Override
|
||||
|
@ -1557,7 +1556,7 @@ public class HBaseAdmin implements Admin {
|
|||
MetaTableAccessor.getTableRegionsAndLocations(connection, tableName);
|
||||
Map<ServerName,
|
||||
List<RegionInfo>> regionInfoByServerName = pairs.stream()
|
||||
.filter(pair -> !(pair.getFirst().isOffline())).filter(pair -> pair.getSecond() != null)
|
||||
.filter(pair -> !pair.getFirst().isOffline()).filter(pair -> pair.getSecond() != null)
|
||||
.collect(Collectors.groupingBy(pair -> pair.getSecond(),
|
||||
Collectors.mapping(pair -> pair.getFirst(), Collectors.toList())));
|
||||
|
||||
|
@ -2836,7 +2835,7 @@ public class HBaseAdmin implements Admin {
|
|||
|
||||
@Override
|
||||
public List<SnapshotDescription> listSnapshots(Pattern pattern) throws IOException {
|
||||
List<SnapshotDescription> matched = new LinkedList<>();
|
||||
List<SnapshotDescription> matched = new ArrayList<>();
|
||||
List<SnapshotDescription> snapshots = listSnapshots();
|
||||
for (SnapshotDescription snapshot : snapshots) {
|
||||
if (pattern.matcher(snapshot.getName()).matches()) {
|
||||
|
@ -2857,7 +2856,7 @@ public class HBaseAdmin implements Admin {
|
|||
Pattern snapshotNamePattern) throws IOException {
|
||||
TableName[] tableNames = listTableNames(tableNamePattern);
|
||||
|
||||
List<SnapshotDescription> tableSnapshots = new LinkedList<>();
|
||||
List<SnapshotDescription> tableSnapshots = new ArrayList<>();
|
||||
List<SnapshotDescription> snapshots = listSnapshots(snapshotNamePattern);
|
||||
|
||||
List<TableName> listOfTableNames = Arrays.asList(tableNames);
|
||||
|
@ -2959,7 +2958,7 @@ public class HBaseAdmin implements Admin {
|
|||
|
||||
@Override
|
||||
public List<QuotaSettings> getQuota(QuotaFilter filter) throws IOException {
|
||||
List<QuotaSettings> quotas = new LinkedList<>();
|
||||
List<QuotaSettings> quotas = new ArrayList<>();
|
||||
try (QuotaRetriever retriever = QuotaRetriever.open(conf, filter)) {
|
||||
Iterator<QuotaSettings> iterator = retriever.iterator();
|
||||
while (iterator.hasNext()) {
|
||||
|
@ -3017,21 +3016,6 @@ public class HBaseAdmin implements Admin {
|
|||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Simple {@link Abortable}, throwing RuntimeException on abort.
|
||||
*/
|
||||
private static class ThrowableAbortable implements Abortable {
|
||||
@Override
|
||||
public void abort(String why, Throwable e) {
|
||||
throw new RuntimeException(why, e);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isAborted() {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public CoprocessorRpcChannel coprocessorService(final ServerName serverName) {
|
||||
return new SyncCoprocessorRpcChannel() {
|
||||
|
|
|
@ -131,6 +131,7 @@ public class HRegionLocator implements RegionLocator {
|
|||
return this.tableName;
|
||||
}
|
||||
|
||||
@SuppressWarnings("MixedMutabilityReturnType")
|
||||
private List<RegionLocations> listRegionLocations() throws IOException {
|
||||
if (TableName.isMetaTableName(tableName)) {
|
||||
return Collections
|
||||
|
|
|
@ -53,7 +53,6 @@ import org.apache.hadoop.hbase.HRegionLocation;
|
|||
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.client.trace.TableOperationSpanBuilder;
|
||||
import org.apache.hadoop.hbase.client.trace.TableSpanBuilder;
|
||||
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
|
||||
|
@ -481,8 +480,8 @@ public class HTable implements Table {
|
|||
}
|
||||
|
||||
public static <R> void doBatchWithCallback(List<? extends Row> actions, Object[] results,
|
||||
Callback<R> callback, ClusterConnection connection, ExecutorService pool, TableName tableName)
|
||||
throws InterruptedIOException, RetriesExhaustedWithDetailsException {
|
||||
Batch.Callback<R> callback, ClusterConnection connection, ExecutorService pool,
|
||||
TableName tableName) throws InterruptedIOException, RetriesExhaustedWithDetailsException {
|
||||
int operationTimeout = connection.getConnectionConfiguration().getOperationTimeout();
|
||||
int writeTimeout = connection.getConfiguration().getInt(HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY,
|
||||
connection.getConfiguration().getInt(HConstants.HBASE_RPC_TIMEOUT_KEY,
|
||||
|
@ -1264,7 +1263,7 @@ public class HTable implements Table {
|
|||
@Override
|
||||
public <R extends Message> void batchCoprocessorService(
|
||||
final Descriptors.MethodDescriptor methodDescriptor, final Message request, byte[] startKey,
|
||||
byte[] endKey, final R responsePrototype, final Callback<R> callback)
|
||||
byte[] endKey, final R responsePrototype, final Batch.Callback<R> callback)
|
||||
throws ServiceException, Throwable {
|
||||
final Supplier<Span> supplier = new TableOperationSpanBuilder(connection)
|
||||
.setTableName(tableName).setOperation(HBaseSemanticAttributes.Operation.COPROC_EXEC);
|
||||
|
@ -1310,7 +1309,7 @@ public class HTable implements Table {
|
|||
RpcRetryingCallerFactory.instantiate(configuration, connection.getStatisticsTracker()),
|
||||
RpcControllerFactory.instantiate(configuration));
|
||||
|
||||
Callback<ClientProtos.CoprocessorServiceResult> resultsCallback =
|
||||
Batch.Callback<ClientProtos.CoprocessorServiceResult> resultsCallback =
|
||||
(byte[] region, byte[] row, ClientProtos.CoprocessorServiceResult serviceResult) -> {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Received result for endpoint {}: region={}, row={}, value={}",
|
||||
|
|
|
@ -222,6 +222,7 @@ public class HTableMultiplexer {
|
|||
}
|
||||
|
||||
@InterfaceAudience.Private
|
||||
@SuppressWarnings("FutureReturnValueIgnored")
|
||||
LinkedBlockingQueue<PutStatus> getQueue(HRegionLocation addr) {
|
||||
FlushWorker worker = serverToFlushWorkerMap.get(addr);
|
||||
if (worker == null) {
|
||||
|
@ -455,6 +456,7 @@ public class HTableMultiplexer {
|
|||
return this.maxLatency.getAndSet(0);
|
||||
}
|
||||
|
||||
@SuppressWarnings("FutureReturnValueIgnored")
|
||||
boolean resubmitFailedPut(PutStatus ps, HRegionLocation oldLoc) throws IOException {
|
||||
// Decrease the retry count
|
||||
final int retryCount = ps.maxAttempCount - 1;
|
||||
|
|
|
@ -94,6 +94,7 @@ public class Increment extends Mutation {
|
|||
* Add the specified KeyValue to this operation.
|
||||
* @param cell individual Cell n * @throws java.io.IOException e
|
||||
*/
|
||||
@Override
|
||||
public Increment add(Cell cell) throws IOException {
|
||||
super.add(cell);
|
||||
return this;
|
||||
|
@ -274,7 +275,7 @@ public class Increment extends Mutation {
|
|||
if (this == obj) {
|
||||
return true;
|
||||
}
|
||||
if (obj == null || getClass() != obj.getClass()) {
|
||||
if (!(obj instanceof Increment)) {
|
||||
return false;
|
||||
}
|
||||
Row other = (Row) obj;
|
||||
|
|
|
@ -29,11 +29,8 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
@InterfaceAudience.Public
|
||||
public enum IsolationLevel {
|
||||
|
||||
READ_COMMITTED(1),
|
||||
READ_UNCOMMITTED(2);
|
||||
|
||||
IsolationLevel(int value) {
|
||||
}
|
||||
READ_COMMITTED,
|
||||
READ_UNCOMMITTED;
|
||||
|
||||
public byte[] toBytes() {
|
||||
return new byte[] { toByte() };
|
||||
|
|
|
@ -112,13 +112,10 @@ public class LogQueryFilter {
|
|||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
if (!(o instanceof LogQueryFilter)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
LogQueryFilter that = (LogQueryFilter) o;
|
||||
|
||||
return new EqualsBuilder().append(limit, that.limit).append(regionName, that.regionName)
|
||||
.append(clientAddress, that.clientAddress).append(tableName, that.tableName)
|
||||
.append(userName, that.userName).append(type, that.type)
|
||||
|
|
|
@ -77,11 +77,11 @@ class MasterCoprocessorRpcChannelImpl implements RpcChannel {
|
|||
Message responsePrototype, RpcCallback<Message> done) {
|
||||
addListener(
|
||||
callerBuilder.action((c, s) -> rpcCall(method, request, responsePrototype, c, s)).call(),
|
||||
((r, e) -> {
|
||||
(r, e) -> {
|
||||
if (e != null) {
|
||||
((ClientCoprocessorRpcController) controller).setFailed(e);
|
||||
}
|
||||
done.run(r);
|
||||
}));
|
||||
});
|
||||
}
|
||||
}
|
||||
|
|
|
@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.util.DNS.ServerType;
|
|||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
|
||||
import org.apache.hbase.thirdparty.com.google.common.base.Splitter;
|
||||
import org.apache.hbase.thirdparty.com.google.common.base.Strings;
|
||||
import org.apache.hbase.thirdparty.com.google.common.net.HostAndPort;
|
||||
|
||||
|
@ -78,7 +79,8 @@ public class MasterRegistry extends AbstractRpcBasedConnectionRegistry {
|
|||
public static Set<ServerName> parseMasterAddrs(Configuration conf) throws UnknownHostException {
|
||||
Set<ServerName> masterAddrs = new HashSet<>();
|
||||
String configuredMasters = getMasterAddr(conf);
|
||||
for (String masterAddr : configuredMasters.split(MASTER_ADDRS_CONF_SEPARATOR)) {
|
||||
for (String masterAddr : Splitter.onPattern(MASTER_ADDRS_CONF_SEPARATOR)
|
||||
.split(configuredMasters)) {
|
||||
HostAndPort masterHostPort =
|
||||
HostAndPort.fromString(masterAddr.trim()).withDefaultPort(HConstants.DEFAULT_MASTER_PORT);
|
||||
masterAddrs.add(ServerName.valueOf(masterHostPort.toString(), ServerName.NON_STARTCODE));
|
||||
|
|
|
@ -36,8 +36,10 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
public final class MultiAction {
|
||||
// TODO: This class should not be visible outside of the client package.
|
||||
|
||||
// map of regions to lists of puts/gets/deletes for that region.
|
||||
protected Map<byte[], List<Action>> actions = new TreeMap<>(Bytes.BYTES_COMPARATOR);
|
||||
/**
|
||||
* A map of regions to lists of puts/gets/deletes for that region. Package visible.
|
||||
*/
|
||||
Map<byte[], List<Action>> actions = new TreeMap<>(Bytes.BYTES_COMPARATOR);
|
||||
|
||||
private long nonceGroup = HConstants.NO_NONCE;
|
||||
|
||||
|
@ -60,7 +62,7 @@ public final class MultiAction {
|
|||
/**
|
||||
* Add an Action to this container based on it's regionName. If the regionName is wrong, the
|
||||
* initial execution will fail, but will be automatically retried after looking up the correct
|
||||
* region. nn
|
||||
* region.
|
||||
*/
|
||||
public void add(byte[] regionName, Action a) {
|
||||
add(regionName, Collections.singletonList(a));
|
||||
|
@ -69,7 +71,7 @@ public final class MultiAction {
|
|||
/**
|
||||
* Add an Action to this container based on it's regionName. If the regionName is wrong, the
|
||||
* initial execution will fail, but will be automatically retried after looking up the correct
|
||||
* region. n * @param actionList list of actions to add for the region
|
||||
* region.
|
||||
*/
|
||||
public void add(byte[] regionName, List<Action> actionList) {
|
||||
List<Action> rsActions = actions.get(regionName);
|
||||
|
|
|
@ -53,11 +53,7 @@ public class MultiResponse extends AbstractResponse {
|
|||
return size;
|
||||
}
|
||||
|
||||
/**
|
||||
* Add the pair to the container, grouped by the regionName n * @param originalIndex the original
|
||||
* index of the Action (request).
|
||||
* @param resOrEx the result or error; will be empty for successful Put and Delete actions.
|
||||
*/
|
||||
/** Add the pair to the container, grouped by the regionName. */
|
||||
public void add(byte[] regionName, int originalIndex, Object resOrEx) {
|
||||
getResult(regionName).addResult(originalIndex, resOrEx);
|
||||
}
|
||||
|
|
|
@ -44,7 +44,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActi
|
|||
* Callable that handles the <code>multi</code> method call going against a single regionserver;
|
||||
* i.e. A RegionServerCallable for the multi call (It is NOT a RegionServerCallable that goes
|
||||
* against multiple regions).
|
||||
* @param <R>
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
class MultiServerCallable extends CancellableRegionServerCallable<MultiResponse> {
|
||||
|
|
|
@ -230,6 +230,7 @@ class MutableRegionInfo implements RegionInfo {
|
|||
}
|
||||
|
||||
/**
|
||||
* Change the split status flag.
|
||||
* @param split set split status n
|
||||
*/
|
||||
public MutableRegionInfo setSplit(boolean split) {
|
||||
|
|
|
@ -152,7 +152,7 @@ public abstract class Mutation extends OperationWithAttributes
|
|||
return list;
|
||||
}
|
||||
|
||||
/*
|
||||
/**
|
||||
* Create a KeyValue with this objects row key and the Put identifier.
|
||||
* @return a KeyValue with this objects row key and the Put identifier.
|
||||
*/
|
||||
|
@ -161,8 +161,7 @@ public abstract class Mutation extends OperationWithAttributes
|
|||
}
|
||||
|
||||
/**
|
||||
* Create a KeyValue with this objects row key and the Put identifier. nnnn * @param tags -
|
||||
* Specify the Tags as an Array
|
||||
* Create a KeyValue with this objects row key and the Put identifier.
|
||||
* @return a KeyValue with this objects row key and the Put identifier.
|
||||
*/
|
||||
KeyValue createPutKeyValue(byte[] family, byte[] qualifier, long ts, byte[] value, Tag[] tags) {
|
||||
|
@ -170,7 +169,7 @@ public abstract class Mutation extends OperationWithAttributes
|
|||
return kvWithTag;
|
||||
}
|
||||
|
||||
/*
|
||||
/**
|
||||
* Create a KeyValue with this objects row key and the Put identifier.
|
||||
* @return a KeyValue with this objects row key and the Put identifier.
|
||||
*/
|
||||
|
@ -183,7 +182,7 @@ public abstract class Mutation extends OperationWithAttributes
|
|||
|
||||
/**
|
||||
* Compile the column family (i.e. schema) information into a Map. Useful for parsing and
|
||||
* aggregation by debugging, logging, and administration tools. n
|
||||
* aggregation by debugging, logging, and administration tools.
|
||||
*/
|
||||
@Override
|
||||
public Map<String, Object> getFingerprint() {
|
||||
|
@ -257,7 +256,7 @@ public abstract class Mutation extends OperationWithAttributes
|
|||
if (tags != null) {
|
||||
List<String> tagsString = new ArrayList<>(tags.size());
|
||||
for (Tag t : tags) {
|
||||
tagsString.add((t.getType()) + ":" + Bytes.toStringBinary(Tag.cloneValue(t)));
|
||||
tagsString.add(t.getType() + ":" + Bytes.toStringBinary(Tag.cloneValue(t)));
|
||||
}
|
||||
stringMap.put("tag", tagsString);
|
||||
}
|
||||
|
@ -473,6 +472,7 @@ public abstract class Mutation extends OperationWithAttributes
|
|||
}
|
||||
|
||||
/**
|
||||
* Set the ACL for this operation.
|
||||
* @param user User short name
|
||||
* @param perms Permissions for the user
|
||||
*/
|
||||
|
@ -483,6 +483,7 @@ public abstract class Mutation extends OperationWithAttributes
|
|||
}
|
||||
|
||||
/**
|
||||
* Set the ACL for this operation.
|
||||
* @param perms A map of permissions for a user or users
|
||||
*/
|
||||
public Mutation setACL(Map<String, Permission> perms) {
|
||||
|
@ -624,11 +625,9 @@ public abstract class Mutation extends OperationWithAttributes
|
|||
return filteredList;
|
||||
}
|
||||
|
||||
/*
|
||||
/**
|
||||
* Private method to determine if this object's familyMap contains the given value assigned to the
|
||||
* given family, qualifier and timestamp respecting the 2 boolean arguments nnnnnn * @return
|
||||
* returns true if the given family, qualifier timestamp and value already has an existing
|
||||
* KeyValue object in the family map.
|
||||
* given family, qualifier and timestamp, respecting the 2 boolean arguments.
|
||||
*/
|
||||
protected boolean has(byte[] family, byte[] qualifier, long ts, byte[] value, boolean ignoreTS,
|
||||
boolean ignoreValue) {
|
||||
|
|
|
@ -86,7 +86,8 @@ class PreemptiveFastFailInterceptor extends RetryingCallerInterceptor {
|
|||
// fast fail mode for any reason.
|
||||
private long fastFailClearingTimeMilliSec;
|
||||
|
||||
private final ThreadLocal<MutableBoolean> threadRetryingInFastFailMode = new ThreadLocal<>();
|
||||
private static final ThreadLocal<MutableBoolean> threadRetryingInFastFailMode =
|
||||
new ThreadLocal<>();
|
||||
|
||||
public PreemptiveFastFailInterceptor(Configuration conf) {
|
||||
this.fastFailThresholdMilliSec = conf.getLong(HConstants.HBASE_CLIENT_FAST_FAIL_THREASHOLD_MS,
|
||||
|
@ -216,8 +217,8 @@ class PreemptiveFastFailInterceptor extends RetryingCallerInterceptor {
|
|||
* @return true, if the thread is already in FF mode.
|
||||
*/
|
||||
private boolean currentThreadInFastFailMode() {
|
||||
return (this.threadRetryingInFastFailMode.get() != null
|
||||
&& (this.threadRetryingInFastFailMode.get().booleanValue() == true));
|
||||
return (threadRetryingInFastFailMode.get() != null
|
||||
&& (threadRetryingInFastFailMode.get().booleanValue() == true));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -232,10 +233,10 @@ class PreemptiveFastFailInterceptor extends RetryingCallerInterceptor {
|
|||
// actively trying to connect. If we are the chosen one, we will retry
|
||||
// and not throw an exception.
|
||||
if (fInfo != null && fInfo.exclusivelyRetringInspiteOfFastFail.compareAndSet(false, true)) {
|
||||
MutableBoolean threadAlreadyInFF = this.threadRetryingInFastFailMode.get();
|
||||
MutableBoolean threadAlreadyInFF = threadRetryingInFastFailMode.get();
|
||||
if (threadAlreadyInFF == null) {
|
||||
threadAlreadyInFF = new MutableBoolean();
|
||||
this.threadRetryingInFastFailMode.set(threadAlreadyInFF);
|
||||
threadRetryingInFastFailMode.set(threadAlreadyInFF);
|
||||
}
|
||||
threadAlreadyInFF.setValue(true);
|
||||
return true;
|
||||
|
|
|
@ -257,6 +257,7 @@ public class Put extends Mutation implements HeapSize {
|
|||
* immutable and its backing array will not be modified for the duration of this Put.
|
||||
* @param cell individual cell n * @throws java.io.IOException e
|
||||
*/
|
||||
@Override
|
||||
public Put add(Cell cell) throws IOException {
|
||||
super.add(cell);
|
||||
return this;
|
||||
|
|
|
@ -88,6 +88,7 @@ public abstract class Query extends OperationWithAttributes {
|
|||
}
|
||||
|
||||
/**
|
||||
* Set the ACL for the operation.
|
||||
* @param user User short name
|
||||
* @param perms Permissions for the user
|
||||
*/
|
||||
|
@ -98,6 +99,7 @@ public abstract class Query extends OperationWithAttributes {
|
|||
}
|
||||
|
||||
/**
|
||||
* Set the ACL for the operation.
|
||||
* @param perms A map of permissions for a user or users
|
||||
*/
|
||||
public Query setACL(Map<String, Permission> perms) {
|
||||
|
@ -159,9 +161,8 @@ public abstract class Query extends OperationWithAttributes {
|
|||
}
|
||||
|
||||
/**
|
||||
* @return The isolation level of this query. If no isolation level was set for this query object,
|
||||
* then it returns READ_COMMITTED.
|
||||
* @return The IsolationLevel for this query
|
||||
* Returns The isolation level of this query. If no isolation level was set for this query object,
|
||||
* then it returns READ_COMMITTED.
|
||||
*/
|
||||
public IsolationLevel getIsolationLevel() {
|
||||
byte[] attr = getAttribute(ISOLATION_LEVEL);
|
||||
|
|
|
@ -469,11 +469,6 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
|
|||
return future;
|
||||
}
|
||||
|
||||
@FunctionalInterface
|
||||
private interface TableOperator {
|
||||
CompletableFuture<Void> operate(TableName table);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompletableFuture<Boolean> tableExists(TableName tableName) {
|
||||
if (TableName.isMetaTableName(tableName)) {
|
||||
|
@ -1539,11 +1534,13 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
|
|||
future.completeExceptionally(err);
|
||||
return;
|
||||
}
|
||||
addListener(this.<Void> newMasterCaller().priority(regionInfo.getTable())
|
||||
.action(((controller, stub) -> this.<AssignRegionRequest, AssignRegionResponse, Void> call(
|
||||
controller, stub, RequestConverter.buildAssignRegionRequest(regionInfo.getRegionName()),
|
||||
(s, c, req, done) -> s.assignRegion(c, req, done), resp -> null)))
|
||||
.call(), (ret, err2) -> {
|
||||
addListener(
|
||||
this.<Void> newMasterCaller().priority(regionInfo.getTable())
|
||||
.action((controller, stub) -> this.<AssignRegionRequest, AssignRegionResponse, Void> call(
|
||||
controller, stub, RequestConverter.buildAssignRegionRequest(regionInfo.getRegionName()),
|
||||
(s, c, req, done) -> s.assignRegion(c, req, done), resp -> null))
|
||||
.call(),
|
||||
(ret, err2) -> {
|
||||
if (err2 != null) {
|
||||
future.completeExceptionally(err2);
|
||||
} else {
|
||||
|
@ -1564,10 +1561,10 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
|
|||
}
|
||||
addListener(
|
||||
this.<Void> newMasterCaller().priority(regionInfo.getTable())
|
||||
.action(((controller, stub) -> this.<UnassignRegionRequest, UnassignRegionResponse,
|
||||
.action((controller, stub) -> this.<UnassignRegionRequest, UnassignRegionResponse,
|
||||
Void> call(controller, stub,
|
||||
RequestConverter.buildUnassignRegionRequest(regionInfo.getRegionName()),
|
||||
(s, c, req, done) -> s.unassignRegion(c, req, done), resp -> null)))
|
||||
(s, c, req, done) -> s.unassignRegion(c, req, done), resp -> null))
|
||||
.call(),
|
||||
(ret, err2) -> {
|
||||
if (err2 != null) {
|
||||
|
@ -1588,14 +1585,11 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
|
|||
future.completeExceptionally(err);
|
||||
return;
|
||||
}
|
||||
addListener(
|
||||
this.<Void> newMasterCaller().priority(regionInfo.getTable())
|
||||
.action(((controller, stub) -> this.<OfflineRegionRequest, OfflineRegionResponse,
|
||||
Void> call(controller, stub,
|
||||
RequestConverter.buildOfflineRegionRequest(regionInfo.getRegionName()),
|
||||
(s, c, req, done) -> s.offlineRegion(c, req, done), resp -> null)))
|
||||
.call(),
|
||||
(ret, err2) -> {
|
||||
addListener(this.<Void> newMasterCaller().priority(regionInfo.getTable())
|
||||
.action((controller, stub) -> this.<OfflineRegionRequest, OfflineRegionResponse, Void> call(
|
||||
controller, stub, RequestConverter.buildOfflineRegionRequest(regionInfo.getRegionName()),
|
||||
(s, c, req, done) -> s.offlineRegion(c, req, done), resp -> null))
|
||||
.call(), (ret, err2) -> {
|
||||
if (err2 != null) {
|
||||
future.completeExceptionally(err2);
|
||||
} else {
|
||||
|
@ -2206,7 +2200,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
|
|||
listSnapshotsFuture = getCompletedSnapshots(tableNamePattern, snapshotNamePattern);
|
||||
}
|
||||
CompletableFuture<Void> future = new CompletableFuture<>();
|
||||
addListener(listSnapshotsFuture, ((snapshotDescriptions, err) -> {
|
||||
addListener(listSnapshotsFuture, (snapshotDescriptions, err) -> {
|
||||
if (err != null) {
|
||||
future.completeExceptionally(err);
|
||||
return;
|
||||
|
@ -2223,7 +2217,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
|
|||
future.complete(v);
|
||||
}
|
||||
});
|
||||
}));
|
||||
});
|
||||
return future;
|
||||
}
|
||||
|
||||
|
@ -3966,9 +3960,9 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
|
|||
|
||||
private CompletableFuture<Boolean> clearSlowLogsResponses(final ServerName serverName) {
|
||||
return this.<Boolean> newAdminCaller()
|
||||
.action(((controller, stub) -> this.adminCall(controller, stub,
|
||||
.action((controller, stub) -> this.adminCall(controller, stub,
|
||||
RequestConverter.buildClearSlowLogResponseRequest(),
|
||||
AdminService.Interface::clearSlowLogsResponses, ProtobufUtil::toClearSlowLogPayload)))
|
||||
AdminService.Interface::clearSlowLogsResponses, ProtobufUtil::toClearSlowLogPayload))
|
||||
.serverName(serverName).call();
|
||||
}
|
||||
|
||||
|
@ -4030,9 +4024,9 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
|
|||
public CompletableFuture<Void> flushMasterStore() {
|
||||
FlushMasterStoreRequest.Builder request = FlushMasterStoreRequest.newBuilder();
|
||||
return this.<Void> newMasterCaller()
|
||||
.action(((controller, stub) -> this.<FlushMasterStoreRequest, FlushMasterStoreResponse,
|
||||
.action((controller, stub) -> this.<FlushMasterStoreRequest, FlushMasterStoreResponse,
|
||||
Void> call(controller, stub, request.build(),
|
||||
(s, c, req, done) -> s.flushMasterStore(c, req, done), resp -> null)))
|
||||
(s, c, req, done) -> s.flushMasterStore(c, req, done), resp -> null))
|
||||
.call();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -814,7 +814,7 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
|
|||
int c = Bytes.compareTo(endKey, region.getEndKey());
|
||||
// 1. if the region contains endKey
|
||||
// 2. endKey is equal to the region's endKey and we do not want to include endKey.
|
||||
return c < 0 || c == 0 && !endKeyInclusive;
|
||||
return c < 0 || (c == 0 && !endKeyInclusive);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -93,7 +93,7 @@ public class RegionCoprocessorServiceExec implements Row {
|
|||
if (this == obj) {
|
||||
return true;
|
||||
}
|
||||
if (obj == null || getClass() != obj.getClass()) {
|
||||
if (!(obj instanceof RegionCoprocessorServiceExec)) {
|
||||
return false;
|
||||
}
|
||||
return compareTo((RegionCoprocessorServiceExec) obj) == 0;
|
||||
|
|
|
@ -160,7 +160,7 @@ public interface RegionInfo extends Comparable<RegionInfo> {
|
|||
long getRegionId();
|
||||
|
||||
/**
|
||||
* @return the regionName as an array of bytes.
|
||||
* Returns the regionName as an array of bytes.
|
||||
* @see #getRegionNameAsString()
|
||||
*/
|
||||
byte[] getRegionName();
|
||||
|
@ -190,7 +190,7 @@ public interface RegionInfo extends Comparable<RegionInfo> {
|
|||
boolean isSplit();
|
||||
|
||||
/**
|
||||
* @return True if this region is offline.
|
||||
* Returns True if this region is offline.
|
||||
* @deprecated since 3.0.0 and will be removed in 4.0.0
|
||||
* @see <a href="https://issues.apache.org/jira/browse/HBASE-25210">HBASE-25210</a>
|
||||
*/
|
||||
|
@ -198,7 +198,7 @@ public interface RegionInfo extends Comparable<RegionInfo> {
|
|||
boolean isOffline();
|
||||
|
||||
/**
|
||||
* @return True if this is a split parent region.
|
||||
* Returns True if this is a split parent region.
|
||||
* @deprecated since 3.0.0 and will be removed in 4.0.0, Use {@link #isSplit()} instead.
|
||||
* @see <a href="https://issues.apache.org/jira/browse/HBASE-25210">HBASE-25210</a>
|
||||
*/
|
||||
|
@ -209,9 +209,9 @@ public interface RegionInfo extends Comparable<RegionInfo> {
|
|||
boolean isMetaRegion();
|
||||
|
||||
/**
|
||||
* @return true if the given inclusive range of rows is fully contained by this region. For
|
||||
* example, if the region is foo,a,g and this is passed ["b","c"] or ["a","c"] it will
|
||||
* return true, but if this is passed ["b","z"] it will return false.
|
||||
* Returns true if the given inclusive range of rows is fully contained by this region. For
|
||||
* example, if the region is foo,a,g and this is passed ["b","c"] or ["a","c"] it will return
|
||||
* true, but if this is passed ["b","z"] it will return false.
|
||||
* @throws IllegalArgumentException if the range passed is invalid (ie. end < start)
|
||||
*/
|
||||
boolean containsRange(byte[] rangeStartKey, byte[] rangeEndKey);
|
||||
|
@ -271,16 +271,16 @@ public interface RegionInfo extends Comparable<RegionInfo> {
|
|||
}
|
||||
|
||||
/**
|
||||
* @return Return a String of short, printable names for <code>hris</code> (usually encoded name)
|
||||
* for us logging.
|
||||
* Returns a String of short, printable names for <code>hris</code> (usually encoded name) for us
|
||||
* logging.
|
||||
*/
|
||||
static String getShortNameToLog(RegionInfo... hris) {
|
||||
return getShortNameToLog(Arrays.asList(hris));
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Return a String of short, printable names for <code>hris</code> (usually encoded name)
|
||||
* for us logging.
|
||||
* Returns a String of short, printable names for <code>hris</code> (usually encoded name) for us
|
||||
* logging.
|
||||
*/
|
||||
static String getShortNameToLog(final List<RegionInfo> ris) {
|
||||
return ris.stream().map(RegionInfo::getEncodedName).collect(Collectors.toList()).toString();
|
||||
|
@ -369,8 +369,7 @@ public interface RegionInfo extends Comparable<RegionInfo> {
|
|||
}
|
||||
|
||||
/**
|
||||
* @param bytes A pb RegionInfo serialized with a pb magic prefix.
|
||||
* @return A deserialized {@link RegionInfo}
|
||||
* Returns A deserialized {@link RegionInfo}
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
static RegionInfo parseFrom(final byte[] bytes) throws DeserializationException {
|
||||
|
@ -379,6 +378,7 @@ public interface RegionInfo extends Comparable<RegionInfo> {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse a serialized representation of {@link RegionInfo}
|
||||
* @param bytes A pb RegionInfo serialized with a pb magic prefix.
|
||||
* @param offset starting point in the byte array
|
||||
* @param len length to read on the byte array
|
||||
|
@ -437,7 +437,7 @@ public interface RegionInfo extends Comparable<RegionInfo> {
|
|||
}
|
||||
|
||||
/**
|
||||
* @return This instance serialized as protobuf w/ a magic pb prefix.
|
||||
* Returns This instance serialized as protobuf w/ a magic pb prefix.
|
||||
* @see #parseFrom(byte[])
|
||||
*/
|
||||
static byte[] toByteArray(RegionInfo ri) {
|
||||
|
@ -765,7 +765,7 @@ public interface RegionInfo extends Comparable<RegionInfo> {
|
|||
}
|
||||
|
||||
/**
|
||||
* @return True if region is next, adjacent but 'after' this one.
|
||||
* Returns True if region is next, adjacent but 'after' this one.
|
||||
* @see #isAdjacent(RegionInfo)
|
||||
* @see #areAdjacent(RegionInfo, RegionInfo)
|
||||
*/
|
||||
|
@ -774,7 +774,7 @@ public interface RegionInfo extends Comparable<RegionInfo> {
|
|||
}
|
||||
|
||||
/**
|
||||
* @return True if region is adjacent, either just before or just after this one.
|
||||
* Returns True if region is adjacent, either just before or just after this one.
|
||||
* @see #isNext(RegionInfo)
|
||||
*/
|
||||
default boolean isAdjacent(RegionInfo other) {
|
||||
|
@ -787,7 +787,7 @@ public interface RegionInfo extends Comparable<RegionInfo> {
|
|||
}
|
||||
|
||||
/**
|
||||
* @return True if an overlap in region range.
|
||||
* Returns True if an overlap in region range.
|
||||
* @see #isDegenerate()
|
||||
*/
|
||||
default boolean isOverlap(RegionInfo other) {
|
||||
|
@ -813,6 +813,7 @@ public interface RegionInfo extends Comparable<RegionInfo> {
|
|||
return Bytes.compareTo(getStartKey(), other.getEndKey()) < 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
default int compareTo(RegionInfo other) {
|
||||
return RegionInfo.COMPARATOR.compare(this, other);
|
||||
}
|
||||
|
|
|
@ -51,7 +51,7 @@ public class RegionReplicaUtil {
|
|||
/**
|
||||
* Returns the RegionInfo for the given replicaId. RegionInfo's correspond to a range of a table,
|
||||
* but more than one "instance" of the same range can be deployed which are differentiated by the
|
||||
* replicaId. n * @param replicaId the replicaId to use
|
||||
* replicaId.
|
||||
* @return an RegionInfo object corresponding to the same range (table, start and end key), but
|
||||
* for the given replicaId.
|
||||
*/
|
||||
|
@ -149,7 +149,7 @@ public class RegionReplicaUtil {
|
|||
if ((newReplicaCount - 1) <= 0) {
|
||||
return regions;
|
||||
}
|
||||
List<RegionInfo> hRegionInfos = new ArrayList<>((newReplicaCount) * regions.size());
|
||||
List<RegionInfo> hRegionInfos = new ArrayList<>(newReplicaCount * regions.size());
|
||||
for (RegionInfo ri : regions) {
|
||||
if (
|
||||
RegionReplicaUtil.isDefaultReplica(ri)
|
||||
|
|
|
@ -77,11 +77,11 @@ public class RegionServerCoprocessorRpcChannelImpl implements RpcChannel {
|
|||
Message responsePrototype, RpcCallback<Message> done) {
|
||||
addListener(
|
||||
callerBuilder.action((c, s) -> rpcCall(method, request, responsePrototype, c, s)).call(),
|
||||
((r, e) -> {
|
||||
(r, e) -> {
|
||||
if (e != null) {
|
||||
((ClientCoprocessorRpcController) controller).setFailed(e);
|
||||
}
|
||||
done.run(r);
|
||||
}));
|
||||
});
|
||||
}
|
||||
}
|
||||
|
|
|
@ -873,6 +873,7 @@ public class Result implements CellScannable, CellScanner {
|
|||
}
|
||||
|
||||
/**
|
||||
* Whether or not the results are partial.
|
||||
* @deprecated the word 'partial' ambiguous, use {@link #mayHaveMoreCellsInRow()} instead.
|
||||
* Deprecated since 1.4.0.
|
||||
* @see #mayHaveMoreCellsInRow()
|
||||
|
@ -904,8 +905,8 @@ public class Result implements CellScannable, CellScanner {
|
|||
}
|
||||
|
||||
/**
|
||||
* @return the associated statistics about the region from which this was returned. Can be
|
||||
* <tt>null</tt> if stats are disabled.
|
||||
* Returns the associated statistics about the region from which this was returned. Can be
|
||||
* <tt>null</tt> if stats are disabled.
|
||||
*/
|
||||
public RegionLoadStats getStats() {
|
||||
return stats;
|
||||
|
|
|
@ -46,13 +46,14 @@ public interface RetryingCallable<T> {
|
|||
void throwable(final Throwable t, boolean retrying);
|
||||
|
||||
/**
|
||||
* @return Some details from the implementation that we would like to add to a terminating
|
||||
* exception; i.e. a fatal exception is being thrown ending retries and we might like to
|
||||
* add more implementation-specific detail on to the exception being thrown.
|
||||
* Returns Some details from the implementation that we would like to add to a terminating
|
||||
* exception; i.e. a fatal exception is being thrown ending retries and we might like to add more
|
||||
* implementation-specific detail on to the exception being thrown.
|
||||
*/
|
||||
String getExceptionMessageAdditionalDetail();
|
||||
|
||||
/**
|
||||
* Sleep and retry.
|
||||
* @param pause time to pause
|
||||
* @param tries amount of tries until till sleep
|
||||
* @return Suggestion on how much to sleep between retries
|
||||
|
|
|
@ -32,9 +32,11 @@ public interface Row extends Comparable<Row> {
|
|||
byte[] getRow();
|
||||
|
||||
/**
|
||||
* Compare this row to another row.
|
||||
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Use
|
||||
* {@link Row#COMPARATOR} instead
|
||||
*/
|
||||
@Deprecated
|
||||
@Override
|
||||
int compareTo(Row var1);
|
||||
}
|
||||
|
|
|
@ -22,7 +22,6 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
/**
|
||||
* Provide a way to access the inner buffer. The purpose is to reduce the elapsed time to move a
|
||||
* large number of elements between collections.
|
||||
* @param <T>
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public interface RowAccess<T> extends Iterable<T> {
|
||||
|
|
|
@ -102,6 +102,7 @@ public class RowMutations implements Row {
|
|||
}
|
||||
|
||||
/**
|
||||
* Add a list of mutations
|
||||
* @param mutations The data to send.
|
||||
* @throws IOException if the row of added mutation doesn't match the original row
|
||||
*/
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.util.ReflectionUtils;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
|
@ -34,9 +33,6 @@ public class RpcRetryingCallerFactory {
|
|||
private final ConnectionConfiguration connectionConf;
|
||||
private final RetryingCallerInterceptor interceptor;
|
||||
private final int startLogErrorsCnt;
|
||||
/* These below data members are UNUSED!!! */
|
||||
private final boolean enableBackPressure;
|
||||
private ServerStatisticTracker stats;
|
||||
|
||||
public RpcRetryingCallerFactory(Configuration conf) {
|
||||
this(conf, RetryingCallerInterceptorFactory.NO_OP_INTERCEPTOR);
|
||||
|
@ -48,15 +44,6 @@ public class RpcRetryingCallerFactory {
|
|||
startLogErrorsCnt = conf.getInt(AsyncProcess.START_LOG_ERRORS_AFTER_COUNT_KEY,
|
||||
AsyncProcess.DEFAULT_START_LOG_ERRORS_AFTER_COUNT);
|
||||
this.interceptor = interceptor;
|
||||
enableBackPressure = conf.getBoolean(HConstants.ENABLE_CLIENT_BACKPRESSURE,
|
||||
HConstants.DEFAULT_ENABLE_CLIENT_BACKPRESSURE);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the tracker that should be used for tracking statistics about the server
|
||||
*/
|
||||
public void setStatisticTracker(ServerStatisticTracker statisticTracker) {
|
||||
this.stats = statisticTracker;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -102,9 +89,6 @@ public class RpcRetryingCallerFactory {
|
|||
factory = ReflectionUtils.instantiateWithCustomCtor(rpcCallerFactoryClazz,
|
||||
new Class[] { Configuration.class }, new Object[] { configuration });
|
||||
}
|
||||
|
||||
// setting for backwards compat with existing caller factories, rather than in the ctor
|
||||
factory.setStatisticTracker(stats);
|
||||
return factory;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -156,7 +156,7 @@ public class RpcRetryingCallerImpl<T> implements RpcRetryingCaller<T> {
|
|||
if (duration > callTimeout) {
|
||||
String msg = "callTimeout=" + callTimeout + ", callDuration=" + duration + ": "
|
||||
+ t.getMessage() + " " + callable.getExceptionMessageAdditionalDetail();
|
||||
throw (SocketTimeoutException) (new SocketTimeoutException(msg).initCause(t));
|
||||
throw (SocketTimeoutException) new SocketTimeoutException(msg).initCause(t);
|
||||
}
|
||||
} finally {
|
||||
interceptor.updateFailureInfo(context);
|
||||
|
@ -235,7 +235,7 @@ public class RpcRetryingCallerImpl<T> implements RpcRetryingCaller<T> {
|
|||
|
||||
private int getTimeout(int callTimeout) {
|
||||
int timeout = tracker.getRemainingTime(callTimeout);
|
||||
if (timeout <= 0 || rpcTimeout > 0 && rpcTimeout < timeout) {
|
||||
if (timeout <= 0 || (rpcTimeout > 0 && rpcTimeout < timeout)) {
|
||||
timeout = rpcTimeout;
|
||||
}
|
||||
return timeout;
|
||||
|
|
|
@ -866,9 +866,9 @@ public class Scan extends Query {
|
|||
}
|
||||
|
||||
/**
|
||||
* @return true when the constructor of this scan understands that the results they will see may
|
||||
* only represent a partial portion of a row. The entire row would be retrieved by
|
||||
* subsequent calls to {@link ResultScanner#next()}
|
||||
* Returns true when the constructor of this scan understands that the results they will see may
|
||||
* only represent a partial portion of a row. The entire row would be retrieved by subsequent
|
||||
* calls to {@link ResultScanner#next()}
|
||||
*/
|
||||
public boolean getAllowPartialResults() {
|
||||
return allowPartialResults;
|
||||
|
|
|
@ -30,8 +30,7 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
public interface ScanResultConsumer extends ScanResultConsumerBase {
|
||||
|
||||
/**
|
||||
* @param result the data fetched from HBase service.
|
||||
* @return {@code false} if you want to terminate the scan process. Otherwise {@code true}
|
||||
* Return {@code false} if you want to terminate the scan process. Otherwise {@code true}
|
||||
*/
|
||||
boolean onNext(Result result);
|
||||
}
|
||||
|
|
|
@ -262,6 +262,7 @@ class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
|
|||
throw new IOException("Imposible? Arrive at an unreachable line...");
|
||||
}
|
||||
|
||||
@SuppressWarnings("FutureReturnValueIgnored")
|
||||
private void updateCurrentlyServingReplica(ScannerCallable scanner, Result[] result,
|
||||
AtomicBoolean done, ExecutorService pool) {
|
||||
if (done.compareAndSet(false, true)) {
|
||||
|
@ -308,10 +309,10 @@ class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
|
|||
}
|
||||
|
||||
/**
|
||||
* @return true when the most recent RPC response indicated that the response was a heartbeat
|
||||
* message. Heartbeat messages are sent back from the server when the processing of the
|
||||
* scan request exceeds a certain time threshold. Heartbeats allow the server to avoid
|
||||
* timeouts during long running scan operations.
|
||||
* Returns true when the most recent RPC response indicated that the response was a heartbeat
|
||||
* message. Heartbeat messages are sent back from the server when the processing of the scan
|
||||
* request exceeds a certain time threshold. Heartbeats allow the server to avoid timeouts during
|
||||
* long running scan operations.
|
||||
*/
|
||||
public boolean isHeartbeatMessage() {
|
||||
return currentScannerCallable != null && currentScannerCallable.isHeartbeatMessage();
|
||||
|
|
|
@ -27,20 +27,16 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
* Usually, it is just a simple lambda expression, like:
|
||||
*
|
||||
* <pre>
|
||||
* <code>
|
||||
* (stub, controller, rpcCallback) -> {
|
||||
* (stub, controller, rpcCallback) -> {
|
||||
* XXXRequest request = ...; // prepare the request
|
||||
* stub.xxx(controller, request, rpcCallback);
|
||||
* }
|
||||
* </code>
|
||||
* </pre>
|
||||
*
|
||||
* And if already have the {@code request}, the lambda expression will be:
|
||||
*
|
||||
* <pre>
|
||||
* <code>
|
||||
* (stub, controller, rpcCallback) -> stub.xxx(controller, request, rpcCallback)
|
||||
* </code>
|
||||
* (stub, controller, rpcCallback) -> stub.xxx(controller, request, rpcCallback)
|
||||
* </pre>
|
||||
*
|
||||
* @param <S> the type of the protobuf Service you want to call.
|
||||
|
|
|
@ -62,13 +62,10 @@ public class SlowLogParams {
|
|||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
if (!(o instanceof SlowLogParams)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
SlowLogParams that = (SlowLogParams) o;
|
||||
|
||||
return new EqualsBuilder().append(regionName, that.regionName).append(params, that.params)
|
||||
.isEquals();
|
||||
}
|
||||
|
|
|
@ -160,6 +160,7 @@ public class SnapshotDescription {
|
|||
}
|
||||
|
||||
/**
|
||||
* Get the table name as a string.
|
||||
* @deprecated since 2.0.0 and will be removed in 3.0.0. Use {@link #getTableName()} or
|
||||
* {@link #getTableNameAsString()} instead.
|
||||
* @see #getTableName()
|
||||
|
|
|
@ -464,11 +464,13 @@ public interface Table extends Closeable {
|
|||
interface CheckAndMutateBuilder {
|
||||
|
||||
/**
|
||||
* Specify a column qualifer
|
||||
* @param qualifier column qualifier to check.
|
||||
*/
|
||||
CheckAndMutateBuilder qualifier(byte[] qualifier);
|
||||
|
||||
/**
|
||||
* Specify a timerange
|
||||
* @param timeRange timeRange to check
|
||||
*/
|
||||
CheckAndMutateBuilder timeRange(TimeRange timeRange);
|
||||
|
@ -487,24 +489,28 @@ public interface Table extends Closeable {
|
|||
}
|
||||
|
||||
/**
|
||||
* Check for match.
|
||||
* @param compareOp comparison operator to use
|
||||
* @param value the expected value
|
||||
*/
|
||||
CheckAndMutateBuilder ifMatches(CompareOperator compareOp, byte[] value);
|
||||
|
||||
/**
|
||||
* Specify a Put to commit if the check succeeds.
|
||||
* @param put data to put if check succeeds
|
||||
* @return {@code true} if the new put was executed, {@code false} otherwise.
|
||||
*/
|
||||
boolean thenPut(Put put) throws IOException;
|
||||
|
||||
/**
|
||||
* Specify a Delete to commit if the check succeeds.
|
||||
* @param delete data to delete if check succeeds
|
||||
* @return {@code true} if the new delete was executed, {@code false} otherwise.
|
||||
*/
|
||||
boolean thenDelete(Delete delete) throws IOException;
|
||||
|
||||
/**
|
||||
* Specify a RowMutations to commit if the check succeeds.
|
||||
* @param mutation mutations to perform if check succeeds
|
||||
* @return true if the new mutation was executed, false otherwise.
|
||||
*/
|
||||
|
@ -541,23 +547,27 @@ public interface Table extends Closeable {
|
|||
interface CheckAndMutateWithFilterBuilder {
|
||||
|
||||
/**
|
||||
* Specify a timerange.
|
||||
* @param timeRange timeRange to check
|
||||
*/
|
||||
CheckAndMutateWithFilterBuilder timeRange(TimeRange timeRange);
|
||||
|
||||
/**
|
||||
* Specify a Put to commit if the check succeeds.
|
||||
* @param put data to put if check succeeds
|
||||
* @return {@code true} if the new put was executed, {@code false} otherwise.
|
||||
*/
|
||||
boolean thenPut(Put put) throws IOException;
|
||||
|
||||
/**
|
||||
* Specify a Delete to commit if the check succeeds.
|
||||
* @param delete data to delete if check succeeds
|
||||
* @return {@code true} if the new delete was executed, {@code false} otherwise.
|
||||
*/
|
||||
boolean thenDelete(Delete delete) throws IOException;
|
||||
|
||||
/**
|
||||
* Specify a RowMutations to commit if the check succeeds.
|
||||
* @param mutation mutations to perform if check succeeds
|
||||
* @return true if the new mutation was executed, false otherwise.
|
||||
*/
|
||||
|
|
|
@ -166,11 +166,12 @@ public interface TableDescriptor {
|
|||
String getRegionSplitPolicyClassName();
|
||||
|
||||
/**
|
||||
* Get the name of the table n
|
||||
* Get the name of the table
|
||||
*/
|
||||
TableName getTableName();
|
||||
|
||||
/**
|
||||
* Get the owner name as a string.
|
||||
* @deprecated since 2.0.0 and will be removed in 3.0.0.
|
||||
* @see <a href="https://issues.apache.org/jira/browse/HBASE-15583">HBASE-15583</a>
|
||||
*/
|
||||
|
@ -280,8 +281,8 @@ public interface TableDescriptor {
|
|||
boolean isReadOnly();
|
||||
|
||||
/**
|
||||
* @return Name of this table and then a map of all of the column family descriptors (with only
|
||||
* the non-default column family attributes)
|
||||
* Returns Name of this table and then a map of all of the column family descriptors (with only
|
||||
* the non-default column family attributes)
|
||||
*/
|
||||
String toStringCustomizedValues();
|
||||
|
||||
|
|
|
@ -148,8 +148,6 @@ public class TableDescriptorBuilder {
|
|||
private static final Bytes REGION_MEMSTORE_REPLICATION_KEY =
|
||||
new Bytes(Bytes.toBytes(REGION_MEMSTORE_REPLICATION));
|
||||
|
||||
private static final Bytes REGION_REPLICA_WAIT_FOR_PRIMARY_FLUSH_CONF_KEY =
|
||||
new Bytes(Bytes.toBytes(RegionReplicaUtil.REGION_REPLICA_WAIT_FOR_PRIMARY_FLUSH_CONF_KEY));
|
||||
/**
|
||||
* Used by shell/rest interface to access this metadata attribute which denotes if the table
|
||||
* should be treated by region normalizer.
|
||||
|
@ -294,10 +292,7 @@ public class TableDescriptorBuilder {
|
|||
|
||||
private final ModifyableTableDescriptor desc;
|
||||
|
||||
/**
|
||||
* @param desc The table descriptor to serialize
|
||||
* @return This instance serialized with pb with pb magic prefix
|
||||
*/
|
||||
/** Returns This instance serialized with pb with pb magic prefix */
|
||||
public static byte[] toByteArray(TableDescriptor desc) {
|
||||
if (desc instanceof ModifyableTableDescriptor) {
|
||||
return ((ModifyableTableDescriptor) desc).toByteArray();
|
||||
|
@ -309,7 +304,7 @@ public class TableDescriptorBuilder {
|
|||
* The input should be created by {@link #toByteArray}.
|
||||
* @param pbBytes A pb serialized TableDescriptor instance with pb magic prefix
|
||||
* @return This instance serialized with pb with pb magic prefix
|
||||
* @throws org.apache.hadoop.hbase.exceptions.DeserializationException
|
||||
* @throws org.apache.hadoop.hbase.exceptions.DeserializationException if an error occurred
|
||||
*/
|
||||
public static TableDescriptor parseFrom(byte[] pbBytes) throws DeserializationException {
|
||||
return ModifyableTableDescriptor.parseFrom(pbBytes);
|
||||
|
@ -506,6 +501,7 @@ public class TableDescriptorBuilder {
|
|||
}
|
||||
|
||||
/**
|
||||
* Set the table owner
|
||||
* @deprecated since 2.0.0 and will be removed in 3.0.0.
|
||||
* @see <a href="https://issues.apache.org/jira/browse/HBASE-15583">HBASE-15583</a>
|
||||
*/
|
||||
|
@ -516,6 +512,7 @@ public class TableDescriptorBuilder {
|
|||
}
|
||||
|
||||
/**
|
||||
* Set the table owner
|
||||
* @deprecated since 2.0.0 and will be removed in 3.0.0.
|
||||
* @see <a href="https://issues.apache.org/jira/browse/HBASE-15583">HBASE-15583</a>
|
||||
*/
|
||||
|
@ -1511,6 +1508,7 @@ public class TableDescriptorBuilder {
|
|||
}
|
||||
|
||||
/**
|
||||
* Set the table owner
|
||||
* @deprecated since 2.0.0 and will be removed in 3.0.0.
|
||||
* @see <a href="https://issues.apache.org/jira/browse/HBASE-15583">HBASE-15583</a>
|
||||
*/
|
||||
|
@ -1520,6 +1518,7 @@ public class TableDescriptorBuilder {
|
|||
}
|
||||
|
||||
/**
|
||||
* Set the table owner.
|
||||
* @deprecated since 2.0.0 and will be removed in 3.0.0.
|
||||
* @see <a href="https://issues.apache.org/jira/browse/HBASE-15583">HBASE-15583</a>
|
||||
*/
|
||||
|
@ -1530,6 +1529,7 @@ public class TableDescriptorBuilder {
|
|||
}
|
||||
|
||||
/**
|
||||
* Set the table owner.
|
||||
* @deprecated since 2.0.0 and will be removed in 3.0.0.
|
||||
* @see <a href="https://issues.apache.org/jira/browse/HBASE-15583">HBASE-15583</a>
|
||||
*/
|
||||
|
@ -1548,6 +1548,7 @@ public class TableDescriptorBuilder {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse the serialized representation of a {@link ModifyableTableDescriptor}
|
||||
* @param bytes A pb serialized {@link ModifyableTableDescriptor} instance with pb magic prefix
|
||||
* @return An instance of {@link ModifyableTableDescriptor} made from <code>bytes</code> n
|
||||
* * @see #toByteArray()
|
||||
|
|
|
@ -209,15 +209,19 @@ public class TableState {
|
|||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
|
||||
TableState that = (TableState) o;
|
||||
|
||||
if (state != that.state) return false;
|
||||
if (tableName != null ? !tableName.equals(that.tableName) : that.tableName != null)
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (!(o instanceof TableState)) {
|
||||
return false;
|
||||
|
||||
}
|
||||
TableState that = (TableState) o;
|
||||
if (state != that.state) {
|
||||
return false;
|
||||
}
|
||||
if (tableName != null ? !tableName.equals(that.tableName) : that.tableName != null) {
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
|
|
|
@ -22,15 +22,11 @@ import org.apache.hadoop.hbase.ServerName;
|
|||
import org.apache.hadoop.hbase.util.ReflectionUtils;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Evolving
|
||||
public final class ClientBackoffPolicyFactory {
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(ClientBackoffPolicyFactory.class);
|
||||
|
||||
private ClientBackoffPolicyFactory() {
|
||||
}
|
||||
|
||||
|
|
|
@ -21,8 +21,6 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
|
||||
|
||||
|
@ -33,8 +31,6 @@ import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
|
|||
@InterfaceAudience.Public
|
||||
public class ExponentialClientBackoffPolicy implements ClientBackoffPolicy {
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(ExponentialClientBackoffPolicy.class);
|
||||
|
||||
private static final long ONE_MINUTE = 60 * 1000;
|
||||
public static final long DEFAULT_MAX_BACKOFF = 5 * ONE_MINUTE;
|
||||
public static final String MAX_BACKOFF_KEY = "hbase.client.exponential-backoff.max";
|
||||
|
|
|
@ -78,7 +78,7 @@ public class BigDecimalColumnInterpreter
|
|||
|
||||
@Override
|
||||
public BigDecimal increment(BigDecimal bd) {
|
||||
return bd == null ? null : (bd.add(BigDecimal.ONE));
|
||||
return bd == null ? null : bd.add(BigDecimal.ONE);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -37,8 +37,6 @@ import org.apache.hadoop.hbase.replication.ReplicationException;
|
|||
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
|
||||
|
||||
|
@ -65,7 +63,6 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
|
|||
@InterfaceAudience.Public
|
||||
@Deprecated
|
||||
public class ReplicationAdmin implements Closeable {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(ReplicationAdmin.class);
|
||||
|
||||
public static final String TNAME = "tableName";
|
||||
public static final String CFNAME = "columnFamilyName";
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.util.ArrayList;
|
|||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
@ -112,8 +113,8 @@ public final class ReplicationPeerConfigUtil {
|
|||
}
|
||||
|
||||
ReplicationProtos.TableCF.Builder tableCFBuilder = ReplicationProtos.TableCF.newBuilder();
|
||||
String[] tables = tableCFsConfig.split(";");
|
||||
List<ReplicationProtos.TableCF> tableCFList = new ArrayList<>(tables.length);
|
||||
List<String> tables = Splitter.on(';').splitToList(tableCFsConfig);
|
||||
List<ReplicationProtos.TableCF> tableCFList = new ArrayList<>(tables.size());
|
||||
|
||||
for (String tab : tables) {
|
||||
// 1 ignore empty table config
|
||||
|
@ -123,9 +124,9 @@ public final class ReplicationPeerConfigUtil {
|
|||
}
|
||||
// 2 split to "table" and "cf1,cf2"
|
||||
// for each table: "table#cf1,cf2" or "table"
|
||||
String[] pair = tab.split(":");
|
||||
String tabName = pair[0].trim();
|
||||
if (pair.length > 2 || tabName.length() == 0) {
|
||||
Iterator<String> i = Splitter.on(':').split(tab).iterator();
|
||||
String tabName = i.next().trim();
|
||||
if (tabName.length() == 0) {
|
||||
LOG.info("incorrect format:" + tableCFsConfig);
|
||||
continue;
|
||||
}
|
||||
|
@ -134,16 +135,17 @@ public final class ReplicationPeerConfigUtil {
|
|||
// split namespace from tableName
|
||||
String ns = "default";
|
||||
String tName = tabName;
|
||||
String[] dbs = tabName.split("\\.");
|
||||
if (dbs != null && dbs.length == 2) {
|
||||
ns = dbs[0];
|
||||
tName = dbs[1];
|
||||
List<String> dbs = Splitter.on('.').splitToList(tabName);
|
||||
if (dbs != null && dbs.size() == 2) {
|
||||
Iterator<String> ii = dbs.iterator();
|
||||
ns = ii.next();
|
||||
tName = ii.next();
|
||||
}
|
||||
tableCFBuilder.setTableName(ProtobufUtil.toProtoTableName(TableName.valueOf(ns, tName)));
|
||||
|
||||
// 3 parse "cf1,cf2" part to List<cf>
|
||||
if (pair.length == 2) {
|
||||
String[] cfsList = pair[1].split(",");
|
||||
if (i.hasNext()) {
|
||||
List<String> cfsList = Splitter.on(',').splitToList(i.next());
|
||||
for (String cf : cfsList) {
|
||||
String cfName = cf.trim();
|
||||
if (cfName.length() > 0) {
|
||||
|
@ -240,6 +242,7 @@ public final class ReplicationPeerConfigUtil {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse the serialized representation of a peer configuration.
|
||||
* @param bytes Content of a peer znode.
|
||||
* @return ClusterKey parsed from the passed bytes.
|
||||
* @throws DeserializationException deserialization exception
|
||||
|
@ -377,10 +380,9 @@ public final class ReplicationPeerConfigUtil {
|
|||
}
|
||||
|
||||
/**
|
||||
* @param peerConfig peer config of replication peer
|
||||
* @return Serialized protobuf of <code>peerConfig</code> with pb magic prefix prepended suitable
|
||||
* for use as content of a this.peersZNode; i.e. the content of PEER_ID znode under
|
||||
* /hbase/replication/peers/PEER_ID
|
||||
* Returns Serialized protobuf of <code>peerConfig</code> with pb magic prefix prepended suitable
|
||||
* for use as content of a this.peersZNode; i.e. the content of PEER_ID znode under
|
||||
* /hbase/replication/peers/PEER_ID
|
||||
*/
|
||||
public static byte[] toByteArray(final ReplicationPeerConfig peerConfig) {
|
||||
byte[] bytes = convert(peerConfig).toByteArray();
|
||||
|
|
|
@ -39,11 +39,12 @@ import org.apache.yetus.audience.InterfaceStability;
|
|||
* {@link ColumnInterpreter#castToReturnType(Object)} which takes a <T> type and returns a
|
||||
* <S> type. The AggregateIm>lementation uses PB messages to initialize the user's
|
||||
* ColumnInterpreter implementation, and for sending the responses back to AggregationClient.
|
||||
* @param T Cell value data type
|
||||
* @param S Promoted data type
|
||||
* @param P PB message that is used to transport initializer specific bytes
|
||||
* @param Q PB message that is used to transport Cell (<T>) instance
|
||||
* @param R PB message that is used to transport Promoted (<S>) instance
|
||||
* <p>
|
||||
* <T> Cell value data type<br>
|
||||
* <S> Promoted data type<br>
|
||||
* <P> PB message that is used to transport initializer specific bytes<br>
|
||||
* <Q> PB message that is used to transport Cell (<T>) instance<br>
|
||||
* <R> PB message that is used to transport Promoted (<S>) instance
|
||||
*/
|
||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
|
||||
@InterfaceStability.Evolving
|
||||
|
|
|
@ -20,8 +20,6 @@ package org.apache.hadoop.hbase.exceptions;
|
|||
import org.apache.hadoop.hbase.NotServingRegionException;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* Subclass if the server knows the region is now on another server. This allows the client to call
|
||||
|
@ -30,7 +28,7 @@ import org.slf4j.LoggerFactory;
|
|||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Evolving
|
||||
public class RegionOpeningException extends NotServingRegionException {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(RegionOpeningException.class);
|
||||
|
||||
private static final long serialVersionUID = -7232903522310558395L;
|
||||
|
||||
public RegionOpeningException(String message) {
|
||||
|
|
|
@ -82,9 +82,10 @@ public class BigDecimalComparator extends ByteArrayComparable {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse a serialized representation of {@link BigDecimalComparator}
|
||||
* @param pbBytes A pb serialized {@link BigDecimalComparator} instance
|
||||
* @return An instance of {@link BigDecimalComparator} made from <code>bytes</code>
|
||||
* @throws DeserializationException A deserialization exception
|
||||
* @throws DeserializationException if an error occurred
|
||||
* @see #toByteArray
|
||||
*/
|
||||
public static BigDecimalComparator parseFrom(final byte[] pbBytes)
|
||||
|
@ -100,10 +101,10 @@ public class BigDecimalComparator extends ByteArrayComparable {
|
|||
}
|
||||
|
||||
/**
|
||||
* @param other the other comparator
|
||||
* @return true if and only if the fields of the comparator that are serialized are equal to the
|
||||
* corresponding fields in other. Used for testing.
|
||||
* Returns true if and only if the fields of the comparator that are serialized are equal to the
|
||||
* corresponding fields in other. Used for testing.
|
||||
*/
|
||||
@SuppressWarnings("ReferenceEquality")
|
||||
boolean areSerializedFieldsEqual(BigDecimalComparator other) {
|
||||
if (other == this) {
|
||||
return true;
|
||||
|
|
|
@ -64,9 +64,11 @@ public class BinaryComparator extends org.apache.hadoop.hbase.filter.ByteArrayCo
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse a serialized representation of {@link BinaryComparator}
|
||||
* @param pbBytes A pb serialized {@link BinaryComparator} instance
|
||||
* @return An instance of {@link BinaryComparator} made from <code>bytes</code> n * @see
|
||||
* #toByteArray
|
||||
* @return An instance of {@link BinaryComparator} made from <code>bytes</code>
|
||||
* @throws DeserializationException if an error occurred
|
||||
* @see #toByteArray
|
||||
*/
|
||||
public static BinaryComparator parseFrom(final byte[] pbBytes) throws DeserializationException {
|
||||
ComparatorProtos.BinaryComparator proto;
|
||||
|
@ -79,14 +81,17 @@ public class BinaryComparator extends org.apache.hadoop.hbase.filter.ByteArrayCo
|
|||
}
|
||||
|
||||
/**
|
||||
* n * @return true if and only if the fields of the comparator that are serialized are equal to
|
||||
* the corresponding fields in other. Used for testing.
|
||||
* Returns true if and only if the fields of the comparator that are serialized are equal to the
|
||||
* corresponding fields in other. Used for testing.
|
||||
*/
|
||||
@Override
|
||||
boolean areSerializedFieldsEqual(ByteArrayComparable other) {
|
||||
if (other == this) return true;
|
||||
if (!(other instanceof BinaryComparator)) return false;
|
||||
|
||||
if (other == this) {
|
||||
return true;
|
||||
}
|
||||
if (!(other instanceof BinaryComparator)) {
|
||||
return false;
|
||||
}
|
||||
return super.areSerializedFieldsEqual(other);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -84,9 +84,10 @@ public class BinaryComponentComparator extends ByteArrayComparable {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse a serialized representation of {@link BinaryComponentComparator}
|
||||
* @param pbBytes A pb serialized {@link BinaryComponentComparator} instance
|
||||
* @return An instance of {@link BinaryComponentComparator} made from <code>bytes</code>
|
||||
* @throws DeserializationException DeserializationException
|
||||
* @throws DeserializationException if an error occurred
|
||||
* @see #toByteArray
|
||||
*/
|
||||
public static BinaryComponentComparator parseFrom(final byte[] pbBytes)
|
||||
|
@ -101,9 +102,8 @@ public class BinaryComponentComparator extends ByteArrayComparable {
|
|||
}
|
||||
|
||||
/**
|
||||
* @param other paramemter to compare against
|
||||
* @return true if and only if the fields of the comparator that are serialized are equal to the
|
||||
* corresponding fields in other. Used for testing.
|
||||
* Returns true if and only if the fields of the comparator that are serialized are equal to the
|
||||
* corresponding fields in other. Used for testing.
|
||||
*/
|
||||
@Override
|
||||
boolean areSerializedFieldsEqual(ByteArrayComparable other) {
|
||||
|
|
|
@ -68,9 +68,11 @@ public class BinaryPrefixComparator extends ByteArrayComparable {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse a serialized representation of {@link BinaryPrefixComparator}
|
||||
* @param pbBytes A pb serialized {@link BinaryPrefixComparator} instance
|
||||
* @return An instance of {@link BinaryPrefixComparator} made from <code>bytes</code> n * @see
|
||||
* #toByteArray
|
||||
* @return An instance of {@link BinaryPrefixComparator} made from <code>bytes</code>
|
||||
* @throws DeserializationException if an error occurred
|
||||
* @see #toByteArray
|
||||
*/
|
||||
public static BinaryPrefixComparator parseFrom(final byte[] pbBytes)
|
||||
throws DeserializationException {
|
||||
|
@ -84,14 +86,17 @@ public class BinaryPrefixComparator extends ByteArrayComparable {
|
|||
}
|
||||
|
||||
/**
|
||||
* n * @return true if and only if the fields of the comparator that are serialized are equal to
|
||||
* the corresponding fields in other. Used for testing.
|
||||
* Returns true if and only if the fields of the comparator that are serialized are equal to the
|
||||
* corresponding fields in other. Used for testing.
|
||||
*/
|
||||
@Override
|
||||
boolean areSerializedFieldsEqual(ByteArrayComparable other) {
|
||||
if (other == this) return true;
|
||||
if (!(other instanceof BinaryPrefixComparator)) return false;
|
||||
|
||||
if (other == this) {
|
||||
return true;
|
||||
}
|
||||
if (!(other instanceof BinaryPrefixComparator)) {
|
||||
return false;
|
||||
}
|
||||
return super.areSerializedFieldsEqual(other);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -74,8 +74,11 @@ public class BitComparator extends ByteArrayComparable {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse a serialized representation of {@link BitComparator}
|
||||
* @param pbBytes A pb serialized {@link BitComparator} instance
|
||||
* @return An instance of {@link BitComparator} made from <code>bytes</code> n * @see #toByteArray
|
||||
* @return An instance of {@link BitComparator} made from <code>bytes</code>
|
||||
* @throws DeserializationException if an error occurred
|
||||
* @see #toByteArray
|
||||
*/
|
||||
public static BitComparator parseFrom(final byte[] pbBytes) throws DeserializationException {
|
||||
ComparatorProtos.BitComparator proto;
|
||||
|
@ -89,14 +92,17 @@ public class BitComparator extends ByteArrayComparable {
|
|||
}
|
||||
|
||||
/**
|
||||
* n * @return true if and only if the fields of the comparator that are serialized are equal to
|
||||
* the corresponding fields in other. Used for testing.
|
||||
* Returns true if and only if the fields of the comparator that are serialized are equal to the
|
||||
* corresponding fields in other. Used for testing.
|
||||
*/
|
||||
@Override
|
||||
boolean areSerializedFieldsEqual(ByteArrayComparable other) {
|
||||
if (other == this) return true;
|
||||
if (!(other instanceof BitComparator)) return false;
|
||||
|
||||
if (other == this) {
|
||||
return true;
|
||||
}
|
||||
if (!(other instanceof BitComparator)) {
|
||||
return false;
|
||||
}
|
||||
BitComparator comparator = (BitComparator) other;
|
||||
return super.areSerializedFieldsEqual(other)
|
||||
&& this.getOperator().equals(comparator.getOperator());
|
||||
|
|
|
@ -94,8 +94,10 @@ public class ColumnCountGetFilter extends FilterBase {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse a serialized representation of {@link ColumnCountGetFilter}
|
||||
* @param pbBytes A pb serialized {@link ColumnCountGetFilter} instance
|
||||
* @return An instance of {@link ColumnCountGetFilter} made from <code>bytes</code>
|
||||
* @throws DeserializationException if an error occurred
|
||||
* @see #toByteArray
|
||||
*/
|
||||
public static ColumnCountGetFilter parseFrom(final byte[] pbBytes)
|
||||
|
@ -110,15 +112,17 @@ public class ColumnCountGetFilter extends FilterBase {
|
|||
}
|
||||
|
||||
/**
|
||||
* @param o the other filter to compare with
|
||||
* @return true if and only if the fields of the filter that are serialized are equal to the
|
||||
* corresponding fields in other. Used for testing.
|
||||
* Returns true if and only if the fields of the filter that are serialized are equal to the
|
||||
* corresponding fields in other. Used for testing.
|
||||
*/
|
||||
@Override
|
||||
boolean areSerializedFieldsEqual(Filter o) {
|
||||
if (o == this) return true;
|
||||
if (!(o instanceof ColumnCountGetFilter)) return false;
|
||||
|
||||
if (o == this) {
|
||||
return true;
|
||||
}
|
||||
if (!(o instanceof ColumnCountGetFilter)) {
|
||||
return false;
|
||||
}
|
||||
ColumnCountGetFilter other = (ColumnCountGetFilter) o;
|
||||
return this.getLimit() == other.getLimit();
|
||||
}
|
||||
|
|
|
@ -78,23 +78,14 @@ public class ColumnPaginationFilter extends FilterBase {
|
|||
this.columnOffset = columnOffset;
|
||||
}
|
||||
|
||||
/**
|
||||
* n
|
||||
*/
|
||||
public int getLimit() {
|
||||
return limit;
|
||||
}
|
||||
|
||||
/**
|
||||
* n
|
||||
*/
|
||||
public int getOffset() {
|
||||
return offset;
|
||||
}
|
||||
|
||||
/**
|
||||
* n
|
||||
*/
|
||||
public byte[] getColumnOffset() {
|
||||
return columnOffset;
|
||||
}
|
||||
|
@ -173,9 +164,11 @@ public class ColumnPaginationFilter extends FilterBase {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse a serialized representation of {@link ColumnPaginationFilter}
|
||||
* @param pbBytes A pb serialized {@link ColumnPaginationFilter} instance
|
||||
* @return An instance of {@link ColumnPaginationFilter} made from <code>bytes</code> n * @see
|
||||
* #toByteArray
|
||||
* @return An instance of {@link ColumnPaginationFilter} made from <code>bytes</code>
|
||||
* @throws DeserializationException if an error occurred
|
||||
* @see #toByteArray
|
||||
*/
|
||||
public static ColumnPaginationFilter parseFrom(final byte[] pbBytes)
|
||||
throws DeserializationException {
|
||||
|
@ -192,15 +185,17 @@ public class ColumnPaginationFilter extends FilterBase {
|
|||
}
|
||||
|
||||
/**
|
||||
* @param o the other filter to compare with
|
||||
* @return true if and only if the fields of the filter that are serialized are equal to the
|
||||
* corresponding fields in other. Used for testing.
|
||||
* Returns true if and only if the fields of the filter that are serialized are equal to the
|
||||
* corresponding fields in other. Used for testing.
|
||||
*/
|
||||
@Override
|
||||
boolean areSerializedFieldsEqual(Filter o) {
|
||||
if (o == this) return true;
|
||||
if (!(o instanceof ColumnPaginationFilter)) return false;
|
||||
|
||||
if (o == this) {
|
||||
return true;
|
||||
}
|
||||
if (!(o instanceof ColumnPaginationFilter)) {
|
||||
return false;
|
||||
}
|
||||
ColumnPaginationFilter other = (ColumnPaginationFilter) o;
|
||||
if (this.columnOffset != null) {
|
||||
return this.getLimit() == other.getLimit()
|
||||
|
|
|
@ -117,9 +117,10 @@ public class ColumnPrefixFilter extends FilterBase {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parses a serialized representation of the {@link ColumnPrefixFilter}
|
||||
* @param pbBytes A pb serialized {@link ColumnPrefixFilter} instance
|
||||
* @return An instance of {@link ColumnPrefixFilter} made from <code>bytes</code>
|
||||
* @throws org.apache.hadoop.hbase.exceptions.DeserializationException
|
||||
* @throws DeserializationException if an error occurred
|
||||
* @see #toByteArray
|
||||
*/
|
||||
public static ColumnPrefixFilter parseFrom(final byte[] pbBytes) throws DeserializationException {
|
||||
|
@ -133,15 +134,17 @@ public class ColumnPrefixFilter extends FilterBase {
|
|||
}
|
||||
|
||||
/**
|
||||
* @param o the other filter to compare with
|
||||
* @return true if and only if the fields of the filter that are serialized are equal to the
|
||||
* corresponding fields in other. Used for testing.
|
||||
* Returns true if and only if the fields of the filter that are serialized are equal to the
|
||||
* corresponding fields in other. Used for testing.
|
||||
*/
|
||||
@Override
|
||||
boolean areSerializedFieldsEqual(Filter o) {
|
||||
if (o == this) return true;
|
||||
if (!(o instanceof ColumnPrefixFilter)) return false;
|
||||
|
||||
if (o == this) {
|
||||
return true;
|
||||
}
|
||||
if (!(o instanceof ColumnPrefixFilter)) {
|
||||
return false;
|
||||
}
|
||||
ColumnPrefixFilter other = (ColumnPrefixFilter) o;
|
||||
return Bytes.equals(this.getPrefix(), other.getPrefix());
|
||||
}
|
||||
|
|
|
@ -163,9 +163,11 @@ public class ColumnRangeFilter extends FilterBase {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse a serialized representation of {@link ColumnRangeFilter}
|
||||
* @param pbBytes A pb serialized {@link ColumnRangeFilter} instance
|
||||
* @return An instance of {@link ColumnRangeFilter} made from <code>bytes</code> n * @see
|
||||
* #toByteArray
|
||||
* @return An instance of {@link ColumnRangeFilter} made from <code>bytes</code>
|
||||
* @throws DeserializationException if an error occurred
|
||||
* @see #toByteArray
|
||||
*/
|
||||
public static ColumnRangeFilter parseFrom(final byte[] pbBytes) throws DeserializationException {
|
||||
FilterProtos.ColumnRangeFilter proto;
|
||||
|
@ -181,9 +183,8 @@ public class ColumnRangeFilter extends FilterBase {
|
|||
}
|
||||
|
||||
/**
|
||||
* @param o filter to serialize.
|
||||
* @return true if and only if the fields of the filter that are serialized are equal to the
|
||||
* corresponding fields in other. Used for testing.
|
||||
* Returns true if and only if the fields of the filter that are serialized are equal to the
|
||||
* corresponding fields in other. Used for testing.
|
||||
*/
|
||||
@Override
|
||||
boolean areSerializedFieldsEqual(Filter o) {
|
||||
|
|
|
@ -168,10 +168,11 @@ public class ColumnValueFilter extends FilterBase {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse protobuf bytes to a ColumnValueFilter
|
||||
* @param pbBytes pbBytes
|
||||
* @return a ColumnValueFilter
|
||||
* @throws DeserializationException deserialization exception
|
||||
* Parse a serialized representation of {@link ColumnValueFilter}
|
||||
* @param pbBytes A pb serialized {@link ColumnValueFilter} instance
|
||||
* @return An instance of {@link ColumnValueFilter} made from <code>bytes</code>
|
||||
* @throws DeserializationException if an error occurred
|
||||
* @see #toByteArray
|
||||
*/
|
||||
public static ColumnValueFilter parseFrom(final byte[] pbBytes) throws DeserializationException {
|
||||
FilterProtos.ColumnValueFilter proto;
|
||||
|
@ -198,6 +199,10 @@ public class ColumnValueFilter extends FilterBase {
|
|||
return convert().toByteArray();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns true if and only if the fields of the filter that are serialized are equal to the
|
||||
* corresponding fields in other. Used for testing.
|
||||
*/
|
||||
@Override
|
||||
boolean areSerializedFieldsEqual(Filter o) {
|
||||
if (o == this) {
|
||||
|
@ -205,7 +210,6 @@ public class ColumnValueFilter extends FilterBase {
|
|||
} else if (!(o instanceof ColumnValueFilter)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
ColumnValueFilter other = (ColumnValueFilter) o;
|
||||
return Bytes.equals(this.getFamily(), other.getFamily())
|
||||
&& Bytes.equals(this.getQualifier(), other.getQualifier())
|
||||
|
|
|
@ -257,7 +257,7 @@ public abstract class CompareFilter extends FilterBase {
|
|||
}
|
||||
}
|
||||
|
||||
// returns an array of heterogeneous objects
|
||||
/** Returns an array of heterogeneous objects */
|
||||
public static ArrayList<Object> extractArguments(ArrayList<byte[]> filterArguments) {
|
||||
Preconditions.checkArgument(filterArguments.size() == 2, "Expected 2 but got: %s",
|
||||
filterArguments.size());
|
||||
|
@ -287,13 +287,17 @@ public abstract class CompareFilter extends FilterBase {
|
|||
}
|
||||
|
||||
/**
|
||||
* n * @return true if and only if the fields of the filter that are serialized are equal to the
|
||||
* Returns true if and only if the fields of the filter that are serialized are equal to the
|
||||
* corresponding fields in other. Used for testing.
|
||||
*/
|
||||
@Override
|
||||
boolean areSerializedFieldsEqual(Filter o) {
|
||||
if (o == this) return true;
|
||||
if (!(o instanceof CompareFilter)) return false;
|
||||
if (o == this) {
|
||||
return true;
|
||||
}
|
||||
if (!(o instanceof CompareFilter)) {
|
||||
return false;
|
||||
}
|
||||
CompareFilter other = (CompareFilter) o;
|
||||
return this.getCompareOperator().equals(other.getCompareOperator())
|
||||
&& (this.getComparator() == other.getComparator()
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue