HDFS-14224. RBF: NPE in getContentSummary() for getEcPolicy() in case of multiple destinations. Contributed by Ayush Saxena.
This commit is contained in:
parent
8b9b58b58a
commit
acdf911c01
|
@ -1635,6 +1635,7 @@ public class RouterClientProtocol implements ClientProtocol {
|
|||
long quota = 0;
|
||||
long spaceConsumed = 0;
|
||||
long spaceQuota = 0;
|
||||
String ecPolicy = "";
|
||||
|
||||
for (ContentSummary summary : summaries) {
|
||||
length += summary.getLength();
|
||||
|
@ -1643,6 +1644,11 @@ public class RouterClientProtocol implements ClientProtocol {
|
|||
quota += summary.getQuota();
|
||||
spaceConsumed += summary.getSpaceConsumed();
|
||||
spaceQuota += summary.getSpaceQuota();
|
||||
// We return from the first response as we assume that the EC policy
|
||||
// of each sub-cluster is same.
|
||||
if (ecPolicy.isEmpty()) {
|
||||
ecPolicy = summary.getErasureCodingPolicy();
|
||||
}
|
||||
}
|
||||
|
||||
ContentSummary ret = new ContentSummary.Builder()
|
||||
|
@ -1652,6 +1658,7 @@ public class RouterClientProtocol implements ClientProtocol {
|
|||
.quota(quota)
|
||||
.spaceConsumed(spaceConsumed)
|
||||
.spaceQuota(spaceQuota)
|
||||
.erasureCodingPolicy(ecPolicy)
|
||||
.build();
|
||||
return ret;
|
||||
}
|
||||
|
|
|
@ -41,6 +41,7 @@ import java.util.TreeSet;
|
|||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
||||
import org.apache.hadoop.hdfs.protocol.DirectoryListing;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
||||
|
@ -229,6 +230,21 @@ public class TestRouterRpcMultiDestination extends TestRouterRpc {
|
|||
testRename2(getRouterContext(), filename1, renamedFile, false);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetContentSummaryEc() throws Exception {
|
||||
DistributedFileSystem routerDFS =
|
||||
(DistributedFileSystem) getRouterFileSystem();
|
||||
Path dir = new Path("/");
|
||||
String expectedECPolicy = "RS-6-3-1024k";
|
||||
try {
|
||||
routerDFS.setErasureCodingPolicy(dir, expectedECPolicy);
|
||||
assertEquals(expectedECPolicy,
|
||||
routerDFS.getContentSummary(dir).getErasureCodingPolicy());
|
||||
} finally {
|
||||
routerDFS.unsetErasureCodingPolicy(dir);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSubclusterDown() throws Exception {
|
||||
final int totalFiles = 6;
|
||||
|
|
Loading…
Reference in New Issue