You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by si...@apache.org on 2018/10/03 15:29:57 UTC

[bookkeeper] branch master updated: [TABLE SERVICE] [CLI] improve table service related cli commands

This is an automated email from the ASF dual-hosted git repository.

sijie pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git


The following commit(s) were added to refs/heads/master by this push:
     new cb7b091  [TABLE SERVICE] [CLI] improve table service related cli commands
cb7b091 is described below

commit cb7b0916978725e4f42814c9ba5824b5dd1b0209
Author: Sijie Guo <gu...@gmail.com>
AuthorDate: Wed Oct 3 08:29:52 2018 -0700

    [TABLE SERVICE] [CLI] improve table service related cli commands
    
    Descriptions of the changes in this PR:
    
    *Motivation*
    
    Add missing commands for table service related operations.
    
    *Changes*
    
    - Add missing commands such as get/delete
    - Fill the default service uri if it is not provided
    - Define the command ops in `Commands` util class
    
    
    
    
    Author: Sijie Guo <si...@apache.org>
    
    Reviewers: Jia Zhai <None>
    
    This closes #1724 from sijie/improve_console
---
 .../apache/bookkeeper/common/net/ServiceURI.java   |  6 ++
 conf/log4j.cli.properties                          |  5 +-
 .../bookkeeper/stream/cluster/StreamCluster.java   |  5 +-
 .../tests/integration/stream/BkCtlTest.java        |  2 +-
 .../integration/standalone/StandaloneTest.java     |  5 +-
 .../org/apache/bookkeeper/stream/cli/Commands.java | 38 ++++++++++
 .../stream/cli/NamespaceCommandGroup.java          |  4 +
 .../stream/cli/TableAdminCommandGroup.java         |  4 +
 .../bookkeeper/stream/cli/TableCommandGroup.java   |  2 +
 .../stream/cli/commands/AbstractStreamCommand.java | 62 +++++++++++++++
 .../stream/cli/commands/AdminCommand.java          | 24 ++----
 .../stream/cli/commands/ClientCommand.java         | 19 ++---
 .../cli/commands/cluster/InitClusterCommand.java   | 10 ++-
 .../commands/namespace/CreateNamespaceCommand.java | 25 ++++---
 .../commands/namespace/DeleteNamespaceCommand.java | 74 ++++++++++++++++++
 .../commands/namespace/GetNamespaceCommand.java    | 76 +++++++++++++++++++
 .../cli/commands/table/CreateTableCommand.java     | 52 +++++++++----
 .../table/{PutCommand.java => DelCommand.java}     | 37 +++++----
 ...teTableCommand.java => DeleteTableCommand.java} | 56 +++++++-------
 .../stream/cli/commands/table/GetCommand.java      |  3 +-
 .../stream/cli/commands/table/GetTableCommand.java | 87 ++++++++++++++++++++++
 .../cli/commands/table/IncrementCommand.java       |  3 +-
 .../stream/cli/commands/table/PutCommand.java      |  3 +-
 23 files changed, 492 insertions(+), 110 deletions(-)

diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/net/ServiceURI.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/net/ServiceURI.java
index 289c3f3..4e2b158 100644
--- a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/net/ServiceURI.java
+++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/net/ServiceURI.java
@@ -123,6 +123,12 @@ public class ServiceURI {
      */
     public static final String SERVICE_BK = "bk";
 
+    /**
+     * The default local bk service uri.
+     */
+    public static final ServiceURI DEFAULT_LOCAL_STREAM_STORAGE_SERVICE_URI =
+        ServiceURI.create("bk://localhost:4181");
+
     private static final String SERVICE_SEP = "+";
     private static final String SERVICE_DLOG_SEP = "-";
 
diff --git a/conf/log4j.cli.properties b/conf/log4j.cli.properties
index ceb77cc..f90d955 100644
--- a/conf/log4j.cli.properties
+++ b/conf/log4j.cli.properties
@@ -54,7 +54,4 @@ log4j.appender.ROLLINGFILE.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{
 
 log4j.logger.verbose=INFO,VERBOSECONSOLE
 log4j.logger.org.apache.zookeeper=ERROR
-log4j.logger.org.apache.bookkeeper=ERROR
-log4j.logger.org.apache.bookkeeper.tools=INFO
-log4j.logger.org.apache.bookkeeper.bookie.BookieShell=INFO
-log4j.logger.org.apache.bookkeeper.client.BookKeeperAdmin=INFO
+log4j.logger.org.apache.bookkeeper=INFO
diff --git a/stream/server/src/main/java/org/apache/bookkeeper/stream/cluster/StreamCluster.java b/stream/server/src/main/java/org/apache/bookkeeper/stream/cluster/StreamCluster.java
index 407b27f..5490dee 100644
--- a/stream/server/src/main/java/org/apache/bookkeeper/stream/cluster/StreamCluster.java
+++ b/stream/server/src/main/java/org/apache/bookkeeper/stream/cluster/StreamCluster.java
@@ -245,10 +245,11 @@ public class StreamCluster
             boolean created = false;
             while (!created) {
                 try {
-                    result(admin.getNamespace(namespaceName));
+                    NamespaceProperties nsProps = result(admin.getNamespace(namespaceName));
+                    log.info("Namespace '{}':\n{}", namespaceName, nsProps);
                     created = true;
                 } catch (NamespaceNotFoundException nnfe) {
-                    log.info("Namespace '{}' is not found.");
+                    log.info("Namespace '{}' is not found.", namespaceName);
                     log.info("Creating namespace '{}' ...", namespaceName);
                     try {
                         NamespaceProperties nsProps = result(
diff --git a/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/BkCtlTest.java b/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/BkCtlTest.java
index 0bb11a1..6067776 100644
--- a/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/BkCtlTest.java
+++ b/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/BkCtlTest.java
@@ -118,7 +118,7 @@ public class BkCtlTest extends StreamClusterTestBase {
             tableName
         );
         assertTrue(
-            result.getStdout().contains("Successfully created stream '" + tableName + "'"));
+            result.getStdout().contains("Successfully created table '" + tableName + "'"));
     }
 
     //
diff --git a/tests/integration/standalone/src/test/java/org/apache/bookkeeper/tests/integration/standalone/StandaloneTest.java b/tests/integration/standalone/src/test/java/org/apache/bookkeeper/tests/integration/standalone/StandaloneTest.java
index f7b7077..29c30c3 100644
--- a/tests/integration/standalone/src/test/java/org/apache/bookkeeper/tests/integration/standalone/StandaloneTest.java
+++ b/tests/integration/standalone/src/test/java/org/apache/bookkeeper/tests/integration/standalone/StandaloneTest.java
@@ -80,17 +80,20 @@ public class StandaloneTest {
 
     @Test
     public void createTable() throws Exception {
+        createNamespace();
         String tableName = testName.getMethodName();
         ExecResult result = bkContainer.execCmd(
             "/opt/bookkeeper/bin/bkctl",
             "-u bk://localhost:4181",
+            "--namespace",
+            testName.getMethodName(),
             "tables",
             "create",
             tableName
         );
         assertTrue(
             result.getStdout(),
-            result.getStdout().contains("Successfully created stream '" + tableName + "'"));
+            result.getStdout().contains("Successfully created table '" + tableName + "'"));
     }
 
 }
diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/Commands.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/Commands.java
new file mode 100644
index 0000000..2c88c00
--- /dev/null
+++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/Commands.java
@@ -0,0 +1,38 @@
+/*
+ * 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
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.bookkeeper.stream.cli;
+
+/**
+ * Utils for commands.
+ */
+public final class Commands {
+
+    private Commands() {}
+
+    public static final String OP_INIT = "init";
+    public static final String OP_CREATE = "create";
+    public static final String OP_DELETE = "delete";
+    public static final String OP_LIST = "list";
+    public static final String OP_GET = "get";
+    public static final String OP_DEL = "del";
+    public static final String OP_PUT = "put";
+    public static final String OP_INC = "inc";
+
+}
diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/NamespaceCommandGroup.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/NamespaceCommandGroup.java
index be9e062..1300bf2 100644
--- a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/NamespaceCommandGroup.java
+++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/NamespaceCommandGroup.java
@@ -18,6 +18,8 @@
 package org.apache.bookkeeper.stream.cli;
 
 import org.apache.bookkeeper.stream.cli.commands.namespace.CreateNamespaceCommand;
+import org.apache.bookkeeper.stream.cli.commands.namespace.DeleteNamespaceCommand;
+import org.apache.bookkeeper.stream.cli.commands.namespace.GetNamespaceCommand;
 import org.apache.bookkeeper.tools.common.BKFlags;
 import org.apache.bookkeeper.tools.framework.CliCommandGroup;
 import org.apache.bookkeeper.tools.framework.CliSpec;
@@ -35,6 +37,8 @@ public class NamespaceCommandGroup extends CliCommandGroup<BKFlags> {
         .withDescription(DESC)
         .withParent("bkctl")
         .addCommand(new CreateNamespaceCommand())
+        .addCommand(new GetNamespaceCommand())
+        .addCommand(new DeleteNamespaceCommand())
         .build();
 
     public NamespaceCommandGroup() {
diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/TableAdminCommandGroup.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/TableAdminCommandGroup.java
index 60b9892..bce4fc5 100644
--- a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/TableAdminCommandGroup.java
+++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/TableAdminCommandGroup.java
@@ -18,6 +18,8 @@
 package org.apache.bookkeeper.stream.cli;
 
 import org.apache.bookkeeper.stream.cli.commands.table.CreateTableCommand;
+import org.apache.bookkeeper.stream.cli.commands.table.DeleteTableCommand;
+import org.apache.bookkeeper.stream.cli.commands.table.GetTableCommand;
 import org.apache.bookkeeper.tools.common.BKFlags;
 import org.apache.bookkeeper.tools.framework.CliCommandGroup;
 import org.apache.bookkeeper.tools.framework.CliSpec;
@@ -35,6 +37,8 @@ public class TableAdminCommandGroup extends CliCommandGroup<BKFlags> {
         .withDescription(DESC)
         .withParent("bkctl")
         .addCommand(new CreateTableCommand())
+        .addCommand(new GetTableCommand())
+        .addCommand(new DeleteTableCommand())
         .build();
 
     public TableAdminCommandGroup() {
diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/TableCommandGroup.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/TableCommandGroup.java
index 09d3be6..f7d96b7 100644
--- a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/TableCommandGroup.java
+++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/TableCommandGroup.java
@@ -17,6 +17,7 @@
  */
 package org.apache.bookkeeper.stream.cli;
 
+import org.apache.bookkeeper.stream.cli.commands.table.DelCommand;
 import org.apache.bookkeeper.stream.cli.commands.table.GetCommand;
 import org.apache.bookkeeper.stream.cli.commands.table.IncrementCommand;
 import org.apache.bookkeeper.stream.cli.commands.table.PutCommand;
@@ -39,6 +40,7 @@ public class TableCommandGroup extends CliCommandGroup<BKFlags> {
         .addCommand(new PutCommand())
         .addCommand(new GetCommand())
         .addCommand(new IncrementCommand())
+        .addCommand(new DelCommand())
         .build();
 
     public TableCommandGroup() {
diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/AbstractStreamCommand.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/AbstractStreamCommand.java
new file mode 100644
index 0000000..583dcff
--- /dev/null
+++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/AbstractStreamCommand.java
@@ -0,0 +1,62 @@
+/*
+ * 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
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.bookkeeper.stream.cli.commands;
+
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.net.ServiceURI;
+import org.apache.bookkeeper.tools.common.BKCommand;
+import org.apache.bookkeeper.tools.common.BKFlags;
+import org.apache.bookkeeper.tools.framework.CliFlags;
+import org.apache.bookkeeper.tools.framework.CliSpec;
+import org.apache.commons.configuration.CompositeConfiguration;
+
+
+/**
+ * Abstract stream storage related commands.
+ */
+@Slf4j
+abstract class AbstractStreamCommand<CommandFlagsT extends CliFlags> extends BKCommand<CommandFlagsT> {
+
+    protected AbstractStreamCommand(CliSpec<CommandFlagsT> spec) {
+        super(spec);
+    }
+
+    @Override
+    protected boolean acceptServiceUri(ServiceURI serviceURI) {
+        return ServiceURI.SERVICE_BK.equals(serviceURI.getServiceName());
+    }
+
+    @Override
+    protected boolean apply(ServiceURI serviceURI,
+                            CompositeConfiguration conf,
+                            BKFlags globalFlags,
+                            CommandFlagsT cmdFlags) {
+        if (serviceURI == null) {
+            serviceURI = ServiceURI.DEFAULT_LOCAL_STREAM_STORAGE_SERVICE_URI;
+            log.info("Service Uri is not specified. Using default service uri : {}", serviceURI);
+        }
+        return doApply(serviceURI, conf, globalFlags, cmdFlags);
+    }
+
+    protected abstract boolean doApply(ServiceURI serviceURI,
+                                       CompositeConfiguration conf,
+                                       BKFlags globalFlags,
+                                       CommandFlagsT cmdFlags);
+}
diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/AdminCommand.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/AdminCommand.java
index eb319aa..bbe47b7 100644
--- a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/AdminCommand.java
+++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/AdminCommand.java
@@ -17,14 +17,11 @@
  */
 package org.apache.bookkeeper.stream.cli.commands;
 
-import static com.google.common.base.Preconditions.checkArgument;
-
 import lombok.extern.slf4j.Slf4j;
 import org.apache.bookkeeper.clients.StorageClientBuilder;
 import org.apache.bookkeeper.clients.admin.StorageAdminClient;
 import org.apache.bookkeeper.clients.config.StorageClientSettings;
 import org.apache.bookkeeper.common.net.ServiceURI;
-import org.apache.bookkeeper.tools.common.BKCommand;
 import org.apache.bookkeeper.tools.common.BKFlags;
 import org.apache.bookkeeper.tools.framework.CliFlags;
 import org.apache.bookkeeper.tools.framework.CliSpec;
@@ -34,26 +31,17 @@ import org.apache.commons.configuration.CompositeConfiguration;
  * An admin command interface provides a run method to execute admin commands.
  */
 @Slf4j
-public abstract class AdminCommand<ClientFlagsT extends CliFlags> extends BKCommand<ClientFlagsT> {
+public abstract class AdminCommand<ClientFlagsT extends CliFlags> extends AbstractStreamCommand<ClientFlagsT> {
 
     protected AdminCommand(CliSpec<ClientFlagsT> spec) {
         super(spec);
     }
 
     @Override
-    protected boolean acceptServiceUri(ServiceURI serviceURI) {
-        return ServiceURI.SERVICE_BK.equals(serviceURI.getServiceName());
-    }
-
-    @Override
-    protected boolean apply(ServiceURI serviceURI,
-                            CompositeConfiguration conf,
-                            BKFlags bkFlags,
-                            ClientFlagsT cmdFlags) {
-        checkArgument(
-            null != serviceURI,
-            "No service URI is provided");
-
+    protected boolean doApply(ServiceURI serviceURI,
+                              CompositeConfiguration conf,
+                              BKFlags bkFlags,
+                              ClientFlagsT cmdFlags) {
         StorageClientSettings settings = StorageClientSettings.newBuilder()
             .clientName("bkctl")
             .serviceUri(serviceURI.getUri().toString())
@@ -66,6 +54,8 @@ public abstract class AdminCommand<ClientFlagsT extends CliFlags> extends BKComm
             return true;
         } catch (Exception e) {
             log.error("Failed to process stream admin command", e);
+            spec.console().println("Failed to process stream admin command");
+            e.printStackTrace(spec.console());
             return false;
         }
     }
diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/ClientCommand.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/ClientCommand.java
index 03232a3..14ad838 100644
--- a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/ClientCommand.java
+++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/ClientCommand.java
@@ -17,14 +17,11 @@
  */
 package org.apache.bookkeeper.stream.cli.commands;
 
-import static com.google.common.base.Preconditions.checkArgument;
-
 import lombok.extern.slf4j.Slf4j;
 import org.apache.bookkeeper.api.StorageClient;
 import org.apache.bookkeeper.clients.StorageClientBuilder;
 import org.apache.bookkeeper.clients.config.StorageClientSettings;
 import org.apache.bookkeeper.common.net.ServiceURI;
-import org.apache.bookkeeper.tools.common.BKCommand;
 import org.apache.bookkeeper.tools.common.BKFlags;
 import org.apache.bookkeeper.tools.framework.CliFlags;
 import org.apache.bookkeeper.tools.framework.CliSpec;
@@ -34,21 +31,17 @@ import org.apache.commons.configuration.CompositeConfiguration;
  * An admin command interface provides a run method to execute admin commands.
  */
 @Slf4j
-public abstract class ClientCommand<ClientFlagsT extends CliFlags> extends BKCommand<ClientFlagsT> {
+public abstract class ClientCommand<ClientFlagsT extends CliFlags> extends AbstractStreamCommand<ClientFlagsT> {
 
     protected ClientCommand(CliSpec<ClientFlagsT> spec) {
         super(spec);
     }
 
     @Override
-    protected boolean apply(ServiceURI serviceURI,
-                            CompositeConfiguration conf,
-                            BKFlags globalFlags,
-                            ClientFlagsT cmdFlags) {
-        checkArgument(
-            null != serviceURI,
-            "No service uri is provided");
-
+    protected boolean doApply(ServiceURI serviceURI,
+                              CompositeConfiguration conf,
+                              BKFlags globalFlags,
+                              ClientFlagsT cmdFlags) {
         StorageClientSettings settings = StorageClientSettings.newBuilder()
             .clientName("bkctl")
             .serviceUri(serviceURI.getUri().toString())
@@ -63,6 +56,8 @@ public abstract class ClientCommand<ClientFlagsT extends CliFlags> extends BKCom
         } catch (Exception e) {
             log.error("Failed to process commands under namespace '{}'",
                 globalFlags.namespace, e);
+            spec.console().println("Failed to process stream admin command");
+            e.printStackTrace(spec.console());
             return false;
         }
     }
diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/cluster/InitClusterCommand.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/cluster/InitClusterCommand.java
index 2337f00..4ee3d39 100644
--- a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/cluster/InitClusterCommand.java
+++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/cluster/InitClusterCommand.java
@@ -19,6 +19,7 @@
 package org.apache.bookkeeper.stream.cli.commands.cluster;
 
 import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.stream.cli.Commands.OP_INIT;
 
 import com.beust.jcommander.Parameter;
 import com.google.common.base.Strings;
@@ -52,7 +53,7 @@ import org.apache.zookeeper.KeeperException.Code;
 @Slf4j
 public class InitClusterCommand extends BKCommand<Flags> {
 
-    private static final String NAME = "init";
+    private static final String NAME = OP_INIT;
     private static final String DESC = "Init a cluster";
 
     /**
@@ -95,6 +96,7 @@ public class InitClusterCommand extends BKCommand<Flags> {
             .withName(NAME)
             .withDescription(DESC)
             .withFlags(new Flags())
+            .withUsage("bkctl cluster init [flags] <service-uri>")
             .build());
     }
 
@@ -105,14 +107,16 @@ public class InitClusterCommand extends BKCommand<Flags> {
     }
 
     @Override
-    protected boolean apply(ServiceURI serviceURI,
+    protected boolean apply(ServiceURI ignored,
                             CompositeConfiguration conf,
                             BKFlags globalFlags,
                             Flags cmdFlags) {
         checkArgument(
-            null != serviceURI,
+            !cmdFlags.arguments.isEmpty(),
             "No service URI is provided");
 
+        ServiceURI serviceURI = ServiceURI.create(cmdFlags.arguments.get(0));
+
         if (null != cmdFlags.clusterName) {
             checkArgument(
                 !cmdFlags.clusterName.contains("/"),
diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/namespace/CreateNamespaceCommand.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/namespace/CreateNamespaceCommand.java
index 6f5e435..2c57209 100644
--- a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/namespace/CreateNamespaceCommand.java
+++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/namespace/CreateNamespaceCommand.java
@@ -19,9 +19,11 @@ package org.apache.bookkeeper.stream.cli.commands.namespace;
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static org.apache.bookkeeper.common.concurrent.FutureUtils.result;
+import static org.apache.bookkeeper.stream.cli.Commands.OP_CREATE;
 import static org.apache.bookkeeper.stream.protocol.ProtocolConstants.DEFAULT_STREAM_CONF;
 
 import org.apache.bookkeeper.clients.admin.StorageAdminClient;
+import org.apache.bookkeeper.clients.exceptions.NamespaceExistsException;
 import org.apache.bookkeeper.stream.cli.commands.AdminCommand;
 import org.apache.bookkeeper.stream.cli.commands.namespace.CreateNamespaceCommand.Flags;
 import org.apache.bookkeeper.stream.proto.NamespaceConfiguration;
@@ -35,7 +37,7 @@ import org.apache.bookkeeper.tools.framework.CliSpec;
  */
 public class CreateNamespaceCommand extends AdminCommand<Flags> {
 
-    private static final String NAME = "create";
+    private static final String NAME = OP_CREATE;
     private static final String DESC = "Create a namespace";
 
     /**
@@ -62,15 +64,18 @@ public class CreateNamespaceCommand extends AdminCommand<Flags> {
 
         String namespaceName = flags.arguments.get(0);
 
-        spec.console().println("Creating namespace '" + namespaceName + "' ...");
-        NamespaceProperties nsProps = result(
-            admin.createNamespace(
-                namespaceName,
-                NamespaceConfiguration.newBuilder()
-                    .setDefaultStreamConf(DEFAULT_STREAM_CONF)
-                    .build()));
-        spec.console().println("Successfully created namespace '" + namespaceName + "':");
-        spec.console().println(nsProps);
+        try {
+            NamespaceProperties nsProps = result(
+                admin.createNamespace(
+                    namespaceName,
+                    NamespaceConfiguration.newBuilder()
+                        .setDefaultStreamConf(DEFAULT_STREAM_CONF)
+                        .build()));
+            spec.console().println("Successfully created namespace '" + namespaceName + "':");
+            spec.console().println(nsProps);
+        } catch (NamespaceExistsException nee) {
+            spec.console().println("Namespace '" + namespaceName + "' already exists");
+        }
     }
 
 }
diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/namespace/DeleteNamespaceCommand.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/namespace/DeleteNamespaceCommand.java
new file mode 100644
index 0000000..cc10655
--- /dev/null
+++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/namespace/DeleteNamespaceCommand.java
@@ -0,0 +1,74 @@
+/*
+ * 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
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.bookkeeper.stream.cli.commands.namespace;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.common.concurrent.FutureUtils.result;
+import static org.apache.bookkeeper.stream.cli.Commands.OP_DELETE;
+
+import org.apache.bookkeeper.clients.admin.StorageAdminClient;
+import org.apache.bookkeeper.clients.exceptions.NamespaceNotFoundException;
+import org.apache.bookkeeper.stream.cli.commands.AdminCommand;
+import org.apache.bookkeeper.stream.cli.commands.namespace.DeleteNamespaceCommand.Flags;
+import org.apache.bookkeeper.tools.common.BKFlags;
+import org.apache.bookkeeper.tools.framework.CliFlags;
+import org.apache.bookkeeper.tools.framework.CliSpec;
+
+/**
+ * Command to get a namespace.
+ */
+public class DeleteNamespaceCommand extends AdminCommand<Flags> {
+
+    private static final String NAME = OP_DELETE;
+    private static final String DESC = "Delete a namespace";
+
+    /**
+     * Flags for the get namespace command.
+     */
+    public static class Flags extends CliFlags {
+    }
+
+    public DeleteNamespaceCommand() {
+        super(CliSpec.<Flags>newBuilder()
+            .withName(NAME)
+            .withDescription(DESC)
+            .withFlags(new Flags())
+            .withArgumentsUsage("<namespace-name>")
+            .build());
+    }
+
+    @Override
+    protected void run(StorageAdminClient admin,
+                       BKFlags globalFlags,
+                       Flags cmdFlags) throws Exception {
+        checkArgument(!cmdFlags.arguments.isEmpty(),
+            "Namespace name is not provided");
+
+        String namespaceName = cmdFlags.arguments.get(0);
+        try {
+            result(
+                admin.deleteNamespace(namespaceName));
+            spec.console().println("Successfully deleted namespace '" + namespaceName + "'");
+        } catch (NamespaceNotFoundException nfe) {
+            spec.console().println("Namespace '" + namespaceName + "' does not exist");
+        }
+    }
+
+}
diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/namespace/GetNamespaceCommand.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/namespace/GetNamespaceCommand.java
new file mode 100644
index 0000000..b8342f5
--- /dev/null
+++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/namespace/GetNamespaceCommand.java
@@ -0,0 +1,76 @@
+/*
+ * 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
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.bookkeeper.stream.cli.commands.namespace;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.common.concurrent.FutureUtils.result;
+import static org.apache.bookkeeper.stream.cli.Commands.OP_GET;
+
+import org.apache.bookkeeper.clients.admin.StorageAdminClient;
+import org.apache.bookkeeper.clients.exceptions.NamespaceNotFoundException;
+import org.apache.bookkeeper.stream.cli.commands.AdminCommand;
+import org.apache.bookkeeper.stream.cli.commands.namespace.GetNamespaceCommand.Flags;
+import org.apache.bookkeeper.stream.proto.NamespaceProperties;
+import org.apache.bookkeeper.tools.common.BKFlags;
+import org.apache.bookkeeper.tools.framework.CliFlags;
+import org.apache.bookkeeper.tools.framework.CliSpec;
+
+/**
+ * Command to get a namespace.
+ */
+public class GetNamespaceCommand extends AdminCommand<Flags> {
+
+    private static final String NAME = OP_GET;
+    private static final String DESC = "Get the details of a namespace";
+
+    /**
+     * Flags for the get namespace command.
+     */
+    public static class Flags extends CliFlags {
+    }
+
+    public GetNamespaceCommand() {
+        super(CliSpec.<Flags>newBuilder()
+            .withName(NAME)
+            .withDescription(DESC)
+            .withFlags(new Flags())
+            .withArgumentsUsage("<namespace-name>")
+            .build());
+    }
+
+    @Override
+    protected void run(StorageAdminClient admin,
+                       BKFlags globalFlags,
+                       Flags cmdFlags) throws Exception {
+        checkArgument(!cmdFlags.arguments.isEmpty(),
+            "Namespace name is not provided");
+
+        String namespaceName = cmdFlags.arguments.get(0);
+        try {
+            NamespaceProperties ns = result(
+                admin.getNamespace(namespaceName));
+            spec.console().println("Namespace '" + namespaceName + "' :");
+            spec.console().println(ns);
+        } catch (NamespaceNotFoundException nfe) {
+            spec.console().println("Namespace '" + namespaceName + "' does not exist");
+        }
+    }
+
+}
diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/CreateTableCommand.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/CreateTableCommand.java
index 5b50d1f..2979b35 100644
--- a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/CreateTableCommand.java
+++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/CreateTableCommand.java
@@ -19,9 +19,13 @@ package org.apache.bookkeeper.stream.cli.commands.table;
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static org.apache.bookkeeper.common.concurrent.FutureUtils.result;
+import static org.apache.bookkeeper.stream.cli.Commands.OP_CREATE;
 import static org.apache.bookkeeper.stream.protocol.ProtocolConstants.DEFAULT_STREAM_CONF;
 
+import com.beust.jcommander.Parameter;
 import org.apache.bookkeeper.clients.admin.StorageAdminClient;
+import org.apache.bookkeeper.clients.exceptions.NamespaceNotFoundException;
+import org.apache.bookkeeper.clients.exceptions.StreamExistsException;
 import org.apache.bookkeeper.stream.cli.commands.AdminCommand;
 import org.apache.bookkeeper.stream.cli.commands.table.CreateTableCommand.Flags;
 import org.apache.bookkeeper.stream.proto.StorageType;
@@ -36,13 +40,22 @@ import org.apache.bookkeeper.tools.framework.CliSpec;
  */
 public class CreateTableCommand extends AdminCommand<Flags> {
 
-    private static final String NAME = "create";
-    private static final String DESC = "Create a stream";
+    private static final String NAME = OP_CREATE;
+    private static final String DESC = "Create a table";
 
     /**
      * Flags for the create table command.
      */
     public static class Flags extends CliFlags {
+
+        @Parameter(
+            names = {
+                "-r", "--num-ranges"
+            },
+            description = "num of ranges for the table"
+        )
+        private int numRanges = 3;
+
     }
 
     public CreateTableCommand() {
@@ -50,7 +63,7 @@ public class CreateTableCommand extends AdminCommand<Flags> {
             .withName(NAME)
             .withDescription(DESC)
             .withFlags(new Flags())
-            .withArgumentsUsage("<stream-name>")
+            .withArgumentsUsage("<table-name>")
             .build());
     }
 
@@ -59,20 +72,31 @@ public class CreateTableCommand extends AdminCommand<Flags> {
                        BKFlags globalFlags,
                        Flags flags) throws Exception {
         checkArgument(!flags.arguments.isEmpty(),
-            "Stream name is not provided");
+            "Table name is not provided");
+        checkArgument(flags.numRanges >= 1,
+            "Invalid number of ranges specified for creating table : " + flags.numRanges);
 
         String streamName = flags.arguments.get(0);
 
-        spec.console().println("Creating stream '" + streamName + "' ...");
-        StreamProperties nsProps = result(
-            admin.createStream(
-                globalFlags.namespace,
-                streamName,
-                StreamConfiguration.newBuilder(DEFAULT_STREAM_CONF)
-                    .setStorageType(StorageType.TABLE)
-                    .build()));
-        spec.console().println("Successfully created stream '" + streamName + "':");
-        spec.console().println(nsProps);
+        try {
+            StreamConfiguration conf = StreamConfiguration.newBuilder(DEFAULT_STREAM_CONF)
+                .setMinNumRanges(flags.numRanges)
+                .setInitialNumRanges(flags.numRanges)
+                .build();
+            StreamProperties nsProps = result(
+                admin.createStream(
+                    globalFlags.namespace,
+                    streamName,
+                    StreamConfiguration.newBuilder(conf)
+                        .setStorageType(StorageType.TABLE)
+                        .build()));
+            spec.console().println("Successfully created table '" + streamName + "':");
+            spec.console().println(nsProps);
+        } catch (NamespaceNotFoundException nfe) {
+            spec.console().println("Namespace '" + globalFlags.namespace + "' does not exist");
+        } catch (StreamExistsException see) {
+            spec.console().println("Table '" + globalFlags.namespace + "' already exists");
+        }
     }
 
 }
diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/PutCommand.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/DelCommand.java
similarity index 63%
copy from tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/PutCommand.java
copy to tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/DelCommand.java
index 5c89c66..44d17d4 100644
--- a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/PutCommand.java
+++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/DelCommand.java
@@ -7,35 +7,37 @@
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
  *
- *     http://www.apache.org/licenses/LICENSE-2.0
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * 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.
+ * 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.bookkeeper.stream.cli.commands.table;
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.bookkeeper.common.concurrent.FutureUtils.result;
+import static org.apache.bookkeeper.stream.cli.Commands.OP_DEL;
 
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.Unpooled;
 import org.apache.bookkeeper.api.StorageClient;
 import org.apache.bookkeeper.api.kv.Table;
 import org.apache.bookkeeper.stream.cli.commands.ClientCommand;
-import org.apache.bookkeeper.stream.cli.commands.table.PutCommand.Flags;
+import org.apache.bookkeeper.stream.cli.commands.table.DelCommand.Flags;
 import org.apache.bookkeeper.tools.framework.CliFlags;
 import org.apache.bookkeeper.tools.framework.CliSpec;
 
 /**
  * Commands to put kvs.
  */
-public class PutCommand extends ClientCommand<Flags> {
+public class DelCommand extends ClientCommand<Flags> {
 
-    private static final String NAME = "put";
+    private static final String NAME = OP_DEL;
     private static final String DESC = "Put key/value pair to a table";
 
     /**
@@ -44,7 +46,7 @@ public class PutCommand extends ClientCommand<Flags> {
     public static class Flags extends CliFlags {
     }
 
-    public PutCommand() {
+    public DelCommand() {
         super(CliSpec.<Flags>newBuilder()
             .withName(NAME)
             .withDescription(DESC)
@@ -55,18 +57,21 @@ public class PutCommand extends ClientCommand<Flags> {
 
     @Override
     protected void run(StorageClient client, Flags flags) throws Exception {
-        checkArgument(flags.arguments.size() >= 3,
+        checkArgument(flags.arguments.size() >= 2,
             "table and key/value are not provided");
 
         String tableName = flags.arguments.get(0);
         String key = flags.arguments.get(1);
-        String value = flags.arguments.get(2);
 
         try (Table<ByteBuf, ByteBuf> table = result(client.openTable(tableName))) {
-            result(table.put(
-                Unpooled.wrappedBuffer(key.getBytes(UTF_8)),
-                Unpooled.wrappedBuffer(value.getBytes(UTF_8))));
-            spec.console().println("Successfully update kv: ('" + key + "', '" + value + "').");
+            ByteBuf value = result(table.delete(
+                Unpooled.wrappedBuffer(key.getBytes(UTF_8))));
+            if (null != value) {
+                value.release();
+                spec.console().println("Successfully deleted key: ('" + key + "').");
+            } else {
+                spec.console().println("key '" + key + "' doesn't exist.");
+            }
         }
     }
 
diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/CreateTableCommand.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/DeleteTableCommand.java
similarity index 50%
copy from tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/CreateTableCommand.java
copy to tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/DeleteTableCommand.java
index 5b50d1f..3022ca2 100644
--- a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/CreateTableCommand.java
+++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/DeleteTableCommand.java
@@ -7,26 +7,26 @@
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
  *
- *     http://www.apache.org/licenses/LICENSE-2.0
+ *   http://www.apache.org/licenses/LICENSE-2.0
  *
- * 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.
+ * 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.bookkeeper.stream.cli.commands.table;
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static org.apache.bookkeeper.common.concurrent.FutureUtils.result;
-import static org.apache.bookkeeper.stream.protocol.ProtocolConstants.DEFAULT_STREAM_CONF;
+import static org.apache.bookkeeper.stream.cli.Commands.OP_DELETE;
 
 import org.apache.bookkeeper.clients.admin.StorageAdminClient;
+import org.apache.bookkeeper.clients.exceptions.NamespaceNotFoundException;
+import org.apache.bookkeeper.clients.exceptions.StreamNotFoundException;
 import org.apache.bookkeeper.stream.cli.commands.AdminCommand;
-import org.apache.bookkeeper.stream.cli.commands.table.CreateTableCommand.Flags;
-import org.apache.bookkeeper.stream.proto.StorageType;
-import org.apache.bookkeeper.stream.proto.StreamConfiguration;
-import org.apache.bookkeeper.stream.proto.StreamProperties;
+import org.apache.bookkeeper.stream.cli.commands.table.DeleteTableCommand.Flags;
 import org.apache.bookkeeper.tools.common.BKFlags;
 import org.apache.bookkeeper.tools.framework.CliFlags;
 import org.apache.bookkeeper.tools.framework.CliSpec;
@@ -34,10 +34,10 @@ import org.apache.bookkeeper.tools.framework.CliSpec;
 /**
  * Command to create a namespace.
  */
-public class CreateTableCommand extends AdminCommand<Flags> {
+public class DeleteTableCommand extends AdminCommand<Flags> {
 
-    private static final String NAME = "create";
-    private static final String DESC = "Create a stream";
+    private static final String NAME = OP_DELETE;
+    private static final String DESC = "Delete a table";
 
     /**
      * Flags for the create table command.
@@ -45,12 +45,12 @@ public class CreateTableCommand extends AdminCommand<Flags> {
     public static class Flags extends CliFlags {
     }
 
-    public CreateTableCommand() {
+    public DeleteTableCommand() {
         super(CliSpec.<Flags>newBuilder()
             .withName(NAME)
             .withDescription(DESC)
             .withFlags(new Flags())
-            .withArgumentsUsage("<stream-name>")
+            .withArgumentsUsage("<table-name>")
             .build());
     }
 
@@ -59,20 +59,22 @@ public class CreateTableCommand extends AdminCommand<Flags> {
                        BKFlags globalFlags,
                        Flags flags) throws Exception {
         checkArgument(!flags.arguments.isEmpty(),
-            "Stream name is not provided");
+            "Table name is not provided");
 
         String streamName = flags.arguments.get(0);
 
-        spec.console().println("Creating stream '" + streamName + "' ...");
-        StreamProperties nsProps = result(
-            admin.createStream(
-                globalFlags.namespace,
-                streamName,
-                StreamConfiguration.newBuilder(DEFAULT_STREAM_CONF)
-                    .setStorageType(StorageType.TABLE)
-                    .build()));
-        spec.console().println("Successfully created stream '" + streamName + "':");
-        spec.console().println(nsProps);
+        try {
+            result(
+                admin.deleteStream(
+                    globalFlags.namespace,
+                    streamName));
+            spec.console().println("Successfully deleted table '" + streamName
+                + "' from namespace '" + globalFlags.namespace + "'");
+        } catch (NamespaceNotFoundException nfe) {
+            spec.console().println("Namespace '" + globalFlags.namespace + "' does not exist");
+        } catch (StreamNotFoundException snfe) {
+            spec.console().println("Table '" + streamName + "' does not exist");
+        }
     }
 
 }
diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/GetCommand.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/GetCommand.java
index b7090fc..093b669 100644
--- a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/GetCommand.java
+++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/GetCommand.java
@@ -20,6 +20,7 @@ package org.apache.bookkeeper.stream.cli.commands.table;
 import static com.google.common.base.Preconditions.checkArgument;
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.bookkeeper.common.concurrent.FutureUtils.result;
+import static org.apache.bookkeeper.stream.cli.Commands.OP_GET;
 
 import com.beust.jcommander.Parameter;
 import io.netty.buffer.ByteBuf;
@@ -38,7 +39,7 @@ import org.apache.bookkeeper.tools.framework.CliSpec;
  */
 public class GetCommand extends ClientCommand<Flags> {
 
-    private static final String NAME = "get";
+    private static final String NAME = OP_GET;
     private static final String DESC = "Get key/value pair from a table";
 
     /**
diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/GetTableCommand.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/GetTableCommand.java
new file mode 100644
index 0000000..bc4b461
--- /dev/null
+++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/GetTableCommand.java
@@ -0,0 +1,87 @@
+/*
+ * 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
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.bookkeeper.stream.cli.commands.table;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.bookkeeper.common.concurrent.FutureUtils.result;
+import static org.apache.bookkeeper.stream.cli.Commands.OP_GET;
+
+import org.apache.bookkeeper.clients.admin.StorageAdminClient;
+import org.apache.bookkeeper.clients.exceptions.NamespaceNotFoundException;
+import org.apache.bookkeeper.clients.exceptions.StreamNotFoundException;
+import org.apache.bookkeeper.stream.cli.commands.AdminCommand;
+import org.apache.bookkeeper.stream.cli.commands.table.GetTableCommand.Flags;
+import org.apache.bookkeeper.stream.proto.StorageType;
+import org.apache.bookkeeper.stream.proto.StreamProperties;
+import org.apache.bookkeeper.tools.common.BKFlags;
+import org.apache.bookkeeper.tools.framework.CliFlags;
+import org.apache.bookkeeper.tools.framework.CliSpec;
+
+/**
+ * Command to create a namespace.
+ */
+public class GetTableCommand extends AdminCommand<Flags> {
+
+    private static final String NAME = OP_GET;
+    private static final String DESC = "Get the details of a table";
+
+    /**
+     * Flags for the create table command.
+     */
+    public static class Flags extends CliFlags {
+    }
+
+    public GetTableCommand() {
+        super(CliSpec.<Flags>newBuilder()
+            .withName(NAME)
+            .withDescription(DESC)
+            .withFlags(new Flags())
+            .withArgumentsUsage("<table-name>")
+            .build());
+    }
+
+    @Override
+    protected void run(StorageAdminClient admin,
+                       BKFlags globalFlags,
+                       Flags flags) throws Exception {
+        checkArgument(!flags.arguments.isEmpty(),
+            "Table name is not provided");
+
+        String streamName = flags.arguments.get(0);
+
+        try {
+            StreamProperties streamProps = result(
+                admin.getStream(
+                    globalFlags.namespace,
+                    streamName));
+            if (streamProps.getStreamConf().getStorageType() != StorageType.TABLE) {
+                spec.console().println("'" + streamName + "' is not a table storage entity, its storage type is "
+                    + streamProps.getStreamConf().getStorageType());
+            } else {
+                spec.console().println("Table '" + streamName + "' :");
+            }
+            spec.console().println(streamProps);
+        } catch (NamespaceNotFoundException nfe) {
+            spec.console().println("Namespace '" + globalFlags.namespace + "' does not exist");
+        } catch (StreamNotFoundException snfe) {
+            spec.console().println("Table '" + streamName + "' does not exist");
+        }
+    }
+
+}
diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/IncrementCommand.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/IncrementCommand.java
index 0e8f2e5..25a2fb5 100644
--- a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/IncrementCommand.java
+++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/IncrementCommand.java
@@ -20,6 +20,7 @@ package org.apache.bookkeeper.stream.cli.commands.table;
 import static com.google.common.base.Preconditions.checkArgument;
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.bookkeeper.common.concurrent.FutureUtils.result;
+import static org.apache.bookkeeper.stream.cli.Commands.OP_INC;
 
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.Unpooled;
@@ -35,7 +36,7 @@ import org.apache.bookkeeper.tools.framework.CliSpec;
  */
 public class IncrementCommand extends ClientCommand<Flags> {
 
-    private static final String NAME = "inc";
+    private static final String NAME = OP_INC;
     private static final String DESC = "Increment the amount of a key in a table";
 
     /**
diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/PutCommand.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/PutCommand.java
index 5c89c66..b43f91e 100644
--- a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/PutCommand.java
+++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/PutCommand.java
@@ -20,6 +20,7 @@ package org.apache.bookkeeper.stream.cli.commands.table;
 import static com.google.common.base.Preconditions.checkArgument;
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.bookkeeper.common.concurrent.FutureUtils.result;
+import static org.apache.bookkeeper.stream.cli.Commands.OP_PUT;
 
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.Unpooled;
@@ -35,7 +36,7 @@ import org.apache.bookkeeper.tools.framework.CliSpec;
  */
 public class PutCommand extends ClientCommand<Flags> {
 
-    private static final String NAME = "put";
+    private static final String NAME = OP_PUT;
     private static final String DESC = "Put key/value pair to a table";
 
     /**