Browse Source

HADOOP-1720 Addition of HQL (Hbase Query Language) support in Hbase Shell

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk@575156 13f79535-47bb-0310-9956-ffa450edef68
Michael Stack 17 years ago
parent
commit
6550e89e6a
34 changed files with 2804 additions and 1178 deletions
  1. 5 0
      src/contrib/hbase/CHANGES.txt
  2. 7 4
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java
  3. 1 1
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegiondirReader.java
  4. 12 2
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/HTableDescriptor.java
  5. 7 7
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/Shell.java
  6. 127 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/AlterCommand.java
  7. 18 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/BasicCommand.java
  8. 4 3
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/ClearCommand.java
  9. 4 3
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/Command.java
  10. 0 2
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/CommandFactory.java
  11. 64 97
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/ConsoleTable.java
  12. 37 39
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/CreateCommand.java
  13. 56 36
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/DeleteCommand.java
  14. 19 20
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/DescCommand.java
  15. 51 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/DisableCommand.java
  16. 19 12
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/DropCommand.java
  17. 50 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/EnableCommand.java
  18. 3 2
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/ExitCommand.java
  19. 4 1
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/FsCommand.java
  20. 461 247
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/HBaseShell.jj
  21. 1 3
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/HelpCommand.java
  22. 63 22
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/HelpContents.java
  23. 8 7
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/HelpManager.java
  24. 18 12
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/InsertCommand.java
  25. 6 4
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/ReturnMsg.java
  26. 106 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/SchemaModificationCommand.java
  27. 128 153
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/SelectCommand.java
  28. 3 1
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/ShowCommand.java
  29. 587 253
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/Parser.java
  30. 86 57
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/ParserConstants.java
  31. 653 165
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/ParserTokenManager.java
  32. 14 14
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestToString.java
  33. 41 0
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/shell/TestConsoleTable.java
  34. 141 11
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/shell/TestHBaseShell.java

+ 5 - 0
src/contrib/hbase/CHANGES.txt

@@ -13,6 +13,11 @@ Trunk (unreleased changes)
                 older than passed timestamp.  Fixed compaction so deleted cells
                 do not make it out into compacted output.  Ensure also that
                 versions > column max are dropped compacting.
+    HADOOP-1720 Addition of HQL (Hbase Query Language) support in Hbase Shell.
+                The old shell syntax has been replaced by HQL, a small SQL-like
+                set of operators, for creating, altering, dropping, inserting,
+                deleting, and selecting, etc., data in hbase.
+                (Inchul Song and Edward Yoon via Stack)
 
   OPTIMIZATIONS
 

+ 7 - 4
src/contrib/hbase/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java

