|
@@ -41,6 +41,7 @@ import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
+import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
@@ -75,6 +76,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.VolumeChoosingPolicy;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
|
|
|
+import org.apache.hadoop.io.nativeio.NativeIO;
|
|
|
import org.apache.hadoop.metrics2.util.MBeans;
|
|
|
import org.apache.hadoop.util.DataChecksum;
|
|
|
import org.apache.hadoop.util.DiskChecker.DiskErrorException;
|
|
@@ -90,6 +92,15 @@ import org.apache.hadoop.util.Time;
|
|
|
@InterfaceAudience.Private
|
|
|
class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
static final Log LOG = LogFactory.getLog(FsDatasetImpl.class);
|
|
|
+ private final static boolean isNativeIOAvailable;
|
|
|
+ static {
|
|
|
+ isNativeIOAvailable = NativeIO.isAvailable();
|
|
|
+ if (Path.WINDOWS && !isNativeIOAvailable) {
|
|
|
+ LOG.warn("Data node cannot fully support concurrent reading"
|
|
|
+ + " and writing without native code extensions on Windows.");
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
|
|
|
@Override // FsDatasetSpi
|
|
|
public List<FsVolumeImpl> getVolumes() {
|
|
@@ -147,6 +158,11 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
if (meta == null || !meta.exists()) {
|
|
|
return null;
|
|
|
}
|
|
|
+ if (isNativeIOAvailable) {
|
|
|
+ return new LengthInputStream(
|
|
|
+ NativeIO.getShareDeleteFileInputStream(meta),
|
|
|
+ meta.length());
|
|
|
+ }
|
|
|
return new LengthInputStream(new FileInputStream(meta), meta.length());
|
|
|
}
|
|
|
|
|
@@ -322,18 +338,22 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
public InputStream getBlockInputStream(ExtendedBlock b,
|
|
|
long seekOffset) throws IOException {
|
|
|
File blockFile = getBlockFileNoExistsCheck(b);
|
|
|
- RandomAccessFile blockInFile;
|
|
|
- try {
|
|
|
- blockInFile = new RandomAccessFile(blockFile, "r");
|
|
|
- } catch (FileNotFoundException fnfe) {
|
|
|
- throw new IOException("Block " + b + " is not valid. " +
|
|
|
- "Expected block file at " + blockFile + " does not exist.");
|
|
|
- }
|
|
|
+ if (isNativeIOAvailable) {
|
|
|
+ return NativeIO.getShareDeleteFileInputStream(blockFile, seekOffset);
|
|
|
+ } else {
|
|
|
+ RandomAccessFile blockInFile;
|
|
|
+ try {
|
|
|
+ blockInFile = new RandomAccessFile(blockFile, "r");
|
|
|
+ } catch (FileNotFoundException fnfe) {
|
|
|
+ throw new IOException("Block " + b + " is not valid. " +
|
|
|
+ "Expected block file at " + blockFile + " does not exist.");
|
|
|
+ }
|
|
|
|
|
|
- if (seekOffset > 0) {
|
|
|
- blockInFile.seek(seekOffset);
|
|
|
+ if (seekOffset > 0) {
|
|
|
+ blockInFile.seek(seekOffset);
|
|
|
+ }
|
|
|
+ return new FileInputStream(blockInFile.getFD());
|
|
|
}
|
|
|
- return new FileInputStream(blockInFile.getFD());
|
|
|
}
|
|
|
|
|
|
/**
|