|
@@ -32,14 +32,18 @@ import static org.junit.Assert.fail;
|
|
|
|
|
|
import org.apache.hadoop.fs.BlockLocation;
|
|
|
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
|
|
+import org.apache.hadoop.fs.CreateFlag;
|
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.HdfsBlockLocation;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
+import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
import org.apache.hadoop.hdfs.*;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNodeFaultInjector;
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
|
|
|
import org.apache.hadoop.ipc.RemoteException;
|
|
@@ -57,6 +61,8 @@ import java.io.IOException;
|
|
|
import java.io.OutputStream;
|
|
|
import java.lang.management.ManagementFactory;
|
|
|
import java.lang.reflect.Field;
|
|
|
+import java.util.Collection;
|
|
|
+import java.util.EnumSet;
|
|
|
import java.util.Map;
|
|
|
import java.util.Random;
|
|
|
import java.util.concurrent.TimeoutException;
|
|
@@ -745,4 +751,48 @@ public class TestSpaceReservation {
|
|
|
}, 500, 30000);
|
|
|
checkReservedSpace(0);
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Ensure that bytes reserved of ReplicaInfo gets cleared
|
|
|
+ * during finalize.
|
|
|
+ *
|
|
|
+ * @throws IOException
|
|
|
+ */
|
|
|
+ @Test(timeout = 300000)
|
|
|
+ public void testReplicaInfoBytesReservedReleasedOnFinalize() throws IOException {
|
|
|
+ short replication = 3;
|
|
|
+ int bufferLength = 4096;
|
|
|
+ startCluster(BLOCK_SIZE, replication, -1);
|
|
|
+
|
|
|
+ String methodName = GenericTestUtils.getMethodName();
|
|
|
+ Path path = new Path("/" + methodName + ".01.dat");
|
|
|
+
|
|
|
+ FSDataOutputStream fos =
|
|
|
+ fs.create(path, FsPermission.getFileDefault(), EnumSet.of(CreateFlag.CREATE), bufferLength,
|
|
|
+ replication, BLOCK_SIZE, null);
|
|
|
+ // Allocate a block.
|
|
|
+ fos.write(new byte[bufferLength]);
|
|
|
+ fos.hsync();
|
|
|
+
|
|
|
+ DataNode dataNode = cluster.getDataNodes().get(0);
|
|
|
+ FsDatasetImpl fsDataSetImpl = (FsDatasetImpl) dataNode.getFSDataset();
|
|
|
+ long expectedReservedSpace = BLOCK_SIZE - bufferLength;
|
|
|
+
|
|
|
+ String bpid = cluster.getNamesystem().getBlockPoolId();
|
|
|
+ Collection<ReplicaInfo> replicas = FsDatasetTestUtil.getReplicas(fsDataSetImpl, bpid);
|
|
|
+ ReplicaInfo r = replicas.iterator().next();
|
|
|
+
|
|
|
+ // Verify Initial Bytes Reserved for Replica and Volume are correct
|
|
|
+ assertEquals(fsDataSetImpl.getVolumeList().get(0).getReservedForReplicas(),
|
|
|
+ expectedReservedSpace);
|
|
|
+ assertEquals(r.getBytesReserved(), expectedReservedSpace);
|
|
|
+
|
|
|
+ // Verify Bytes Reserved for Replica and Volume are correct after finalize
|
|
|
+ fsDataSetImpl.finalizeNewReplica(r, new ExtendedBlock(bpid, r));
|
|
|
+
|
|
|
+ assertEquals(fsDataSetImpl.getVolumeList().get(0).getReservedForReplicas(), 0L);
|
|
|
+ assertEquals(r.getBytesReserved(), 0L);
|
|
|
+
|
|
|
+ fos.close();
|
|
|
+ }
|
|
|
}
|