|
@@ -18,12 +18,9 @@
|
|
|
package org.apache.hadoop.hdfs.server.datanode;
|
|
|
|
|
|
|
|
|
-import java.io.Closeable;
|
|
|
import java.io.File;
|
|
|
-import java.io.FilterInputStream;
|
|
|
import java.io.IOException;
|
|
|
import java.io.InputStream;
|
|
|
-import java.io.OutputStream;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
|
|
@@ -34,11 +31,13 @@ import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
|
|
|
import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
|
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaInputStreams;
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
|
|
|
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.IOUtils;
|
|
|
-import org.apache.hadoop.util.DataChecksum;
|
|
|
import org.apache.hadoop.util.DiskChecker.DiskErrorException;
|
|
|
import org.apache.hadoop.util.ReflectionUtils;
|
|
|
|
|
@@ -50,7 +49,7 @@ import org.apache.hadoop.util.ReflectionUtils;
|
|
|
*
|
|
|
*/
|
|
|
@InterfaceAudience.Private
|
|
|
-public interface FSDatasetInterface<V extends FSDatasetInterface.FSVolumeInterface>
|
|
|
+public interface FSDatasetInterface<V extends FsVolumeSpi>
|
|
|
extends FSDatasetMBean {
|
|
|
/**
|
|
|
* A factory for creating FSDatasetInterface objects.
|
|
@@ -77,24 +76,6 @@ public interface FSDatasetInterface<V extends FSDatasetInterface.FSVolumeInterfa
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * This is an interface for the underlying volume.
|
|
|
- * @see org.apache.hadoop.hdfs.server.datanode.FSDataset.FSVolume
|
|
|
- */
|
|
|
- interface FSVolumeInterface {
|
|
|
- /** @return a list of block pools. */
|
|
|
- public String[] getBlockPoolList();
|
|
|
-
|
|
|
- /** @return the available storage space in bytes. */
|
|
|
- public long getAvailable() throws IOException;
|
|
|
-
|
|
|
- /** @return the path to the volume */
|
|
|
- public String getPath(String bpid) throws IOException;
|
|
|
-
|
|
|
- /** @return the directory for the finalized blocks in the block pool. */
|
|
|
- public File getFinalizedDir(String bpid) throws IOException;
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Create rolling logs.
|
|
|
*
|
|
@@ -121,32 +102,15 @@ public interface FSDatasetInterface<V extends FSDatasetInterface.FSVolumeInterfa
|
|
|
* as corrupted.
|
|
|
*/
|
|
|
public void checkAndUpdate(String bpid, long blockId, File diskFile,
|
|
|
- File diskMetaFile, FSVolumeInterface vol);
|
|
|
+ File diskMetaFile, FsVolumeSpi vol);
|
|
|
|
|
|
- /**
|
|
|
- * This class provides the input stream and length of the metadata
|
|
|
- * of a block
|
|
|
- *
|
|
|
- */
|
|
|
- static class MetaDataInputStream extends FilterInputStream {
|
|
|
- MetaDataInputStream(InputStream stream, long len) {
|
|
|
- super(stream);
|
|
|
- length = len;
|
|
|
- }
|
|
|
- private long length;
|
|
|
-
|
|
|
- public long getLength() {
|
|
|
- return length;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* @param b - the block
|
|
|
* @return a stream if the meta-data of the block exists;
|
|
|
* otherwise, return null.
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- public MetaDataInputStream getMetaDataInputStream(ExtendedBlock b
|
|
|
+ public LengthInputStream getMetaDataInputStream(ExtendedBlock b
|
|
|
) throws IOException;
|
|
|
|
|
|
/**
|
|
@@ -197,58 +161,10 @@ public interface FSDatasetInterface<V extends FSDatasetInterface.FSVolumeInterfa
|
|
|
* starting at the offset
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- public BlockInputStreams getTmpInputStreams(ExtendedBlock b, long blkoff,
|
|
|
+ public ReplicaInputStreams getTmpInputStreams(ExtendedBlock b, long blkoff,
|
|
|
long ckoff) throws IOException;
|
|
|
|
|
|
/**
|
|
|
- *
|
|
|
- * This class contains the output streams for the data and checksum
|
|
|
- * of a block
|
|
|
- *
|
|
|
- */
|
|
|
- static class BlockWriteStreams {
|
|
|
- OutputStream dataOut;
|
|
|
- OutputStream checksumOut;
|
|
|
- DataChecksum checksum;
|
|
|
-
|
|
|
- BlockWriteStreams(OutputStream dOut, OutputStream cOut,
|
|
|
- DataChecksum checksum) {
|
|
|
- dataOut = dOut;
|
|
|
- checksumOut = cOut;
|
|
|
- this.checksum = checksum;
|
|
|
- }
|
|
|
-
|
|
|
- void close() {
|
|
|
- IOUtils.closeStream(dataOut);
|
|
|
- IOUtils.closeStream(checksumOut);
|
|
|
- }
|
|
|
-
|
|
|
- DataChecksum getChecksum() {
|
|
|
- return checksum;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * This class contains the input streams for the data and checksum
|
|
|
- * of a block
|
|
|
- */
|
|
|
- static class BlockInputStreams implements Closeable {
|
|
|
- final InputStream dataIn;
|
|
|
- final InputStream checksumIn;
|
|
|
-
|
|
|
- BlockInputStreams(InputStream dataIn, InputStream checksumIn) {
|
|
|
- this.dataIn = dataIn;
|
|
|
- this.checksumIn = checksumIn;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public void close() {
|
|
|
- IOUtils.closeStream(dataIn);
|
|
|
- IOUtils.closeStream(checksumIn);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
* Creates a temporary replica and returns the meta information of the replica
|
|
|
*
|
|
|
* @param b block
|
|
@@ -395,7 +311,7 @@ public interface FSDatasetInterface<V extends FSDatasetInterface.FSVolumeInterfa
|
|
|
* @param checksumSize number of bytes each checksum has
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- public void adjustCrcChannelPosition(ExtendedBlock b, BlockWriteStreams stream,
|
|
|
+ public void adjustCrcChannelPosition(ExtendedBlock b, ReplicaOutputStreams stream,
|
|
|
int checksumSize) throws IOException;
|
|
|
|
|
|
/**
|