|
@@ -17,9 +17,12 @@
|
|
*/
|
|
*/
|
|
package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
|
|
package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
|
|
|
|
|
|
|
|
+import java.io.InputStream;
|
|
import java.util.Arrays;
|
|
import java.util.Arrays;
|
|
import java.util.Collections;
|
|
import java.util.Collections;
|
|
import java.util.function.Supplier;
|
|
import java.util.function.Supplier;
|
|
|
|
+
|
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner;
|
|
import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
|
|
import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
|
|
|
|
|
|
import java.io.OutputStream;
|
|
import java.io.OutputStream;
|
|
@@ -68,6 +71,7 @@ import org.apache.hadoop.io.MultipleIOException;
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
import org.apache.hadoop.test.LambdaTestUtils;
|
|
import org.apache.hadoop.test.LambdaTestUtils;
|
|
import org.apache.hadoop.util.AutoCloseableLock;
|
|
import org.apache.hadoop.util.AutoCloseableLock;
|
|
|
|
+import org.apache.hadoop.util.DiskChecker;
|
|
import org.apache.hadoop.util.FakeTimer;
|
|
import org.apache.hadoop.util.FakeTimer;
|
|
import org.apache.hadoop.util.StringUtils;
|
|
import org.apache.hadoop.util.StringUtils;
|
|
import org.junit.Assert;
|
|
import org.junit.Assert;
|
|
@@ -1070,24 +1074,43 @@ public class TestFsDatasetImpl {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * When moving blocks using hardLink or copy
|
|
|
|
+ * and append happened in the middle,
|
|
|
|
+ * block movement should fail and hardlink is removed.
|
|
|
|
+ */
|
|
@Test(timeout = 30000)
|
|
@Test(timeout = 30000)
|
|
public void testMoveBlockFailure() {
|
|
public void testMoveBlockFailure() {
|
|
|
|
+ // Test copy
|
|
|
|
+ testMoveBlockFailure(conf);
|
|
|
|
+ // Test hardlink
|
|
|
|
+ conf.setBoolean(DFSConfigKeys
|
|
|
|
+ .DFS_DATANODE_ALLOW_SAME_DISK_TIERING, true);
|
|
|
|
+ conf.setDouble(DFSConfigKeys
|
|
|
|
+ .DFS_DATANODE_RESERVE_FOR_ARCHIVE_DEFAULT_PERCENTAGE, 0.5);
|
|
|
|
+ testMoveBlockFailure(conf);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void testMoveBlockFailure(Configuration config) {
|
|
MiniDFSCluster cluster = null;
|
|
MiniDFSCluster cluster = null;
|
|
try {
|
|
try {
|
|
|
|
+
|
|
cluster = new MiniDFSCluster.Builder(conf)
|
|
cluster = new MiniDFSCluster.Builder(conf)
|
|
.numDataNodes(1)
|
|
.numDataNodes(1)
|
|
- .storageTypes(new StorageType[]{StorageType.DISK, StorageType.DISK})
|
|
|
|
|
|
+ .storageTypes(
|
|
|
|
+ new StorageType[]{StorageType.DISK, StorageType.ARCHIVE})
|
|
.storagesPerDatanode(2)
|
|
.storagesPerDatanode(2)
|
|
.build();
|
|
.build();
|
|
FileSystem fs = cluster.getFileSystem();
|
|
FileSystem fs = cluster.getFileSystem();
|
|
DataNode dataNode = cluster.getDataNodes().get(0);
|
|
DataNode dataNode = cluster.getDataNodes().get(0);
|
|
|
|
|
|
Path filePath = new Path("testData");
|
|
Path filePath = new Path("testData");
|
|
- DFSTestUtil.createFile(fs, filePath, 100, (short) 1, 0);
|
|
|
|
- ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, filePath);
|
|
|
|
|
|
+ long fileLen = 100;
|
|
|
|
+ ExtendedBlock block = createTestFile(fs, fileLen, filePath);
|
|
|
|
|
|
FsDatasetImpl fsDataSetImpl = (FsDatasetImpl) dataNode.getFSDataset();
|
|
FsDatasetImpl fsDataSetImpl = (FsDatasetImpl) dataNode.getFSDataset();
|
|
- ReplicaInfo newReplicaInfo = createNewReplicaObj(block, fsDataSetImpl);
|
|
|
|
|
|
+ ReplicaInfo newReplicaInfo =
|
|
|
|
+ createNewReplicaObjWithLink(block, fsDataSetImpl);
|
|
|
|
|
|
// Append to file to update its GS
|
|
// Append to file to update its GS
|
|
FSDataOutputStream out = fs.append(filePath, (short) 1);
|
|
FSDataOutputStream out = fs.append(filePath, (short) 1);
|
|
@@ -1095,6 +1118,7 @@ public class TestFsDatasetImpl {
|
|
out.hflush();
|
|
out.hflush();
|
|
|
|
|
|
// Call finalizeNewReplica
|
|
// Call finalizeNewReplica
|
|
|
|
+ assertTrue(newReplicaInfo.blockDataExists());
|
|
LOG.info("GenerationStamp of old replica: {}",
|
|
LOG.info("GenerationStamp of old replica: {}",
|
|
block.getGenerationStamp());
|
|
block.getGenerationStamp());
|
|
LOG.info("GenerationStamp of new replica: {}", fsDataSetImpl
|
|
LOG.info("GenerationStamp of new replica: {}", fsDataSetImpl
|
|
@@ -1103,6 +1127,9 @@ public class TestFsDatasetImpl {
|
|
LambdaTestUtils.intercept(IOException.class, "Generation Stamp "
|
|
LambdaTestUtils.intercept(IOException.class, "Generation Stamp "
|
|
+ "should be monotonically increased.",
|
|
+ "should be monotonically increased.",
|
|
() -> fsDataSetImpl.finalizeNewReplica(newReplicaInfo, block));
|
|
() -> fsDataSetImpl.finalizeNewReplica(newReplicaInfo, block));
|
|
|
|
+ assertFalse(newReplicaInfo.blockDataExists());
|
|
|
|
+
|
|
|
|
+ validateFileLen(fs, fileLen, filePath);
|
|
} catch (Exception ex) {
|
|
} catch (Exception ex) {
|
|
LOG.info("Exception in testMoveBlockFailure ", ex);
|
|
LOG.info("Exception in testMoveBlockFailure ", ex);
|
|
fail("Exception while testing testMoveBlockFailure ");
|
|
fail("Exception while testing testMoveBlockFailure ");
|
|
@@ -1143,6 +1170,253 @@ public class TestFsDatasetImpl {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Make sure datanode restart can clean up un-finalized links,
|
|
|
|
+ * if the block is not finalized yet.
|
|
|
|
+ */
|
|
|
|
+ @Test(timeout = 30000)
|
|
|
|
+ public void testDnRestartWithHardLinkInTmp() {
|
|
|
|
+ MiniDFSCluster cluster = null;
|
|
|
|
+ try {
|
|
|
|
+ conf.setBoolean(DFSConfigKeys
|
|
|
|
+ .DFS_DATANODE_ALLOW_SAME_DISK_TIERING, true);
|
|
|
|
+ conf.setDouble(DFSConfigKeys
|
|
|
|
+ .DFS_DATANODE_RESERVE_FOR_ARCHIVE_DEFAULT_PERCENTAGE, 0.5);
|
|
|
|
+ cluster = new MiniDFSCluster.Builder(conf)
|
|
|
|
+ .numDataNodes(1)
|
|
|
|
+ .storageTypes(
|
|
|
|
+ new StorageType[]{StorageType.DISK, StorageType.ARCHIVE})
|
|
|
|
+ .storagesPerDatanode(2)
|
|
|
|
+ .build();
|
|
|
|
+ FileSystem fs = cluster.getFileSystem();
|
|
|
|
+ DataNode dataNode = cluster.getDataNodes().get(0);
|
|
|
|
+
|
|
|
|
+ Path filePath = new Path("testData");
|
|
|
|
+ long fileLen = 100;
|
|
|
|
+
|
|
|
|
+ ExtendedBlock block = createTestFile(fs, fileLen, filePath);
|
|
|
|
+
|
|
|
|
+ FsDatasetImpl fsDataSetImpl = (FsDatasetImpl) dataNode.getFSDataset();
|
|
|
|
+
|
|
|
|
+ ReplicaInfo oldReplicaInfo = fsDataSetImpl.getReplicaInfo(block);
|
|
|
|
+ ReplicaInfo newReplicaInfo =
|
|
|
|
+ createNewReplicaObjWithLink(block, fsDataSetImpl);
|
|
|
|
+
|
|
|
|
+ // Link exists
|
|
|
|
+ assertTrue(Files.exists(Paths.get(newReplicaInfo.getBlockURI())));
|
|
|
|
+
|
|
|
|
+ cluster.restartDataNode(0);
|
|
|
|
+ cluster.waitDatanodeFullyStarted(cluster.getDataNodes().get(0), 60000);
|
|
|
|
+ cluster.triggerBlockReports();
|
|
|
|
+
|
|
|
|
+ // Un-finalized replica data (hard link) is deleted as they were in /tmp
|
|
|
|
+ assertFalse(Files.exists(Paths.get(newReplicaInfo.getBlockURI())));
|
|
|
|
+
|
|
|
|
+ // Old block is there.
|
|
|
|
+ assertTrue(Files.exists(Paths.get(oldReplicaInfo.getBlockURI())));
|
|
|
|
+
|
|
|
|
+ validateFileLen(fs, fileLen, filePath);
|
|
|
|
+
|
|
|
|
+ } catch (Exception ex) {
|
|
|
|
+ LOG.info("Exception in testDnRestartWithHardLinkInTmp ", ex);
|
|
|
|
+ fail("Exception while testing testDnRestartWithHardLinkInTmp ");
|
|
|
|
+ } finally {
|
|
|
|
+ if (cluster.isClusterUp()) {
|
|
|
|
+ cluster.shutdown();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * If new block is finalized and DN restarted,
|
|
|
|
+ * DiskScanner should clean up the hardlink correctly.
|
|
|
|
+ */
|
|
|
|
+ @Test(timeout = 30000)
|
|
|
|
+ public void testDnRestartWithHardLink() {
|
|
|
|
+ MiniDFSCluster cluster = null;
|
|
|
|
+ try {
|
|
|
|
+ conf.setBoolean(DFSConfigKeys
|
|
|
|
+ .DFS_DATANODE_ALLOW_SAME_DISK_TIERING, true);
|
|
|
|
+ conf.setDouble(DFSConfigKeys
|
|
|
|
+ .DFS_DATANODE_RESERVE_FOR_ARCHIVE_DEFAULT_PERCENTAGE, 0.5);
|
|
|
|
+ cluster = new MiniDFSCluster.Builder(conf)
|
|
|
|
+ .numDataNodes(1)
|
|
|
|
+ .storageTypes(
|
|
|
|
+ new StorageType[]{StorageType.DISK, StorageType.ARCHIVE})
|
|
|
|
+ .storagesPerDatanode(2)
|
|
|
|
+ .build();
|
|
|
|
+ FileSystem fs = cluster.getFileSystem();
|
|
|
|
+ DataNode dataNode = cluster.getDataNodes().get(0);
|
|
|
|
+
|
|
|
|
+ Path filePath = new Path("testData");
|
|
|
|
+ long fileLen = 100;
|
|
|
|
+
|
|
|
|
+ ExtendedBlock block = createTestFile(fs, fileLen, filePath);
|
|
|
|
+
|
|
|
|
+ FsDatasetImpl fsDataSetImpl = (FsDatasetImpl) dataNode.getFSDataset();
|
|
|
|
+
|
|
|
|
+ final ReplicaInfo oldReplicaInfo = fsDataSetImpl.getReplicaInfo(block);
|
|
|
|
+
|
|
|
|
+ fsDataSetImpl.finalizeNewReplica(
|
|
|
|
+ createNewReplicaObjWithLink(block, fsDataSetImpl), block);
|
|
|
|
+
|
|
|
|
+ ReplicaInfo newReplicaInfo = fsDataSetImpl.getReplicaInfo(block);
|
|
|
|
+
|
|
|
|
+ cluster.restartDataNode(0);
|
|
|
|
+ cluster.waitDatanodeFullyStarted(cluster.getDataNodes().get(0), 60000);
|
|
|
|
+ cluster.triggerBlockReports();
|
|
|
|
+
|
|
|
|
+ assertTrue(Files.exists(Paths.get(newReplicaInfo.getBlockURI())));
|
|
|
|
+ assertTrue(Files.exists(Paths.get(oldReplicaInfo.getBlockURI())));
|
|
|
|
+
|
|
|
|
+ DirectoryScanner scanner = new DirectoryScanner(
|
|
|
|
+ cluster.getDataNodes().get(0).getFSDataset(), conf);
|
|
|
|
+ scanner.start();
|
|
|
|
+ scanner.run();
|
|
|
|
+
|
|
|
|
+ GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
|
|
|
+ @Override public Boolean get() {
|
|
|
|
+ return !Files.exists(Paths.get(oldReplicaInfo.getBlockURI()));
|
|
|
|
+ }
|
|
|
|
+ }, 100, 10000);
|
|
|
|
+ assertTrue(Files.exists(Paths.get(newReplicaInfo.getBlockURI())));
|
|
|
|
+
|
|
|
|
+ validateFileLen(fs, fileLen, filePath);
|
|
|
|
+
|
|
|
|
+ } catch (Exception ex) {
|
|
|
|
+ LOG.info("Exception in testDnRestartWithHardLink ", ex);
|
|
|
|
+ fail("Exception while testing testDnRestartWithHardLink ");
|
|
|
|
+ } finally {
|
|
|
|
+ if (cluster.isClusterUp()) {
|
|
|
|
+ cluster.shutdown();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Test(timeout = 30000)
|
|
|
|
+ public void testMoveBlockSuccessWithSameMountMove() {
|
|
|
|
+ MiniDFSCluster cluster = null;
|
|
|
|
+ try {
|
|
|
|
+ conf.setBoolean(DFSConfigKeys
|
|
|
|
+ .DFS_DATANODE_ALLOW_SAME_DISK_TIERING, true);
|
|
|
|
+ conf.setDouble(DFSConfigKeys
|
|
|
|
+ .DFS_DATANODE_RESERVE_FOR_ARCHIVE_DEFAULT_PERCENTAGE, 0.5);
|
|
|
|
+ cluster = new MiniDFSCluster.Builder(conf)
|
|
|
|
+ .numDataNodes(1)
|
|
|
|
+ .storageTypes(
|
|
|
|
+ new StorageType[]{StorageType.DISK, StorageType.ARCHIVE})
|
|
|
|
+ .storagesPerDatanode(2)
|
|
|
|
+ .build();
|
|
|
|
+ FileSystem fs = cluster.getFileSystem();
|
|
|
|
+ DataNode dataNode = cluster.getDataNodes().get(0);
|
|
|
|
+ Path filePath = new Path("testData");
|
|
|
|
+ long fileLen = 100;
|
|
|
|
+
|
|
|
|
+ ExtendedBlock block = createTestFile(fs, fileLen, filePath);
|
|
|
|
+
|
|
|
|
+ FsDatasetImpl fsDataSetImpl = (FsDatasetImpl) dataNode.getFSDataset();
|
|
|
|
+ assertEquals(StorageType.DISK,
|
|
|
|
+ fsDataSetImpl.getReplicaInfo(block).getVolume().getStorageType());
|
|
|
|
+
|
|
|
|
+ FsDatasetImpl fsDataSetImplSpy =
|
|
|
|
+ spy((FsDatasetImpl) dataNode.getFSDataset());
|
|
|
|
+ fsDataSetImplSpy.moveBlockAcrossStorage(
|
|
|
|
+ block, StorageType.ARCHIVE, null);
|
|
|
|
+
|
|
|
|
+ // Make sure it is done thru hardlink
|
|
|
|
+ verify(fsDataSetImplSpy).moveBlock(any(), any(), any(), eq(true));
|
|
|
|
+
|
|
|
|
+ assertEquals(StorageType.ARCHIVE,
|
|
|
|
+ fsDataSetImpl.getReplicaInfo(block).getVolume().getStorageType());
|
|
|
|
+ validateFileLen(fs, fileLen, filePath);
|
|
|
|
+
|
|
|
|
+ } catch (Exception ex) {
|
|
|
|
+ LOG.info("Exception in testMoveBlockSuccessWithSameMountMove ", ex);
|
|
|
|
+ fail("testMoveBlockSuccessWithSameMountMove operation should succeed");
|
|
|
|
+ } finally {
|
|
|
|
+ if (cluster.isClusterUp()) {
|
|
|
|
+ cluster.shutdown();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // Move should fail if the volume on same mount has no space.
|
|
|
|
+ @Test(timeout = 30000)
|
|
|
|
+ public void testMoveBlockWithSameMountMoveWithoutSpace() {
|
|
|
|
+ MiniDFSCluster cluster = null;
|
|
|
|
+ try {
|
|
|
|
+ conf.setBoolean(DFSConfigKeys
|
|
|
|
+ .DFS_DATANODE_ALLOW_SAME_DISK_TIERING, true);
|
|
|
|
+ conf.setDouble(DFSConfigKeys
|
|
|
|
+ .DFS_DATANODE_RESERVE_FOR_ARCHIVE_DEFAULT_PERCENTAGE, 0.0);
|
|
|
|
+ cluster = new MiniDFSCluster.Builder(conf)
|
|
|
|
+ .numDataNodes(1)
|
|
|
|
+ .storageTypes(
|
|
|
|
+ new StorageType[]{StorageType.DISK, StorageType.ARCHIVE})
|
|
|
|
+ .storagesPerDatanode(2)
|
|
|
|
+ .build();
|
|
|
|
+ FileSystem fs = cluster.getFileSystem();
|
|
|
|
+ DataNode dataNode = cluster.getDataNodes().get(0);
|
|
|
|
+ Path filePath = new Path("testData");
|
|
|
|
+ long fileLen = 100;
|
|
|
|
+
|
|
|
|
+ ExtendedBlock block = createTestFile(fs, fileLen, filePath);
|
|
|
|
+
|
|
|
|
+ FsDatasetImpl fsDataSetImpl = (FsDatasetImpl) dataNode.getFSDataset();
|
|
|
|
+ assertEquals(StorageType.DISK,
|
|
|
|
+ fsDataSetImpl.getReplicaInfo(block).getVolume().getStorageType());
|
|
|
|
+
|
|
|
|
+ FsDatasetImpl fsDataSetImplSpy =
|
|
|
|
+ spy((FsDatasetImpl) dataNode.getFSDataset());
|
|
|
|
+ fsDataSetImplSpy.moveBlockAcrossStorage(
|
|
|
|
+ block, StorageType.ARCHIVE, null);
|
|
|
|
+
|
|
|
|
+ fail("testMoveBlockWithSameMountMoveWithoutSpace operation" +
|
|
|
|
+ " should failed");
|
|
|
|
+ } catch (Exception ex) {
|
|
|
|
+ assertTrue(ex instanceof DiskChecker.DiskOutOfSpaceException);
|
|
|
|
+ } finally {
|
|
|
|
+ if (cluster.isClusterUp()) {
|
|
|
|
+ cluster.shutdown();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // More tests on shouldConsiderSameMountVolume.
|
|
|
|
+ @Test(timeout = 10000)
|
|
|
|
+ public void testShouldConsiderSameMountVolume() throws IOException {
|
|
|
|
+ FsVolumeImpl volume = new FsVolumeImplBuilder()
|
|
|
|
+ .setConf(conf)
|
|
|
|
+ .setDataset(dataset)
|
|
|
|
+ .setStorageID("storage-id")
|
|
|
|
+ .setStorageDirectory(
|
|
|
|
+ new StorageDirectory(StorageLocation.parse(BASE_DIR)))
|
|
|
|
+ .build();
|
|
|
|
+ assertFalse(dataset.shouldConsiderSameMountVolume(volume,
|
|
|
|
+ StorageType.ARCHIVE, null));
|
|
|
|
+
|
|
|
|
+ conf.setBoolean(DFSConfigKeys
|
|
|
|
+ .DFS_DATANODE_ALLOW_SAME_DISK_TIERING, true);
|
|
|
|
+ conf.setDouble(DFSConfigKeys
|
|
|
|
+ .DFS_DATANODE_RESERVE_FOR_ARCHIVE_DEFAULT_PERCENTAGE,
|
|
|
|
+ 0.5);
|
|
|
|
+ volume = new FsVolumeImplBuilder()
|
|
|
|
+ .setConf(conf)
|
|
|
|
+ .setDataset(dataset)
|
|
|
|
+ .setStorageID("storage-id")
|
|
|
|
+ .setStorageDirectory(
|
|
|
|
+ new StorageDirectory(StorageLocation.parse(BASE_DIR)))
|
|
|
|
+ .build();
|
|
|
|
+ assertTrue(dataset.shouldConsiderSameMountVolume(volume,
|
|
|
|
+ StorageType.ARCHIVE, null));
|
|
|
|
+ assertTrue(dataset.shouldConsiderSameMountVolume(volume,
|
|
|
|
+ StorageType.ARCHIVE, ""));
|
|
|
|
+ assertFalse(dataset.shouldConsiderSameMountVolume(volume,
|
|
|
|
+ StorageType.DISK, null));
|
|
|
|
+ assertFalse(dataset.shouldConsiderSameMountVolume(volume,
|
|
|
|
+ StorageType.ARCHIVE, "target"));
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Create a new temporary replica of replicaInfo object in another volume.
|
|
* Create a new temporary replica of replicaInfo object in another volume.
|
|
*
|
|
*
|
|
@@ -1158,6 +1432,38 @@ public class TestFsDatasetImpl {
|
|
destVolume.obtainReference());
|
|
destVolume.obtainReference());
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Create a new temporary replica of replicaInfo object in another volume.
|
|
|
|
+ *
|
|
|
|
+ * @param block - Extended Block
|
|
|
|
+ * @param fsDataSetImpl - FsDatasetImpl reference
|
|
|
|
+ * @throws IOException
|
|
|
|
+ */
|
|
|
|
+ private ReplicaInfo createNewReplicaObjWithLink(ExtendedBlock block,
|
|
|
|
+ FsDatasetImpl fsDataSetImpl) throws IOException {
|
|
|
|
+ ReplicaInfo replicaInfo = fsDataSetImpl.getReplicaInfo(block);
|
|
|
|
+ FsVolumeSpi destVolume = getDestinationVolume(block, fsDataSetImpl);
|
|
|
|
+ return fsDataSetImpl.moveReplicaToVolumeOnSameMount(block, replicaInfo,
|
|
|
|
+ destVolume.obtainReference());
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private ExtendedBlock createTestFile(FileSystem fs,
|
|
|
|
+ long fileLen, Path filePath) throws IOException {
|
|
|
|
+ DFSTestUtil.createFile(fs, filePath, fileLen, (short) 1, 0);
|
|
|
|
+ return DFSTestUtil.getFirstBlock(fs, filePath);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void validateFileLen(FileSystem fs,
|
|
|
|
+ long fileLen, Path filePath) throws IOException {
|
|
|
|
+ // Read data file to make sure it is good.
|
|
|
|
+ InputStream in = fs.open(filePath);
|
|
|
|
+ int bytesCount = 0;
|
|
|
|
+ while (in.read() != -1) {
|
|
|
|
+ bytesCount++;
|
|
|
|
+ }
|
|
|
|
+ assertTrue(fileLen <= bytesCount);
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Finds a new destination volume for block.
|
|
* Finds a new destination volume for block.
|
|
*
|
|
*
|
|
@@ -1225,7 +1531,8 @@ public class TestFsDatasetImpl {
|
|
ReplicaInfo replicaInfo = fsDataSetImpl.getReplicaInfo(block);
|
|
ReplicaInfo replicaInfo = fsDataSetImpl.getReplicaInfo(block);
|
|
FsVolumeSpi destVolume = getDestinationVolume(block, fsDataSetImpl);
|
|
FsVolumeSpi destVolume = getDestinationVolume(block, fsDataSetImpl);
|
|
assertNotNull("Destination volume should not be null.", destVolume);
|
|
assertNotNull("Destination volume should not be null.", destVolume);
|
|
- fsDataSetImpl.moveBlock(block, replicaInfo, destVolume.obtainReference());
|
|
|
|
|
|
+ fsDataSetImpl.moveBlock(block, replicaInfo,
|
|
|
|
+ destVolume.obtainReference(), false);
|
|
// Trigger block report to update block info in NN
|
|
// Trigger block report to update block info in NN
|
|
cluster.triggerBlockReports();
|
|
cluster.triggerBlockReports();
|
|
blkReader.read(buf, 512, 512);
|
|
blkReader.read(buf, 512, 512);
|