HBASE-7336 HFileBlock.readAtOffset does not work well with multiple threads

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1421440 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
larsh 2012-12-13 19:34:26 +00:00
parent 8c68353d63
commit e23ba9b3ce
1 changed files with 22 additions and 16 deletions

View File

@ -28,6 +28,8 @@ import java.io.DataOutputStream;
import java.io.IOException; import java.io.IOException;
import java.io.InputStream; import java.io.InputStream;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataInputStream;
@ -1143,6 +1145,8 @@ public class HFileBlock implements Cacheable {
/** The path (if any) where this data is coming from */ /** The path (if any) where this data is coming from */
protected Path path; protected Path path;
private final Lock streamLock = new ReentrantLock();
/** The default buffer size for our buffered streams */ /** The default buffer size for our buffered streams */
public static final int DEFAULT_BUFFER_SIZE = 1 << 20; public static final int DEFAULT_BUFFER_SIZE = 1 << 20;
@ -1217,23 +1221,9 @@ public class HFileBlock implements Cacheable {
"-byte array at offset " + destOffset); "-byte array at offset " + destOffset);
} }
if (pread) { if (!pread && streamLock.tryLock()) {
// Positional read. Better for random reads.
int extraSize = peekIntoNextBlock ? hdrSize : 0;
int ret = istream.read(fileOffset, dest, destOffset, size + extraSize);
if (ret < size) {
throw new IOException("Positional read of " + size + " bytes " +
"failed at offset " + fileOffset + " (returned " + ret + ")");
}
if (ret == size || ret < size + extraSize) {
// Could not read the next block's header, or did not try.
return -1;
}
} else {
// Seek + read. Better for scanning. // Seek + read. Better for scanning.
synchronized (istream) { try {
istream.seek(fileOffset); istream.seek(fileOffset);
long realOffset = istream.getPos(); long realOffset = istream.getPos();
@ -1251,6 +1241,22 @@ public class HFileBlock implements Cacheable {
// Try to read the next block header. // Try to read the next block header.
if (!readWithExtra(istream, dest, destOffset, size, hdrSize)) if (!readWithExtra(istream, dest, destOffset, size, hdrSize))
return -1; return -1;
} finally {
streamLock.unlock();
}
} else {
// Positional read. Better for random reads; or when the streamLock is already locked.
int extraSize = peekIntoNextBlock ? hdrSize : 0;
int ret = istream.read(fileOffset, dest, destOffset, size + extraSize);
if (ret < size) {
throw new IOException("Positional read of " + size + " bytes " +
"failed at offset " + fileOffset + " (returned " + ret + ")");
}
if (ret == size || ret < size + extraSize) {
// Could not read the next block's header, or did not try.
return -1;
} }
} }