Index: hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java £¨ÐÞ¶©°æ 1573563£© +++ hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java £¨¹¤×÷¿½±´£© @@ -1260,6 +1260,7 @@ private HFileBlockDefaultDecodingContext defaultDecodingCtx; + private PrefetchedHeader prefetchedHeaderGlobal; private ThreadLocal prefetchedHeaderForThread = new ThreadLocal() { @Override @@ -1278,6 +1279,7 @@ new HFileBlockDefaultDecodingContext(fileContext); encodedBlockDecodingCtx = new HFileBlockDefaultDecodingContext(fileContext); + prefetchedHeaderGlobal = new PrefetchedHeader(); } /** @@ -1408,9 +1410,24 @@ // read this block's header as part of the previous read's look-ahead. // And we also want to skip reading the header again if it has already // been read. - PrefetchedHeader prefetchedHeader = prefetchedHeaderForThread.get(); - ByteBuffer headerBuf = prefetchedHeader.offset == offset ? - prefetchedHeader.buf : null; + PrefetchedHeader prefetchedHeaderGlobalCopied = new PrefetchedHeader(); + if ( prefetchedHeaderGlobal.offset == offset ) { + synchronized (prefetchedHeaderGlobal) { + if (prefetchedHeaderGlobal.offset == offset) { + prefetchedHeaderGlobalCopied.offset = prefetchedHeaderGlobal.offset; + System.arraycopy(prefetchedHeaderGlobal.buf.array(), 0, + prefetchedHeaderGlobalCopied.buf.array(), 0, HConstants.HFILEBLOCK_HEADER_SIZE); + } + } + } + ByteBuffer headerBuf = prefetchedHeaderGlobalCopied.offset == offset ? + prefetchedHeaderGlobalCopied.buf : null; + PrefetchedHeader prefetchedHeaderThisThread = prefetchedHeaderGlobalCopied; + if (headerBuf == null) { + prefetchedHeaderThisThread = prefetchedHeaderForThread.get(); + headerBuf = prefetchedHeaderThisThread.offset == offset ? + prefetchedHeaderThisThread.buf : null; + } int nextBlockOnDiskSize = 0; // Allocate enough space to fit the next block's header too. @@ -1455,9 +1472,9 @@ + ", preReadHeaderSize=" + hdrSize + ", header.length=" - + prefetchedHeader.header.length + + prefetchedHeaderThisThread.header.length + ", header bytes: " - + Bytes.toStringBinary(prefetchedHeader.header, 0, + + Bytes.toStringBinary(prefetchedHeaderThisThread.header, 0, hdrSize), ex); } // if the caller specifies a onDiskSizeWithHeader, validate it. @@ -1543,9 +1560,14 @@ // Set prefetched header if (b.nextBlockOnDiskSizeWithHeader > 0) { - prefetchedHeader.offset = offset + b.getOnDiskSizeWithHeader(); + synchronized (prefetchedHeaderGlobal) { + prefetchedHeaderGlobal.offset = offset + b.getOnDiskSizeWithHeader(); + System.arraycopy(onDiskBlock, onDiskSizeWithHeader, + prefetchedHeaderGlobal.header, 0, hdrSize); + } + prefetchedHeaderForThread.get().offset = offset + b.getOnDiskSizeWithHeader(); System.arraycopy(onDiskBlock, onDiskSizeWithHeader, - prefetchedHeader.header, 0, hdrSize); + prefetchedHeaderForThread.get().header, 0, hdrSize); } b.offset = offset;