HBASE-11921 Minor fixups that come of testing branch-1 (Stack)
This commit is contained in:
parent
0259d27341
commit
f7bcf3df6b
|
@ -1204,7 +1204,7 @@ class AsyncProcess {
|
|||
.append("/").append(numTries).append(" ");
|
||||
|
||||
if (failureCount > 0 || error != null){
|
||||
sb.append("failed ").append(failureCount).append(" ops").append(", last exception: ").
|
||||
sb.append("failed=").append(failureCount).append("ops").append(", last exception: ").
|
||||
append(error == null ? "null" : error);
|
||||
} else {
|
||||
sb.append("succeeded");
|
||||
|
@ -1213,8 +1213,8 @@ class AsyncProcess {
|
|||
sb.append(" on ").append(sn).append(", tracking started ").append(startTime);
|
||||
|
||||
if (willRetry) {
|
||||
sb.append(", retrying after ").append(backOffTime).append(" ms").
|
||||
append(", replay ").append(replaySize).append(" ops");
|
||||
sb.append(", retrying after=").append(backOffTime).append("ms").
|
||||
append(", replay=").append(replaySize).append("ops");
|
||||
} else if (failureCount > 0) {
|
||||
if (stopped > 0) {
|
||||
sb.append("; not retrying ").append(stopped).append(" due to success from other replica");
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.Callable;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
|
|
|
@ -53,7 +53,6 @@ import org.apache.hadoop.hbase.util.Pair;
|
|||
* in which case, the other replicas are queried (as in (3) above).
|
||||
*
|
||||
*/
|
||||
|
||||
@InterfaceAudience.Private
|
||||
class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
|
||||
private final Log LOG = LogFactory.getLog(this.getClass());
|
||||
|
@ -115,8 +114,8 @@ class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
|
|||
if (currentScannerCallable != null && currentScannerCallable.closed) {
|
||||
// For closing we target that exact scanner (and not do replica fallback like in
|
||||
// the case of normal reads)
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Closing scanner " + currentScannerCallable.scannerId);
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Closing scanner id=" + currentScannerCallable.scannerId);
|
||||
}
|
||||
Result[] r = currentScannerCallable.call(timeout);
|
||||
currentScannerCallable = null;
|
||||
|
@ -211,15 +210,17 @@ class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
|
|||
// store where to start the replica scanner from if we need to.
|
||||
if (result != null && result.length != 0) this.lastResult = result[result.length - 1];
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Setting current scanner as " + currentScannerCallable.scannerId +
|
||||
" associated with " + currentScannerCallable.getHRegionInfo().getReplicaId());
|
||||
LOG.trace("Setting current scanner as id=" + currentScannerCallable.scannerId +
|
||||
" associated with replica=" + currentScannerCallable.getHRegionInfo().getReplicaId());
|
||||
}
|
||||
// close all outstanding replica scanners but the one we heard back from
|
||||
outstandingCallables.remove(scanner);
|
||||
for (ScannerCallable s : outstandingCallables) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Closing scanner " + s.scannerId +
|
||||
" because this was slow and another replica succeeded");
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Closing scanner id=" + s.scannerId +
|
||||
", replica=" + s.getHRegionInfo().getRegionId() +
|
||||
" because slow and replica=" +
|
||||
this.currentScannerCallable.getHRegionInfo().getReplicaId() + " succeeded");
|
||||
}
|
||||
// Submit the "close" to the pool since this might take time, and we don't
|
||||
// want to wait for the "close" to happen yet. The "wait" will happen when
|
||||
|
|
|
@ -65,7 +65,7 @@ public enum ChecksumType {
|
|||
// check if hadoop library is available
|
||||
try {
|
||||
ctor = ChecksumFactory.newConstructor(PURECRC32);
|
||||
LOG.debug("Checksum using " + PURECRC32);
|
||||
LOG.debug(PURECRC32 + " available");
|
||||
} catch (Exception e) {
|
||||
LOG.trace(PURECRC32 + " not available.");
|
||||
}
|
||||
|
@ -74,7 +74,7 @@ public enum ChecksumType {
|
|||
// This is available on all JVMs.
|
||||
if (ctor == null) {
|
||||
ctor = ChecksumFactory.newConstructor(JDKCRC);
|
||||
LOG.debug("Checksum can use " + JDKCRC);
|
||||
LOG.debug(JDKCRC + " available");
|
||||
}
|
||||
} catch (Exception e) {
|
||||
LOG.trace(JDKCRC + " not available.");
|
||||
|
@ -108,7 +108,7 @@ public enum ChecksumType {
|
|||
LOG = LogFactory.getLog(ChecksumType.class);
|
||||
try {
|
||||
ctor = ChecksumFactory.newConstructor(PURECRC32C);
|
||||
LOG.debug("Checksum can use " + PURECRC32C);
|
||||
LOG.debug(PURECRC32C + " available");
|
||||
} catch (Exception e) {
|
||||
LOG.trace(PURECRC32C + " not available.");
|
||||
}
|
||||
|
|
|
@ -180,6 +180,7 @@ public class HFile {
|
|||
* The number of bytes per checksum.
|
||||
*/
|
||||
public static final int DEFAULT_BYTES_PER_CHECKSUM = 16 * 1024;
|
||||
// TODO: This define is done in three places. Fix.
|
||||
public static final ChecksumType DEFAULT_CHECKSUM_TYPE = ChecksumType.CRC32;
|
||||
|
||||
// For measuring number of checksum failures
|
||||
|
|
|
@ -870,7 +870,7 @@ public class HRegionServer extends HasThread implements
|
|||
}
|
||||
|
||||
// so callers waiting for meta without timeout can stop
|
||||
metaTableLocator.stop();
|
||||
if (this.metaTableLocator != null) this.metaTableLocator.stop();
|
||||
if (this.shortCircuitConnection != null && !shortCircuitConnection.isClosed()) {
|
||||
try {
|
||||
this.shortCircuitConnection.close();
|
||||
|
|
|
@ -56,7 +56,6 @@ import org.apache.hadoop.hbase.util.BloomFilter;
|
|||
import org.apache.hadoop.hbase.util.BloomFilterFactory;
|
||||
import org.apache.hadoop.hbase.util.BloomFilterWriter;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.ChecksumType;
|
||||
import org.apache.hadoop.hbase.util.Writables;
|
||||
import org.apache.hadoop.io.WritableUtils;
|
||||
|
||||
|
@ -704,10 +703,6 @@ public class StoreFile {
|
|||
private KeyValue lastDeleteFamilyKV = null;
|
||||
private long deleteFamilyCnt = 0;
|
||||
|
||||
|
||||
/** Checksum type */
|
||||
protected ChecksumType checksumType;
|
||||
|
||||
/** Bytes per Checksum */
|
||||
protected int bytesPerChecksum;
|
||||
|
||||
|
|
|
@ -93,7 +93,6 @@ public class TestChecksum {
|
|||
.withCompression(algo)
|
||||
.withIncludesMvcc(true)
|
||||
.withIncludesTags(useTags)
|
||||
.withChecksumType(HFile.DEFAULT_CHECKSUM_TYPE)
|
||||
.withBytesPerCheckSum(HFile.DEFAULT_BYTES_PER_CHECKSUM)
|
||||
.build();
|
||||
HFileBlock.Writer hbw = new HFileBlock.Writer(null, meta);
|
||||
|
@ -202,7 +201,6 @@ public class TestChecksum {
|
|||
.withIncludesTags(useTags)
|
||||
.withHBaseCheckSum(true)
|
||||
.withBytesPerCheckSum(bytesPerChecksum)
|
||||
.withChecksumType(HFile.DEFAULT_CHECKSUM_TYPE)
|
||||
.build();
|
||||
HFileBlock.Writer hbw = new HFileBlock.Writer(null,
|
||||
meta);
|
||||
|
|
|
@ -205,7 +205,6 @@ public class TestHFileBlock {
|
|||
.withIncludesMvcc(includesMemstoreTS)
|
||||
.withIncludesTags(includesTag)
|
||||
.withBytesPerCheckSum(HFile.DEFAULT_BYTES_PER_CHECKSUM)
|
||||
.withChecksumType(HFile.DEFAULT_CHECKSUM_TYPE)
|
||||
.build();
|
||||
HFileBlock.Writer hbw = new HFileBlock.Writer(null, meta);
|
||||
DataOutputStream dos = hbw.startWriting(blockType);
|
||||
|
@ -289,7 +288,6 @@ public class TestHFileBlock {
|
|||
.withIncludesMvcc(includesMemstoreTS)
|
||||
.withIncludesTags(includesTag)
|
||||
.withBytesPerCheckSum(HFile.DEFAULT_BYTES_PER_CHECKSUM)
|
||||
.withChecksumType(HFile.DEFAULT_CHECKSUM_TYPE)
|
||||
.build();
|
||||
HFileBlock.Writer hbw = new HFileBlock.Writer(null,
|
||||
meta);
|
||||
|
@ -372,7 +370,6 @@ public class TestHFileBlock {
|
|||
.withIncludesMvcc(includesMemstoreTS)
|
||||
.withIncludesTags(includesTag)
|
||||
.withBytesPerCheckSum(HFile.DEFAULT_BYTES_PER_CHECKSUM)
|
||||
.withChecksumType(HFile.DEFAULT_CHECKSUM_TYPE)
|
||||
.build();
|
||||
HFileBlock.Writer hbw = new HFileBlock.Writer(dataBlockEncoder, meta);
|
||||
long totalSize = 0;
|
||||
|
@ -722,7 +719,6 @@ public class TestHFileBlock {
|
|||
.withIncludesTags(includesTag)
|
||||
.withCompression(compressAlgo)
|
||||
.withBytesPerCheckSum(HFile.DEFAULT_BYTES_PER_CHECKSUM)
|
||||
.withChecksumType(HFile.DEFAULT_CHECKSUM_TYPE)
|
||||
.build();
|
||||
HFileBlock.Writer hbw = new HFileBlock.Writer(null, meta);
|
||||
Map<BlockType, Long> prevOffsetByType = new HashMap<BlockType, Long>();
|
||||
|
|
|
@ -251,7 +251,6 @@ public class TestHFileBlockIndex {
|
|||
.withIncludesMvcc(includesMemstoreTS)
|
||||
.withIncludesTags(useTags)
|
||||
.withCompression(compr)
|
||||
.withChecksumType(HFile.DEFAULT_CHECKSUM_TYPE)
|
||||
.withBytesPerCheckSum(HFile.DEFAULT_BYTES_PER_CHECKSUM)
|
||||
.build();
|
||||
HFileBlock.Writer hbw = new HFileBlock.Writer(null,
|
||||
|
|
Loading…
Reference in New Issue