HDFS-11679. Ozone: SCM CLI: Implement list container command. Contributed by Yuanbo Liu.

This commit is contained in:
Weiwei Yang 2017-07-07 15:54:21 +08:00 committed by Owen O'Malley
parent eb18b86e00
commit 25c1b296e6
14 changed files with 524 additions and 1 deletions

View File

@ -28,6 +28,7 @@
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.List;
import java.util.UUID;
/**
@ -156,6 +157,17 @@ public void deleteContainer(Pipeline pipeline, boolean force)
}
}
/**
* {@inheritDoc}
*/
@Override
public List<Pipeline> listContainer(String startName,
String prefixName, int count)
throws IOException {
return storageContainerLocationClient.listContainer(
startName, prefixName, count);
}
/**
* Get meta data from an existing container.
*

View File

@ -22,6 +22,7 @@
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
import java.io.IOException;
import java.util.List;
/**
* The interface to call into underlying container layer.
@ -59,6 +60,21 @@ public interface ScmClient {
*/
void deleteContainer(Pipeline pipeline, boolean force) throws IOException;
/**
* Lists a range of containers and get the pipelines info.
*
* @param startName start name, if null, start searching at the head.
* @param prefixName prefix name, if null, then filter is disabled.
* @param count count, if count < 0, the max size is unlimited.(
* Usually the count will be replace with a very big
* value instead of being unlimited in case the db is very big)
*
* @return a list of pipeline.
* @throws IOException
*/
List<Pipeline> listContainer(String startName, String prefixName, int count)
throws IOException;
/**
* Read meta data from an existing container.
* @param pipeline - Pipeline that represents the container.

View File

@ -18,11 +18,21 @@
package org.apache.hadoop.scm.container.common.helpers;
import com.fasterxml.jackson.annotation.JsonAutoDetect;
import com.fasterxml.jackson.annotation.JsonFilter;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.PropertyAccessor;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.ObjectWriter;
import com.fasterxml.jackson.databind.ser.FilterProvider;
import com.fasterxml.jackson.databind.ser.impl.SimpleBeanPropertyFilter;
import com.fasterxml.jackson.databind.ser.impl.SimpleFilterProvider;
import com.google.common.base.Preconditions;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
@ -43,6 +53,26 @@ public class Pipeline {
*/
private byte[] data;
static final String PIPELINE_INFO = "PIPELINE_INFO_FILTER";
private static final ObjectWriter WRITER;
static {
ObjectMapper mapper = new ObjectMapper();
String[] ignorableFieldNames = {"data", "leaderID", "datanodes"};
FilterProvider filters = new SimpleFilterProvider()
.addFilter(PIPELINE_INFO, SimpleBeanPropertyFilter
.serializeAllExcept(ignorableFieldNames));
mapper.setVisibility(PropertyAccessor.FIELD,
JsonAutoDetect.Visibility.ANY);
mapper.addMixIn(Object.class, MixIn.class);
WRITER = mapper.writer(filters);
}
@JsonFilter(PIPELINE_INFO)
class MixIn {
}
/**
* Constructs a new pipeline data structure.
*
@ -86,24 +116,49 @@ public void addMember(DatanodeID dataNodeId) {
*
* @return First Machine.
*/
@JsonIgnore
public DatanodeID getLeader() {
return datanodes.get(leaderID);
}
/**
* Returns the leader host.
*
* @return First Machine.
*/
public String getLeaderHost() {
return datanodes.get(leaderID).getHostName();
}
/**
* Returns all machines that make up this pipeline.
*
* @return List of Machines.
*/
@JsonIgnore
public List<DatanodeID> getMachines() {
return new ArrayList<>(datanodes.values());
}
/**
* Returns all machines that make up this pipeline.
*
* @return List of Machines.
*/
public List<String> getDatanodeHosts() {
List<String> dataHosts = new ArrayList<>();
for (DatanodeID id : datanodes.values()) {
dataHosts.add(id.getHostName());
}
return dataHosts;
}
/**
* Return a Protobuf Pipeline message from pipeline.
*
* @return Protobuf message
*/
@JsonIgnore
public OzoneProtos.Pipeline getProtobufMessage() {
OzoneProtos.Pipeline.Builder builder =
OzoneProtos.Pipeline.newBuilder();
@ -165,4 +220,14 @@ public String toString() {
b.append("] container:").append(containerName);
return b.toString();
}
/**
* Returns a JSON string of this object.
*
* @return String - json string
* @throws IOException
*/
public String toJsonString() throws IOException {
return WRITER.writeValueAsString(this);
}
}

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.scm.protocol;
import java.io.IOException;
import java.util.List;
import org.apache.hadoop.scm.client.ScmClient;
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
@ -59,6 +60,25 @@ Pipeline allocateContainer(String containerName,
*/
Pipeline getContainer(String containerName) throws IOException;
/**
* Ask SCM a list of pipelines with a range of container names
* and the limit of count.
* Search container names between start name(exclusive), and
* use prefix name to filter the result. the max size of the
* searching range cannot exceed the value of count.
*
* @param startName start name, if null, start searching at the head.
* @param prefixName prefix name, if null, then filter is disabled.
* @param count count, if count < 0, the max size is unlimited.(
* Usually the count will be replace with a very big
* value instead of being unlimited in case the db is very big)
*
* @return a list of pipeline.
* @throws IOException
*/
List<Pipeline> listContainer(String startName, String prefixName, int count)
throws IOException;
/**
* Deletes a container in SCM.
*

View File

@ -25,6 +25,7 @@
import org.apache.hadoop.ipc.ProtobufHelper;
import org.apache.hadoop.ipc.ProtocolTranslator;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
import org.apache.hadoop.scm.client.ScmClient;
import org.apache.hadoop.scm.protocol.StorageContainerLocationProtocol;
import org.apache.hadoop.ozone.protocol.proto.StorageContainerLocationProtocolProtos.ContainerRequestProto;
@ -32,10 +33,14 @@
import org.apache.hadoop.ozone.protocol.proto.StorageContainerLocationProtocolProtos.GetContainerRequestProto;
import org.apache.hadoop.ozone.protocol.proto.StorageContainerLocationProtocolProtos.GetContainerResponseProto;
import org.apache.hadoop.ozone.protocol.proto.StorageContainerLocationProtocolProtos.DeleteContainerRequestProto;
import org.apache.hadoop.ozone.protocol.proto.StorageContainerLocationProtocolProtos.ListContainerRequestProto;
import org.apache.hadoop.ozone.protocol.proto.StorageContainerLocationProtocolProtos.ListContainerResponseProto;
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
/**
* This class is the client-side translator to translate the requests made on
@ -128,6 +133,36 @@ public Pipeline getContainer(String containerName) throws IOException {
}
}
/**
* {@inheritDoc}
*/
@Override
public List<Pipeline> listContainer(String startName, String prefixName,
int count) throws IOException {
ListContainerRequestProto.Builder builder = ListContainerRequestProto
.newBuilder();
if (prefixName != null) {
builder.setPrefixName(prefixName);
}
if (startName != null) {
builder.setStartName(startName);
}
builder.setCount(count);
ListContainerRequestProto request = builder.build();
try {
ListContainerResponseProto response =
rpcProxy.listContainer(NULL_RPC_CONTROLLER, request);
List<Pipeline> pipelineList = new ArrayList<>();
for (OzoneProtos.Pipeline pipelineProto : response.getPipelineList()) {
pipelineList.add(Pipeline.getFromProtoBuf(pipelineProto));
}
return pipelineList;
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
}
/**
* Ask SCM to delete a container by name. SCM will remove
* the container mapping in its database.

View File

@ -66,6 +66,16 @@ message GetContainerResponseProto {
required hadoop.hdfs.ozone.Pipeline pipeline = 1;
}
message ListContainerRequestProto {
required uint32 count = 1;
optional string startName = 2;
optional string prefixName = 3;
}
message ListContainerResponseProto {
repeated hadoop.hdfs.ozone.Pipeline pipeline = 1;
}
message DeleteContainerRequestProto {
required string containerName = 1;
}
@ -91,6 +101,8 @@ service StorageContainerLocationProtocolService {
*/
rpc getContainer(GetContainerRequestProto) returns (GetContainerResponseProto);
rpc listContainer(ListContainerRequestProto) returns (ListContainerResponseProto);
/**
* Deletes a container in SCM.
*/

View File

@ -18,6 +18,8 @@
package org.apache.hadoop.ozone.protocolPB;
import java.io.IOException;
import java.util.List;
import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException;
@ -36,6 +38,10 @@
.StorageContainerLocationProtocolProtos.DeleteContainerRequestProto;
import org.apache.hadoop.ozone.protocol.proto
.StorageContainerLocationProtocolProtos.DeleteContainerResponseProto;
import org.apache.hadoop.ozone.protocol.proto
.StorageContainerLocationProtocolProtos.ListContainerResponseProto;
import org.apache.hadoop.ozone.protocol.proto
.StorageContainerLocationProtocolProtos.ListContainerRequestProto;
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.scm.protocolPB.StorageContainerLocationProtocolPB;
@ -89,6 +95,38 @@ public GetContainerResponseProto getContainer(
}
}
@Override
public ListContainerResponseProto listContainer(RpcController controller,
ListContainerRequestProto request) throws ServiceException {
try {
String startName = null;
String prefixName = null;
int count = -1;
// Arguments check.
if (request.hasPrefixName()) {
// End container name is given.
prefixName = request.getPrefixName();
}
if (request.hasStartName()) {
// End container name is given.
startName = request.getStartName();
}
count = request.getCount();
List<Pipeline> pipelineList = impl.listContainer(startName,
prefixName, count);
ListContainerResponseProto.Builder builder =
ListContainerResponseProto.newBuilder();
for (Pipeline pipeline : pipelineList) {
builder.addPipeline(pipeline.getProtobufMessage());
}
return builder.build();
} catch (IOException e) {
throw new ServiceException(e);
}
}
@Override
public DeleteContainerResponseProto deleteContainer(
RpcController controller, DeleteContainerRequestProto request)

View File

@ -402,6 +402,15 @@ public Pipeline getContainer(String containerName) throws IOException {
return scmContainerManager.getContainer(containerName);
}
/**
* {@inheritDoc}
*/
@Override
public List<Pipeline> listContainer(String startName,
String prefixName, int count) throws IOException {
return scmContainerManager.listContainer(startName, prefixName, count);
}
/**
* {@inheritDoc}
*/

View File

@ -35,6 +35,8 @@
.DeleteContainerHandler.CONTAINER_DELETE;
import static org.apache.hadoop.ozone.scm.cli.container
.InfoContainerHandler.CONTAINER_INFO;
import static org.apache.hadoop.ozone.scm.cli.container
.ListContainerHandler.CONTAINER_LIST;
/**
* The handler class of container-specific commands, e.g. createContainer.
@ -61,6 +63,8 @@ public void execute(CommandLine cmd) throws IOException {
handler = new DeleteContainerHandler(getScmClient());
} else if (cmd.hasOption(CONTAINER_INFO)) {
handler = new InfoContainerHandler(getScmClient());
} else if (cmd.hasOption(CONTAINER_LIST)) {
handler = new ListContainerHandler(getScmClient());
}
// execute the sub command, throw exception if no sub command found
@ -95,10 +99,13 @@ private static void addCommandsOption(Options options) {
new Option(CONTAINER_INFO, false, "Info container");
Option deleteContainer =
new Option(CONTAINER_DELETE, false, "Delete container");
Option listContainer =
new Option(CONTAINER_LIST, false, "List container");
options.addOption(createContainer);
options.addOption(deleteContainer);
options.addOption(infoContainer);
options.addOption(listContainer);
// TODO : add other options such as delete, close etc.
}
@ -108,6 +115,7 @@ public static void addOptions(Options options) {
CreateContainerHandler.addOptions(options);
DeleteContainerHandler.addOptions(options);
InfoContainerHandler.addOptions(options);
ListContainerHandler.addOptions(options);
// TODO : add other options such as delete, close etc.
}
}

View File

@ -0,0 +1,120 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.ozone.scm.cli.container;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.HelpFormatter;
import org.apache.commons.cli.Option;
import org.apache.commons.cli.Options;
import org.apache.hadoop.ozone.scm.cli.OzoneCommandHandler;
import org.apache.hadoop.ozone.web.utils.JsonUtils;
import org.apache.hadoop.scm.client.ScmClient;
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
import java.io.IOException;
import java.util.List;
import static org.apache.hadoop.ozone.scm.cli.SCMCLI.CMD_WIDTH;
import static org.apache.hadoop.ozone.scm.cli.SCMCLI.HELP_OP;
/**
* This is the handler that process container list command.
*/
public class ListContainerHandler extends OzoneCommandHandler {
public static final String CONTAINER_LIST = "list";
public static final String OPT_START_CONTAINER = "start";
public static final String OPT_PREFIX_CONTAINER = "prefix";
public static final String OPT_COUNT = "count";
/**
* Constructs a handler object.
*
* @param scmClient scm client
*/
public ListContainerHandler(ScmClient scmClient) {
super(scmClient);
}
@Override
public void execute(CommandLine cmd) throws IOException {
if (!cmd.hasOption(CONTAINER_LIST)) {
throw new IOException("Expecting container list");
}
if (cmd.hasOption(HELP_OP)) {
displayHelp();
return;
}
if (!cmd.hasOption(OPT_COUNT)) {
displayHelp();
if (!cmd.hasOption(HELP_OP)) {
throw new IOException("Expecting container count");
} else {
return;
}
}
String startName = cmd.getOptionValue(OPT_START_CONTAINER);
String prefixName = cmd.getOptionValue(OPT_PREFIX_CONTAINER);
int count = 0;
if (cmd.hasOption(OPT_COUNT)) {
count = Integer.parseInt(cmd.getOptionValue(OPT_COUNT));
if (count < 0) {
displayHelp();
throw new IOException("-count should not be negative");
}
}
List<Pipeline> pipelineList =
getScmClient().listContainer(startName, prefixName, count);
// Output data list
for (Pipeline pipeline : pipelineList) {
outputContainerPipeline(pipeline);
}
}
private void outputContainerPipeline(Pipeline pipeline) throws IOException {
// Print container report info.
logOut("%s", JsonUtils.toJsonStringWithDefaultPrettyPrinter(
pipeline.toJsonString()));
}
@Override
public void displayHelp() {
Options options = new Options();
addOptions(options);
HelpFormatter helpFormatter = new HelpFormatter();
helpFormatter.printHelp(CMD_WIDTH, "hdfs scm -container -list <option>",
"where <option> can be the following", options, "");
}
public static void addOptions(Options options) {
Option startContainerOpt = new Option(OPT_START_CONTAINER,
true, "Specify start container name");
Option endContainerOpt = new Option(OPT_PREFIX_CONTAINER,
true, "Specify prefix container name");
Option countOpt = new Option(OPT_COUNT, true,
"Specify count number, required");
options.addOption(countOpt);
options.addOption(startContainerOpt);
options.addOption(endContainerOpt);
}
}

View File

@ -20,6 +20,7 @@
import com.google.common.base.Preconditions;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.ozone.OzoneConsts;
@ -31,6 +32,7 @@
import org.apache.hadoop.scm.ScmConfigKeys;
import org.apache.hadoop.scm.client.ScmClient;
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.utils.LevelDBKeyFilters.KeyPrefixFilter;
import org.apache.hadoop.utils.LevelDBStore;
import org.iq80.leveldb.Options;
import org.slf4j.Logger;
@ -41,7 +43,9 @@
import java.lang.reflect.Constructor;
import java.lang.reflect.InvocationTargetException;
import java.nio.charset.Charset;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
@ -175,6 +179,37 @@ public Pipeline getContainer(final String containerName) throws IOException {
}
}
/**
* {@inheritDoc}
*/
@Override
public List<Pipeline> listContainer(String startName,
String prefixName, int count)
throws IOException {
List<Pipeline> pipelineList = new ArrayList<>();
lock.lock();
try {
if(containerStore.isEmpty()) {
throw new IOException("No container exists in current db");
}
KeyPrefixFilter prefixFilter = new KeyPrefixFilter(prefixName);
byte[] startKey = startName == null ?
null : DFSUtil.string2Bytes(startName);
List<Map.Entry<byte[], byte[]>> range =
containerStore.getRangeKVs(startKey, count, prefixFilter);
// Transform the values into the pipelines.
for (Map.Entry<byte[], byte[]> entry : range) {
Pipeline pipeline = Pipeline.getFromProtoBuf(
OzoneProtos.Pipeline.PARSER.parseFrom(entry.getValue()));
pipelineList.add(pipeline);
}
} finally {
lock.unlock();
}
return pipelineList;
}
/**
* Allocates a new container.
*

View File

@ -22,6 +22,7 @@
import java.io.Closeable;
import java.io.IOException;
import java.util.List;
/**
* Mapping class contains the mapping from a name to a pipeline mapping. This is
@ -37,6 +38,25 @@ public interface Mapping extends Closeable {
*/
Pipeline getContainer(String containerName) throws IOException;
/**
* Returns pipelines under certain conditions.
* Search container names from start name(exclusive),
* and use prefix name to filter the result. The max
* size of the searching range cannot exceed the
* value of count.
*
* @param startName start name, if null, start searching at the head.
* @param prefixName prefix name, if null, then filter is disabled.
* @param count count, if count < 0, the max size is unlimited.(
* Usually the count will be replace with a very big
* value instead of being unlimited in case the db is very big)
*
* @return a list of pipeline.
* @throws IOException
*/
List<Pipeline> listContainer(String startName, String prefixName, int count)
throws IOException;
/**
* Allocates a new container for a given keyName.
*

View File

@ -23,6 +23,8 @@
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
/**
* This class is the one that directly talks to SCM server.
@ -61,6 +63,24 @@ public void deleteContainer(Pipeline pipeline, boolean force)
}
/**
* This is a mock class, so returns the pipelines of start container
* and end container.
*
* @param startName start container name.
* @param prefixName prefix container name.
* @param count count.
* @return a list of pipeline.
* @throws IOException
*/
@Override
public List<Pipeline> listContainer(String startName,
String prefixName, int count) throws IOException {
List<Pipeline> dataList = new ArrayList<>();
dataList.add(getContainer(startName));
return dataList;
}
/**
* Create a instance of ContainerData by a given container id,
* since this is a testing class, there is no need set up the hold

View File

@ -352,6 +352,107 @@ public void testNonExistCommand() throws Exception {
System.setOut(init);
}
@Test
public void testListContainerCommand() throws Exception {
// Create 20 containers for testing.
String prefix = "ContainerForTesting";
for (int index = 0; index < 20; index++) {
String containerName = String.format("%s%02d", prefix, index);
Pipeline pipeline = scm.allocateContainer(containerName);
ContainerData data = new ContainerData(containerName);
containerManager.createContainer(pipeline, data);
}
ByteArrayOutputStream out = new ByteArrayOutputStream();
ByteArrayOutputStream err = new ByteArrayOutputStream();
// Test without -start, -prefix and -count
String[] args = new String[] {"-container", "-list"};
int exitCode = runCommandAndGetOutput(args, out, err);
assertEquals(ResultCode.EXECUTION_ERROR, exitCode);
assertTrue(err.toString()
.contains("Expecting container count"));
out.reset();
err.reset();
// Test with -start and -count, the value of -count is negative.
args = new String[] {"-container", "-list",
"-start", prefix + 0, "-count", "-1"};
exitCode = runCommandAndGetOutput(args, out, err);
assertEquals(ResultCode.EXECUTION_ERROR, exitCode);
assertTrue(err.toString()
.contains("-count should not be negative"));
out.reset();
err.reset();
String startName = String.format("%s%02d", prefix, 0);
// Test with -start and -count.
args = new String[] {"-container", "-list", "-start",
startName, "-count", "10"};
exitCode = runCommandAndGetOutput(args, out, err);
assertEquals(ResultCode.SUCCESS, exitCode);
for (int index = 0; index < 10; index++) {
String containerName = String.format("%s%02d", prefix, index);
assertTrue(out.toString().contains(containerName));
}
out.reset();
err.reset();
// Test with -start, -prefix and -count.
startName = String.format("%s%02d", prefix, 0);
String prefixName = String.format("%s0", prefix);
args = new String[] {"-container", "-list", "-start",
startName, "-prefix", prefixName, "-count", "20"};
exitCode = runCommandAndGetOutput(args, out, err);
assertEquals(ResultCode.SUCCESS, exitCode);
for (int index = 0; index < 10; index++) {
String containerName = String.format("%s%02d", prefix, index);
assertTrue(out.toString().contains(containerName));
}
out.reset();
err.reset();
startName = String.format("%s%02d", prefix, 0);
prefixName = String.format("%s0", prefix);
args = new String[] {"-container", "-list", "-start",
startName, "-prefix", prefixName, "-count", "4"};
exitCode = runCommandAndGetOutput(args, out, err);
assertEquals(ResultCode.SUCCESS, exitCode);
for (int index = 0; index < 4; index++) {
String containerName = String.format("%s%02d", prefix, index);
assertTrue(out.toString().contains(containerName));
}
out.reset();
err.reset();
prefixName = String.format("%s0", prefix);
args = new String[] {"-container", "-list",
"-prefix", prefixName, "-count", "6"};
exitCode = runCommandAndGetOutput(args, out, err);
assertEquals(ResultCode.SUCCESS, exitCode);
for (int index = 0; index < 6; index++) {
String containerName = String.format("%s%02d", prefix, index);
assertTrue(out.toString().contains(containerName));
}
out.reset();
err.reset();
// Test with -start and -prefix, while -count doesn't exist.
prefixName = String.format("%s%02d", prefix, 20);
args = new String[] {"-container", "-list", "-start",
startName, "-prefix", prefixName, "-count", "10"};
exitCode = runCommandAndGetOutput(args, out, err);
assertEquals(ResultCode.SUCCESS, exitCode);
assertTrue(out.toString().isEmpty());
}
@Test
public void testHelp() throws Exception {
// TODO : this test assertion may break for every new help entry added
@ -377,7 +478,8 @@ public void testHelp() throws Exception {
"where <commands> can be one of the following\n" +
" -create Create container\n" +
" -delete Delete container\n" +
" -info Info container\n";
" -info Info container\n" +
" -list List container\n";
assertEquals(expected1, testContent.toString());
testContent.reset();
@ -410,6 +512,17 @@ public void testHelp() throws Exception {
assertEquals(expected4, testContent.toString());
testContent.reset();
String[] args5 = {"-container", "-list", "-help"};
assertEquals(ResultCode.SUCCESS, cli.run(args5));
String expected5 =
"usage: hdfs scm -container -list <option>\n" +
"where <option> can be the following\n" +
" -count <arg> Specify count number, required\n" +
" -prefix <arg> Specify prefix container name\n" +
" -start <arg> Specify start container name\n";
assertEquals(expected5, testContent.toString());
testContent.reset();
System.setOut(init);
}
}