@@ -214,10 +214,13 @@ public class HColumnDescriptor implements WritableComparable {
   /** {@inheritDoc} */
   @Override
   public String toString() {
-    return "(" + name + ", max versions: " + maxVersions + ", compression: "
-      + this.compressionType + ", in memory: " + inMemory +
-      ", max value length: " + maxValueLength + ", bloom filter: "
-      + (bloomFilterSpecified ? bloomFilter.toString() : "none") + ")";
+    // Output a name minus ':'.
+    String tmp = name.toString();
+    return "{name: " + tmp.substring(0, tmp.length() - 1) +
+      ", max versions: " + maxVersions +
+      ", compression: " + this.compressionType + ", in memory: " + inMemory +
+      ", max length: " + maxValueLength + ", bloom filter: " +
+      (bloomFilterSpecified ? bloomFilter.toString() : "none") + "}";
   }
   
   /** {@inheritDoc} */

+ 1 - 1
src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegiondirReader.java

@@ -171,7 +171,7 @@ class HRegiondirReader {
     HRegion r = new HRegion(this.parentdir, null,
         FileSystem.get(this.conf), conf, info, null);
     Text [] families = info.tableDesc.families().keySet().toArray(
-        new Text [info.tableDesc.families.size()]);
+        new Text [info.tableDesc.getFamilies().size()]);
     HInternalScannerInterface scanner =
       r.getScanner(families, new Text(), System.currentTimeMillis(), null);
     

+ 12 - 2
src/contrib/hbase/src/java/org/apache/hadoop/hbase/HTableDescriptor.java

@@ -22,8 +22,10 @@ package org.apache.hadoop.hbase;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.Map;
+import java.util.SortedMap;
 import java.util.TreeMap;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
@@ -36,8 +38,9 @@ import org.apache.hadoop.io.WritableComparable;
  * column families.
  */
 public class HTableDescriptor implements WritableComparable {
-  Text name;
-  TreeMap<Text, HColumnDescriptor> families;
+  private Text name;
+  // TODO: Does this need to be a treemap?  Can it be a HashMap?
+  private final TreeMap<Text, HColumnDescriptor> families;
   
   /*
    * Legal table names can only contain 'word characters':
@@ -183,4 +186,11 @@ public class HTableDescriptor implements WritableComparable {
     }
     return result;
   }
+
+  /**
+   * @return Immutable sorted map of families.
+   */
+  public SortedMap<Text, HColumnDescriptor> getFamilies() {
+    return Collections.unmodifiableSortedMap(this.families);
+  }
 }

+ 7 - 7
src/contrib/hbase/src/java/org/apache/hadoop/hbase/Shell.java

@@ -34,18 +34,19 @@ import org.apache.hadoop.hbase.shell.generated.TokenMgrError;
 /**
  * An hbase shell.
  * 
- * @see <a href="http://wiki.apache.org/lucene-hadoop/Hbase/HbaseShell">HBaseShell</a>
+ * @see <a href="http://wiki.apache.org/lucene-hadoop/Hbase/HbaseShell">HbaseShell</a>
  */
 public class Shell {
   /** audible keyboard bells */
   public static final boolean DEFAULT_BELL_ENABLED = true;
 
-  /** Main method
-   * 
+  /**
+   * Main method
    * @param args not used
    * @throws IOException
    */
-  public static void main(@SuppressWarnings("unused") String args[]) throws IOException {
+  public static void main(@SuppressWarnings("unused") String args[])
+  throws IOException {
     Configuration conf = new HBaseConfiguration();
     ConsoleReader reader = new ConsoleReader();
     reader.setBellEnabled(conf.getBoolean("hbaseshell.jline.bell.enabled",
@@ -73,7 +74,6 @@ public class Shell {
         }
 
         long end = System.currentTimeMillis();
-
         if (rs != null && rs.getType() > -1)
           System.out.println(rs.getMsg()
               + executeTime((rs.getType() == 1), start, end));
@@ -92,7 +92,7 @@ public class Shell {
 
   /** Return the string of prompt start string */
   private static String getPrompt(final StringBuilder queryStr) {
-    return (queryStr.toString().equals("")) ? "HBase > " : "    --> ";
+    return (queryStr.toString().equals("")) ? "Hbase > " : "    --> ";
   }
 
   /**
@@ -105,4 +105,4 @@ public class Shell {
         "(" + String.format("%.2f", (end - start) * 0.001) + " sec)" :
           "";
   }
-}
+}

+ 127 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/AlterCommand.java

@@ -0,0 +1,127 @@
+/**
+ * Copyright 2007 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.shell;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseAdmin;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.io.Text;
+
+/**
+ * Alters tables.
+ */
+public class AlterCommand extends SchemaModificationCommand {
+  
+  public enum OperationType {ADD, DROP, CHANGE, NOOP}
+  private OperationType operationType = OperationType.NOOP;
+  private Map<String, Map<String, Object>> columnSpecMap =
+    new HashMap<String, Map<String, Object>>();
+  private String table;
+  private String column; // column to be dropped
+
+  public ReturnMsg execute(Configuration conf) {
+    try {
+      HBaseAdmin admin = new HBaseAdmin(conf);
+      Set<String> columns = null;
+      HColumnDescriptor columnDesc = null;
+      switch (operationType) {
+      case ADD:
+        disableTable(admin, table);
+        columns = columnSpecMap.keySet();
+        for (String c : columns) {
+          columnDesc = getColumnDescriptor(c, columnSpecMap.get(c));
+          System.out.println("Adding " + c + " to " + table +
+            "... Please wait.");
+          admin.addColumn(new Text(table), columnDesc);
+        }
+        enableTable(admin, table);
+        break;
+      case DROP:
+        disableTable(admin, table);
+        System.out.println("Dropping " + column + " from " + table +
+          "... Please wait.");
+        column = appendDelimiter(column);
+        admin.deleteColumn(new Text(table), new Text(column));
+        enableTable(admin, table);
+        break;
+      case CHANGE:
+        // Not yet supported
+        return new ReturnMsg(0, "" + operationType + " is not yet supported.");
+      case NOOP:
+        return new ReturnMsg(0, "Invalid operation type.");
+      }
+      return new ReturnMsg(0, "Table altered successfully.");
+    } catch (Exception e) {
+      return new ReturnMsg(0, extractErrMsg(e));
+    }
+  }
+
+  private void disableTable(HBaseAdmin admin, String t) throws IOException {
+    System.out.println("Disabling " + t + "... Please wait.");
+    admin.disableTable(new Text(t));
+  }
+
+  private void enableTable(HBaseAdmin admin, String t) throws IOException {
+    System.out.println("Enabling " + t + "... Please wait.");
+    admin.enableTable(new Text(t));
+  }
+
+  /**
+   * Sets the table to be altered.
+   * 
+   * @param t Table to be altered.
+   */
+  public void setTable(String t) {
+    this.table = t;
+  }
+
+  /**
+   * Adds a column specification.
+   * 
+   * @param columnSpec Column specification
+   */
+  public void addColumnSpec(String c, Map<String, Object> columnSpec) {
+    columnSpecMap.put(c, columnSpec);
+  }
+
+  /**
+   * Sets the column to be dropped. Only applicable to the DROP operation.
+   * 
+   * @param c Column to be dropped.
+   */
+  public void setColumn(String c) {
+    this.column = c;
+  }
+
+  /**
+   * Sets the operation type of this alteration.
+   * 
+   * @param operationType Operation type
+   * @see OperationType
+   */
+  public void setOperationType(OperationType operationType) {
+    this.operationType = operationType;
+  }
+}

+ 18 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/BasicCommand.java

@@ -33,4 +33,22 @@ public abstract class BasicCommand implements Command, CommandFactory {
     return this;
   }
   
+  protected String extractErrMsg(String msg) {
+    int index = msg.indexOf(":");
+    int eofIndex = msg.indexOf("\n");
+    return msg.substring(index + 1, eofIndex);
+  }
+  
+  protected String extractErrMsg(Exception e) {
+    return extractErrMsg(e.getMessage());
+  }
+ 
+  /**
+   * Appends, if it does not exist, a delimiter (colon) 
+   * at the end of the column name.
+   */
+  protected String appendDelimiter(String column) {
+    return (!column.endsWith(FAMILY_INDICATOR))?
+      column + FAMILY_INDICATOR: column;
+  } 
 }

+ 4 - 3
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/ClearCommand.java

@@ -23,9 +23,11 @@ import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
 
+/**
+ * Clears the console screen. 
+ */
 public class ClearCommand extends BasicCommand {
-
-  public ReturnMsg execute(Configuration conf) {
+  public ReturnMsg execute(@SuppressWarnings("unused") Configuration conf) {
     clear();
     return null;
   }
@@ -42,5 +44,4 @@ public class ClearCommand extends BasicCommand {
       System.out.print("\033c");
     }
   }
-
 }

+ 4 - 3
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/Command.java

@@ -22,11 +22,12 @@ package org.apache.hadoop.hbase.shell;
 import org.apache.hadoop.conf.Configuration;
 
 public interface Command {
-  
   /** family indicator */
   public static final String FAMILY_INDICATOR = ":";
 
-  /** Execute a command */
+  /** Execute a command
+   * @param conf Configuration
+   * @return Result of command execution
+   */
   public ReturnMsg execute(Configuration conf);
-  
 }

+ 0 - 2
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/CommandFactory.java

@@ -23,7 +23,5 @@ package org.apache.hadoop.hbase.shell;
  * Parser uses command factories to create command.
  */
 public interface CommandFactory {
-  
   Command getCommand();
-  
 }

+ 64 - 97
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/ConsoleTable.java

@@ -26,7 +26,11 @@ import java.io.UnsupportedEncodingException;
  * Manufactures console table, but stupid.
  */
 public class ConsoleTable {
+  
   private static PrintStream out;
+  private static final String sBar = "+------+----------------------+";
+  private static final String lBar = "----------------------+----------------------+";
+  
   static {
     try {
       out = new PrintStream(System.out, true, "UTF-8");
@@ -36,29 +40,24 @@ public class ConsoleTable {
   }
   
   public static void printHead(String name) {
-    out.println("+------+----------------------+");
+    out.println(sBar);
     out.print("| No.  | ");
-    out.printf("%-20s", name);
-    out.println(" |");
+    printCell(name, " |", true);
   }
 
   public static void printFoot() {
-    out.println("+------+----------------------+");
+    out.println(sBar);
     out.println();
   }
 
   public static void printTable(int count, String name) {
-    out.println("+------+----------------------+");
-
+    out.println(sBar);
     if (name.length() > 20) {
       int interval = 20;
-
       out.print("| ");
-      out.printf("%-4s", count + 1);
+      out.printf("%-4s", Integer.valueOf(count + 1));
       out.print(" | ");
-      out.printf("%-20s", name.substring(0, interval));
-      out.println(" |");
-
+      printCell(name.substring(0, interval), " |", true);
       for (int i = 0; i < name.length() / interval; i++) {
         out.print("| ");
         out.printf("%-4s", "");
@@ -66,64 +65,42 @@ public class ConsoleTable {
 
         int end = ((interval * i) + interval + interval);
         if (end > name.length()) {
-          out.printf("%-20s", name.substring(end - interval,
-            name.length()));
+          printCell(name.substring(end - interval, name.length()), " |", true);
         } else {
-          out.printf("%-20s", name.substring(end - interval, end));
+          printCell(name.substring(end - interval, end), " |", true);
         }
-        out.println(" |");
       }
-
     } else {
       out.print("| ");
-      out.printf("%-4s", count + 1);
+      out.printf("%-4s", Integer.valueOf(count + 1));
       out.print(" | ");
-      out.printf("%-20s", name);
-      out.println(" |");
+      printCell(name, " |", true);
     }
   }
 
   public static void selectHead() {
-    out.println("+------+----------------------+" +
-      "----------------------+----------------------+");
+    out.println(sBar + lBar);
     out.print("| No.  | ");
-    out.printf("%-20s", "Row");
-    out.printf(" | ");
-    out.printf("%-20s", "Column");
-    out.printf(" | ");
-    out.printf("%-20s", "Cell");
-    out.println(" | ");
+    printCell("Row", " | ", false);
+    printCell("Column", " | ", false);
+    printCell("Cell", " | ", true);
   }
 
   public static void printLine(int count, String key, String column,
       String cellData) {
-    out.println("+------+----------------------+" +
-      "----------------------+----------------------+");
-
+    out.println(sBar + lBar);
     if (key.length() > 20 || column.length() > 20 || cellData.length() > 20) {
       int interval = 20;
       out.print("| ");
-      out.printf("%-4s", count + 1);
-      out.print(" | ");
-      if (key.length() > 20)
-        out.printf("%-20s", key.substring(0, interval));
-      else
-        out.printf("%-20s", key);
-      out.print(" | ");
-      if (column.length() > 20)
-        out.printf("%-20s", column.substring(0, interval));
-      else
-        out.printf("%-20s", column);
+      out.printf("%-4s", Integer.valueOf(count + 1));
       out.print(" | ");
-      if (cellData.length() > 20)
-        out.printf("%-20s", cellData.substring(0, interval));
-      else
-        out.printf("%-20s", cellData);
-      out.println(" |");
 
-      // out.println(getBiggerInt(new int[]{ 3, 1, 9}));
+      printLongCell(key, interval);
+      printLongCell(column, interval);
+      printLongCell(cellData, interval);
+
       int biggerStrLength = getBiggerInt(new int[] { key.length(),
-        column.length(), cellData.length() });
+          column.length(), cellData.length() });
 
       for (int i = 0; i < (biggerStrLength / interval); i++) {
         out.print("| ");
@@ -132,58 +109,50 @@ public class ConsoleTable {
 
         int end = ((interval * i) + interval + interval);
 
-        if (end > key.length()) {
-          if (key.length() > interval && end - interval < key.length()) {
-            out.printf("%-20s", key.substring(end - interval,
-              key.length()));
-          } else {
-            out.printf("%-20s", "");
-          }
-        } else {
-          out.printf("%-20s", key.substring(end - interval, end));
-        }
-
-        out.print(" | ");
-
-        if (end > column.length()) {
-          if (column.length() > interval && end - interval < column.length()) {
-            out.printf("%-20s", column.substring(end - interval,
-              column.length()));
-          } else {
-            out.printf("%-20s", "");
-          }
-        } else {
-          out.printf("%-20s", column.substring(end - interval, end));
-        }
-
-        out.print(" | ");
-        if (end > cellData.length()) {
-          if (cellData.length() > interval &&
-              end - interval < cellData.length()) {
-            out.printf("%-20s",
-              cellData.substring(end - interval, cellData.length()));
-          } else {
-            out.printf("%-20s", "");
-          }
-        } else {
-          out.printf("%-20s", cellData.substring(end - interval, end));
-        }
-        out.println(" |");
+        printLongCellData(key, end, interval, false);
+        printLongCellData(column, end, interval, false);
+        printLongCellData(cellData, end, interval, false);
       }
-
     } else {
       out.print("| ");
-      out.printf("%-4s", count + 1);
+      out.printf("%-4s", Integer.valueOf(count + 1));
       out.print(" | ");
-      out.printf("%-20s", key);
-      out.print(" | ");
-      out.printf("%-20s", column);
-      out.print(" | ");
-      out.printf("%-20s", cellData);
-      out.println(" |");
+      printCell(key, " | ", false);
+      printCell(column, " | ", false);
+      printCell(cellData, " |", true);
     }
   }
 
+  private static void printLongCellData(String key, int end, int interval,
+      boolean newLine) {
+    if (end > key.length()) {
+      if (key.length() > interval && end - interval < key.length()) {
+        out.printf("%-20s", key.substring(end - interval, key.length()));
+      } else {
+        out.printf("%-20s", "");
+      }
+    } else {
+      out.printf("%-20s", key.substring(end - interval, end));
+    }
+    out.print(" | ");
+    if (newLine)
+      out.println();
+  }
+
+  private static void printLongCell(String iKey, int interval) {
+    if (iKey.length() > 20)
+      printCell(iKey.substring(0, interval), " | ", true);
+    else
+      printCell(iKey, " | ", true);
+  }
+
+  private static void printCell(String data, String end, boolean newLine) {
+    out.printf("%-20s", data);
+    out.printf(end);
+    if (newLine)
+      out.println();
+  }
+
   public static int getBiggerInt(int[] integers) {
     int result = -1;
     for (int i = 0; i < integers.length; i++) {
@@ -195,9 +164,7 @@ public class ConsoleTable {
   }
 
   public static void selectFoot() {
-    out.println("+------+----------------------+" +
-      "----------------------+----------------------+");
+    out.println(sBar + lBar);
     out.println();
   }
-  
-}
+}

+ 37 - 39
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/CreateCommand.java

@@ -19,60 +19,58 @@
  */
 package org.apache.hadoop.hbase.shell;
 
-import java.io.IOException;
-import java.util.List;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseAdmin;
 import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConnection;
-import org.apache.hadoop.hbase.HConnectionManager;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.io.Text;
 
-public class CreateCommand extends BasicCommand {
+/**
+ * Creates tables.
+ */
+public class CreateCommand extends SchemaModificationCommand {
   
-  private Text table;
-  private List<String> columnfamilies;
-  @SuppressWarnings("unused")
-  private int limit;
+  private String tableName;
+  private Map<String, Map<String, Object>> columnSpecMap =
+    new HashMap<String, Map<String, Object>>();
 
   public ReturnMsg execute(Configuration conf) {
-    if (this.table == null || this.columnfamilies == null)
-      return new ReturnMsg(0, "Syntax error : Please check 'Create' syntax.");
-
     try {
-      HConnection conn = HConnectionManager.getConnection(conf);
       HBaseAdmin admin = new HBaseAdmin(conf);
-      
-      if (conn.tableExists(this.table)) {
-        return new ReturnMsg(0, "Table was already exsits.");
-      }
-      HTableDescriptor desc = new HTableDescriptor(this.table.toString());
-      for (int i = 0; i < this.columnfamilies.size(); i++) {
-        String columnFamily = columnfamilies.get(i);
-        if (columnFamily.lastIndexOf(':') == (columnFamily.length() - 1)) {
-          columnFamily = columnFamily.substring(0, columnFamily.length() - 1);
-        }
-        desc.addFamily(new HColumnDescriptor(columnFamily + FAMILY_INDICATOR));
+      HTableDescriptor tableDesc = new HTableDescriptor(tableName);
+      HColumnDescriptor columnDesc = null;
+      Set<String> columns = columnSpecMap.keySet();
+      for (String column : columns) {
+        columnDesc = getColumnDescriptor(column, columnSpecMap.get(column));
+        tableDesc.addFamily(columnDesc);
       }
-      admin.createTable(desc);
-      return new ReturnMsg(1, "Table created successfully.");
-    } catch (IOException e) {
-      return new ReturnMsg(0, "error msg : " + e.toString());
+      
+      System.out.println("Creating table... Please wait.");
+      
+      admin.createTable(tableDesc);
+      return new ReturnMsg(0, "Table created successfully.");
+    }
+    catch (Exception e) {
+      return new ReturnMsg(0, extractErrMsg(e));
     }
   }
 
+  /**
+   * Sets the table to be created.
+   * @param table Table to be created
+   */
   public void setTable(String table) {
-    this.table = new Text(table);
-  }
-
-  public void setColumnfamilies(List<String> columnfamilies) {
-    this.columnfamilies = columnfamilies;
+    this.tableName = table;
   }
 
-  public void setLimit(int limit) {
-    this.limit = limit;
-  }
-  
-}
+  /**
+   * Adds a column specification.  
+   * @param columnSpec Column specification
+   */
+  public void addColumnSpec(String column, Map<String, Object> columnSpec) {
+    columnSpecMap.put(column, columnSpec);
+  } 
+}

+ 56 - 36
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/DeleteCommand.java

@@ -20,64 +20,84 @@
 package org.apache.hadoop.hbase.shell;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.List;
-import java.util.Map;
-import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseAdmin;
 import org.apache.hadoop.hbase.HTable;
 import org.apache.hadoop.io.Text;
 
+/**
+ * Deletes values from tables.
+ */
 public class DeleteCommand extends BasicCommand {
   
-  private Text table;
-  private Map<String, List<String>> condition;
+  private String tableName;
+  private String rowKey;
+  private List<String> columnList;
 
   public ReturnMsg execute(Configuration conf) {
-    if (this.table == null || condition == null)
-      return new ReturnMsg(0, "Syntax error : Please check 'Delete' syntax.");
-
+    if (columnList == null) {
+      throw new IllegalArgumentException("Column list is null");
+    }
     try {
-      HTable table = new HTable(conf, this.table);
-      long lockId = table.startUpdate(getRow());
-
-      if (getColumn() != null) {
-        table.delete(lockId, getColumn());
-      } else {
-        Set<Text> keySet = table.getRow(getRow()).keySet();
-        Text[] columnKey = keySet.toArray(new Text[keySet.size()]);
-
-        for (int i = 0; i < columnKey.length; i++) {
-          table.delete(lockId, columnKey[i]);
-        }
+      HBaseAdmin admin = new HBaseAdmin(conf);
+      HTable hTable = new HTable(conf, new Text(tableName));
+      long lockID = hTable.startUpdate(new Text(rowKey));
+      for (Text column : getColumnList(admin, hTable)) {
+        hTable.delete(lockID, new Text(column));
       }
-
-      table.commit(lockId);
-
-      return new ReturnMsg(1, "1 deleted successfully. ");
+      hTable.commit(lockID);
+      return new ReturnMsg(1, "Column(s) deleted successfully.");
     } catch (IOException e) {
-      return new ReturnMsg(0, "error msg : " + e.toString());
+      String[] msg = e.getMessage().split("[\n]");
+      return new ReturnMsg(0, msg[0]);
     }
   }
 
   public void setTable(String table) {
-    this.table = new Text(table);
+    this.tableName = table;
   }
 
-  public void setCondition(Map<String, List<String>> cond) {
-    this.condition = cond;
+  public void setRow(String row) {
+    this.rowKey = row;
   }
 
-  public Text getRow() {
-    return new Text(this.condition.get("row").get(1));
+  /**
+   * Sets the column list.
+   * @param columnList
+   */
+  public void setColumnList(List<String> columnList) {
+    this.columnList = columnList;
   }
 
-  public Text getColumn() {
-    if (this.condition.containsKey("column")) {
-      return new Text(this.condition.get("column").get(1));
-    } else {
-      return null;
+  /**
+   * @param admin
+   * @param hTable
+   * @return return the column list.
+   */
+  public Text[] getColumnList(HBaseAdmin admin, HTable hTable) {
+    Text[] columns = null;
+    try {
+      if (this.columnList.contains("*")) {
+        columns = hTable.getRow(new Text(this.rowKey)).keySet().toArray(new Text[] {});
+      } else {
+        List<Text> tmpList = new ArrayList<Text>();
+        for (int i = 0; i < this.columnList.size(); i++) {
+          Text column = null;
+          if (this.columnList.get(i).contains(":"))
+            column = new Text(this.columnList.get(i));
+          else
+            column = new Text(this.columnList.get(i) + ":");
+
+          tmpList.add(column);
+        }
+        columns = tmpList.toArray(new Text[] {});
+      }
+    } catch (IOException e) {
+      e.printStackTrace();
     }
+    return columns;
   }
-  
-}
+}

+ 19 - 20
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/DescCommand.java

@@ -22,42 +22,42 @@ package org.apache.hadoop.hbase.shell;
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConnection;
 import org.apache.hadoop.hbase.HConnectionManager;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.io.Text;
 
+/**
+ * Prints information about tables.
+ */
 public class DescCommand extends BasicCommand {
   
-  private Text table;
+  private Text tableName;
 
   public ReturnMsg execute(Configuration conf) {
-    if (this.table == null) 
+    if (this.tableName == null) 
       return new ReturnMsg(0, "Syntax error : Please check 'Describe' syntax.");
-
     try {
       HConnection conn = HConnectionManager.getConnection(conf);
-      
-      if (!conn.tableExists(this.table)) {
+      if (!conn.tableExists(this.tableName)) {
         return new ReturnMsg(0, "Table not found.");
       }
-
-      HTableDescriptor[] tables = conn.listTables();
-      Text[] columns = null;
-
+      HTableDescriptor [] tables = conn.listTables();
+      HColumnDescriptor [] columns = null;
       for (int i = 0; i < tables.length; i++) {
-        if (tables[i].getName().equals(this.table)) {
-          columns = tables[i].families().keySet().toArray(new Text[] {});
+        if (tables[i].getName().equals(this.tableName)) {
+          columns = tables[i].getFamilies().values().
+            toArray(new HColumnDescriptor [] {});
+          break;
         }
       }
-
-      ConsoleTable.printHead("ColumnFamily Name");
+      ConsoleTable.printHead("ColumnFamily");
       for (int ii = 0; ii < columns.length; ii++) {
-        String familyName = columns[ii].toString().replace(FAMILY_INDICATOR, "");
-        ConsoleTable.printTable(ii, familyName);
+        String tmp = columns[ii].toString();
+        ConsoleTable.printTable(ii, tmp.substring(1, tmp.length() - 1));
       }
       ConsoleTable.printFoot();
-
       return new ReturnMsg(1, columns.length + " columnfamilie(s) found.");
     } catch (IOException e) {
       return new ReturnMsg(0, "error msg : " + e.toString());
@@ -65,7 +65,6 @@ public class DescCommand extends BasicCommand {
   }
 
   public void setArgument(String table) {
-    this.table = new Text(table);
-  }
-  
-}
+    this.tableName = new Text(table);
+  } 
+}

+ 51 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/DisableCommand.java

@@ -0,0 +1,51 @@
+/**
+ * Copyright 2007 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.shell;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseAdmin;
+import org.apache.hadoop.io.Text;
+
+/**
+ * Disables tables.
+ */
+public class DisableCommand extends BasicCommand {
+  private String tableName;
+ 
+  public ReturnMsg execute(Configuration conf) {
+    assert tableName != null;
+    
+    try {
+      HBaseAdmin admin = new HBaseAdmin(conf);
+      admin.disableTable(new Text(tableName));
+      
+      return new ReturnMsg(1, "Table disabled successfully.");
+    } catch (IOException e) {
+      String[] msg = e.getMessage().split("[\n]");
+      return new ReturnMsg(0, msg[0]);
+    }
+  }
+
+  public void setTable(String table) {
+    this.tableName = table;
+  } 
+}

+ 19 - 12
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/DropCommand.java

@@ -20,31 +20,38 @@
 package org.apache.hadoop.hbase.shell;
 
 import java.io.IOException;
+import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseAdmin;
 import org.apache.hadoop.io.Text;
 
+/**
+ * Drops tables.
+ */
 public class DropCommand extends BasicCommand {
-  
-  private Text table;
+  private List<String> tableList;
 
   public ReturnMsg execute(Configuration conf) {
-    if (this.table == null) 
-      return new ReturnMsg(0, "Syntax error : Please check 'Drop' syntax.");
-
+    if (tableList == null) {
+      throw new IllegalArgumentException("List of tables is null");
+    }
+ 
     try {
       HBaseAdmin admin = new HBaseAdmin(conf);
-      admin.deleteTable(this.table);
       
-      return new ReturnMsg(1, "Table droped successfully.");
+      for (String table : tableList) {
+        System.out.println("Dropping " + table + "... Please wait.");
+        admin.deleteTable(new Text(table));
+      }
+      
+      return new ReturnMsg(1, "Table(s) dropped successfully.");
     } catch (IOException e) {
-      return new ReturnMsg(0, "error msg : " + e.toString());
+      return new ReturnMsg(0, extractErrMsg(e));
     }
   }
 
-  public void setArgument(String table) {
-    this.table = new Text(table);
+  public void setTableList(List<String> tableList) {
+    this.tableList = tableList;
   }
-  
-}
+}

+ 50 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/EnableCommand.java

@@ -0,0 +1,50 @@
+/**
+ * Copyright 2007 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.shell;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseAdmin;
+import org.apache.hadoop.io.Text;
+
+/**
+ * Enables tables.
+ */
+public class EnableCommand extends BasicCommand {
+  private String tableName;
+ 
+  public ReturnMsg execute(Configuration conf) {
+    assert tableName != null;
+    try {
+      HBaseAdmin admin = new HBaseAdmin(conf);
+      admin.enableTable(new Text(tableName));
+      
+      return new ReturnMsg(1, "Table enabled successfully.");
+    } catch (IOException e) {
+      String[] msg = e.getMessage().split("[\n]");
+      return new ReturnMsg(0, msg[0]);
+    }
+  }
+
+  public void setTable(String table) {
+    this.tableName = table;
+  }
+}

+ 3 - 2
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/ExitCommand.java

@@ -23,9 +23,10 @@ import org.apache.hadoop.conf.Configuration;
 
 public class ExitCommand extends BasicCommand {
 
-  public ReturnMsg execute(Configuration conf) {
+  public ReturnMsg execute(@SuppressWarnings("unused") Configuration conf) {
+    // TOD: Is this the best way to exit?  Would be a problem if shell is run
+    // inside another program -- St.Ack 09/11/2007
     System.exit(1);
     return null;
   }
-
 }

+ 4 - 1
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/FsCommand.java

@@ -25,10 +25,13 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.util.ToolRunner;
 
+/**
+ * Run hadoop filesystem commands.
+ */
 public class FsCommand extends BasicCommand {
   private List<String> query;
 
-  public ReturnMsg execute(Configuration conf) {
+  public ReturnMsg execute(@SuppressWarnings("unused") Configuration conf) {
     FsShell shell = new FsShell();
     try {
       ToolRunner.run(shell, getQuery());

+ 461 - 247
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/HBaseShell.jj

@@ -4,6 +4,8 @@ options {
 }
 
 PARSER_BEGIN(Parser)
+package org.apache.hadoop.hbase.shell.generated;
+
 /**
  * Copyright 2007 The Apache Software Foundation
  *
@@ -23,7 +25,6 @@ PARSER_BEGIN(Parser)
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hbase.shell.generated;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -60,43 +61,72 @@ SKIP :
  | "\n"
 }
 
-TOKEN:
+TOKEN: /** for HQL statements */
 {
    <HELP: "help">
- | <CLEAR: "clear">
- | <SHOW: "show">
- | <DESCRIBE: "describe">
- | <CREATE: "create">	
- | <DROP: "drop">
- | <FS: "fs">
- | <EXIT: "exit">
- | <INSERT: "insert">
- | <DELETE: "delete">
- | <SELECT: "select">
- | <ROW: "row">
- | <COLUMN: "column">
- | <TIME: "time">
- | <VALUES: "values">
- | <COLUMNFAMILIES: "columnfamilies">
- | <WHERE: "where">
- | <LIMIT: "limit">
- | <AND: "and">
- | <OR:  "or">
- | <COMMA: ",">
- | <DOT: ".">
- | <LPAREN: "(">
- | <RPAREN: ")">
- | <EQUALS: "=">
- | <NOTEQUAL: "<>">
- | <OPTIONS: "-">
-}
-
-TOKEN :
-{
-   <ID: ["a"-"z","A"-"Z","_","-",".","/"] ( ["a"-"z","A"-"Z","_","0"-"9","-",".","/"] )* >
- | <NUM: ( ["0"-"9"] )+ >
- | <STRING: (["A"-"Z","a"-"z","0"-"9"])+ >
- | <QUOTED_STRING: "\"" (~["\""])+ "\"" >
+   | <ALTER: "alter">
+   | <CLEAR: "clear">
+   | <SHOW: "show">
+   | <DESCRIBE: "describe">
+   | <DESC: "desc">
+   | <CREATE: "create">   
+   | <DROP: "drop">
+   | <FS: "fs">
+   | <EXIT: "exit">
+   | <INSERT: "insert">
+   | <INTO: "into">
+   | <TABLE: "table">
+   | <DELETE: "delete">
+   | <SELECT: "select">
+   | <ENABLE: "enable">
+   | <DISABLE: "disable">
+   | <STARTING: "starting">
+   | <WHERE: "where">
+   | <FROM: "from">
+   | <ROW: "row">
+   | <VALUES: "values">
+   | <COLUMNFAMILIES: "columnfamilies">
+   | <TIMESTAMP: "timestamp">
+   | <NUM_VERSIONS: "num_versions">
+   | <LIMIT: "limit">
+   | <AND: "and">
+   | <OR:  "or">
+   | <COMMA: ",">
+   | <DOT: ".">
+   | <LPAREN: "(">
+   | <RPAREN: ")">
+   | <EQUALS: "=">
+   | <NOTEQUAL: "<>">
+   | <ASTERISK: "*">
+   | <MAX_VERSIONS: "max_versions">
+   | <MAX_LENGTH: "max_length">
+   | <COMPRESSION: "compression">
+   | <NONE: "none">
+   | <BLOCK: "block">
+   | <RECORD: "record">
+   | <IN_MEMORY: "in_memory">
+   | <BLOOMFILTER: "bloomfilter">
+   | <COUNTING_BLOOMFILTER: "counting_bloomfilter">
+   | <RETOUCHED_BLOOMFILTER: "retouched_bloomfilter">
+   | <VECTOR_SIZE: "vector_size">
+   | <NUM_HASH: "num_hash">
+   | <NUM_ENTRIES: "num_entries">
+   | <ADD: "add">
+   | <CHANGE: "change">
+}
+
+TOKEN : /** Literals */
+{
+   <ID: (["A"-"Z","a"-"z","_","-",".",":","/"])+ >
+ | <INTEGER_LITERAL: (["0"-"9"])+ >
+ | <FLOATING_POINT_LITERAL:
+   (["0"-"9"])+ "." (["0"-"9"])+ (<EXPONENT>)?
+  | "." (["0"-"9"])+ (<EXPONENT>)?
+  | (["0"-"9"])+ <EXPONENT>
+  | (["0"-"9"])+ (<EXPONENT>)?
+   >
+ | <#EXPONENT: ["e","E"] (["+","-"])? (["0"-"9"])+ >
+ | <QUOTED_IDENTIFIER: "\"" (~["\""])+ "\"" >
  | <STRING_LITERAL: "'" (~["'"])* ( "''" (~["'"])* )* "'" >
 }
 
@@ -120,17 +150,20 @@ Command cmdStatement() :
 }
 {
   (
-       cmd = exitCommand()
-     | cmd = helpCommand()
-     | cmd = showCommand()
-     | cmd = descCommand()
-     | cmd = createCommand()
-     | cmd = dropCommand()
-     | cmd = insertCommand()
-     | cmd = deleteCommand()
-     | cmd = selectCommand()
-     | cmd = clearCommand()
-     | cmd = fsCommand()
+      cmd = exitCommand()
+      | cmd = helpCommand()
+      | cmd = showCommand()
+      | cmd = descCommand()
+      | cmd = createCommand()
+      | cmd = dropCommand()
+      | cmd = alterCommand()
+      | cmd = insertCommand()
+      | cmd = deleteCommand()
+      | cmd = selectCommand()
+      | cmd = enableCommand()
+      | cmd = disableCommand()
+      | cmd = clearCommand()
+      | cmd = fsCommand()
   )
    {
        return cmd;
@@ -182,6 +215,7 @@ HelpCommand helpCommand() :
        | t=<INSERT>
        | t=<DELETE>
        | t=<SELECT>
+       | t=<ALTER>
        | t=<CLEAR>
        | t=<FS>
        | t=<ID>
@@ -202,9 +236,8 @@ ShowCommand showCommand() :
 {
    <SHOW>
    [
-       argument = getString()
+       argument = Identifier()
    ]
-   
   { 
       show.setArgument(argument);
       return show;
@@ -217,265 +250,352 @@ DescCommand descCommand() :
   String argument = null;
 }
 {
-   <DESCRIBE>
-   [
-       argument = getString()
-   ]
-   
+  ( <DESCRIBE> | <DESC> )
+  argument = Identifier()
   { 
-      desc.setArgument(argument);
-      return desc;
+    desc.setArgument(argument);
+    return desc;
   }
 }
 
-CreateCommand createCommand() :
+Map<String, Object> ColumnSpec() :
 {
-  CreateCommand create = new CreateCommand();
-  String argument = null;
-  List<String> columnfamilies = null;
-  int limit = 1;
+  Map<String, Object> columnSpec = new HashMap<String, Object>();
+  int n = -1;
+  Token t = null;
 }
 {
-    <CREATE>
-    argument = getString() 
+  (
+    <MAX_VERSIONS> 
+     <EQUALS> n = Number() 
     { 
-         create.setTable(argument); 
+      columnSpec.put("MAX_VERSIONS", n); 
     }
-    
-    <COLUMNFAMILIES>
-    columnfamilies = getLiteralValues()
+   |
+    <MAX_LENGTH>
+     <EQUALS> n = Number()
+    {  
+      columnSpec.put("MAX_LENGTH", n); 
+    }
+   |
+    <COMPRESSION>
+     <EQUALS> 
+    (  t=<NONE>
+     | t=<BLOCK>
+     | t=<RECORD> )
+    { 
+      columnSpec.put("COMPRESSION", t.image.toString()); 
+    }
+   |
+    <IN_MEMORY> 
+    { 
+      columnSpec.put("IN_MEMORY", true); 
+    } 
+   |  
+    <BLOOMFILTER>
+    <EQUALS> 
+    (  t=<BLOOMFILTER> 
+     | t=<COUNTING_BLOOMFILTER> 
+     | t=<RETOUCHED_BLOOMFILTER> 
+    )
     { 
-         create.setColumnfamilies(columnfamilies); 
+      columnSpec.put("BLOOMFILTER", t.image.toString()); 
     }
+   |  
+    <VECTOR_SIZE> 
+    <EQUALS> n = Number() 
+    { 
+      columnSpec.put("VECTOR_SIZE", n); 
+    }
+   |
+    <NUM_HASH>
+    <EQUALS> n = Number()
+    { 
+      columnSpec.put("NUM_HASH", n); 
+    }  
+   |
+    <NUM_ENTRIES> <EQUALS> n = Number()
+    { 
+      columnSpec.put("NUM_ENTRIES", n); 
+    }
+  )*
 
-    [ <LIMIT><EQUALS> limit = getInt() {
-            try{
-               create.setLimit(limit);
-            }catch(ClassCastException ce) {
-               throw generateParseException();
-        }
-   } ]
-  { return create; }
+  { return columnSpec; }  
 }
 
-DropCommand dropCommand() :
+CreateCommand createCommand() :
 {
-  DropCommand drop = new DropCommand();
-  String argument = null;
+  CreateCommand createCommand = new CreateCommand();
+  String table = null;
+  Map<String, Object> columnSpec = null;
+  String column = null;
 }
 {
-   <DROP>
-   [
-       argument = getString()
-   ]
-   
+  <CREATE> 
+  <TABLE> 
+  table = Identifier() 
   { 
-      drop.setArgument(argument);
-      return drop;
+    createCommand.setTable(table); 
   }
+  
+  <LPAREN>
+  
+  column = Identifier() 
+  columnSpec = ColumnSpec()
+  { 
+    createCommand.addColumnSpec(column, columnSpec); 
+  }
+  
+  (
+     <COMMA> 
+      column = Identifier() 
+      columnSpec = ColumnSpec() 
+      { 
+        createCommand.addColumnSpec(column, columnSpec); 
+      }
+  )*
+  
+  <RPAREN> 
+  { return createCommand; }
 }
 
-InsertCommand insertCommand() :
+AlterCommand alterCommand() :
 {
-    InsertCommand in = new InsertCommand();
-    Map<String, List<String>> cond = null;
-    List<String> columnfamilies = null;
-    List<String> values = null;
-    String table = null;
+  AlterCommand alterCommand = new AlterCommand();
+  String table = null;
+  String column = null;
+  Map<String, Object> columnSpec = null;
 }
 {
-    <INSERT>
-    table = getString()
+  <ALTER> 
+  <TABLE> table = Identifier() 
+  { alterCommand.setTable(table); }
+
+  (
+    LOOKAHEAD(2)
+      <ADD> column = Identifier() columnSpec = ColumnSpec() 
+      { 
+         alterCommand.setOperationType(AlterCommand.OperationType.ADD);
+         alterCommand.addColumnSpec(column, columnSpec); 
+      }
+  |
+    <ADD>
+    <LPAREN> 
     { 
-         in.setTable(table); 
+      alterCommand.setOperationType(AlterCommand.OperationType.ADD); 
     }
-    
-    columnfamilies = getLiteralValues()
-    {
-        in.setColumnfamilies(columnfamilies);
-    }
-    
-    <VALUES> values = getLiteralValues()
-    {
-        in.setValues(values);
+      
+    column = Identifier() columnSpec = ColumnSpec() 
+    { 
+      alterCommand.addColumnSpec(column, columnSpec); 
     }
       
-    <WHERE> cond = WhereClause() 
-    {
-        try{
-           in.setCondition(cond);
-        }catch(ClassCastException ce) {
-           throw generateParseException();
+    ( 
+        <COMMA> 
+        column = Identifier()
+        columnSpec = ColumnSpec() 
+        { 
+          alterCommand.addColumnSpec(column, columnSpec); 
         }
-    } 
-    {
-       return in;
+    )*
+    <RPAREN>
+  |
+    <DROP> column = Identifier()
+    { 
+      alterCommand.setOperationType(AlterCommand.OperationType.DROP);
+      alterCommand.setColumn(column); 
+    }
+  |
+    <CHANGE> column = Identifier() columnSpec = ColumnSpec()  
+    { 
+      alterCommand.setOperationType(AlterCommand.OperationType.CHANGE);
+      alterCommand.addColumnSpec(column, columnSpec);
     }
+  )
+  { return alterCommand; }
 }
 
-DeleteCommand deleteCommand() :
+DropCommand dropCommand() :
 {
-    DeleteCommand del = new DeleteCommand();
-    Map<String, List<String>> cond = null;
-    String argument = null;
+  DropCommand drop = new DropCommand();
+  List<String> tableList = null;
 }
 {
-    <DELETE>
-    argument = getString()
-    { 
-         del.setTable(argument); 
-    }
-    
-     <WHERE> cond = WhereClause() {
-        try{
-           del.setCondition(cond);
-        }catch(ClassCastException ce) {
-            throw generateParseException();
-       }
-    } 
-    {
-       return del;
-    }
+   <DROP>
+   <TABLE>
+   tableList = TableList()
+   { 
+     drop.setTableList(tableList);
+     return drop;
+   }
 }
 
-SelectCommand selectCommand() :
+InsertCommand insertCommand() :
 {
-    SelectCommand select = new SelectCommand();
-    Map<String, List<String>> cond = null;
-    String argument = null;
-    int limit;
+  InsertCommand in = new InsertCommand();
+  List<String> columnfamilies = null;
+  List<String> values = null;
+  String table = null;
+  Token t = null;       
 }
 {
-    <SELECT>
-    argument = getString()
-    { 
-         select.setTable(argument); 
-    }
-
-     [ <WHERE> cond = WhereClause() {
-        try{
-           select.setCondition(cond);
-        }catch(ClassCastException ce) {
-            throw generateParseException();
-        }
-    } ] 
+  <INSERT>
+  <INTO>
+  table = Identifier()
+  { 
+     in.setTable(table); 
+  }
     
-    [ <LIMIT><EQUALS> limit = getInt() {
-            try{
-               select.setLimit(limit);
-            }catch(ClassCastException ce) {
-               throw generateParseException();
-        }
-   } ]
-  { return select; }
+  columnfamilies = getColumns()
+  {
+     in.setColumnfamilies(columnfamilies);
+  }
+    
+  <VALUES> values = getLiteralValues()
+  {
+     in.setValues(values);
+  }
+      
+  <WHERE> 
+  <ROW> <EQUALS> ( t=<STRING_LITERAL> | t=<QUOTED_IDENTIFIER> )
+  { 
+     in.setRow(t.image.substring(1, t.image.length()-1)); 
+  }
+  {
+    return in;
+  }
 }
 
-ClearCommand clearCommand() :
+DeleteCommand deleteCommand() :
 {
-  ClearCommand clear = new ClearCommand();
+  DeleteCommand deleteCommand = new DeleteCommand();
+  List<String> columnList = null;
+  Token t = null;
+  String table = null;
 }
 {
-   <CLEAR>  { return clear; }
-}
+  <DELETE>
+  columnList = ColumnList()
+  { 
+    deleteCommand.setColumnList(columnList); 
+  }
 
-/**
-* TODO : expressions codes need more love.
-*/
+  <FROM>
+  table = Identifier()
+  { 
+    deleteCommand.setTable(table); 
+  }
 
-String getString():
-{ Token t = null; }
-{
-    (  t=<ID>
-     | t=<QUOTED_STRING>
-    ) 
-    { return t.image.toString(); }
-}
+  <WHERE> 
+  <ROW> <EQUALS> ( t=<STRING_LITERAL> | t=<QUOTED_IDENTIFIER> )
+  { 
+    deleteCommand.setRow(t.image.substring(1, t.image.length()-1)); 
+  }
 
-int getInt():
-{ Token t = null; }
-{
-   t = <NUM>
-   { return Integer.parseInt(t.image.toString()); }
+  { return deleteCommand; }
 }
 
-Map<String, List<String>> WhereClause() :
+SelectCommand selectCommand() :
 {
-    Map<String, List<String>> result = 
-               new HashMap<String, List<String>>();
-    List<String> exception = 
-               new ArrayList<String>();
+  SelectCommand select = new SelectCommand();
+  List<String> columns = null;
+  String rowKey = "";
+  String timestamp = null;
+  int numVersion = 0;
+  String tableName = null;
+  int limit;
 }
 {
+  <SELECT>
+  columns = ColumnList()
+  <FROM>
+  tableName = Identifier()
   { 
-      try{
-        result.putAll(ConditionExpression()); 
-      }catch(ParseException pe) {
-        exception.add(pe.toString());
-        result.put("error", exception);
-      }
+     select.setColumns(columns);
+     select.setTable(tableName);
   }
-	(
-     <AND> { 
+
+  [ ( <WHERE> <ROW> <EQUALS>
+     { select.setWhere(true); }
+     | <STARTING> <FROM> )
+    
+     rowKey = getStringLiteral()
+     {
+       select.setRowKey(rowKey); 
+     }
+  ]
+
+  [ <TIMESTAMP> 
+    timestamp = getStringLiteral()
+    {
+       select.setTimestamp(timestamp);
+    }
+  ]
+   
+  [
+     <NUM_VERSIONS>
+      numVersion = Number()
+      {
+        select.setVersion(numVersion);
+      }
+  ]
+
+  [ <LIMIT><EQUALS> limit = Number() {
       try{
-        result.putAll(ConditionExpression()); 
-      }catch(ParseException pe) {
-        exception.add(pe.toString());
-        result.put("error", exception);
+        select.setLimit(limit);
+      }catch(ClassCastException ce) {
+        throw generateParseException();
       }
-     }
-	)*
-	
- { return result; }
+   } ]
+  { return select; }
 }
 
-Map<String, List<String>> ConditionExpression() :
+EnableCommand enableCommand() :
 {
-    Token tSearchName, tComparator, tComparand;
-    Map<String, List<String>> tmp = 
-               new HashMap<String, List<String>>();
-    List<String> values = 
-               new ArrayList<String>();
+  EnableCommand enableCommand = new EnableCommand();
+  String table = null;
 }
 {
-	( 
-	   tSearchName=<ROW>
-	 | tSearchName=<COLUMN>
-	 | tSearchName=<TIME> 
-	 | tSearchName=<ID>
-	 | tSearchName=<VALUES>
-	 | tSearchName=<COLUMNFAMILIES>
-	 )
-	 
-	( tComparator=<EQUALS> | tComparator=<NOTEQUAL> )
+  <ENABLE> 
+  table = Identifier()
+  { 
+    enableCommand.setTable(table);
+    return enableCommand;
+  }
+}
 
- ( tComparand=<QUOTED_STRING>
-    { 
-          values.add("quoted string"); 
-          tmp.put("error", values); 
-          return tmp; 
-    } 
-  |  tComparand=<STRING_LITERAL> {
-    values.add(tComparator.image);
-    values.add(tComparand.image.substring(1,tComparand.image.length() - 1));
-  
-  if(tSearchName.image.toString().equals("row") || 
-  tSearchName.image.toString().equals("column") || 
-  tSearchName.image.toString().equals("time"))
-    { tmp.put(tSearchName.image, values); }
-  else   
-    {
-    values.add(tSearchName.image.toString());
-    tmp.put("error", values); 
-    }
-    
-    return tmp; 
-} )
+DisableCommand disableCommand() :
+{
+  DisableCommand disableCommand = new DisableCommand();
+  String table = null;
 }
+{
+  <DISABLE> 
+  table = Identifier()
+  { 
+    disableCommand.setTable(table); 
+    return disableCommand;
+  }
+}
+
+ClearCommand clearCommand() :
+{
+  ClearCommand clear = new ClearCommand();
+}
+{
+   <CLEAR>
+   { 
+     return clear; 
+   }
+}
+
+////////////////////////////////////////////////
+// Utility expansion units...
 
 List<String> getLiteralValues() :
 {
-    List<String> values = new ArrayList<String>();
-    String literal = null;
+  List<String> values = new ArrayList<String>();
+  String literal = null;
 }
 {    
 <LPAREN>
@@ -490,10 +610,9 @@ List<String> getLiteralValues() :
 | ( 
        <ID>
      | <STRING_LITERAL>
-     | <QUOTED_STRING>
-     | <STRING> 
+     | <QUOTED_IDENTIFIER>
   )  { values.removeAll(values); }
-	)*
+        )*
 <RPAREN>
    { 
      return values;
@@ -502,10 +621,105 @@ List<String> getLiteralValues() :
 
 String getStringLiteral() :
 {
-    Token stringLiteral;
+  Token s;
 }
 {
- stringLiteral=<STRING_LITERAL> 
- { return stringLiteral.image.substring(1,stringLiteral.image.length() - 1); }
- | <QUOTED_STRING> { return null; }
+ ( s=<STRING_LITERAL> | s=<QUOTED_IDENTIFIER> )
+ { 
+   String value = s.image.toString();
+   return value.substring(1,value.length() - 1);
+ }
 }
+
+List<String> getColumns() : // return parenthesized column list
+{
+  List<String> values = new ArrayList<String>();
+  String literal = null;
+}
+{    
+<LPAREN>
+ { literal = getColumn();
+ if(literal != null) values.add(literal); 
+ }
+    (
+        <COMMA> 
+      { 
+        literal = getColumn(); 
+        if(literal != null) values.add(literal); 
+      }
+    )*
+<RPAREN>
+   { 
+     return values;
+    }
+}
+
+String getColumn() :
+{
+  Token col;
+}
+{
+  (
+    ( col=<ID> | col=<ASTERISK> )
+      { return col.image.toString(); }
+    | (col=<QUOTED_IDENTIFIER> | col=<STRING_LITERAL> )
+      { return col.image.substring(1,col.image.toString().length() - 1); }
+  )
+}
+
+List<String> TableList() :
+{
+  List<String> tableList = new ArrayList<String>();
+  String table = null;
+}
+{
+  table = Identifier() { tableList.add(table); }
+  ( <COMMA> table = Identifier()
+    { tableList.add(table); } 
+  )*
+  
+  { return tableList; }
+}
+
+List<String> ColumnList() :
+{
+  List<String> columnList = new ArrayList<String>();  
+  String column = null;
+}
+{ 
+    column = getColumn() 
+    { 
+      if(column != null) {
+        columnList.add(column);
+      } else {
+        return columnList;
+      }
+    }
+  ( <COMMA> column = getColumn()  
+    { columnList.add(column); }
+  )*
+  
+  { return columnList; }
+}
+
+int Number() :
+{
+  Token t = null;
+}
+{
+  t = <INTEGER_LITERAL>
+  { return Integer.parseInt(t.image.toString()); }
+}
+
+String Identifier() :
+{
+  Token t = null;
+}
+{
+  (
+     t = <ID>
+     { return t.image.toString(); }
+   | ( t=<QUOTED_IDENTIFIER> | t=<STRING_LITERAL> )
+     { return t.image.substring(1,t.image.toString().length() - 1); }
+  )
+}

+ 1 - 3
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/HelpCommand.java

@@ -22,10 +22,9 @@ package org.apache.hadoop.hbase.shell;
 import org.apache.hadoop.conf.Configuration;
 
 public class HelpCommand extends BasicCommand {
-  
   private String argument;
 
-  public ReturnMsg execute(Configuration conf) {
+  public ReturnMsg execute(@SuppressWarnings("unused") Configuration conf) {
     HelpManager.printHelp(this.argument);
     return null;
   }
@@ -33,5 +32,4 @@ public class HelpCommand extends BasicCommand {
   public void setArgument(String argument) {
     this.argument = argument;
   }
-  
 }

+ 63 - 22
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/HelpContents.java

@@ -30,36 +30,77 @@ public class HelpContents {
   public static Map<? extends String, ? extends String[]> Load() {
     Map<String, String[]> load = new HashMap<String, String[]>();
 
-    load.put("SHOW", new String[] { "List all tables.", "SHOW TABLES;" });
+    String columnName = "column_name: " 
+      + "\n\t  column_family_name"
+      + "\n\t| column_family_name:column_label_name";    
+    String columnList = "{column_name, [, column_name] ... | *}";
+
+    load.put("SHOW", new String[] {"List all available tables", "SHOW TABLES;"});
+
     load.put("FS", new String[] { "Hadoop FsShell operations.",
       "FS -copyFromLocal /home/user/backup.dat fs/user/backup;" });
-    load.put("CLEAR", new String[] {"Clear the screen.", "CLEAR;"} );
-    load.put("DESCRIBE", new String[] { "Describe a table's columnfamilies.",
-        "DESCRIBE <table_name>;" });
+    
+    load.put("CLEAR", new String[] {"Clear the screen", "CLEAR;"} );
+    
+    load.put("DESCRIBE", new String[] { "Print information about tables",
+    "[DESCRIBE|DESC] table_name;" });
+    
     load.put("CREATE", new String[] {
-        "Create a table",
-        "CREATE <table_name>"
-            + "\n\t  COLUMNFAMILIES('cf_name1'[, 'cf_name2', ...]);"
-            + "\n    [LIMIT=versions_limit];" });
+        "Create tables",
+        "CREATE TABLE table_name"
+            + "\n\t(column_family_spec [, column_family_spec] ...);"
+            + "\n\n"
+      + "column_family_spec:"
+      + "\n\tcolumn_family_name"
+      + "\n\t[MAX_VERSIONS=n]"
+      + "\n\t[MAX_LENGTH=n]"
+      + "\n\t[COMPRESSION=NONE|RECORD|BLOCK]"
+      + "\n\t[IN_MEMORY]"
+      + "\n\t[BLOOMFILTER=NONE|BLOOM|COUNTING|RETOUCHED VECTOR_SIZE=n NUM_HASH=n]"
+    });
+    
     load.put("DROP", new String[] {
-        "Drop columnfamilie(s) from a table or drop table(s)",
-        "DROP table_name1[, table_name2, ...] | cf_name1[, cf_name2, ...];" });
+        "Drop tables",
+        "DROP TABLE table_name [, table_name] ...;" });
+    
     load.put("INSERT", new String[] {
-        "Insert row into table",
-        "INSERT <table_name>" + "\n\t('column_name1'[, 'column_name2', ...])"
-            + "\n\t    VALUES('entry1'[, 'entry2', ...])"
-            + "\n    WHERE row='row_key';" });
+        "Insert values into tables",
+        "INSERT INTO table_name"
+            + "\n\t(column_name, ...) VALUES ('value', ...)"
+            + "\n\tWHERE row='row_key';"
+            + "\n\n" + columnName            
+    });
+    
     load.put("DELETE", new String[] {
-        "Delete cell or row in table.",
-        "DELETE <table_name>" + "\n\t    WHERE row='row_key;"
-            + "\n    [AND column='column_name'];" });
+        "Delete a subset of the data in a table",
+        "DELETE " + columnList 
+            + "\n\tFROM table_name"
+            + "\n\tWHERE row='row-key';" 
+            + "\n\n"
+            + columnName
+    });
+    
     load.put("SELECT",
         new String[] {
-            "Select values from a table",
-            "SELECT <table_name>" + "\n\t    [WHERE row='row_key']"
-                + "\n    [AND column='column_name'];"
-                + "\n    [AND time='timestamp'];"
-                + "\n    [LIMIT=versions_limit];" });
+            "Select values from tables",
+            "SELECT " + columnList + " FROM table_name" 
+                + "\n\t[WHERE row='row_key' | STARTING FROM 'row-key']"
+                + "\n\t[NUM_VERSIONS = version_count]"
+                + "\n\t[TIMESTAMP 'timestamp']"
+                + "\n\t[LIMIT = row_count]"
+                + "\n\t[INTO FILE 'file_name'];"
+    });
+                
+    load.put("ALTER",
+        new String[] {
+            "Alter the structure of a table",
+            "ALTER TABLE table_name" 
+                + "\n\t  ADD column_spec"
+                + "\n\t| ADD (column_spec, column_spec, ...)"
+                + "\n\t| DROP column_family_name"
+                + "\n\t| CHANGE column_spec;" 
+    });
+
     load.put("EXIT", new String[] { "Exit shell", "EXIT;" });
 
     return load;

+ 8 - 7
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/HelpManager.java

@@ -27,13 +27,14 @@ import java.util.Map;
  */
 public class HelpManager {
   /** application name */
-  public static final String APP_NAME = "HBase Shell";
+  public static final String APP_NAME = "Hbase Shell";
 
   /** version of the code */
-  public static final String APP_VERSION = "0.0.1";
+  public static final String APP_VERSION = "0.0.2";
 
   /** help contents map */
-  public static final Map<String, String[]> help = new HashMap<String, String[]>();
+  public static final Map<String, String[]> help =
+    new HashMap<String, String[]>();
 
   public HelpManager() {
     help.putAll(HelpContents.Load());
@@ -41,7 +42,6 @@ public class HelpManager {
 
   /** Print out the program version. */
   public void printVersion() {
-    ClearCommand.clear();
     System.out.println(APP_NAME + ", " + APP_VERSION + " version.\n"
         + "Copyright (c) 2007 by udanax, "
         + "licensed to Apache Software Foundation.\n"
@@ -55,12 +55,13 @@ public class HelpManager {
       for (Map.Entry<String, String[]> helpMap : help.entrySet()) {
         wrapping(helpMap.getKey(), helpMap.getValue(), false);
       }
+      System.out.println();
     } else {
       if (help.containsKey(cmd.toUpperCase())) {
         String[] msg = help.get(cmd.toUpperCase());
         wrapping(cmd.toUpperCase(), msg, true);
       } else {
-        System.out.println("Unknown Command : Type 'help' for usage.");
+        System.out.println("Unknown Command : Type 'help;' for usage.");
       }
     }
   }
@@ -76,6 +77,6 @@ public class HelpManager {
     }
 
     if (example)
-      System.out.println("\n>>> " + cmdType[1]);
+      System.out.println("\nSyntax:\n" + cmdType[1] + "\n");
   }
-}
+}

+ 18 - 12
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/InsertCommand.java

@@ -21,21 +21,23 @@ package org.apache.hadoop.hbase.shell;
 
 import java.io.IOException;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HTable;
 import org.apache.hadoop.io.Text;
 
+/**
+ * Inserts values into tables.
+ */
 public class InsertCommand extends BasicCommand {
   
-  private Text table;
+  private Text tableName;
   private List<String> columnfamilies;
   private List<String> values;
-  private Map<String, List<String>> condition;
+  private String rowKey;
 
   public ReturnMsg execute(Configuration conf) {
-    if (this.table == null || this.values == null || this.condition == null)
+    if (this.tableName == null || this.values == null || this.rowKey == null)
       return new ReturnMsg(0, "Syntax error : Please check 'Insert' syntax.");
 
     if (this.columnfamilies.size() != this.values.size())
@@ -43,12 +45,16 @@ public class InsertCommand extends BasicCommand {
           "Mismatch between values list and columnfamilies list");
 
     try {
-      HTable table = new HTable(conf, this.table);
+      HTable table = new HTable(conf, this.tableName);
       long lockId = table.startUpdate(getRow());
 
       for (int i = 0; i < this.values.size(); i++) {
-        table.put(lockId, getColumn(i), getValue(i));
-        
+        Text column = null;
+        if(getColumn(i).toString().contains(":"))
+          column = getColumn(i);
+        else
+          column = new Text(getColumn(i) + ":");
+        table.put(lockId, column, getValue(i));
       }
       table.commit(lockId);
 
@@ -60,7 +66,7 @@ public class InsertCommand extends BasicCommand {
   }
 
   public void setTable(String table) {
-    this.table = new Text(table);
+    this.tableName = new Text(table);
   }
 
   public void setColumnfamilies(List<String> columnfamilies) {
@@ -71,12 +77,12 @@ public class InsertCommand extends BasicCommand {
     this.values = values;
   }
 
-  public void setCondition(Map<String, List<String>> cond) {
-    this.condition = cond;
+  public void setRow(String row) {
+    this.rowKey = row;
   }
 
   public Text getRow() {
-    return new Text(this.condition.get("row").get(1));
+    return new Text(this.rowKey);
   }
 
   public Text getColumn(int i) {
@@ -85,5 +91,5 @@ public class InsertCommand extends BasicCommand {
 
   public byte[] getValue(int i) {
     return this.values.get(i).getBytes();
-  }
+  } 
 }

+ 6 - 4
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/ReturnMsg.java

@@ -19,10 +19,13 @@
  */
 package org.apache.hadoop.hbase.shell;
 
+/**
+ * Message returned when a {@link Command} is
+ * {@link Command#execute(org.apache.hadoop.conf.Configuration)}'ed.
+ */
 public class ReturnMsg {
-  
-  private String msg;
-  private int type;
+  private final String msg;
+  private final int type;
 
   public ReturnMsg(int i, String string) {
     this.type = i;
@@ -41,5 +44,4 @@ public class ReturnMsg {
   public int getType() {
     return this.type;
   }
-  
 }

+ 106 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/SchemaModificationCommand.java

@@ -0,0 +1,106 @@
+/**
+ * Copyright 2007 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.shell;
+
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.BloomFilterDescriptor;
+import org.apache.hadoop.hbase.BloomFilterDescriptor.BloomFilterType;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.io.Text;
+
+/**
+ * The base class of schema modification commands, CreateCommand and Alter
+ * Command. Provides utility methods for alteration operations.
+ */
+public abstract class SchemaModificationCommand extends BasicCommand {
+
+  private int maxVersions;
+  private int maxLength;
+  private HColumnDescriptor.CompressionType compression;
+  private boolean inMemory;
+  private BloomFilterDescriptor bloomFilterDesc;
+  private BloomFilterType bloomFilterType;
+  private int vectorSize;
+  private int numHash;
+  private int numEntries;
+
+  private void initOptions() {
+    maxVersions = HColumnDescriptor.DEFAULT_N_VERSIONS;
+    maxLength = HColumnDescriptor.DEFAULT_MAX_VALUE_LENGTH;
+    compression = HColumnDescriptor.DEFAULT_COMPRESSION_TYPE;
+    inMemory = HColumnDescriptor.DEFAULT_IN_MEMORY;
+    bloomFilterDesc = HColumnDescriptor.DEFAULT_BLOOM_FILTER_DESCRIPTOR;
+  }
+
+  /**
+   * Given a column name and column spec, returns an instance of
+   * HColumnDescriptor representing the column spec.
+   */
+  protected HColumnDescriptor getColumnDescriptor(String column,
+      Map<String, Object> columnSpec) throws IllegalArgumentException {
+    initOptions();
+
+    Set<String> specs = columnSpec.keySet();
+    for (String spec : specs) {
+      spec = spec.toUpperCase();
+
+      if (spec.equals("MAX_VERSIONS")) {
+        maxVersions = (Integer) columnSpec.get(spec);
+      } else if (spec.equals("MAX_LENGTH")) {
+        maxLength = (Integer) columnSpec.get(spec);
+      } else if (spec.equals("COMPRESSION")) {
+        compression = HColumnDescriptor.CompressionType
+            .valueOf(((String) columnSpec.get(spec)).toUpperCase());
+      } else if (spec.equals("IN_MEMORY")) {
+        inMemory = (Boolean) columnSpec.get(spec);
+      } else if (spec.equals("BLOOMFILTER")) {
+        bloomFilterType = BloomFilterType.valueOf(((String) columnSpec
+            .get(spec)).toUpperCase());
+      } else if (spec.equals("VECTOR_SIZE")) {
+        vectorSize = (Integer) columnSpec.get(spec);
+      } else if (spec.equals("NUM_HASH")) {
+        numHash = (Integer) columnSpec.get(spec);
+      } else if (spec.equals("NUM_ENTRIES")) {
+        numEntries = (Integer) columnSpec.get(spec);
+      } else {
+        throw new IllegalArgumentException("Invalid option: " + spec);
+      }
+    }
+
+    // Now we gather all the specified options for this column.
+    if (bloomFilterType != null) {
+      if (specs.contains("NUM_ENTRIES")) {
+        bloomFilterDesc = new BloomFilterDescriptor(bloomFilterType, numEntries);
+      } else {
+        bloomFilterDesc = new BloomFilterDescriptor(bloomFilterType,
+            vectorSize, numHash);
+      }
+    }
+
+    column = appendDelimiter(column);
+
+    HColumnDescriptor columnDesc = new HColumnDescriptor(new Text(column),
+        maxVersions, compression, inMemory, maxLength, bloomFilterDesc);
+
+    return columnDesc;
+  }
+}

+ 128 - 153
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/SelectCommand.java

@@ -20,6 +20,7 @@
 package org.apache.hadoop.hbase.shell;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
@@ -35,75 +36,66 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.Text;
 
+/**
+ * Selects values from tables.
+ * 
+ * TODO: INTO FILE is not yet implemented.
+ */
 public class SelectCommand extends BasicCommand {
   
-  private Text table;
+  private Text tableName;
+  private Text rowKey = new Text("");
+  private List<String> columns;
+  private long timestamp;
   private int limit;
-  private Map<String, List<String>> condition;
+  private int version;
+  private boolean whereClause = false;
 
   public ReturnMsg execute(Configuration conf) {
-    if (this.condition != null && this.condition.containsKey("error"))
+    if (this.tableName.equals("") || this.rowKey == null ||
+        this.columns.size() == 0) {
       return new ReturnMsg(0, "Syntax error : Please check 'Select' syntax.");
-
+    } 
+    
     try {
-      HTable table = new HTable(conf, this.table);
+      HTable table = new HTable(conf, this.tableName);
       HBaseAdmin admin = new HBaseAdmin(conf);
-      
-      switch (getCondition()) {
-      case 0:
-
-        HTableDescriptor[] tables = admin.listTables();
-        Text[] columns = null;
+      if (this.whereClause) {
+        compoundWherePrint(table, admin);
+      } else {
+        scanPrint(table, admin);
+      }
+      return new ReturnMsg(1, "Successfully print out the selected data.");
+    } catch (IOException e) {
+      String[] msg = e.getMessage().split("[,]");
+      return new ReturnMsg(0, msg[0]);
+    }
+  }
 
-        if (this.table.equals(HConstants.ROOT_TABLE_NAME)
-            || this.table.equals(HConstants.META_TABLE_NAME)) {
-          columns = HConstants.COLUMN_FAMILY_ARRAY;
-        } else {
-          for (int i = 0; i < tables.length; i++) {
-            if (tables[i].getName().equals(this.table)) {
-              columns = tables[i].families().keySet().toArray(new Text[] {});
-            }
+  private void compoundWherePrint(HTable table, HBaseAdmin admin) {
+    try {
+      if (this.version != 0) {
+        byte[][] result = null;
+        Text[] cols = getColumns(admin);
+        for (int i = 0; i < cols.length; i++) {
+          if (this.timestamp == 0) {
+            result = table.get(this.rowKey, cols[i], this.timestamp, this.version);
+          } else {
+            result = table.get(this.rowKey, cols[i], this.version);
           }
-        }
-
-        HScannerInterface scan = table.obtainScanner(columns, new Text(""));
-        HStoreKey key = new HStoreKey();
-        TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
-
-        ConsoleTable.selectHead();
-        int count = 0;
-        while (scan.next(key, results)) {
-          Text rowKey = key.getRow();
-
-          for (Text columnKey : results.keySet()) {
-            byte[] value = results.get(columnKey);
-            String cellData = new String(value, HConstants.UTF8_ENCODING);
-
-            if (columnKey.equals(HConstants.COL_REGIONINFO)) {
-              DataInputBuffer inbuf = new DataInputBuffer();
-              HRegionInfo info = new HRegionInfo();
-              inbuf.reset(value, value.length);
-              info.readFields(inbuf);
 
-              cellData = "ID : " + String.valueOf(info.getRegionId());
-            }
-            ConsoleTable.printLine(count, rowKey.toString(), columnKey.toString(),
-                cellData);
-            count++;
+          ConsoleTable.selectHead();
+          for (int ii = 0; ii < result.length; ii++) {
+            ConsoleTable.printLine(i, this.rowKey.toString(), cols[i].toString(),
+                new String(result[ii], HConstants.UTF8_ENCODING));
           }
-          results = new TreeMap<Text, byte[]>();
+          ConsoleTable.selectFoot();
         }
-        ConsoleTable.selectFoot();
-        scan.close();
-
-        break;
-
-      case 1:
-
-        count = 0;
+      } else {
+        int count = 0;
         ConsoleTable.selectHead();
-        for (Map.Entry<Text, byte[]> entry : table.getRow(new Text(getRow())).entrySet()) {
-
+        
+        for (Map.Entry<Text, byte[]> entry : table.getRow(this.rowKey).entrySet()) {
           byte[] value = entry.getValue();
           String cellData = new String(value, HConstants.UTF8_ENCODING);
 
@@ -112,138 +104,121 @@ public class SelectCommand extends BasicCommand {
             HRegionInfo info = new HRegionInfo();
             inbuf.reset(value, value.length);
             info.readFields(inbuf);
-
-            cellData = "ID : " + String.valueOf(info.getRegionId());
+            cellData = String.valueOf(info.getRegionId());
           }
-          ConsoleTable.printLine(count, getRow().toString(), entry.getKey().toString(),
-              cellData);
-          count++;
-        }
-        ConsoleTable.selectFoot();
-
-        break;
-
-      case 2:
 
-        Text[] column = new Text[] { new Text(getColumn()) };
-
-        HScannerInterface scanner = table.obtainScanner(column, new Text(""));
-        HStoreKey k = new HStoreKey();
-        TreeMap<Text, byte[]> r = new TreeMap<Text, byte[]>();
-
-        ConsoleTable.selectHead();
-        count = 0;
-        while (scanner.next(k, r)) {
-          Text rowKey = k.getRow();
-
-          for (Text columnKey : r.keySet()) {
-            byte[] value = r.get(columnKey);
-            String cellData = new String(value, HConstants.UTF8_ENCODING);
-            ConsoleTable.printLine(count, rowKey.toString(), columnKey.toString(),
-                cellData);
+          if (columns.contains(entry.getKey().toString()) || columns.contains("*")) {
+            ConsoleTable.printLine(count, this.rowKey.toString(), entry.getKey()
+                .toString(), cellData);
             count++;
           }
-          results = new TreeMap<Text, byte[]>();
         }
         ConsoleTable.selectFoot();
-        scanner.close();
-
-        break;
-
-      case 3:
-
-        byte[] rs1 = table.get(new Text(getRow()), new Text(getColumn()));
+      }
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+  }
 
-        ConsoleTable.selectHead();
-        ConsoleTable.printLine(0, getRow(), getColumn(),
-          new String(rs1, HConstants.UTF8_ENCODING));
-        ConsoleTable.selectFoot();
+  private void scanPrint(HTable table, HBaseAdmin admin) {
+    HScannerInterface scan = null;
+    try {
+      if (this.timestamp == 0) {
+        scan = table.obtainScanner(getColumns(admin), this.rowKey);
+      } else {
+        scan = table.obtainScanner(getColumns(admin), this.rowKey, this.timestamp);
+      }
 
-        break;
+      HStoreKey key = new HStoreKey();
+      TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
 
-      case 4:
+      ConsoleTable.selectHead();
+      int count = 0;
 
-        byte[][] rs2 = table.get(new Text(getRow()), new Text(getColumn()), this.limit);
+      while (scan.next(key, results) && checkLimit(count)) {
+        Text rowKey = key.getRow();
 
-        ConsoleTable.selectHead();
-        for (int i = 0; i < rs2.length; i++) {
-          ConsoleTable.printLine(i, getRow(), getColumn(),
-            new String(rs2[i], HConstants.UTF8_ENCODING));
+        for (Text columnKey : results.keySet()) {
+          String cellData = new String(results.get(columnKey), HConstants.UTF8_ENCODING);
+          ConsoleTable.printLine(count, rowKey.toString(), columnKey.toString(), cellData);
         }
-        ConsoleTable.selectFoot();
-
-        break;
-
-      case 5:
+        count++;
+      }
+      ConsoleTable.selectFoot();
+      scan.close();
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+  }
 
-        byte[][] rs3 = table.get(new Text(getRow()), new Text(getColumn()), getTime(), this.limit);
+  public Text[] getColumns(HBaseAdmin admin) {
+    Text[] cols = null;
 
-        ConsoleTable.selectHead();
-        for (int i = 0; i < rs3.length; i++) {
-          ConsoleTable.printLine(i, getRow(), getColumn(), new String(rs3[i], HConstants.UTF8_ENCODING));
+    try {
+      if (this.columns.contains("*")) {
+        HTableDescriptor[] tables = admin.listTables();
+        if (this.tableName.equals(HConstants.ROOT_TABLE_NAME)
+            || this.tableName.equals(HConstants.META_TABLE_NAME)) {
+          cols = HConstants.COLUMN_FAMILY_ARRAY;
+        } else {
+          for (int i = 0; i < tables.length; i++) {
+            if (tables[i].getName().equals(this.tableName)) {
+              cols = tables[i].families().keySet().toArray(new Text[] {});
+            }
+          }
         }
-        ConsoleTable.selectFoot();
-
-        break;
-
+      } else {
+        List<Text> tmpList = new ArrayList<Text>();
+        for (int i = 0; i < this.columns.size(); i++) {
+          Text column = null;
+          if(this.columns.get(i).contains(":"))
+            column = new Text(this.columns.get(i));
+          else
+            column = new Text(this.columns.get(i) + ":");
+          
+          tmpList.add(column);
+        }
+        cols = tmpList.toArray(new Text[] {});
       }
-
-      return new ReturnMsg(1, "Successfully print out the selected data.");
     } catch (IOException e) {
-      String[] msg = e.getMessage().split("[,]");
-      return new ReturnMsg(0, msg[0]);
+      e.printStackTrace();
     }
+    return cols;
+  }
+
+  private boolean checkLimit(int count) {
+    return (this.limit == 0)? true: (this.limit > count) ? true : false;
   }
 
   public void setTable(String table) {
-    this.table = new Text(table);
+    this.tableName = new Text(table);
   }
 
   public void setLimit(int limit) {
     this.limit = limit;
   }
 
-  public void setCondition(Map<String, List<String>> cond) {
-    this.condition = cond;
+  public void setWhere(boolean isWhereClause) {
+    if (isWhereClause)
+      this.whereClause = true;
   }
 
-  public String getRow() {
-    return this.condition.get("row").get(1);
+  public void setTimestamp(String timestamp) {
+    this.timestamp = Long.parseLong(timestamp);
   }
 
-  public String getColumn() {
-    return this.condition.get("column").get(1);
+  public void setColumns(List<String> columns) {
+    this.columns = columns;
   }
 
-  public long getTime() {
-    return Long.parseLong(this.condition.get("time").get(1));
+  public void setRowKey(String rowKey) {
+    if(rowKey == null) 
+      this.rowKey = null; 
+    else
+      this.rowKey = new Text(rowKey);
   }
 
-  public int getConditionSize() {
-    return this.condition.size();
-  }
-
-  public int getCondition() {
-    int type = 0;
-    if (this.condition == null) {
-      type = 0;
-    } else if (this.condition.containsKey("row")) {
-      if (getConditionSize() == 1) {
-        type = 1;
-      } else if (this.condition.containsKey("column")) {
-        if (getConditionSize() == 2) {
-          if (this.limit == 0) {
-            type = 3;
-          } else {
-            type = 4;
-          }
-        } else {
-          type = 5;
-        }
-      }
-    } else if (this.condition.containsKey("column")) {
-      type = 2;
-    }
-    return type;
+  public void setVersion(int version) {
+    this.version = version;
   }
 }

+ 3 - 1
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/ShowCommand.java

@@ -25,6 +25,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseAdmin;
 import org.apache.hadoop.hbase.HTableDescriptor;
 
+/**
+ * Shows all available tables.
+ */
 public class ShowCommand extends BasicCommand {
 
   private String command;
@@ -62,5 +65,4 @@ public class ShowCommand extends BasicCommand {
   public void setArgument(String argument) {
     this.command = argument;
   }
-
 }

File diff suppressed because it is too large
+ 587 - 253
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/Parser.java


+ 86 - 57
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/ParserConstants.java

@@ -1,60 +1,65 @@
 /* Generated By:JavaCC: Do not edit this line. ParserConstants.java */
-/**
- * Copyright 2007 The Apache Software Foundation
- *
- * 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.hadoop.hbase.shell.generated;
 
 public interface ParserConstants {
 
   int EOF = 0;
   int HELP = 5;
-  int CLEAR = 6;
-  int SHOW = 7;
-  int DESCRIBE = 8;
-  int CREATE = 9;
-  int DROP = 10;
-  int FS = 11;
-  int EXIT = 12;
-  int INSERT = 13;
-  int DELETE = 14;
-  int SELECT = 15;
-  int ROW = 16;
-  int COLUMN = 17;
-  int TIME = 18;
-  int VALUES = 19;
-  int COLUMNFAMILIES = 20;
-  int WHERE = 21;
-  int LIMIT = 22;
-  int AND = 23;
-  int OR = 24;
-  int COMMA = 25;
-  int DOT = 26;
-  int LPAREN = 27;
-  int RPAREN = 28;
-  int EQUALS = 29;
-  int NOTEQUAL = 30;
-  int OPTIONS = 31;
-  int ID = 32;
-  int NUM = 33;
-  int STRING = 34;
-  int QUOTED_STRING = 35;
-  int STRING_LITERAL = 36;
+  int ALTER = 6;
+  int CLEAR = 7;
+  int SHOW = 8;
+  int DESCRIBE = 9;
+  int DESC = 10;
+  int CREATE = 11;
+  int DROP = 12;
+  int FS = 13;
+  int EXIT = 14;
+  int INSERT = 15;
+  int INTO = 16;
+  int TABLE = 17;
+  int DELETE = 18;
+  int SELECT = 19;
+  int ENABLE = 20;
+  int DISABLE = 21;
+  int STARTING = 22;
+  int WHERE = 23;
+  int FROM = 24;
+  int ROW = 25;
+  int VALUES = 26;
+  int COLUMNFAMILIES = 27;
+  int TIMESTAMP = 28;
+  int NUM_VERSIONS = 29;
+  int LIMIT = 30;
+  int AND = 31;
+  int OR = 32;
+  int COMMA = 33;
+  int DOT = 34;
+  int LPAREN = 35;
+  int RPAREN = 36;
+  int EQUALS = 37;
+  int NOTEQUAL = 38;
+  int ASTERISK = 39;
+  int MAX_VERSIONS = 40;
+  int MAX_LENGTH = 41;
+  int COMPRESSION = 42;
+  int NONE = 43;
+  int BLOCK = 44;
+  int RECORD = 45;
+  int IN_MEMORY = 46;
+  int BLOOMFILTER = 47;
+  int COUNTING_BLOOMFILTER = 48;
+  int RETOUCHED_BLOOMFILTER = 49;
+  int VECTOR_SIZE = 50;
+  int NUM_HASH = 51;
+  int NUM_ENTRIES = 52;
+  int ADD = 53;
+  int CHANGE = 54;
+  int ID = 55;
+  int INTEGER_LITERAL = 56;
+  int FLOATING_POINT_LITERAL = 57;
+  int EXPONENT = 58;
+  int QUOTED_IDENTIFIER = 59;
+  int STRING_LITERAL = 60;
 
   int DEFAULT = 0;
 
@@ -65,22 +70,30 @@ public interface ParserConstants {
     "\"\\r\"",
     "\"\\n\"",
     "\"help\"",
+    "\"alter\"",
     "\"clear\"",
     "\"show\"",
     "\"describe\"",
+    "\"desc\"",
     "\"create\"",
     "\"drop\"",
     "\"fs\"",
     "\"exit\"",
     "\"insert\"",
+    "\"into\"",
+    "\"table\"",
     "\"delete\"",
     "\"select\"",
+    "\"enable\"",
+    "\"disable\"",
+    "\"starting\"",
+    "\"where\"",
+    "\"from\"",
     "\"row\"",
-    "\"column\"",
-    "\"time\"",
     "\"values\"",
     "\"columnfamilies\"",
-    "\"where\"",
+    "\"timestamp\"",
+    "\"num_versions\"",
     "\"limit\"",
     "\"and\"",
     "\"or\"",
@@ -90,11 +103,27 @@ public interface ParserConstants {
     "\")\"",
     "\"=\"",
     "\"<>\"",
-    "\"-\"",
+    "\"*\"",
+    "\"max_versions\"",
+    "\"max_length\"",
+    "\"compression\"",
+    "\"none\"",
+    "\"block\"",
+    "\"record\"",
+    "\"in_memory\"",
+    "\"bloomfilter\"",
+    "\"counting_bloomfilter\"",
+    "\"retouched_bloomfilter\"",
+    "\"vector_size\"",
+    "\"num_hash\"",
+    "\"num_entries\"",
+    "\"add\"",
+    "\"change\"",
     "<ID>",
-    "<NUM>",
-    "<STRING>",
-    "<QUOTED_STRING>",
+    "<INTEGER_LITERAL>",
+    "<FLOATING_POINT_LITERAL>",
+    "<EXPONENT>",
+    "<QUOTED_IDENTIFIER>",
     "<STRING_LITERAL>",
     "\";\"",
   };

File diff suppressed because it is too large
+ 653 - 165
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/ParserTokenManager.java


+ 14 - 14
src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestToString.java

@@ -47,23 +47,23 @@ public class TestToString extends TestCase {
     HTableDescriptor htd = new HTableDescriptor("hank");
     htd.addFamily(new HColumnDescriptor("hankfamily:"));
     htd.addFamily(new HColumnDescriptor(new Text("hankotherfamily:"), 10,
-        HColumnDescriptor.CompressionType.BLOCK, true, 1000, null));
+      HColumnDescriptor.CompressionType.BLOCK, true, 1000, null));
     System. out.println(htd.toString());
-    assertEquals("Table descriptor", "name: hank, families: "
-        + "{hankfamily:=(hankfamily:, max versions: 3, compression: NONE, "
-        + "in memory: false, max value length: 2147483647, bloom filter: none), "
-        + "hankotherfamily:=(hankotherfamily:, max versions: 10, "
-        + "compression: BLOCK, in memory: true, max value length: 1000, "
-        + "bloom filter: none)}", htd.toString());
+    assertEquals("Table descriptor", "name: hank, families: " +
+      "{hankfamily:={name: hankfamily, max versions: 3, compression: NONE, " +
+      "in memory: false, max length: 2147483647, bloom filter: none}, " +
+      "hankotherfamily:={name: hankotherfamily, max versions: 10, " +
+      "compression: BLOCK, in memory: true, max length: 1000, " +
+      "bloom filter: none}}", htd.toString());
     HRegionInfo hri = new HRegionInfo(-1, htd, new Text(), new Text("10"));
     System.out.println(hri.toString());
     assertEquals("HRegionInfo", 
-        "regionname: hank,,-1, startKey: <>, tableDesc: {" + "name: hank, "
-        + "families: {hankfamily:=(hankfamily:, max versions: 3, "
-        + "compression: NONE, in memory: false, max value length: 2147483647, "
-        + "bloom filter: none), hankotherfamily:=(hankotherfamily:, "
-        + "max versions: 10, compression: BLOCK, in memory: true, max value "
-        + "length: 1000, bloom filter: none)}}",
-        hri.toString());
+      "regionname: hank,,-1, startKey: <>, tableDesc: {name: hank, " +
+      "families: {hankfamily:={name: hankfamily, max versions: 3, " +
+      "compression: NONE, in memory: false, max length: 2147483647, " +
+      "bloom filter: none}, hankotherfamily:={name: hankotherfamily, " +
+      "max versions: 10, compression: BLOCK, in memory: true, " +
+      "max length: 1000, bloom filter: none}}}",
+      hri.toString());
   }
 }

+ 41 - 0
src/contrib/hbase/src/test/org/apache/hadoop/hbase/shell/TestConsoleTable.java

@@ -0,0 +1,41 @@
+/**
+ * Copyright 2007 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.shell;
+
+import junit.framework.TestCase;
+import junit.framework.TestSuite;
+import junit.textui.TestRunner;
+
+/**
+ * Test the console table class
+ * TODO: Console table needs fixing.
+ */
+public class TestConsoleTable extends TestCase {
+  public void testPrintLine() {
+    ConsoleTable.printLine(0, "smallkey", "smallcolumn", "smallcelldata");
+    ConsoleTable.printLine(0, "a large key too big for column", "smallcolumn",
+      "smallcelldata");
+    ConsoleTable.printLine(0, "smallkey", "smallcolumn", "smallcelldata");
+  }
+  
+  public static void main(String[] args) {
+    TestRunner.run(new TestSuite(TestConsoleTable.class));
+  }
+}

+ 141 - 11
src/contrib/hbase/src/test/org/apache/hadoop/hbase/shell/TestHBaseShell.java

@@ -19,19 +19,149 @@
  */
 package org.apache.hadoop.hbase.shell;
 
-import junit.framework.TestCase;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.io.UnsupportedEncodingException;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HBaseAdmin;
+import org.apache.hadoop.hbase.HBaseClusterTestCase;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.shell.generated.ParseException;
 import org.apache.hadoop.hbase.shell.generated.Parser;
+import org.apache.hadoop.io.Text;
 
 /**
- * Tests for HBase shell
+ * Tests for Hbase shell
  */
-public class TestHBaseShell extends TestCase {
-  /** test parsing */
-  public void testParse() {
-    String queryString1 = "SELECT test_table WHERE row='row_key' and " +
-      "column='column_key';";
-    new Parser(queryString1);
-    
-  }
-}
+public class TestHBaseShell extends HBaseClusterTestCase {
+  
+  protected final Log LOG = LogFactory.getLog(this.getClass().getName());
+  private ByteArrayOutputStream baos;
+  private HBaseAdmin admin;
+  
+  public TestHBaseShell() {
+    super(1 /*One region server only*/);
+  }
+  
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    // Capture System.out so we can grep for stuff in it.  Have to do it once
+    // only because ConsoleTable sets up STDOUT in a static initialization
+    this.baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(this.baos));
+    this.admin = new HBaseAdmin(this.conf);
+  }
+
+  /**
+   * Create and then drop a table.
+   * Tests also that I can use single or double quotes around table and 
+   * column family names.
+   * @throws Exception
+   */
+  public void testCreateDeleteTable() throws Exception {
+    final String tableName = getName();
+    final String columnFamily = tableName;
+    // Create table
+    createTable("create table " + tableName + " (" + columnFamily + ");",
+      tableName, columnFamily);
+    // Try describe
+    runCommand("describe " + tableName + ";");
+    // Try describe with single quotes
+    runCommand("describe '" + tableName + "';");
+    // Try describe with double-quotes
+    runCommand("describe \"" + tableName + "\";");
+    // Try dropping the table.
+    dropTable("drop table " + tableName + ";", tableName);
+    // Use double-quotes creating table.
+    final String dblQuoteSuffix = "DblQuote";
+    final String dblQuotedTableName = tableName + dblQuoteSuffix;
+    createTable("create table \"" + dblQuotedTableName + "\" (" +
+      columnFamily + ");", dblQuotedTableName, columnFamily);
+    // Use single-quotes creating table.
+    final String sglQuoteSuffix = "SglQuote";
+    final String snglQuotedTableName = tableName + sglQuoteSuffix;
+    createTable("create table '" + snglQuotedTableName + "' (" +
+      columnFamily + ");", snglQuotedTableName, columnFamily);
+    // Use double-quotes around columnfamily name.
+    final String dblQuotedColumnFamily = columnFamily + dblQuoteSuffix;
+    String tmpTableName = tableName + dblQuotedColumnFamily;
+    createTable("create table " + tmpTableName + " (\"" +
+      dblQuotedColumnFamily + "\");", tmpTableName,
+      dblQuotedColumnFamily);
+    // Use single-quotes around columnfamily name.
+    final String sglQuotedColumnFamily = columnFamily + sglQuoteSuffix;
+    tmpTableName = tableName + sglQuotedColumnFamily;
+    createTable("create table " + tmpTableName + " ('" +
+        sglQuotedColumnFamily + "');", tmpTableName, sglQuotedColumnFamily);
+  }
+
+  public void testInsertSelectDelete() throws Exception {
+    final String tableName = getName();
+    final String columnFamily = tableName;
+    createTable("create table " + tableName + " (" + columnFamily + ");",
+      tableName, columnFamily);
+    // TODO: Add asserts that inserts, selects and deletes worked.
+    runCommand("insert into " + tableName + " (" + columnFamily +
+      ") values ('" + columnFamily + "') where row='" + columnFamily + "';");
+    // Insert with double-quotes on row.
+    runCommand("insert into " + tableName + " (" + columnFamily +
+      ") values ('" + columnFamily + "') where row=\"" + columnFamily + "\";");
+    // Insert with double-quotes on row and value.
+    runCommand("insert into " + tableName + " (" + columnFamily +
+      ") values (\"" + columnFamily + "\") where row=\"" + columnFamily +
+      "\";");
+    runCommand("select \"" + columnFamily + "\" from \"" + tableName +
+      "\" where row=\"" + columnFamily + "\";");
+    runCommand("delete \"" + columnFamily + ":\" from \"" + tableName +
+        "\" where row=\"" + columnFamily + "\";");
+  }
+  
+  private void createTable(final String cmdStr, final String tableName,
+    final String columnFamily)
+  throws ParseException, IOException {
+    // Run create command.
+    runCommand(cmdStr);
+    // Assert table was created.
+    assertTrue(this.admin.tableExists(new Text(tableName)));
+    HTableDescriptor [] tables = this.admin.listTables();
+    HTableDescriptor td = null;
+    for (int i = 0; i < tables.length; i++) {
+      if (tableName.equals(tables[i].getName().toString())) {
+        td = tables[i];
+      }
+    }
+    assertNotNull(td);
+    assertTrue(td.hasFamily(new Text(columnFamily + ":")));
+  }
+  
+  private void dropTable(final String cmdStr, final String tableName)
+  throws ParseException, IOException {
+    runCommand(cmdStr);
+    // Assert its gone
+    HTableDescriptor [] tables = this.admin.listTables();
+    for (int i = 0; i < tables.length; i++) {
+      assertNotSame(tableName, tables[i].getName().toString());
+    }
+  }
+  
+  private ReturnMsg runCommand(final String cmdStr)
+  throws ParseException, UnsupportedEncodingException {
+    LOG.info("Running command: " + cmdStr);
+    Parser parser = new Parser(cmdStr);
+    Command cmd = parser.terminatedCommand();
+    ReturnMsg rm = cmd.execute(this.conf);
+    dumpStdout();
+    return rm;
+  }
+  
+  private void dumpStdout() throws UnsupportedEncodingException {
+    LOG.info("STDOUT: " +
+      new String(this.baos.toByteArray(), HConstants.UTF8_ENCODING));
+    this.baos.reset();
+  }
+}

Some files were not shown because too many files changed in this diff