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 @@ default ResultScanner getScanner(byte[] family, byte[] qualifier) {
*
*
*
- *
- * 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()));
* }
* });
- *
*
*
* If your result set is very large, you should use other scan method to get a scanner or use
@@ -574,9 +577,7 @@ default CompletableFuture> batchAll(List extends Row> actions) {
* one line lambda expression, like:
*
*
- *
- * channel -> xxxService.newStub(channel)
- *
+ * channel -> xxxService.newStub(channel)
*
*
* @param stubMaker a delegation to the actual {@code newStub} call.
@@ -609,7 +610,7 @@ CompletableFuture coprocessorService(Function stubMaker
*
*
* locateThenCall(byte[] row) {
- * locate(row).whenComplete((location, locateError) -> {
+ * locate(row).whenComplete((location, locateError) -> {
* if (locateError != null) {
* callback.onError(locateError);
* return;
@@ -621,7 +622,7 @@ CompletableFuture coprocessorService(Function stubMaker
* } else {
* locateThenCall(region.getEndKey());
* }
- * sendCall().whenComplete((resp, error) -> {
+ * sendCall().whenComplete((resp, error) -> {
* if (error != null) {
* callback.onRegionError(region, error);
* } else {
@@ -639,12 +640,14 @@ CompletableFuture coprocessorService(Function stubMaker
interface CoprocessorCallback {
/**
+ * 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 @@ interface CoprocessorCallback {
interface CoprocessorServiceBuilder {
/**
+ * Specify a start row
* @param startKey start region selection with region containing this row, inclusive.
*/
default CoprocessorServiceBuilder fromRow(byte[] startKey) {
@@ -682,12 +686,14 @@ default CoprocessorServiceBuilder fromRow(byte[] startKey) {
}
/**
+ * Specify a start row
* @param startKey start region selection with region containing this row
* @param inclusive whether to include the startKey
*/
CoprocessorServiceBuilder 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 toRow(byte[] endKey) {
@@ -695,6 +701,7 @@ default CoprocessorServiceBuilder toRow(byte[] endKey) {
}
/**
+ * 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 @@ default CoprocessorServiceBuilder toRow(byte[] endKey) {
* is only a one line lambda expression, like:
*
*
- *
- * channel -> xxxService.newStub(channel)
- *
+ * channel -> xxxService.newStub(channel)
*
*
* @param stubMaker a delegation to the actual {@code newStub} call.
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CatalogReplicaLoadBalanceSimpleSelector.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CatalogReplicaLoadBalanceSimpleSelector.java
index 2e704c5bdc10..e990fa6d65ff 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CatalogReplicaLoadBalanceSimpleSelector.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CatalogReplicaLoadBalanceSimpleSelector.java
@@ -128,6 +128,7 @@ public String toString() {
* Selector's internal state.
* @param loc the location which causes exception.
*/
+ @Override
public void onError(HRegionLocation loc) {
ConcurrentNavigableMap tableCache = computeIfAbsent(staleCache,
loc.getRegion().getTable(), () -> new ConcurrentSkipListMap<>(BYTES_COMPARATOR));
@@ -159,18 +160,19 @@ private int getRandomReplicaId() {
* 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 tableCache = staleCache.get(tablename);
+ ConcurrentNavigableMap tableCache = staleCache.get(tableName);
// If there is no entry in StaleCache, select a random replica id.
if (tableCache == null) {
@@ -199,7 +201,7 @@ public int select(final TableName tablename, final byte[] row,
(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();
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CheckAndMutate.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CheckAndMutate.java
index 56402fb42f88..b31a0b27242f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CheckAndMutate.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CheckAndMutate.java
@@ -97,6 +97,7 @@ public Builder ifEquals(byte[] family, byte[] qualifier, byte[] value) {
}
/**
+ * Check for match
* @param family family to check
* @param qualifier qualifier to check
* @param compareOp comparison operator to use
@@ -113,6 +114,7 @@ public Builder ifMatches(byte[] family, byte[] qualifier, CompareOperator compar
}
/**
+ * Check for match
* @param filter filter to check
* @return the CheckAndMutate object
*/
@@ -122,6 +124,7 @@ public Builder ifMatches(Filter filter) {
}
/**
+ * Specify a timerange
* @param timeRange time range to check
* @return the CheckAndMutate object
*/
@@ -144,6 +147,7 @@ private void preCheck(Row action) {
}
/**
+ * Build the CheckAndMutate object
* @param put data to put if check succeeds
* @return a CheckAndMutate object
*/
@@ -157,6 +161,7 @@ public CheckAndMutate build(Put put) {
}
/**
+ * Build the CheckAndMutate object
* @param delete data to delete if check succeeds
* @return a CheckAndMutate object
*/
@@ -170,6 +175,7 @@ public CheckAndMutate build(Delete delete) {
}
/**
+ * 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 CheckAndMutate build(Increment increment) {
}
/**
+ * 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 CheckAndMutate build(Append append) {
}
/**
+ * 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
*/
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientIdGenerator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientIdGenerator.java
index f5bedbe88d2e..ee497ef56c5f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientIdGenerator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientIdGenerator.java
@@ -19,6 +19,7 @@
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.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 @@ private 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 @@ public static byte[] generateClientId() {
/** 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 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 @@ public static Long getPid() {
}
/**
- * @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 {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptor.java
index b3d8e4fd6009..d73e67ee0fbf 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptor.java
@@ -98,10 +98,7 @@ public interface ColumnFamilyDescriptor {
/** Returns an unmodifiable map. */
Map 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 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();
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java
index 80178027b6f2..cf4de2edfecf 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java
@@ -42,9 +42,6 @@
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 static Unit getUnit(String key) {
}
/**
+ * Check if the column family name is legal.
* @param b Family name.
* @return b
* @throws IllegalArgumentException If not null and not a legitimate family name: i.e. 'printable'
@@ -398,6 +396,7 @@ private ColumnFamilyDescriptorBuilder(final ColumnFamilyDescriptor desc) {
}
/**
+ * 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 Map getValues() {
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 @@ private ModifyableColumnFamilyDescriptor setValue(Bytes key, String value) {
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 int getMaxVersions() {
}
/**
+ * Set the maximum number of versions to retain.
* @param maxVersions maximum number of versions
* @return this (for chained invocation)
*/
@@ -768,7 +758,7 @@ public ModifyableColumnFamilyDescriptor setMaxVersions(int maxVersions) {
}
/**
- * 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 int getBlocksize() {
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 boolean isInMemory() {
}
/**
+ * 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 MemoryCompactionPolicy getInMemoryCompaction() {
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 KeepDeletedCells getKeepDeletedCells() {
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 int getTimeToLive() {
}
/**
+ * 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 ModifyableColumnFamilyDescriptor setTimeToLive(int timeToLive) {
}
/**
+ * 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 int getMinVersions() {
}
/**
+ * 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 boolean isBlockCacheEnabled() {
}
/**
+ * 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 int getScope() {
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 boolean isCacheDataOnWrite() {
}
/**
+ * 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 boolean isCacheIndexesOnWrite() {
}
/**
+ * 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 boolean isCacheBloomsOnWrite() {
}
/**
+ * 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 boolean isEvictBlocksOnClose() {
}
/**
+ * 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 boolean isPrefetchBlocksOnOpen() {
}
/**
+ * 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 int compareTo(ModifyableColumnFamilyDescriptor other) {
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 bytes
- * n * @see #toByteArray()
+ * @see #toByteArray()
*/
private static ColumnFamilyDescriptor parseFrom(final byte[] bytes)
throws DeserializationException {
@@ -1290,8 +1272,8 @@ public ModifyableColumnFamilyDescriptor setConfiguration(String key, String valu
}
/**
- * 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 String getEncryptionType() {
}
/**
- * 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 byte[] getEncryptionKey() {
}
/**
- * 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));
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CompactType.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CompactType.java
index 225bb072db70..412fd76d9ecf 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CompactType.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CompactType.java
@@ -26,9 +26,7 @@
@InterfaceAudience.Public
public enum CompactType {
- NORMAL(0),
- MOB(1);
+ NORMAL,
+ MOB
- CompactType(int value) {
- }
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
index 755c0ca0b8c0..545e8c38c4b0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
@@ -111,8 +111,6 @@ public Delete(final byte[] row, final int rowOffset, final int rowLength) {
*
* 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 Delete(final byte[] row, final int rowOffset, final int rowLength, long t
}
/**
+ * Create a Delete operation using another Delete as template.
* @param deleteToCopy delete to copy
*/
public Delete(final Delete deleteToCopy) {
@@ -144,6 +143,7 @@ public Delete(byte[] row, long ts, NavigableMap> familyMap) {
* @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;
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java
index 17975ff631de..f4e06101255d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java
@@ -445,7 +445,7 @@ public boolean equals(Object obj) {
if (this == obj) {
return true;
}
- if (obj == null || getClass() != obj.getClass()) {
+ if (!(obj instanceof Row)) {
return false;
}
Row other = (Row) obj;
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java
index dea15cac8d50..e4b177e3bca8 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java
@@ -96,6 +96,7 @@ public Increment(byte[] row, long ts, NavigableMap> familyMap
* 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;
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/IsolationLevel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/IsolationLevel.java
index 7804e48de9f0..3f94fc30f9c7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/IsolationLevel.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/IsolationLevel.java
@@ -29,11 +29,8 @@
@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() };
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/LogQueryFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/LogQueryFilter.java
index b2d217da3de0..61137076a128 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/LogQueryFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/LogQueryFilter.java
@@ -112,13 +112,10 @@ public boolean equals(Object o) {
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)
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCoprocessorRpcChannelImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCoprocessorRpcChannelImpl.java
index 4be0362be85d..f36b3e162187 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCoprocessorRpcChannelImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCoprocessorRpcChannelImpl.java
@@ -78,11 +78,11 @@ public void callMethod(MethodDescriptor method, RpcController controller, Messag
Message responsePrototype, RpcCallback 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);
- }));
+ });
}
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
index a031d3530971..28ac5bb45282 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
@@ -34,6 +34,7 @@
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 parseMasterAddrs(Configuration conf) throws UnknownHostException {
Set 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));
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiResponse.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiResponse.java
index b768b76fdb65..341341b4336e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiResponse.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiResponse.java
@@ -53,11 +53,7 @@ public int size() {
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);
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MutableRegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MutableRegionInfo.java
index 7627f8dc0e5a..fbb76ea4f65d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MutableRegionInfo.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MutableRegionInfo.java
@@ -230,6 +230,7 @@ public boolean isSplit() {
}
/**
+ * Change the split status flag.
* @param split set split status n
*/
public MutableRegionInfo setSplit(boolean split) {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
index 3f82e4110eb2..df9e92f74dcd 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
@@ -156,7 +156,7 @@ List getCellList(byte[] family) {
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 @@ KeyValue createPutKeyValue(byte[] family, byte[] qualifier, long ts, byte[] valu
}
/**
- * 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 @@ KeyValue createPutKeyValue(byte[] family, byte[] qualifier, long ts, byte[] valu
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 @@ KeyValue createPutKeyValue(byte[] family, ByteBuffer qualifier, long ts, ByteBuf
/**
* 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 getFingerprint() {
@@ -261,7 +260,7 @@ private static Map cellToStringMap(Cell c) {
if (tags != null) {
List 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 byte[] getACL() {
}
/**
+ * Set the ACL for this operation.
* @param user User short name
* @param perms Permissions for the user
*/
@@ -455,6 +455,7 @@ public Mutation setACL(String user, Permission perms) {
}
/**
+ * Set the ACL for this operation.
* @param perms A map of permissions for a user or users
*/
public Mutation setACL(Map perms) {
@@ -596,11 +597,9 @@ public List get(byte[] family, byte[] qualifier) {
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) {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java
index bcc1bda9ef02..5e821f075461 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java
@@ -203,6 +203,7 @@ public Put addColumn(byte[] family, ByteBuffer qualifier, long ts, ByteBuffer va
* 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;
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Query.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Query.java
index 750993c1a8a2..cf892ae0d74b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Query.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Query.java
@@ -88,6 +88,7 @@ public byte[] getACL() {
}
/**
+ * Set the ACL for the operation.
* @param user User short name
* @param perms Permissions for the user
*/
@@ -98,6 +99,7 @@ public Query setACL(String user, Permission perms) {
}
/**
+ * Set the ACL for the operation.
* @param perms A map of permissions for a user or users
*/
public Query setACL(Map perms) {
@@ -159,9 +161,8 @@ public Query setIsolationLevel(IsolationLevel level) {
}
/**
- * @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);
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
index 3ec2c741293e..4d614907326e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
@@ -502,11 +502,6 @@ private CompletableFuture procedureCall(
return future;
}
- @FunctionalInterface
- private interface TableOperator {
- CompletableFuture operate(TableName table);
- }
-
@Override
public CompletableFuture tableExists(TableName tableName) {
if (TableName.isMetaTableName(tableName)) {
@@ -1559,11 +1554,13 @@ public CompletableFuture assign(byte[] regionName) {
future.completeExceptionally(err);
return;
}
- addListener(this. newMasterCaller().priority(regionInfo.getTable())
- .action(((controller, stub) -> this. call(
- controller, stub, RequestConverter.buildAssignRegionRequest(regionInfo.getRegionName()),
- (s, c, req, done) -> s.assignRegion(c, req, done), resp -> null)))
- .call(), (ret, err2) -> {
+ addListener(
+ this. newMasterCaller().priority(regionInfo.getTable())
+ .action((controller, stub) -> this. 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 @@ public CompletableFuture unassign(byte[] regionName) {
}
addListener(
this. newMasterCaller().priority(regionInfo.getTable())
- .action(((controller, stub) -> this. this. 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 @@ public CompletableFuture offline(byte[] regionName) {
future.completeExceptionally(err);
return;
}
- addListener(
- this. newMasterCaller().priority(regionInfo.getTable())
- .action(((controller, stub) -> this. call(controller, stub,
- RequestConverter.buildOfflineRegionRequest(regionInfo.getRegionName()),
- (s, c, req, done) -> s.offlineRegion(c, req, done), resp -> null)))
- .call(),
- (ret, err2) -> {
+ addListener(this. newMasterCaller().priority(regionInfo.getTable())
+ .action((controller, stub) -> this. 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 @@ private CompletableFuture internalDeleteSnapshots(Pattern tableNamePattern
listSnapshotsFuture = getCompletedSnapshots(tableNamePattern, snapshotNamePattern);
}
CompletableFuture future = new CompletableFuture<>();
- addListener(listSnapshotsFuture, ((snapshotDescriptions, err) -> {
+ addListener(listSnapshotsFuture, (snapshotDescriptions, err) -> {
if (err != null) {
future.completeExceptionally(err);
return;
@@ -2253,7 +2247,7 @@ private CompletableFuture internalDeleteSnapshots(Pattern tableNamePattern
future.complete(v);
}
});
- }));
+ });
return future;
}
@@ -4010,10 +4004,9 @@ public CompletableFuture moveServersToRSGroup(Set servers, String
@Override
public CompletableFuture addRSGroup(String groupName) {
return this. newMasterCaller()
- .action(
- ((controller, stub) -> this. call(controller,
- stub, AddRSGroupRequest.newBuilder().setRSGroupName(groupName).build(),
- (s, c, req, done) -> s.addRSGroup(c, req, done), resp -> null)))
+ .action((controller, stub) -> this. call(
+ controller, stub, AddRSGroupRequest.newBuilder().setRSGroupName(groupName).build(),
+ (s, c, req, done) -> s.addRSGroup(c, req, done), resp -> null))
.call();
}
@@ -4081,9 +4074,9 @@ private CompletableFuture> getSlowLogResponseFromServer(ServerNam
private CompletableFuture clearSlowLogsResponses(final ServerName serverName) {
return this. 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 @@ Pair, List>> call(controller, stub,
@Override
public CompletableFuture getRSGroup(Address hostPort) {
return this. newMasterCaller()
- .action(
- ((controller, stub) -> this. call(controller, stub, GetRSGroupInfoOfServerRequest.newBuilder()
+ .action((controller, stub) -> this. 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 @@ public CompletableFuture setRSGroup(Set tables, String groupNam
}
});
}
- 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 @@ public CompletableFuture setRSGroup(Set tables, String groupNam
future.complete(v);
}
});
- }));
+ });
return future;
}
@Override
public CompletableFuture getRSGroup(TableName table) {
return this. newMasterCaller()
- .action(((controller, stub) -> this. this. 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 getRSGroup(String groupName) {
return this. newMasterCaller()
- .action(((controller, stub) -> this. this. 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 renameRSGroup(String oldName, String newName) {
return this. newMasterCaller()
- .action(((controller, stub) -> this. call(
+ .action((controller, stub) -> this. 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 @@ public CompletableFuture updateRSGroupConfig(String groupName,
NameStringPair.newBuilder().setName(e.getKey()).setValue(e.getValue()).build()));
}
return this. newMasterCaller()
- .action(((controller, stub) -> this. this. 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 @@ public CompletableFuture> getLogEntries(Set serverNam
public CompletableFuture flushMasterStore() {
FlushMasterStoreRequest.Builder request = FlushMasterStoreRequest.newBuilder();
return this. newMasterCaller()
- .action(((controller, stub) -> this. this. 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();
}
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
index af0b20908031..ff75c0725ce5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
@@ -776,7 +776,7 @@ private boolean locateFinished(RegionInfo region, byte[] endKey, boolean endKeyI
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);
}
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorServiceExec.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorServiceExec.java
index 4bf726079463..de3bc9a90d4a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorServiceExec.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorServiceExec.java
@@ -92,7 +92,7 @@ public boolean equals(Object obj) {
if (this == obj) {
return true;
}
- if (obj == null || getClass() != obj.getClass()) {
+ if (!(obj instanceof RegionCoprocessorServiceExec)) {
return false;
}
return compareTo((RegionCoprocessorServiceExec) obj) == 0;
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
index 32c7da05887c..d1384d71469a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
@@ -160,7 +160,7 @@ public interface RegionInfo extends Comparable {
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 {
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 HBASE-25210
*/
@@ -198,7 +198,7 @@ public interface RegionInfo extends Comparable {
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 HBASE-25210
*/
@@ -209,9 +209,9 @@ public interface RegionInfo extends Comparable {
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 @@ static String getRegionNameAsString(@CheckForNull RegionInfo ri, byte[] regionNa
}
/**
- * @return Return a String of short, printable names for hris (usually encoded name)
- * for us logging.
+ * Returns a String of short, printable names for hris (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 hris (usually encoded name)
- * for us logging.
+ * Returns a String of short, printable names for hris (usually encoded name) for us
+ * logging.
*/
static String getShortNameToLog(final List ris) {
return ris.stream().map(RegionInfo::getEncodedName).collect(Collectors.toList()).toString();
@@ -369,8 +369,7 @@ static RegionInfo parseFromOrNull(final byte[] bytes, int offset, int len) {
}
/**
- * @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 @@ static RegionInfo parseFrom(final byte[] bytes) throws DeserializationException
}
/**
+ * 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 @@ static boolean areAdjacent(RegionInfo regionA, RegionInfo regionB) {
}
/**
- * @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 @@ default boolean isLast() {
}
/**
- * @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 @@ default boolean isNext(RegionInfo after) {
}
/**
- * @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 @@ default boolean isDegenerate() {
}
/**
- * @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 @@ default boolean isOverlap(RegionInfo other) {
return Bytes.compareTo(getStartKey(), other.getEndKey()) < 0;
}
+ @Override
default int compareTo(RegionInfo other) {
return RegionInfo.COMPARATOR.compare(this, other);
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionReplicaUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionReplicaUtil.java
index 4475a01c0f91..1d6708b49d14 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionReplicaUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionReplicaUtil.java
@@ -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 static List addReplicas(final List regions, int o
if ((newReplicaCount - 1) <= 0) {
return regions;
}
- List hRegionInfos = new ArrayList<>((newReplicaCount) * regions.size());
+ List hRegionInfos = new ArrayList<>(newReplicaCount * regions.size());
for (RegionInfo ri : regions) {
if (
RegionReplicaUtil.isDefaultReplica(ri)
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCoprocessorRpcChannelImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCoprocessorRpcChannelImpl.java
index 29b092cad883..ed5fd16dab3f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCoprocessorRpcChannelImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCoprocessorRpcChannelImpl.java
@@ -78,11 +78,11 @@ public void callMethod(MethodDescriptor method, RpcController controller, Messag
Message responsePrototype, RpcCallback 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);
- }));
+ });
}
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
index 1717a9f6964c..a1ab6075fe7b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
@@ -894,8 +894,8 @@ public void setStatistics(RegionLoadStats loadStats) {
}
/**
- * @return the associated statistics about the region from which this was returned. Can be
- * null if stats are disabled.
+ * Returns the associated statistics about the region from which this was returned. Can be
+ * null if stats are disabled.
*/
public RegionLoadStats getStats() {
return stats;
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowAccess.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowAccess.java
index cd1f47560798..1af8798fd87c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowAccess.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowAccess.java
@@ -22,7 +22,6 @@
/**
* 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
*/
@InterfaceAudience.Private
public interface RowAccess extends Iterable {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowMutations.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowMutations.java
index 0cf7f69e9b48..be44c26190b1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowMutations.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowMutations.java
@@ -70,6 +70,7 @@ public RowMutations(byte[] row, int initialCapacity) {
}
/**
+ * 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 RowMutations add(Mutation mutation) throws IOException {
}
/**
+ * 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
*/
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java
index 7e557fcfdb74..bdca990ca549 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java
@@ -710,9 +710,9 @@ public Scan setAllowPartialResults(final boolean allowPartialResults) {
}
/**
- * @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;
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScanResultConsumer.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScanResultConsumer.java
index b574b2c2bd5e..44f8e1940459 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScanResultConsumer.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScanResultConsumer.java
@@ -30,8 +30,7 @@
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);
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ServiceCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ServiceCaller.java
index 501f412bc575..fb4acbdf6574 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ServiceCaller.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ServiceCaller.java
@@ -28,20 +28,16 @@
* Usually, it is just a simple lambda expression, like:
*
*
- *
- * (stub, controller, rpcCallback) -> {
+ * (stub, controller, rpcCallback) -> {
* XXXRequest request = ...; // prepare the request
* stub.xxx(controller, request, rpcCallback);
* }
- *
*
*
* And if already have the {@code request}, the lambda expression will be:
*
*
- *
- * (stub, controller, rpcCallback) -> stub.xxx(controller, request, rpcCallback)
- *
+ * (stub, controller, rpcCallback) -> stub.xxx(controller, request, rpcCallback)
*
*
* @param the type of the protobuf Service you want to call.
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SlowLogParams.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SlowLogParams.java
index 3311539c2614..b1460c0b116c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SlowLogParams.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SlowLogParams.java
@@ -62,13 +62,10 @@ public boolean equals(Object o) {
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();
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java
index 0f93ab21a2c1..53c33a667c3c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java
@@ -30,7 +30,6 @@
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 @@ default CheckAndMutateBuilder checkAndMutate(byte[] row, byte[] family) {
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 @@ default CheckAndMutateBuilder ifEquals(byte[] value) {
}
/**
+ * 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 @@ default CheckAndMutateWithFilterBuilder checkAndMutate(byte[] row, Filter filter
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 @@ default Map batchCoprocessorService(
final Map results =
Collections.synchronizedMap(new TreeMap(Bytes.BYTES_COMPARATOR));
batchCoprocessorService(methodDescriptor, request, startKey, endKey, responsePrototype,
- new Callback() {
+ new Batch.Callback() {
@Override
public void update(byte[] region, byte[] row, R result) {
if (region != null) {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java
index f960f1c748cb..f500a1128a51 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java
@@ -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();
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
index 3fa39ee8e855..d0d3e36aa8ff 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
@@ -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 static PrettyPrinter.Unit getUnit(String key) {
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 static byte[] toByteArray(TableDescriptor desc) {
* 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);
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableOverAsyncTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableOverAsyncTable.java
index d4db9eb49e67..e1565f18159a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableOverAsyncTable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableOverAsyncTable.java
@@ -47,8 +47,7 @@
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 @@ public void batch(List extends Row> actions, Object[] results) throws IOExcept
}
@Override
- public void batchCallback(List extends Row> actions, Object[] results, Callback callback)
- throws IOException, InterruptedException {
+ public void batchCallback(List extends Row> actions, Object[] results,
+ Batch.Callback callback) throws IOException, InterruptedException {
ConcurrentLinkedQueue errors = new ConcurrentLinkedQueue<>();
CountDownLatch latch = new CountDownLatch(actions.size());
AsyncTableRegionLocator locator = conn.getRegionLocator(getName());
@@ -467,7 +466,7 @@ private interface StubCall {
}
private void coprocessorService(String serviceName, byte[] startKey, byte[] endKey,
- Callback callback, StubCall call) throws Throwable {
+ Batch.Callback callback, StubCall call) throws Throwable {
// get regions covered by the row range
ExecutorService pool = Context.current().wrap(this.poolSupplier.get());
List keys = getStartKeysInRange(startKey, endKey);
@@ -509,7 +508,8 @@ private void coprocessorService(String serviceName, byte[] startKey, byte[]
@Override
public void coprocessorService(Class service, byte[] startKey,
- byte[] endKey, Call callable, Callback callback) throws ServiceException, Throwable {
+ byte[] endKey, Batch.Call callable, Batch.Callback callback)
+ throws ServiceException, Throwable {
final Supplier supplier = new TableOperationSpanBuilder(conn)
.setTableName(table.getName()).setOperation(HBaseSemanticAttributes.Operation.COPROC_EXEC);
TraceUtil.trace(() -> {
@@ -526,8 +526,8 @@ public void coprocessorService(Class service, byte[] s
@SuppressWarnings("unchecked")
@Override
public void batchCoprocessorService(MethodDescriptor methodDescriptor,
- Message request, byte[] startKey, byte[] endKey, R responsePrototype, Callback callback)
- throws ServiceException, Throwable {
+ Message request, byte[] startKey, byte[] endKey, R responsePrototype,
+ Batch.Callback callback) throws ServiceException, Throwable {
final Supplier supplier = new TableOperationSpanBuilder(conn)
.setTableName(table.getName()).setOperation(HBaseSemanticAttributes.Operation.COPROC_EXEC);
TraceUtil.trace(() -> {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableState.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableState.java
index 6ecd97a75c9d..4e20302be45e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableState.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableState.java
@@ -209,15 +209,19 @@ public boolean isInStates(State... target) {
@Override
public boolean equals(Object o) {
- if (this == o) return true;
- if (o == null || getClass() != o.getClass()) return false;
-
+ 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)
+ if (state != that.state) {
return false;
-
+ }
+ if (tableName != null ? !tableName.equals(that.tableName) : that.tableName != null) {
+ return false;
+ }
return true;
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/backoff/ClientBackoffPolicyFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/backoff/ClientBackoffPolicyFactory.java
index a786702b1693..6993a4ac21a9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/backoff/ClientBackoffPolicyFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/backoff/ClientBackoffPolicyFactory.java
@@ -22,15 +22,11 @@
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() {
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/backoff/ExponentialClientBackoffPolicy.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/backoff/ExponentialClientBackoffPolicy.java
index aa84207e1ed1..0e3339948eaa 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/backoff/ExponentialClientBackoffPolicy.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/backoff/ExponentialClientBackoffPolicy.java
@@ -21,8 +21,6 @@
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 @@
@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";
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/BigDecimalColumnInterpreter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/BigDecimalColumnInterpreter.java
index 900f96440dca..1e42be4baad8 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/BigDecimalColumnInterpreter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/BigDecimalColumnInterpreter.java
@@ -80,7 +80,7 @@ public BigDecimal getMaxValue() {
@Override
public BigDecimal increment(BigDecimal bd) {
- return bd == null ? null : (bd.add(BigDecimal.ONE));
+ return bd == null ? null : bd.add(BigDecimal.ONE);
}
@Override
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
index 4145a348b08c..b2ddc4eaec15 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
@@ -22,6 +22,7 @@
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 static ReplicationProtos.TableCF[] convert(String tableCFsConfig) {
}
ReplicationProtos.TableCF.Builder tableCFBuilder = ReplicationProtos.TableCF.newBuilder();
- String[] tables = tableCFsConfig.split(";");
- List tableCFList = new ArrayList<>(tables.length);
+ List tables = Splitter.on(';').splitToList(tableCFsConfig);
+ List tableCFList = new ArrayList<>(tables.size());
for (String tab : tables) {
// 1 ignore empty table config
@@ -124,9 +125,9 @@ public static ReplicationProtos.TableCF[] convert(String tableCFsConfig) {
}
// 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 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 static ReplicationProtos.TableCF[] convert(String tableCFsConfig) {
// 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 dbs = Splitter.on('.').splitToList(tabName);
+ if (dbs != null && dbs.size() == 2) {
+ Iterator ii = dbs.iterator();
+ ns = ii.next();
+ tName = ii.next();
}
tableCFBuilder.setTableName(ProtobufUtil.toProtoTableName(TableName.valueOf(ns, tName)));
// 3 parse "cf1,cf2" part to List | |