HDFS-11791. [READ] Test for increasing replication of provided files.
This commit is contained in:
parent
89b9faf529
commit
4851f06bc2
@ -23,6 +23,7 @@
|
|||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.OutputStreamWriter;
|
import java.io.OutputStreamWriter;
|
||||||
import java.io.Writer;
|
import java.io.Writer;
|
||||||
|
import java.net.InetSocketAddress;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.nio.channels.Channels;
|
import java.nio.channels.Channels;
|
||||||
import java.nio.channels.ReadableByteChannel;
|
import java.nio.channels.ReadableByteChannel;
|
||||||
@ -34,10 +35,15 @@
|
|||||||
import org.apache.hadoop.fs.FileUtil;
|
import org.apache.hadoop.fs.FileUtil;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.fs.StorageType;
|
import org.apache.hadoop.fs.StorageType;
|
||||||
|
import org.apache.hadoop.hdfs.DFSClient;
|
||||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||||
import org.apache.hadoop.hdfs.DFSTestUtil;
|
import org.apache.hadoop.hdfs.DFSTestUtil;
|
||||||
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||||
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
||||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
||||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockFormatProvider;
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockFormatProvider;
|
||||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockProvider;
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockProvider;
|
||||||
import org.apache.hadoop.hdfs.server.common.BlockFormat;
|
import org.apache.hadoop.hdfs.server.common.BlockFormat;
|
||||||
@ -378,4 +384,53 @@ public void testClusterWithEmptyImage() throws IOException {
|
|||||||
assertEquals(1, locations.length);
|
assertEquals(1, locations.length);
|
||||||
assertEquals(2, locations[0].getHosts().length);
|
assertEquals(2, locations[0].getHosts().length);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private DatanodeInfo[] getAndCheckBlockLocations(DFSClient client,
|
||||||
|
String filename, int expectedLocations) throws IOException {
|
||||||
|
LocatedBlocks locatedBlocks = client.getLocatedBlocks(
|
||||||
|
filename, 0, baseFileLen);
|
||||||
|
//given the start and length in the above call,
|
||||||
|
//only one LocatedBlock in LocatedBlocks
|
||||||
|
assertEquals(1, locatedBlocks.getLocatedBlocks().size());
|
||||||
|
LocatedBlock locatedBlock = locatedBlocks.getLocatedBlocks().get(0);
|
||||||
|
assertEquals(expectedLocations, locatedBlock.getLocations().length);
|
||||||
|
return locatedBlock.getLocations();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tests setting replication of provided files.
|
||||||
|
* @throws Exception
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testSetReplicationForProvidedFiles() throws Exception {
|
||||||
|
createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
|
||||||
|
FixedBlockResolver.class);
|
||||||
|
startCluster(NNDIRPATH, 2, null,
|
||||||
|
new StorageType[][] {
|
||||||
|
{StorageType.PROVIDED},
|
||||||
|
{StorageType.DISK}},
|
||||||
|
false);
|
||||||
|
|
||||||
|
String filename = "/" + filePrefix + (numFiles - 1) + fileSuffix;
|
||||||
|
Path file = new Path(filename);
|
||||||
|
FileSystem fs = cluster.getFileSystem();
|
||||||
|
|
||||||
|
//set the replication to 2, and test that the file has
|
||||||
|
//the required replication.
|
||||||
|
fs.setReplication(file, (short) 2);
|
||||||
|
DFSTestUtil.waitForReplication((DistributedFileSystem) fs,
|
||||||
|
file, (short) 2, 10000);
|
||||||
|
DFSClient client = new DFSClient(new InetSocketAddress("localhost",
|
||||||
|
cluster.getNameNodePort()), cluster.getConfiguration(0));
|
||||||
|
getAndCheckBlockLocations(client, filename, 2);
|
||||||
|
|
||||||
|
//set the replication back to 1
|
||||||
|
fs.setReplication(file, (short) 1);
|
||||||
|
DFSTestUtil.waitForReplication((DistributedFileSystem) fs,
|
||||||
|
file, (short) 1, 10000);
|
||||||
|
//the only replica left should be the PROVIDED datanode
|
||||||
|
DatanodeInfo[] infos = getAndCheckBlockLocations(client, filename, 1);
|
||||||
|
assertEquals(cluster.getDataNodes().get(0).getDatanodeUuid(),
|
||||||
|
infos[0].getDatanodeUuid());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
Loading…
Reference in New Issue
Block a user