HBASE-27203 Clean up error-prone findings in hbase-client (#4644)
Signed-off-by: Duo Zhang <zhangduo@apache.org>
This commit is contained in:
parent
9ba868d2ad
commit
9573466112
|
@ -349,8 +349,8 @@ public class CatalogFamilyFormat {
|
|||
}
|
||||
|
||||
/**
|
||||
* @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) {
|
||||
|
@ -376,8 +376,8 @@ public class CatalogFamilyFormat {
|
|||
}
|
||||
|
||||
/**
|
||||
* @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) {
|
||||
|
@ -386,8 +386,8 @@ public class CatalogFamilyFormat {
|
|||
}
|
||||
|
||||
/**
|
||||
* @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) {
|
||||
|
|
|
@ -59,6 +59,7 @@ public final class ClientMetaTableAccessor {
|
|||
}
|
||||
|
||||
@InterfaceAudience.Private
|
||||
@SuppressWarnings("ImmutableEnumChecker")
|
||||
public enum QueryType {
|
||||
ALL(HConstants.TABLE_FAMILY, HConstants.CATALOG_FAMILY),
|
||||
REGION(HConstants.CATALOG_FAMILY),
|
||||
|
@ -100,11 +101,7 @@ public final class ClientMetaTableAccessor {
|
|||
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<>();
|
||||
|
@ -126,11 +123,7 @@ public final class ClientMetaTableAccessor {
|
|||
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<>();
|
||||
|
@ -167,8 +160,9 @@ public final class ClientMetaTableAccessor {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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}.
|
||||
*/
|
||||
|
@ -191,8 +185,9 @@ public final class ClientMetaTableAccessor {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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}.
|
||||
|
@ -221,9 +216,11 @@ public final class ClientMetaTableAccessor {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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) {
|
||||
|
@ -232,11 +229,13 @@ public final class ClientMetaTableAccessor {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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) {
|
||||
|
@ -456,19 +455,12 @@ public final class ClientMetaTableAccessor {
|
|||
return scan;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns an HRegionLocationList extracted from the result.
|
||||
* @return an HRegionLocationList containing all locations for the region range or null if we
|
||||
* can't deserialize the result.
|
||||
*/
|
||||
/** Returns an HRegionLocationList extracted from the result. */
|
||||
private static Optional<RegionLocations> getRegionLocations(Result r) {
|
||||
return Optional.ofNullable(CatalogFamilyFormat.getRegionLocations(r));
|
||||
}
|
||||
|
||||
/**
|
||||
* @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) {
|
||||
return null;
|
||||
|
@ -490,10 +482,7 @@ public final class ClientMetaTableAccessor {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @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) {
|
||||
return null;
|
||||
|
|
|
@ -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,13 +67,13 @@ 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()))
|
||||
.addAllDecommissionedServers(metrics.getDecommissionedServerNames().stream()
|
||||
.map(ProtobufUtil::toServerName).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()
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -100,8 +100,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());
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -53,8 +53,8 @@ public interface RegionMetrics {
|
|||
public long getCpRequestCount();
|
||||
|
||||
/**
|
||||
* @return the number of write requests and read requests and coprocessor service requests made to
|
||||
* region
|
||||
* Returns the number of write requests and read requests and coprocessor service requests made to
|
||||
* region
|
||||
*/
|
||||
default long getRequestCount() {
|
||||
return getReadRequestCount() + getWriteRequestCount() + getCpRequestCount();
|
||||
|
@ -113,8 +113,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();
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
@ -300,6 +296,7 @@ public final class ServerMetricsBuilder {
|
|||
return versionNumber;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getVersion() {
|
||||
return version;
|
||||
}
|
||||
|
@ -414,16 +411,18 @@ 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();
|
||||
cpRequestsCount += r.getCpRequestCount();
|
||||
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
|
||||
|
|
|
@ -115,6 +115,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
|
||||
|
@ -187,8 +188,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;
|
||||
|
||||
|
@ -197,7 +198,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
|
||||
|
@ -223,7 +224,7 @@ public interface Admin extends Abortable, Closeable {
|
|||
* occurs
|
||||
* @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
|
||||
|
@ -242,7 +243,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
|
||||
|
@ -381,6 +382,7 @@ public interface Admin extends Abortable, Closeable {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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
|
||||
|
@ -388,6 +390,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
|
||||
|
@ -395,6 +398,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
|
||||
|
@ -1100,6 +1104,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
|
||||
*/
|
||||
|
@ -1108,6 +1113,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
|
||||
*/
|
||||
|
@ -1116,6 +1122,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
|
||||
*/
|
||||
|
@ -1224,7 +1231,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
|
||||
*/
|
||||
|
@ -1336,9 +1343,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;
|
||||
|
||||
|
|
|
@ -379,6 +379,7 @@ class AdminOverAsyncAdmin implements Admin {
|
|||
return get(admin.balancerSwitch(onOrOff, synchronous));
|
||||
}
|
||||
|
||||
@Override
|
||||
public BalanceResponse balance(BalanceRequest request) throws IOException {
|
||||
return get(admin.balance(request));
|
||||
}
|
||||
|
|
|
@ -49,7 +49,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();
|
||||
|
||||
/**
|
||||
|
|
|
@ -66,6 +66,7 @@ import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
|
|||
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}.
|
||||
|
@ -215,6 +216,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}.
|
||||
|
@ -222,6 +224,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}.
|
||||
|
@ -229,6 +232,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}.
|
||||
|
@ -591,11 +595,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);
|
||||
|
@ -1183,19 +1191,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);
|
||||
|
@ -1256,8 +1266,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();
|
||||
|
||||
|
@ -1401,9 +1411,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.
|
||||
|
@ -1424,9 +1432,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.
|
||||
|
@ -1635,7 +1641,6 @@ public interface AsyncAdmin {
|
|||
/**
|
||||
* Creates a new RegionServer group with the given name
|
||||
* @param groupName the name of the group
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
*/
|
||||
CompletableFuture<Void> addRSGroup(String groupName);
|
||||
|
||||
|
@ -1643,34 +1648,29 @@ public interface AsyncAdmin {
|
|||
* Get group info for the given group name
|
||||
* @param groupName the group name
|
||||
* @return group info
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
*/
|
||||
CompletableFuture<RSGroupInfo> getRSGroup(String groupName);
|
||||
|
||||
/**
|
||||
* Get group info for the given hostPort
|
||||
* @param hostPort HostPort to get RSGroupInfo for
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
*/
|
||||
CompletableFuture<RSGroupInfo> getRSGroup(Address hostPort);
|
||||
|
||||
/**
|
||||
* Get group info for the given table
|
||||
* @param tableName table name to get RSGroupInfo for
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
*/
|
||||
CompletableFuture<RSGroupInfo> getRSGroup(TableName tableName);
|
||||
|
||||
/**
|
||||
* Lists current set of RegionServer groups
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
*/
|
||||
CompletableFuture<List<RSGroupInfo>> listRSGroups();
|
||||
|
||||
/**
|
||||
* Get all tables in this RegionServer group.
|
||||
* @param groupName the group name
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
* @see #getConfiguredNamespacesAndTablesInRSGroup(String)
|
||||
*/
|
||||
CompletableFuture<List<TableName>> listTablesInRSGroup(String groupName);
|
||||
|
@ -1685,7 +1685,6 @@ public interface AsyncAdmin {
|
|||
* in the group 'A', but this method will not return these tables but only the namespace 'nsA',
|
||||
* while the {@link #listTablesInRSGroup(String)} will return all these tables.
|
||||
* @param groupName the group name
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
* @see #listTablesInRSGroup(String)
|
||||
*/
|
||||
CompletableFuture<Pair<List<String>, List<TableName>>>
|
||||
|
@ -1694,7 +1693,6 @@ public interface AsyncAdmin {
|
|||
/**
|
||||
* Remove RegionServer group associated with the given name
|
||||
* @param groupName the group name
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
*/
|
||||
CompletableFuture<Void> removeRSGroup(String groupName);
|
||||
|
||||
|
@ -1704,7 +1702,6 @@ public interface AsyncAdmin {
|
|||
* servers to join other clusters. So we need to remove these servers from the group. 2.
|
||||
* Dead/recovering/live servers will be disallowed.
|
||||
* @param servers set of servers to remove
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
*/
|
||||
CompletableFuture<Void> removeServersFromRSGroup(Set<Address> servers);
|
||||
|
||||
|
@ -1712,7 +1709,6 @@ public interface AsyncAdmin {
|
|||
* Move given set of servers to the specified target RegionServer group
|
||||
* @param servers set of servers to move
|
||||
* @param groupName the group to move servers to
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
*/
|
||||
CompletableFuture<Void> moveServersToRSGroup(Set<Address> servers, String groupName);
|
||||
|
||||
|
@ -1720,7 +1716,6 @@ public interface AsyncAdmin {
|
|||
* Set the RegionServer group for tables
|
||||
* @param tables tables to set group for
|
||||
* @param groupName group name for tables
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
*/
|
||||
CompletableFuture<Void> setRSGroup(Set<TableName> tables, String groupName);
|
||||
|
||||
|
@ -1728,7 +1723,6 @@ public interface AsyncAdmin {
|
|||
* Balance regions in the given RegionServer group
|
||||
* @param groupName the group name
|
||||
* @return BalanceResponse details about the balancer run
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
*/
|
||||
default CompletableFuture<BalanceResponse> balanceRSGroup(String groupName) {
|
||||
return balanceRSGroup(groupName, BalanceRequest.defaultInstance());
|
||||
|
@ -1739,7 +1733,6 @@ public interface AsyncAdmin {
|
|||
* @param groupName the group name
|
||||
* @param request options to define how the balancer should run
|
||||
* @return BalanceResponse details about the balancer run
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
*/
|
||||
CompletableFuture<BalanceResponse> balanceRSGroup(String groupName, BalanceRequest request);
|
||||
|
||||
|
@ -1747,7 +1740,6 @@ public interface AsyncAdmin {
|
|||
* Rename rsgroup
|
||||
* @param oldName old rsgroup name
|
||||
* @param newName new rsgroup name
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
*/
|
||||
CompletableFuture<Void> renameRSGroup(String oldName, String newName);
|
||||
|
||||
|
@ -1755,7 +1747,6 @@ public interface AsyncAdmin {
|
|||
* Update RSGroup configuration
|
||||
* @param groupName the group name
|
||||
* @param configuration new configuration of the group name to be set
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
*/
|
||||
CompletableFuture<Void> updateRSGroupConfig(String groupName, Map<String, String> configuration);
|
||||
|
||||
|
@ -1769,7 +1760,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> {
|
||||
|
||||
|
|
|
@ -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 3.0.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 3.0.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.
|
||||
|
|
|
@ -128,6 +128,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));
|
||||
|
@ -159,18 +160,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) {
|
||||
|
@ -199,7 +201,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
|
||||
*/
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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
|
||||
*/
|
||||
|
@ -690,11 +689,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()));
|
||||
|
@ -708,11 +702,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);
|
||||
|
@ -749,6 +738,7 @@ public class ColumnFamilyDescriptorBuilder {
|
|||
}
|
||||
|
||||
/**
|
||||
* Set the maximum number of versions to retain.
|
||||
* @param maxVersions maximum number of versions
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
|
@ -768,7 +758,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)
|
||||
|
@ -795,10 +785,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));
|
||||
}
|
||||
|
@ -901,6 +887,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)
|
||||
|
@ -915,10 +902,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());
|
||||
|
@ -930,10 +913,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());
|
||||
}
|
||||
|
@ -959,6 +938,7 @@ public class ColumnFamilyDescriptorBuilder {
|
|||
}
|
||||
|
||||
/**
|
||||
* Set the time to live
|
||||
* @param timeToLive Time-to-live of cell contents, in seconds.
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
|
@ -967,9 +947,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)));
|
||||
|
@ -981,6 +962,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)
|
||||
*/
|
||||
|
@ -1010,6 +992,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)
|
||||
|
@ -1034,10 +1017,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));
|
||||
}
|
||||
|
@ -1049,6 +1028,7 @@ public class ColumnFamilyDescriptorBuilder {
|
|||
}
|
||||
|
||||
/**
|
||||
* Set the setCacheDataOnWrite flag
|
||||
* @param value true if we should cache data blocks on write
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
|
@ -1063,6 +1043,7 @@ public class ColumnFamilyDescriptorBuilder {
|
|||
}
|
||||
|
||||
/**
|
||||
* Set the setCacheIndexesOnWrite flag
|
||||
* @param value true if we should cache index blocks on write
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
|
@ -1077,6 +1058,7 @@ public class ColumnFamilyDescriptorBuilder {
|
|||
}
|
||||
|
||||
/**
|
||||
* Set the setCacheBloomsOnWrite flag.
|
||||
* @param value true if we should cache bloomfilter blocks on write
|
||||
* @return this (for chained invocation)
|
||||
*/
|
||||
|
@ -1091,6 +1073,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)
|
||||
*/
|
||||
|
@ -1105,6 +1088,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)
|
||||
*/
|
||||
|
@ -1232,19 +1216,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 {
|
||||
|
@ -1290,8 +1272,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);
|
||||
|
@ -1303,8 +1285,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);
|
||||
|
@ -1316,7 +1298,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) {
|
||||
}
|
||||
}
|
||||
|
|
|
@ -111,8 +111,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);
|
||||
|
@ -121,6 +119,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) {
|
||||
|
@ -144,6 +143,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;
|
||||
|
|
|
@ -445,7 +445,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;
|
||||
|
|
|
@ -96,6 +96,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;
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -78,11 +78,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));
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -230,6 +230,7 @@ class MutableRegionInfo implements RegionInfo {
|
|||
}
|
||||
|
||||
/**
|
||||
* Change the split status flag.
|
||||
* @param split set split status n
|
||||
*/
|
||||
public MutableRegionInfo setSplit(boolean split) {
|
||||
|
|
|
@ -156,7 +156,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.
|
||||
*/
|
||||
|
@ -165,8 +165,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) {
|
||||
|
@ -174,7 +173,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.
|
||||
*/
|
||||
|
@ -187,7 +186,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() {
|
||||
|
@ -261,7 +260,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);
|
||||
}
|
||||
|
@ -445,6 +444,7 @@ public abstract class Mutation extends OperationWithAttributes
|
|||
}
|
||||
|
||||
/**
|
||||
* Set the ACL for this operation.
|
||||
* @param user User short name
|
||||
* @param perms Permissions for the user
|
||||
*/
|
||||
|
@ -455,6 +455,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) {
|
||||
|
@ -596,11 +597,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) {
|
||||
|
|
|
@ -203,6 +203,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);
|
||||
|
|
|
@ -502,11 +502,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)) {
|
||||
|
@ -1559,11 +1554,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 {
|
||||
|
@ -1584,10 +1581,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) {
|
||||
|
@ -1608,14 +1605,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 {
|
||||
|
@ -2236,7 +2230,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;
|
||||
|
@ -2253,7 +2247,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
|
|||
future.complete(v);
|
||||
}
|
||||
});
|
||||
}));
|
||||
});
|
||||
return future;
|
||||
}
|
||||
|
||||
|
@ -4010,10 +4004,9 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
|
|||
@Override
|
||||
public CompletableFuture<Void> addRSGroup(String groupName) {
|
||||
return this.<Void> newMasterCaller()
|
||||
.action(
|
||||
((controller, stub) -> this.<AddRSGroupRequest, AddRSGroupResponse, Void> call(controller,
|
||||
stub, AddRSGroupRequest.newBuilder().setRSGroupName(groupName).build(),
|
||||
(s, c, req, done) -> s.addRSGroup(c, req, done), resp -> null)))
|
||||
.action((controller, stub) -> this.<AddRSGroupRequest, AddRSGroupResponse, Void> call(
|
||||
controller, stub, AddRSGroupRequest.newBuilder().setRSGroupName(groupName).build(),
|
||||
(s, c, req, done) -> s.addRSGroup(c, req, done), resp -> null))
|
||||
.call();
|
||||
}
|
||||
|
||||
|
@ -4081,9 +4074,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();
|
||||
}
|
||||
|
||||
|
@ -4124,15 +4117,14 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
|
|||
@Override
|
||||
public CompletableFuture<RSGroupInfo> getRSGroup(Address hostPort) {
|
||||
return this.<RSGroupInfo> newMasterCaller()
|
||||
.action(
|
||||
((controller, stub) -> this.<GetRSGroupInfoOfServerRequest, GetRSGroupInfoOfServerResponse,
|
||||
RSGroupInfo> call(controller, stub, GetRSGroupInfoOfServerRequest.newBuilder()
|
||||
.action((controller, stub) -> this.<GetRSGroupInfoOfServerRequest,
|
||||
GetRSGroupInfoOfServerResponse, RSGroupInfo> call(controller, stub,
|
||||
GetRSGroupInfoOfServerRequest.newBuilder()
|
||||
.setServer(HBaseProtos.ServerName.newBuilder().setHostName(hostPort.getHostname())
|
||||
.setPort(hostPort.getPort()).build())
|
||||
.build(), (s, c, req, done) -> s.getRSGroupInfoOfServer(c, req, done),
|
||||
resp -> resp.hasRSGroupInfo()
|
||||
? ProtobufUtil.toGroupInfo(resp.getRSGroupInfo())
|
||||
: null)))
|
||||
.build(),
|
||||
(s, c, req, done) -> s.getRSGroupInfoOfServer(c, req, done),
|
||||
resp -> resp.hasRSGroupInfo() ? ProtobufUtil.toGroupInfo(resp.getRSGroupInfo()) : null))
|
||||
.call();
|
||||
}
|
||||
|
||||
|
@ -4160,7 +4152,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
|
|||
}
|
||||
});
|
||||
}
|
||||
addListener(listTableDescriptors(new ArrayList<>(tables)), ((tableDescriptions, err) -> {
|
||||
addListener(listTableDescriptors(new ArrayList<>(tables)), (tableDescriptions, err) -> {
|
||||
if (err != null) {
|
||||
future.completeExceptionally(err);
|
||||
return;
|
||||
|
@ -4184,40 +4176,40 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
|
|||
future.complete(v);
|
||||
}
|
||||
});
|
||||
}));
|
||||
});
|
||||
return future;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompletableFuture<RSGroupInfo> getRSGroup(TableName table) {
|
||||
return this.<RSGroupInfo> newMasterCaller()
|
||||
.action(((controller, stub) -> this.<GetRSGroupInfoOfTableRequest,
|
||||
.action((controller, stub) -> this.<GetRSGroupInfoOfTableRequest,
|
||||
GetRSGroupInfoOfTableResponse, RSGroupInfo> call(controller, stub,
|
||||
GetRSGroupInfoOfTableRequest.newBuilder()
|
||||
.setTableName(ProtobufUtil.toProtoTableName(table)).build(),
|
||||
(s, c, req, done) -> s.getRSGroupInfoOfTable(c, req, done),
|
||||
resp -> resp.hasRSGroupInfo() ? ProtobufUtil.toGroupInfo(resp.getRSGroupInfo()) : null)))
|
||||
resp -> resp.hasRSGroupInfo() ? ProtobufUtil.toGroupInfo(resp.getRSGroupInfo()) : null))
|
||||
.call();
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompletableFuture<RSGroupInfo> getRSGroup(String groupName) {
|
||||
return this.<RSGroupInfo> newMasterCaller()
|
||||
.action(((controller, stub) -> this.<GetRSGroupInfoRequest, GetRSGroupInfoResponse,
|
||||
.action((controller, stub) -> this.<GetRSGroupInfoRequest, GetRSGroupInfoResponse,
|
||||
RSGroupInfo> call(controller, stub,
|
||||
GetRSGroupInfoRequest.newBuilder().setRSGroupName(groupName).build(),
|
||||
(s, c, req, done) -> s.getRSGroupInfo(c, req, done),
|
||||
resp -> resp.hasRSGroupInfo() ? ProtobufUtil.toGroupInfo(resp.getRSGroupInfo()) : null)))
|
||||
resp -> resp.hasRSGroupInfo() ? ProtobufUtil.toGroupInfo(resp.getRSGroupInfo()) : null))
|
||||
.call();
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompletableFuture<Void> renameRSGroup(String oldName, String newName) {
|
||||
return this.<Void> newMasterCaller()
|
||||
.action(((controller, stub) -> this.<RenameRSGroupRequest, RenameRSGroupResponse, Void> call(
|
||||
.action((controller, stub) -> this.<RenameRSGroupRequest, RenameRSGroupResponse, Void> call(
|
||||
controller, stub, RenameRSGroupRequest.newBuilder().setOldRsgroupName(oldName)
|
||||
.setNewRsgroupName(newName).build(),
|
||||
(s, c, req, done) -> s.renameRSGroup(c, req, done), resp -> null)))
|
||||
(s, c, req, done) -> s.renameRSGroup(c, req, done), resp -> null))
|
||||
.call();
|
||||
}
|
||||
|
||||
|
@ -4231,9 +4223,9 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
|
|||
NameStringPair.newBuilder().setName(e.getKey()).setValue(e.getValue()).build()));
|
||||
}
|
||||
return this.<Void> newMasterCaller()
|
||||
.action(((controller, stub) -> this.<UpdateRSGroupConfigRequest, UpdateRSGroupConfigResponse,
|
||||
.action((controller, stub) -> this.<UpdateRSGroupConfigRequest, UpdateRSGroupConfigResponse,
|
||||
Void> call(controller, stub, request.build(),
|
||||
(s, c, req, done) -> s.updateRSGroupConfig(c, req, done), resp -> null)))
|
||||
(s, c, req, done) -> s.updateRSGroupConfig(c, req, done), resp -> null))
|
||||
.call();
|
||||
}
|
||||
|
||||
|
@ -4287,9 +4279,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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -776,7 +776,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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -92,7 +92,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
|
||||
|
@ -423,7 +423,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) {
|
||||
|
@ -751,7 +751,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)
|
||||
*/
|
||||
|
@ -760,7 +760,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) {
|
||||
|
@ -773,7 +773,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) {
|
||||
|
@ -799,6 +799,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)
|
||||
|
|
|
@ -78,11 +78,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);
|
||||
}));
|
||||
});
|
||||
}
|
||||
}
|
||||
|
|
|
@ -894,8 +894,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;
|
||||
|
|
|
@ -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> {
|
||||
|
|
|
@ -70,6 +70,7 @@ public class RowMutations implements Row {
|
|||
}
|
||||
|
||||
/**
|
||||
* Add a mutation
|
||||
* @param mutation The data to send.
|
||||
* @throws IOException if the row of added mutation doesn't match the original row
|
||||
*/
|
||||
|
@ -78,6 +79,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
|
||||
*/
|
||||
|
|
|
@ -710,9 +710,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);
|
||||
}
|
||||
|
|
|
@ -28,20 +28,16 @@ import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
|||
* 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();
|
||||
}
|
||||
|
|
|
@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.Cell;
|
|||
import org.apache.hadoop.hbase.CompareOperator;
|
||||
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.filter.Filter;
|
||||
import org.apache.hadoop.hbase.io.TimeRange;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
|
||||
|
@ -302,11 +301,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);
|
||||
|
@ -325,24 +326,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.
|
||||
*/
|
||||
|
@ -379,23 +384,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.
|
||||
*/
|
||||
|
@ -660,7 +669,7 @@ public interface Table extends Closeable {
|
|||
final Map<byte[], R> results =
|
||||
Collections.synchronizedMap(new TreeMap<byte[], R>(Bytes.BYTES_COMPARATOR));
|
||||
batchCoprocessorService(methodDescriptor, request, startKey, endKey, responsePrototype,
|
||||
new Callback<R>() {
|
||||
new Batch.Callback<R>() {
|
||||
@Override
|
||||
public void update(byte[] region, byte[] row, R result) {
|
||||
if (region != null) {
|
||||
|
|
|
@ -267,8 +267,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();
|
||||
|
||||
|
|
|
@ -143,8 +143,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.
|
||||
|
@ -302,10 +300,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();
|
||||
|
@ -317,7 +312,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);
|
||||
|
|
|
@ -47,8 +47,7 @@ import org.apache.hadoop.hbase.HConstants;
|
|||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.RetriesExhaustedException.ThrowableWithExtraContext;
|
||||
import org.apache.hadoop.hbase.client.coprocessor.Batch.Call;
|
||||
import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
|
||||
import org.apache.hadoop.hbase.client.coprocessor.Batch;
|
||||
import org.apache.hadoop.hbase.client.trace.TableOperationSpanBuilder;
|
||||
import org.apache.hadoop.hbase.filter.Filter;
|
||||
import org.apache.hadoop.hbase.io.TimeRange;
|
||||
|
@ -143,8 +142,8 @@ class TableOverAsyncTable implements Table {
|
|||
}
|
||||
|
||||
@Override
|
||||
public <R> void batchCallback(List<? extends Row> actions, Object[] results, Callback<R> callback)
|
||||
throws IOException, InterruptedException {
|
||||
public <R> void batchCallback(List<? extends Row> actions, Object[] results,
|
||||
Batch.Callback<R> callback) throws IOException, InterruptedException {
|
||||
ConcurrentLinkedQueue<ThrowableWithExtraContext> errors = new ConcurrentLinkedQueue<>();
|
||||
CountDownLatch latch = new CountDownLatch(actions.size());
|
||||
AsyncTableRegionLocator locator = conn.getRegionLocator(getName());
|
||||
|
@ -467,7 +466,7 @@ class TableOverAsyncTable implements Table {
|
|||
}
|
||||
|
||||
private <R> void coprocessorService(String serviceName, byte[] startKey, byte[] endKey,
|
||||
Callback<R> callback, StubCall<R> call) throws Throwable {
|
||||
Batch.Callback<R> callback, StubCall<R> call) throws Throwable {
|
||||
// get regions covered by the row range
|
||||
ExecutorService pool = Context.current().wrap(this.poolSupplier.get());
|
||||
List<byte[]> keys = getStartKeysInRange(startKey, endKey);
|
||||
|
@ -509,7 +508,8 @@ class TableOverAsyncTable implements Table {
|
|||
|
||||
@Override
|
||||
public <T extends Service, R> void coprocessorService(Class<T> service, byte[] startKey,
|
||||
byte[] endKey, Call<T, R> callable, Callback<R> callback) throws ServiceException, Throwable {
|
||||
byte[] endKey, Batch.Call<T, R> callable, Batch.Callback<R> callback)
|
||||
throws ServiceException, Throwable {
|
||||
final Supplier<Span> supplier = new TableOperationSpanBuilder(conn)
|
||||
.setTableName(table.getName()).setOperation(HBaseSemanticAttributes.Operation.COPROC_EXEC);
|
||||
TraceUtil.trace(() -> {
|
||||
|
@ -526,8 +526,8 @@ class TableOverAsyncTable implements Table {
|
|||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public <R extends Message> void batchCoprocessorService(MethodDescriptor methodDescriptor,
|
||||
Message request, byte[] startKey, byte[] endKey, R responsePrototype, Callback<R> callback)
|
||||
throws ServiceException, Throwable {
|
||||
Message request, byte[] startKey, byte[] endKey, R responsePrototype,
|
||||
Batch.Callback<R> callback) throws ServiceException, Throwable {
|
||||
final Supplier<Span> supplier = new TableOperationSpanBuilder(conn)
|
||||
.setTableName(table.getName()).setOperation(HBaseSemanticAttributes.Operation.COPROC_EXEC);
|
||||
TraceUtil.trace(() -> {
|
||||
|
|
|
@ -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";
|
||||
|
|
|
@ -80,7 +80,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
|
||||
|
|
|
@ -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;
|
||||
|
@ -113,8 +114,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
|
||||
|
@ -124,9 +125,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;
|
||||
}
|
||||
|
@ -135,16 +136,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) {
|
||||
|
@ -241,6 +243,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
|
||||
|
@ -384,10 +387,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();
|
||||
|
|
|
@ -40,11 +40,12 @@ import org.apache.hbase.thirdparty.com.google.protobuf.Message;
|
|||
* {@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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -86,9 +86,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)
|
||||
|
@ -103,9 +104,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());
|
||||
|
|
|
@ -88,8 +88,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)
|
||||
|
@ -104,15 +106,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;
|
||||
}
|
||||
|
@ -167,9 +158,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 {
|
||||
|
@ -186,15 +179,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()
|
||||
|
|
|
@ -111,9 +111,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 {
|
||||
|
@ -127,15 +128,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());
|
||||
}
|
||||
|
|
|
@ -157,9 +157,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;
|
||||
|
@ -175,9 +177,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())
|
||||
|
|
|
@ -136,7 +136,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());
|
||||
|
@ -166,13 +166,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()
|
||||
|
|
|
@ -199,9 +199,11 @@ public class DependentColumnFilter extends CompareFilter {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse a seralized representation of {@link DependentColumnFilter}
|
||||
* @param pbBytes A pb serialized {@link DependentColumnFilter} instance
|
||||
* @return An instance of {@link DependentColumnFilter} made from <code>bytes</code> n * @see
|
||||
* #toByteArray
|
||||
* @return An instance of {@link DependentColumnFilter} made from <code>bytes</code>
|
||||
* @throws DeserializationException if an error occurred
|
||||
* @see #toByteArray
|
||||
*/
|
||||
public static DependentColumnFilter parseFrom(final byte[] pbBytes)
|
||||
throws DeserializationException {
|
||||
|
@ -228,16 +230,19 @@ public class DependentColumnFilter extends CompareFilter {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings(
|
||||
value = "RCN_REDUNDANT_NULLCHECK_OF_NONNULL_VALUE")
|
||||
@Override
|
||||
boolean areSerializedFieldsEqual(Filter o) {
|
||||
if (o == this) return true;
|
||||
if (!(o instanceof DependentColumnFilter)) return false;
|
||||
|
||||
if (o == this) {
|
||||
return true;
|
||||
}
|
||||
if (!(o instanceof DependentColumnFilter)) {
|
||||
return false;
|
||||
}
|
||||
DependentColumnFilter other = (DependentColumnFilter) o;
|
||||
return other != null && super.areSerializedFieldsEqual(other)
|
||||
&& Bytes.equals(this.getFamily(), other.getFamily())
|
||||
|
|
|
@ -82,8 +82,11 @@ public class FamilyFilter extends CompareFilter {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse the serialized representation of {@link FamilyFilter}
|
||||
* @param pbBytes A pb serialized {@link FamilyFilter} instance
|
||||
* @return An instance of {@link FamilyFilter} made from <code>bytes</code> n * @see #toByteArray
|
||||
* @return An instance of {@link FamilyFilter} made from <code>bytes</code>
|
||||
* @throws DeserializationException if an error occurred
|
||||
* @see #toByteArray
|
||||
*/
|
||||
public static FamilyFilter parseFrom(final byte[] pbBytes) throws DeserializationException {
|
||||
FilterProtos.FamilyFilter proto;
|
||||
|
@ -106,14 +109,17 @@ public class FamilyFilter extends CompareFilter {
|
|||
}
|
||||
|
||||
/**
|
||||
* @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 FamilyFilter)) return false;
|
||||
|
||||
if (o == this) {
|
||||
return true;
|
||||
}
|
||||
if (!(o instanceof FamilyFilter)) {
|
||||
return false;
|
||||
}
|
||||
FamilyFilter other = (FamilyFilter) o;
|
||||
return super.areSerializedFieldsEqual(other);
|
||||
}
|
||||
|
|
|
@ -205,7 +205,9 @@ public abstract class Filter {
|
|||
* Concrete implementers can signal a failure condition in their code by throwing an
|
||||
* {@link IOException}.
|
||||
* @param pbBytes A pb serialized {@link Filter} instance
|
||||
* @return An instance of {@link Filter} made from <code>bytes</code> n * @see #toByteArray
|
||||
* @return An instance of {@link Filter} made from <code>bytes</code>
|
||||
* @throws DeserializationException if an error occurred
|
||||
* @see #toByteArray
|
||||
*/
|
||||
public static Filter parseFrom(final byte[] pbBytes) throws DeserializationException {
|
||||
throw new DeserializationException(
|
||||
|
@ -216,7 +218,6 @@ public abstract class Filter {
|
|||
* Concrete implementers can signal a failure condition in their code by throwing an
|
||||
* {@link IOException}. n * @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.
|
||||
* @throws IOException in case an I/O or an filter specific failure needs to be signaled.
|
||||
*/
|
||||
abstract boolean areSerializedFieldsEqual(Filter other);
|
||||
|
||||
|
|
|
@ -194,8 +194,11 @@ final public class FilterList extends FilterBase {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse a seralized representation of {@link FilterList}
|
||||
* @param pbBytes A pb serialized {@link FilterList} instance
|
||||
* @return An instance of {@link FilterList} made from <code>bytes</code> n * @see #toByteArray
|
||||
* @return An instance of {@link FilterList} made from <code>bytes</code>
|
||||
* @throws DeserializationException if an error occurred
|
||||
* @see #toByteArray
|
||||
*/
|
||||
public static FilterList parseFrom(final byte[] pbBytes) throws DeserializationException {
|
||||
FilterProtos.FilterList proto;
|
||||
|
@ -218,14 +221,17 @@ final public class FilterList 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 other) {
|
||||
if (other == this) return true;
|
||||
if (!(other instanceof FilterList)) return false;
|
||||
|
||||
if (other == this) {
|
||||
return true;
|
||||
}
|
||||
if (!(other instanceof FilterList)) {
|
||||
return false;
|
||||
}
|
||||
FilterList o = (FilterList) other;
|
||||
return this.getOperator().equals(o.getOperator())
|
||||
&& ((this.getFilters() == o.getFilters()) || this.getFilters().equals(o.getFilters()));
|
||||
|
|
|
@ -270,12 +270,12 @@ public class FilterListWithAND extends FilterListBase {
|
|||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (!(obj instanceof FilterListWithAND)) {
|
||||
return false;
|
||||
}
|
||||
if (this == obj) {
|
||||
return true;
|
||||
}
|
||||
if (!(obj instanceof FilterListWithAND)) {
|
||||
return false;
|
||||
}
|
||||
FilterListWithAND f = (FilterListWithAND) obj;
|
||||
return this.filters.equals(f.getFilters()) && this.seekHintFilters.equals(f.seekHintFilters);
|
||||
}
|
||||
|
|
|
@ -394,12 +394,12 @@ public class FilterListWithOR extends FilterListBase {
|
|||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (obj == null || (!(obj instanceof FilterListWithOR))) {
|
||||
return false;
|
||||
}
|
||||
if (this == obj) {
|
||||
return true;
|
||||
}
|
||||
if (!(obj instanceof FilterListWithOR)) {
|
||||
return false;
|
||||
}
|
||||
FilterListWithOR f = (FilterListWithOR) obj;
|
||||
return this.filters.equals(f.getFilters()) && this.prevFilterRCList.equals(f.prevFilterRCList)
|
||||
&& this.prevCellList.equals(f.prevCellList);
|
||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.filter;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Objects;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
@ -71,6 +70,7 @@ public class FirstKeyOnlyFilter extends FilterBase {
|
|||
}
|
||||
|
||||
/**
|
||||
* Set or clear the indication if the first KV has been found.
|
||||
* @param value update {@link #foundKV} flag with value.
|
||||
*/
|
||||
protected void setFoundKV(boolean value) {
|
||||
|
@ -85,9 +85,10 @@ public class FirstKeyOnlyFilter extends FilterBase {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse a serialized representation of {@link FirstKeyOnlyFilter}
|
||||
* @param pbBytes A pb serialized {@link FirstKeyOnlyFilter} instance
|
||||
* @return An instance of {@link FirstKeyOnlyFilter} made from <code>bytes</code>
|
||||
* @throws org.apache.hadoop.hbase.exceptions.DeserializationException
|
||||
* @throws DeserializationException if an error occurred
|
||||
* @see #toByteArray
|
||||
*/
|
||||
public static FirstKeyOnlyFilter parseFrom(final byte[] pbBytes) throws DeserializationException {
|
||||
|
@ -102,15 +103,17 @@ public class FirstKeyOnlyFilter 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 FirstKeyOnlyFilter)) return false;
|
||||
|
||||
if (o == this) {
|
||||
return true;
|
||||
}
|
||||
if (!(o instanceof FirstKeyOnlyFilter)) {
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -121,6 +124,6 @@ public class FirstKeyOnlyFilter extends FilterBase {
|
|||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hashCode(foundKV);
|
||||
return Boolean.hashCode(foundKV);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -32,9 +32,12 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
public class FirstKeyValueMatchingQualifiersFilter extends FirstKeyOnlyFilter {
|
||||
|
||||
/**
|
||||
* Parses a serialized representation of {@link FirstKeyValueMatchingQualifiersFilter}
|
||||
* @param pbBytes A pb serialized {@link FirstKeyValueMatchingQualifiersFilter} instance
|
||||
* @return An instance of {@link FirstKeyValueMatchingQualifiersFilter} made from
|
||||
* <code>bytes</code> n * @see #toByteArray
|
||||
* <code>bytes</code>
|
||||
* @throws DeserializationException if an error occurred
|
||||
* @see #toByteArray
|
||||
*/
|
||||
public static FirstKeyValueMatchingQualifiersFilter parseFrom(final byte[] pbBytes)
|
||||
throws DeserializationException {
|
||||
|
|
|
@ -265,9 +265,11 @@ public class FuzzyRowFilter extends FilterBase {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse a serialized representation of {@link FuzzyRowFilter}
|
||||
* @param pbBytes A pb serialized {@link FuzzyRowFilter} instance
|
||||
* @return An instance of {@link FuzzyRowFilter} made from <code>bytes</code> n * @see
|
||||
* #toByteArray
|
||||
* @return An instance of {@link FuzzyRowFilter} made from <code>bytes</code>
|
||||
* @throws DeserializationException if an error occurred
|
||||
* @see #toByteArray
|
||||
*/
|
||||
public static FuzzyRowFilter parseFrom(final byte[] pbBytes) throws DeserializationException {
|
||||
FilterProtos.FuzzyRowFilter proto;
|
||||
|
@ -340,7 +342,7 @@ public class FuzzyRowFilter extends FilterBase {
|
|||
long fuzzyBytes = Bytes.toLong(fuzzyKeyBytes, i);
|
||||
long fuzzyMeta = Bytes.toLong(fuzzyKeyMeta, i);
|
||||
long rowValue = Bytes.toLong(row, offset + i);
|
||||
if ((rowValue & fuzzyMeta) != (fuzzyBytes)) {
|
||||
if ((rowValue & fuzzyMeta) != fuzzyBytes) {
|
||||
// We always return NEXT_EXISTS
|
||||
return SatisfiesCode.NEXT_EXISTS;
|
||||
}
|
||||
|
@ -352,7 +354,7 @@ public class FuzzyRowFilter extends FilterBase {
|
|||
int fuzzyBytes = Bytes.toInt(fuzzyKeyBytes, off);
|
||||
int fuzzyMeta = Bytes.toInt(fuzzyKeyMeta, off);
|
||||
int rowValue = Bytes.toInt(row, offset + off);
|
||||
if ((rowValue & fuzzyMeta) != (fuzzyBytes)) {
|
||||
if ((rowValue & fuzzyMeta) != fuzzyBytes) {
|
||||
// We always return NEXT_EXISTS
|
||||
return SatisfiesCode.NEXT_EXISTS;
|
||||
}
|
||||
|
@ -363,7 +365,7 @@ public class FuzzyRowFilter extends FilterBase {
|
|||
short fuzzyBytes = Bytes.toShort(fuzzyKeyBytes, off);
|
||||
short fuzzyMeta = Bytes.toShort(fuzzyKeyMeta, off);
|
||||
short rowValue = Bytes.toShort(row, offset + off);
|
||||
if ((rowValue & fuzzyMeta) != (fuzzyBytes)) {
|
||||
if ((rowValue & fuzzyMeta) != fuzzyBytes) {
|
||||
// We always return NEXT_EXISTS
|
||||
// even if it does not (in this case getNextForFuzzyRule
|
||||
// will return null)
|
||||
|
@ -376,7 +378,7 @@ public class FuzzyRowFilter extends FilterBase {
|
|||
int fuzzyBytes = fuzzyKeyBytes[off] & 0xff;
|
||||
int fuzzyMeta = fuzzyKeyMeta[off] & 0xff;
|
||||
int rowValue = row[offset + off] & 0xff;
|
||||
if ((rowValue & fuzzyMeta) != (fuzzyBytes)) {
|
||||
if ((rowValue & fuzzyMeta) != fuzzyBytes) {
|
||||
// We always return NEXT_EXISTS
|
||||
return SatisfiesCode.NEXT_EXISTS;
|
||||
}
|
||||
|
@ -603,14 +605,17 @@ public class FuzzyRowFilter extends FilterBase {
|
|||
}
|
||||
|
||||
/**
|
||||
* @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 FuzzyRowFilter)) return false;
|
||||
|
||||
if (o == this) {
|
||||
return true;
|
||||
}
|
||||
if (!(o instanceof FuzzyRowFilter)) {
|
||||
return false;
|
||||
}
|
||||
FuzzyRowFilter other = (FuzzyRowFilter) o;
|
||||
if (this.fuzzyKeysData.size() != other.fuzzyKeysData.size()) return false;
|
||||
for (int i = 0; i < fuzzyKeysData.size(); ++i) {
|
||||
|
|
|
@ -86,9 +86,11 @@ public class InclusiveStopFilter extends FilterBase {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse a serialized representation of {@link InclusiveStopFilter}
|
||||
* @param pbBytes A pb serialized {@link InclusiveStopFilter} instance
|
||||
* @return An instance of {@link InclusiveStopFilter} made from <code>bytes</code> n * @see
|
||||
* #toByteArray
|
||||
* @return An instance of {@link InclusiveStopFilter} made from <code>bytes</code>
|
||||
* @throws DeserializationException if an error occurred
|
||||
* @see #toByteArray
|
||||
*/
|
||||
public static InclusiveStopFilter parseFrom(final byte[] pbBytes)
|
||||
throws DeserializationException {
|
||||
|
@ -103,15 +105,17 @@ public class InclusiveStopFilter 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 InclusiveStopFilter)) return false;
|
||||
|
||||
if (o == this) {
|
||||
return true;
|
||||
}
|
||||
if (!(o instanceof InclusiveStopFilter)) {
|
||||
return false;
|
||||
}
|
||||
InclusiveStopFilter other = (InclusiveStopFilter) o;
|
||||
return Bytes.equals(this.getStopRowKey(), other.getStopRowKey());
|
||||
}
|
||||
|
|
|
@ -102,8 +102,11 @@ public class KeyOnlyFilter extends FilterBase {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse a serialized representation of {@link KeyOnlyFilter}
|
||||
* @param pbBytes A pb serialized {@link KeyOnlyFilter} instance
|
||||
* @return An instance of {@link KeyOnlyFilter} made from <code>bytes</code> n * @see #toByteArray
|
||||
* @return An instance of {@link KeyOnlyFilter} made from <code>bytes</code>
|
||||
* @throws DeserializationException if an error occurred
|
||||
* @see #toByteArray
|
||||
*/
|
||||
public static KeyOnlyFilter parseFrom(final byte[] pbBytes) throws DeserializationException {
|
||||
FilterProtos.KeyOnlyFilter proto;
|
||||
|
@ -116,15 +119,17 @@ public class KeyOnlyFilter 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 KeyOnlyFilter)) return false;
|
||||
|
||||
if (o == this) {
|
||||
return true;
|
||||
}
|
||||
if (!(o instanceof KeyOnlyFilter)) {
|
||||
return false;
|
||||
}
|
||||
KeyOnlyFilter other = (KeyOnlyFilter) o;
|
||||
return this.lenAsVal == other.lenAsVal;
|
||||
}
|
||||
|
|
|
@ -62,9 +62,10 @@ public class LongComparator extends ByteArrayComparable {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parses a serialized representation of {@link LongComparator}
|
||||
* @param pbBytes A pb serialized {@link LongComparator} instance
|
||||
* @return An instance of {@link LongComparator} made from <code>bytes</code>
|
||||
* @throws org.apache.hadoop.hbase.exceptions.DeserializationException
|
||||
* @throws DeserializationException if an error occurred
|
||||
* @see #toByteArray
|
||||
*/
|
||||
public static LongComparator parseFrom(final byte[] pbBytes) throws DeserializationException {
|
||||
|
@ -78,11 +79,16 @@ public class LongComparator 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.
|
||||
*/
|
||||
boolean areSerializedFieldsEqual(LongComparator other) {
|
||||
if (other == this) return true;
|
||||
if (other == this) {
|
||||
return true;
|
||||
}
|
||||
if (other == null) {
|
||||
return false;
|
||||
}
|
||||
return super.areSerializedFieldsEqual(other);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -197,9 +197,11 @@ public class MultiRowRangeFilter extends FilterBase {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse a serialized representation of {@link MultiRowRangeFilter}
|
||||
* @param pbBytes A pb serialized instance
|
||||
* @return An instance of MultiRowRangeFilter
|
||||
* @throws org.apache.hadoop.hbase.exceptions.DeserializationException
|
||||
* @return An instance of {@link MultiRowRangeFilter}
|
||||
* @throws DeserializationException if an error occurred
|
||||
* @see #toByteArray
|
||||
*/
|
||||
public static MultiRowRangeFilter parseFrom(final byte[] pbBytes)
|
||||
throws DeserializationException {
|
||||
|
@ -224,15 +226,17 @@ public class MultiRowRangeFilter extends FilterBase {
|
|||
}
|
||||
|
||||
/**
|
||||
* @param o the filter to compare
|
||||
* @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 MultiRowRangeFilter)) return false;
|
||||
|
||||
if (o == this) {
|
||||
return true;
|
||||
}
|
||||
if (!(o instanceof MultiRowRangeFilter)) {
|
||||
return false;
|
||||
}
|
||||
MultiRowRangeFilter other = (MultiRowRangeFilter) o;
|
||||
if (this.rangeList.size() != other.rangeList.size()) return false;
|
||||
for (int i = 0; i < rangeList.size(); ++i) {
|
||||
|
@ -719,7 +723,7 @@ public class MultiRowRangeFilter extends FilterBase {
|
|||
/**
|
||||
* Gets the RowRange at the given offset.
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
@SuppressWarnings({ "unchecked", "TypeParameterUnusedInFormals" })
|
||||
public <T extends BasicRowRange> T get(int i) {
|
||||
return (T) ranges.get(i);
|
||||
}
|
||||
|
|
|
@ -128,9 +128,11 @@ public class MultipleColumnPrefixFilter extends FilterBase {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse a serialized representation of {@link MultipleColumnPrefixFilter}
|
||||
* @param pbBytes A pb serialized {@link MultipleColumnPrefixFilter} instance
|
||||
* @return An instance of {@link MultipleColumnPrefixFilter} made from <code>bytes</code> n * @see
|
||||
* #toByteArray
|
||||
* @return An instance of {@link MultipleColumnPrefixFilter} made from <code>bytes</code>
|
||||
* @throws DeserializationException if an error occurred
|
||||
* @see #toByteArray
|
||||
*/
|
||||
public static MultipleColumnPrefixFilter parseFrom(final byte[] pbBytes)
|
||||
throws DeserializationException {
|
||||
|
@ -150,15 +152,17 @@ public class MultipleColumnPrefixFilter 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 MultipleColumnPrefixFilter)) return false;
|
||||
|
||||
if (o == this) {
|
||||
return true;
|
||||
}
|
||||
if (!(o instanceof MultipleColumnPrefixFilter)) {
|
||||
return false;
|
||||
}
|
||||
MultipleColumnPrefixFilter other = (MultipleColumnPrefixFilter) o;
|
||||
return this.sortedPrefixes.equals(other.sortedPrefixes);
|
||||
}
|
||||
|
|
|
@ -71,9 +71,11 @@ public class NullComparator extends ByteArrayComparable {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse the serialized representation of {@link NullComparator}
|
||||
* @param pbBytes A pb serialized {@link NullComparator} instance
|
||||
* @return An instance of {@link NullComparator} made from <code>bytes</code> n * @see
|
||||
* #toByteArray
|
||||
* @return An instance of {@link NullComparator} made from <code>bytes</code>
|
||||
* @throws DeserializationException if an error occurred
|
||||
* @see #toByteArray
|
||||
*/
|
||||
public static NullComparator parseFrom(final byte[] pbBytes) throws DeserializationException {
|
||||
try {
|
||||
|
@ -86,14 +88,17 @@ public class NullComparator 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 NullComparator)) return false;
|
||||
|
||||
if (other == this) {
|
||||
return true;
|
||||
}
|
||||
if (!(other instanceof NullComparator)) {
|
||||
return false;
|
||||
}
|
||||
return super.areSerializedFieldsEqual(other);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -100,8 +100,11 @@ public class PageFilter extends FilterBase {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse a serialized representation of {@link PageFilter}
|
||||
* @param pbBytes A pb serialized {@link PageFilter} instance
|
||||
* @return An instance of {@link PageFilter} made from <code>bytes</code> n * @see #toByteArray
|
||||
* @return An instance of {@link PageFilter} made from <code>bytes</code>
|
||||
* @throws DeserializationException if an error occurred
|
||||
* @see #toByteArray
|
||||
*/
|
||||
public static PageFilter parseFrom(final byte[] pbBytes) throws DeserializationException {
|
||||
FilterProtos.PageFilter proto;
|
||||
|
@ -114,9 +117,8 @@ public class PageFilter extends FilterBase {
|
|||
}
|
||||
|
||||
/**
|
||||
* @param o 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) {
|
||||
|
|
|
@ -156,7 +156,7 @@ public class ParseFilter {
|
|||
operatorStack.pop();
|
||||
continue;
|
||||
}
|
||||
while (!(argumentOnTopOfStack.equals(ParseConstants.LPAREN_BUFFER))) {
|
||||
while (!argumentOnTopOfStack.equals(ParseConstants.LPAREN_BUFFER)) {
|
||||
filterStack.push(popArguments(operatorStack, filterStack));
|
||||
if (operatorStack.empty()) {
|
||||
throw new IllegalArgumentException("Mismatched parenthesis");
|
||||
|
@ -367,7 +367,7 @@ public class ParseFilter {
|
|||
public void reduce(Stack<ByteBuffer> operatorStack, Stack<Filter> filterStack,
|
||||
ByteBuffer operator) {
|
||||
while (
|
||||
!operatorStack.empty() && !(ParseConstants.LPAREN_BUFFER.equals(operatorStack.peek()))
|
||||
!operatorStack.empty() && !ParseConstants.LPAREN_BUFFER.equals(operatorStack.peek())
|
||||
&& hasHigherPriority(operatorStack.peek(), operator)
|
||||
) {
|
||||
filterStack.push(popArguments(operatorStack, filterStack));
|
||||
|
|
|
@ -110,9 +110,10 @@ public class PrefixFilter extends FilterBase {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse a serialized representation of {@link PrefixFilter}
|
||||
* @param pbBytes A pb serialized {@link PrefixFilter} instance
|
||||
* @return An instance of {@link PrefixFilter} made from <code>bytes</code>
|
||||
* @throws org.apache.hadoop.hbase.exceptions.DeserializationException
|
||||
* @throws DeserializationException if an error occurred
|
||||
* @see #toByteArray
|
||||
*/
|
||||
public static PrefixFilter parseFrom(final byte[] pbBytes) throws DeserializationException {
|
||||
|
@ -126,15 +127,17 @@ public class PrefixFilter 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 PrefixFilter)) return false;
|
||||
|
||||
if (o == this) {
|
||||
return true;
|
||||
}
|
||||
if (!(o instanceof PrefixFilter)) {
|
||||
return false;
|
||||
}
|
||||
PrefixFilter other = (PrefixFilter) o;
|
||||
return Bytes.equals(this.getPrefix(), other.getPrefix());
|
||||
}
|
||||
|
|
|
@ -76,9 +76,10 @@ public class QualifierFilter extends CompareFilter {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse a serialized representation of {@link QualifierFilter}
|
||||
* @param pbBytes A pb serialized {@link QualifierFilter} instance
|
||||
* @return An instance of {@link QualifierFilter} made from <code>bytes</code>
|
||||
* @throws org.apache.hadoop.hbase.exceptions.DeserializationException
|
||||
* @throws DeserializationException if an error occurred
|
||||
* @see #toByteArray
|
||||
*/
|
||||
public static QualifierFilter parseFrom(final byte[] pbBytes) throws DeserializationException {
|
||||
|
@ -102,14 +103,17 @@ public class QualifierFilter extends CompareFilter {
|
|||
}
|
||||
|
||||
/**
|
||||
* @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 QualifierFilter)) return false;
|
||||
|
||||
if (o == this) {
|
||||
return true;
|
||||
}
|
||||
if (!(o instanceof QualifierFilter)) {
|
||||
return false;
|
||||
}
|
||||
return super.areSerializedFieldsEqual(o);
|
||||
}
|
||||
|
||||
|
|
|
@ -107,9 +107,11 @@ public class RandomRowFilter extends FilterBase {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse a serialized representation of {@link RandomRowFilter}
|
||||
* @param pbBytes A pb serialized {@link RandomRowFilter} instance
|
||||
* @return An instance of {@link RandomRowFilter} made from <code>bytes</code> n * @see
|
||||
* #toByteArray
|
||||
* @return An instance of {@link RandomRowFilter} made from <code>bytes</code>
|
||||
* @throws DeserializationException if an error occurred
|
||||
* @see #toByteArray
|
||||
*/
|
||||
public static RandomRowFilter parseFrom(final byte[] pbBytes) throws DeserializationException {
|
||||
FilterProtos.RandomRowFilter proto;
|
||||
|
@ -122,15 +124,17 @@ public class RandomRowFilter 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 RandomRowFilter)) return false;
|
||||
|
||||
if (o == this) {
|
||||
return true;
|
||||
}
|
||||
if (!(o instanceof RandomRowFilter)) {
|
||||
return false;
|
||||
}
|
||||
RandomRowFilter other = (RandomRowFilter) o;
|
||||
return this.getChance() == other.getChance();
|
||||
}
|
||||
|
|
|
@ -152,9 +152,11 @@ public class RegexStringComparator extends ByteArrayComparable {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse a serialized representation of {@link RegexStringComparator}
|
||||
* @param pbBytes A pb serialized {@link RegexStringComparator} instance
|
||||
* @return An instance of {@link RegexStringComparator} made from <code>bytes</code> n * @see
|
||||
* #toByteArray
|
||||
* @return An instance of {@link RegexStringComparator} made from <code>bytes</code>
|
||||
* @throws DeserializationException if an error occurred
|
||||
* @see #toByteArray
|
||||
*/
|
||||
public static RegexStringComparator parseFrom(final byte[] pbBytes)
|
||||
throws DeserializationException {
|
||||
|
@ -183,13 +185,17 @@ public class RegexStringComparator 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 RegexStringComparator)) return false;
|
||||
if (other == this) {
|
||||
return true;
|
||||
}
|
||||
if (!(other instanceof RegexStringComparator)) {
|
||||
return false;
|
||||
}
|
||||
RegexStringComparator comparator = (RegexStringComparator) other;
|
||||
return super.areSerializedFieldsEqual(comparator)
|
||||
&& engine.getClass().isInstance(comparator.getEngine())
|
||||
|
|
|
@ -96,8 +96,11 @@ public class RowFilter extends CompareFilter {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse a serialized representation of {@link RowFilter}
|
||||
* @param pbBytes A pb serialized {@link RowFilter} instance
|
||||
* @return An instance of {@link RowFilter} made from <code>bytes</code> n * @see #toByteArray
|
||||
* @return An instance of {@link RowFilter} made from <code>bytes</code>
|
||||
* @throws DeserializationException if an error occurred
|
||||
* @see #toByteArray
|
||||
*/
|
||||
public static RowFilter parseFrom(final byte[] pbBytes) throws DeserializationException {
|
||||
FilterProtos.RowFilter proto;
|
||||
|
@ -120,14 +123,17 @@ public class RowFilter extends CompareFilter {
|
|||
}
|
||||
|
||||
/**
|
||||
* @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 RowFilter)) return false;
|
||||
|
||||
if (o == this) {
|
||||
return true;
|
||||
}
|
||||
if (!(o instanceof RowFilter)) {
|
||||
return false;
|
||||
}
|
||||
return super.areSerializedFieldsEqual(o);
|
||||
}
|
||||
|
||||
|
|
|
@ -122,9 +122,11 @@ public class SingleColumnValueExcludeFilter extends SingleColumnValueFilter {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse a serialized representation of {@link SingleColumnValueExcludeFilter}
|
||||
* @param pbBytes A pb serialized {@link SingleColumnValueExcludeFilter} instance
|
||||
* @return An instance of {@link SingleColumnValueExcludeFilter} made from <code>bytes</code> n
|
||||
* * @see #toByteArray
|
||||
* @return An instance of {@link SingleColumnValueExcludeFilter} made from <code>bytes</code>
|
||||
* @throws DeserializationException if an error occurred
|
||||
* @see #toByteArray
|
||||
*/
|
||||
public static SingleColumnValueExcludeFilter parseFrom(final byte[] pbBytes)
|
||||
throws DeserializationException {
|
||||
|
@ -151,14 +153,17 @@ public class SingleColumnValueExcludeFilter extends SingleColumnValueFilter {
|
|||
}
|
||||
|
||||
/**
|
||||
* @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 SingleColumnValueExcludeFilter)) return false;
|
||||
|
||||
if (o == this) {
|
||||
return true;
|
||||
}
|
||||
if (!(o instanceof SingleColumnValueExcludeFilter)) {
|
||||
return false;
|
||||
}
|
||||
return super.areSerializedFieldsEqual(o);
|
||||
}
|
||||
|
||||
|
|
|
@ -284,8 +284,10 @@ public class SingleColumnValueFilter extends FilterBase {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse a serialized representation of {@link SingleColumnValueFilter}
|
||||
* @param pbBytes A pb serialized {@link SingleColumnValueFilter} instance
|
||||
* @return An instance of {@link SingleColumnValueFilter} made from <code>bytes</code>
|
||||
* @throws DeserializationException if an error occurred
|
||||
* @see #toByteArray
|
||||
*/
|
||||
public static SingleColumnValueFilter parseFrom(final byte[] pbBytes)
|
||||
|
@ -312,8 +314,8 @@ public class SingleColumnValueFilter extends FilterBase {
|
|||
}
|
||||
|
||||
/**
|
||||
* @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) {
|
||||
|
|
|
@ -105,8 +105,11 @@ public class SkipFilter extends FilterBase {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse a serialized representation of {@link SkipFilter}
|
||||
* @param pbBytes A pb serialized {@link SkipFilter} instance
|
||||
* @return An instance of {@link SkipFilter} made from <code>bytes</code> n * @see #toByteArray
|
||||
* @return An instance of {@link SkipFilter} made from <code>bytes</code>
|
||||
* @throws DeserializationException if an error occurred
|
||||
* @see #toByteArray
|
||||
*/
|
||||
public static SkipFilter parseFrom(final byte[] pbBytes) throws DeserializationException {
|
||||
FilterProtos.SkipFilter proto;
|
||||
|
@ -123,15 +126,17 @@ public class SkipFilter 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 SkipFilter)) return false;
|
||||
|
||||
if (o == this) {
|
||||
return true;
|
||||
}
|
||||
if (!(o instanceof SkipFilter)) {
|
||||
return false;
|
||||
}
|
||||
SkipFilter other = (SkipFilter) o;
|
||||
return getFilter().areSerializedFieldsEqual(other.getFilter());
|
||||
}
|
||||
|
|
|
@ -76,9 +76,11 @@ public class SubstringComparator extends ByteArrayComparable {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse a serialized representation of {@link SubstringComparator}
|
||||
* @param pbBytes A pb serialized {@link SubstringComparator} instance
|
||||
* @return An instance of {@link SubstringComparator} made from <code>bytes</code> n * @see
|
||||
* #toByteArray
|
||||
* @return An instance of {@link SubstringComparator} made from <code>bytes</code>
|
||||
* @throws DeserializationException if an error occurred
|
||||
* @see #toByteArray
|
||||
*/
|
||||
public static SubstringComparator parseFrom(final byte[] pbBytes)
|
||||
throws DeserializationException {
|
||||
|
@ -92,14 +94,17 @@ public class SubstringComparator 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 SubstringComparator)) return false;
|
||||
|
||||
if (other == this) {
|
||||
return true;
|
||||
}
|
||||
if (!(other instanceof SubstringComparator)) {
|
||||
return false;
|
||||
}
|
||||
SubstringComparator comparator = (SubstringComparator) other;
|
||||
return super.areSerializedFieldsEqual(comparator) && this.substr.equals(comparator.substr);
|
||||
}
|
||||
|
|
|
@ -167,8 +167,10 @@ public class TimestampsFilter extends FilterBase {
|
|||
}
|
||||
|
||||
/**
|
||||
* Parse a serialized representation of {@link TimestampsFilter}
|
||||
* @param pbBytes A pb serialized {@link TimestampsFilter} instance
|
||||
* @return An instance of {@link TimestampsFilter} made from <code>bytes</code>
|
||||
* @throws DeserializationException if an error occurred
|
||||
* @see #toByteArray
|
||||
*/
|
||||
public static TimestampsFilter parseFrom(final byte[] pbBytes) throws DeserializationException {
|
||||
|
@ -183,15 +185,17 @@ public class TimestampsFilter 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 TimestampsFilter)) return false;
|
||||
|
||||
if (o == this) {
|
||||
return true;
|
||||
}
|
||||
if (!(o instanceof TimestampsFilter)) {
|
||||
return false;
|
||||
}
|
||||
TimestampsFilter other = (TimestampsFilter) o;
|
||||
return this.getTimestamps().equals(other.getTimestamps());
|
||||
}
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue