HBASE-HBASE-15203 Reduce garbage created by path.toString() during
Checksum verification (Ram)
This commit is contained in:
parent
360bb62469
commit
1404d5a973
|
@ -25,7 +25,6 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.fs.ChecksumException;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.util.ChecksumType;
|
||||
import org.apache.hadoop.util.DataChecksum;
|
||||
|
||||
|
@ -87,7 +86,7 @@ public class ChecksumUtil {
|
|||
* The header is extracted from the specified HFileBlock while the
|
||||
* data-to-be-verified is extracted from 'data'.
|
||||
*/
|
||||
static boolean validateBlockChecksum(Path path, HFileBlock block,
|
||||
static boolean validateBlockChecksum(String pathName, HFileBlock block,
|
||||
byte[] data, int hdrSize) throws IOException {
|
||||
|
||||
// If this is an older version of the block that does not have
|
||||
|
@ -120,14 +119,13 @@ public class ChecksumUtil {
|
|||
LOG.info("length of data = " + data.length
|
||||
+ " OnDiskDataSizeWithHeader = " + sizeWithHeader
|
||||
+ " checksum type = " + cktype.getName()
|
||||
+ " file =" + path.toString()
|
||||
+ " file =" + pathName
|
||||
+ " header size = " + hdrSize
|
||||
+ " bytesPerChecksum = " + bytesPerChecksum);
|
||||
}
|
||||
try {
|
||||
dataChecksum.verifyChunkedSums(ByteBuffer.wrap(data, 0, sizeWithHeader),
|
||||
ByteBuffer.wrap(data, sizeWithHeader, data.length - sizeWithHeader),
|
||||
path.toString(), 0);
|
||||
ByteBuffer.wrap(data, sizeWithHeader, data.length - sizeWithHeader), pathName, 0);
|
||||
} catch (ChecksumException e) {
|
||||
return false;
|
||||
}
|
||||
|
|
|
@ -1317,21 +1317,22 @@ public class HFileBlock implements Cacheable {
|
|||
/** The filesystem used to access data */
|
||||
protected HFileSystem hfs;
|
||||
|
||||
/** The path (if any) where this data is coming from */
|
||||
protected Path path;
|
||||
|
||||
private final Lock streamLock = new ReentrantLock();
|
||||
|
||||
/** The default buffer size for our buffered streams */
|
||||
public static final int DEFAULT_BUFFER_SIZE = 1 << 20;
|
||||
|
||||
protected HFileContext fileContext;
|
||||
// Cache the fileName
|
||||
protected String pathName;
|
||||
|
||||
public AbstractFSReader(long fileSize, HFileSystem hfs, Path path, HFileContext fileContext)
|
||||
throws IOException {
|
||||
this.fileSize = fileSize;
|
||||
this.hfs = hfs;
|
||||
this.path = path;
|
||||
if (path != null) {
|
||||
this.pathName = path.toString();
|
||||
}
|
||||
this.fileContext = fileContext;
|
||||
this.hdrSize = headerSize(fileContext.isUseHBaseChecksum());
|
||||
}
|
||||
|
@ -1509,13 +1510,13 @@ public class HFileBlock implements Cacheable {
|
|||
doVerificationThruHBaseChecksum);
|
||||
if (blk == null) {
|
||||
HFile.LOG.warn("HBase checksum verification failed for file " +
|
||||
path + " at offset " +
|
||||
pathName + " at offset " +
|
||||
offset + " filesize " + fileSize +
|
||||
". Retrying read with HDFS checksums turned on...");
|
||||
|
||||
if (!doVerificationThruHBaseChecksum) {
|
||||
String msg = "HBase checksum verification failed for file " +
|
||||
path + " at offset " +
|
||||
pathName + " at offset " +
|
||||
offset + " filesize " + fileSize +
|
||||
" but this cannot happen because doVerify is " +
|
||||
doVerificationThruHBaseChecksum;
|
||||
|
@ -1537,13 +1538,13 @@ public class HFileBlock implements Cacheable {
|
|||
doVerificationThruHBaseChecksum);
|
||||
if (blk != null) {
|
||||
HFile.LOG.warn("HDFS checksum verification suceeded for file " +
|
||||
path + " at offset " +
|
||||
pathName + " at offset " +
|
||||
offset + " filesize " + fileSize);
|
||||
}
|
||||
}
|
||||
if (blk == null && !doVerificationThruHBaseChecksum) {
|
||||
String msg = "readBlockData failed, possibly due to " +
|
||||
"checksum verification failed for file " + path +
|
||||
"checksum verification failed for file " + pathName +
|
||||
" at offset " + offset + " filesize " + fileSize;
|
||||
HFile.LOG.warn(msg);
|
||||
throw new IOException(msg);
|
||||
|
@ -1745,7 +1746,7 @@ public class HFileBlock implements Cacheable {
|
|||
*/
|
||||
protected boolean validateBlockChecksum(HFileBlock block, byte[] data, int hdrSize)
|
||||
throws IOException {
|
||||
return ChecksumUtil.validateBlockChecksum(path, block, data, hdrSize);
|
||||
return ChecksumUtil.validateBlockChecksum(pathName, block, data, hdrSize);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1755,7 +1756,7 @@ public class HFileBlock implements Cacheable {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "hfs=" + hfs + ", path=" + path + ", fileContext=" + fileContext;
|
||||
return "hfs=" + hfs + ", path=" + pathName + ", fileContext=" + fileContext;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue