瀏覽代碼

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 年之前
父節點
當前提交
6550e89e6a
共有 34 個文件被更改,包括 2804 次插入1178 次删除
  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
                 older than passed timestamp.  Fixed compaction so deleted cells
                 do not make it out into compacted output.  Ensure also that
                 do not make it out into compacted output.  Ensure also that
                 versions > column max are dropped compacting.
                 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
   OPTIMIZATIONS
 
 

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

@@ -214,10 +214,13 @@ public class HColumnDescriptor implements WritableComparable {
   /** {@inheritDoc} */
   /** {@inheritDoc} */
   @Override
   @Override
   public String toString() {
   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} */
   /** {@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,
     HRegion r = new HRegion(this.parentdir, null,
         FileSystem.get(this.conf), conf, info, null);
         FileSystem.get(this.conf), conf, info, null);
     Text [] families = info.tableDesc.families().keySet().toArray(
     Text [] families = info.tableDesc.families().keySet().toArray(
-        new Text [info.tableDesc.families.size()]);
+        new Text [info.tableDesc.getFamilies().size()]);
     HInternalScannerInterface scanner =
     HInternalScannerInterface scanner =
       r.getScanner(families, new Text(), System.currentTimeMillis(), null);
       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.DataInput;
 import java.io.DataOutput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.io.IOException;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Map;
+import java.util.SortedMap;
 import java.util.TreeMap;
 import java.util.TreeMap;
 import java.util.regex.Matcher;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import java.util.regex.Pattern;
@@ -36,8 +38,9 @@ import org.apache.hadoop.io.WritableComparable;
  * column families.
  * column families.
  */
  */
 public class HTableDescriptor implements WritableComparable {
 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':
    * Legal table names can only contain 'word characters':
@@ -183,4 +186,11 @@ public class HTableDescriptor implements WritableComparable {
     }
     }
     return result;
     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.
  * 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 {
 public class Shell {
   /** audible keyboard bells */
   /** audible keyboard bells */
   public static final boolean DEFAULT_BELL_ENABLED = true;
   public static final boolean DEFAULT_BELL_ENABLED = true;
 
 
-  /** Main method
-   * 
+  /**
+   * Main method
    * @param args not used
    * @param args not used
    * @throws IOException
    * @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();
     Configuration conf = new HBaseConfiguration();
     ConsoleReader reader = new ConsoleReader();
     ConsoleReader reader = new ConsoleReader();
     reader.setBellEnabled(conf.getBoolean("hbaseshell.jline.bell.enabled",
     reader.setBellEnabled(conf.getBoolean("hbaseshell.jline.bell.enabled",
@@ -73,7 +74,6 @@ public class Shell {
         }
         }
 
 
         long end = System.currentTimeMillis();
         long end = System.currentTimeMillis();
-
         if (rs != null && rs.getType() > -1)
         if (rs != null && rs.getType() > -1)
           System.out.println(rs.getMsg()
           System.out.println(rs.getMsg()
               + executeTime((rs.getType() == 1), start, end));
               + executeTime((rs.getType() == 1), start, end));
@@ -92,7 +92,7 @@ public class Shell {
 
 
   /** Return the string of prompt start string */
   /** Return the string of prompt start string */
   private static String getPrompt(final StringBuilder queryStr) {
   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)" :
         "(" + 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;
     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;
 import org.apache.hadoop.conf.Configuration;
 
 
+/**
+ * Clears the console screen. 
+ */
 public class ClearCommand extends BasicCommand {
 public class ClearCommand extends BasicCommand {
-
-  public ReturnMsg execute(Configuration conf) {
+  public ReturnMsg execute(@SuppressWarnings("unused") Configuration conf) {
     clear();
     clear();
     return null;
     return null;
   }
   }
@@ -42,5 +44,4 @@ public class ClearCommand extends BasicCommand {
       System.out.print("\033c");
       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;
 import org.apache.hadoop.conf.Configuration;
 
 
 public interface Command {
 public interface Command {
-  
   /** family indicator */
   /** family indicator */
   public static final String 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);
   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.
  * Parser uses command factories to create command.
  */
  */
 public interface CommandFactory {
 public interface CommandFactory {
-  
   Command getCommand();
   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.
  * Manufactures console table, but stupid.
  */
  */
 public class ConsoleTable {
 public class ConsoleTable {
+  
   private static PrintStream out;
   private static PrintStream out;
+  private static final String sBar = "+------+----------------------+";
+  private static final String lBar = "----------------------+----------------------+";
+  
   static {
   static {
     try {
     try {
       out = new PrintStream(System.out, true, "UTF-8");
       out = new PrintStream(System.out, true, "UTF-8");
@@ -36,29 +40,24 @@ public class ConsoleTable {
   }
   }
   
   
   public static void printHead(String name) {
   public static void printHead(String name) {
-    out.println("+------+----------------------+");
+    out.println(sBar);
     out.print("| No.  | ");
     out.print("| No.  | ");
-    out.printf("%-20s", name);
-    out.println(" |");
+    printCell(name, " |", true);
   }
   }
 
 
   public static void printFoot() {
   public static void printFoot() {
-    out.println("+------+----------------------+");
+    out.println(sBar);
     out.println();
     out.println();
   }
   }
 
 
   public static void printTable(int count, String name) {
   public static void printTable(int count, String name) {
-    out.println("+------+----------------------+");
-
+    out.println(sBar);
     if (name.length() > 20) {
     if (name.length() > 20) {
       int interval = 20;
       int interval = 20;
-
       out.print("| ");
       out.print("| ");
-      out.printf("%-4s", count + 1);
+      out.printf("%-4s", Integer.valueOf(count + 1));
       out.print(" | ");
       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++) {
       for (int i = 0; i < name.length() / interval; i++) {
         out.print("| ");
         out.print("| ");
         out.printf("%-4s", "");
         out.printf("%-4s", "");
@@ -66,64 +65,42 @@ public class ConsoleTable {
 
 
         int end = ((interval * i) + interval + interval);
         int end = ((interval * i) + interval + interval);
         if (end > name.length()) {
         if (end > name.length()) {
-          out.printf("%-20s", name.substring(end - interval,
-            name.length()));
+          printCell(name.substring(end - interval, name.length()), " |", true);
         } else {
         } else {
-          out.printf("%-20s", name.substring(end - interval, end));
+          printCell(name.substring(end - interval, end), " |", true);
         }
         }
-        out.println(" |");
       }
       }
-
     } else {
     } else {
       out.print("| ");
       out.print("| ");
-      out.printf("%-4s", count + 1);
+      out.printf("%-4s", Integer.valueOf(count + 1));
       out.print(" | ");
       out.print(" | ");
-      out.printf("%-20s", name);
-      out.println(" |");
+      printCell(name, " |", true);
     }
     }
   }
   }
 
 
   public static void selectHead() {
   public static void selectHead() {
-    out.println("+------+----------------------+" +
-      "----------------------+----------------------+");
+    out.println(sBar + lBar);
     out.print("| No.  | ");
     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,
   public static void printLine(int count, String key, String column,
       String cellData) {
       String cellData) {
-    out.println("+------+----------------------+" +
-      "----------------------+----------------------+");
-
+    out.println(sBar + lBar);
     if (key.length() > 20 || column.length() > 20 || cellData.length() > 20) {
     if (key.length() > 20 || column.length() > 20 || cellData.length() > 20) {
       int interval = 20;
       int interval = 20;
       out.print("| ");
       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(" | ");
       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(),
       int biggerStrLength = getBiggerInt(new int[] { key.length(),
-        column.length(), cellData.length() });
+          column.length(), cellData.length() });
 
 
       for (int i = 0; i < (biggerStrLength / interval); i++) {
       for (int i = 0; i < (biggerStrLength / interval); i++) {
         out.print("| ");
         out.print("| ");
@@ -132,58 +109,50 @@ public class ConsoleTable {
 
 
         int end = ((interval * i) + interval + interval);
         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 {
     } else {
       out.print("| ");
       out.print("| ");
-      out.printf("%-4s", count + 1);
+      out.printf("%-4s", Integer.valueOf(count + 1));
       out.print(" | ");
       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) {
   public static int getBiggerInt(int[] integers) {
     int result = -1;
     int result = -1;
     for (int i = 0; i < integers.length; i++) {
     for (int i = 0; i < integers.length; i++) {
@@ -195,9 +164,7 @@ public class ConsoleTable {
   }
   }
 
 
   public static void selectFoot() {
   public static void selectFoot() {
-    out.println("+------+----------------------+" +
-      "----------------------+----------------------+");
+    out.println(sBar + lBar);
     out.println();
     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;
 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.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseAdmin;
 import org.apache.hadoop.hbase.HBaseAdmin;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 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.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) {
   public ReturnMsg execute(Configuration conf) {
-    if (this.table == null || this.columnfamilies == null)
-      return new ReturnMsg(0, "Syntax error : Please check 'Create' syntax.");
-
     try {
     try {
-      HConnection conn = HConnectionManager.getConnection(conf);
       HBaseAdmin admin = new HBaseAdmin(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) {
   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;
 package org.apache.hadoop.hbase.shell;
 
 
 import java.io.IOException;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.List;
-import java.util.Map;
-import java.util.Set;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseAdmin;
 import org.apache.hadoop.hbase.HTable;
 import org.apache.hadoop.hbase.HTable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 
 
+/**
+ * Deletes values from tables.
+ */
 public class DeleteCommand extends BasicCommand {
 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) {
   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 {
     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) {
     } 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) {
   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 java.io.IOException;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConnection;
 import org.apache.hadoop.hbase.HConnection;
 import org.apache.hadoop.hbase.HConnectionManager;
 import org.apache.hadoop.hbase.HConnectionManager;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 
 
+/**
+ * Prints information about tables.
+ */
 public class DescCommand extends BasicCommand {
 public class DescCommand extends BasicCommand {
   
   
-  private Text table;
+  private Text tableName;
 
 
   public ReturnMsg execute(Configuration conf) {
   public ReturnMsg execute(Configuration conf) {
-    if (this.table == null) 
+    if (this.tableName == null) 
       return new ReturnMsg(0, "Syntax error : Please check 'Describe' syntax.");
       return new ReturnMsg(0, "Syntax error : Please check 'Describe' syntax.");
-
     try {
     try {
       HConnection conn = HConnectionManager.getConnection(conf);
       HConnection conn = HConnectionManager.getConnection(conf);
-      
-      if (!conn.tableExists(this.table)) {
+      if (!conn.tableExists(this.tableName)) {
         return new ReturnMsg(0, "Table not found.");
         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++) {
       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++) {
       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();
       ConsoleTable.printFoot();
-
       return new ReturnMsg(1, columns.length + " columnfamilie(s) found.");
       return new ReturnMsg(1, columns.length + " columnfamilie(s) found.");
     } catch (IOException e) {
     } catch (IOException e) {
       return new ReturnMsg(0, "error msg : " + e.toString());
       return new ReturnMsg(0, "error msg : " + e.toString());
@@ -65,7 +65,6 @@ public class DescCommand extends BasicCommand {
   }
   }
 
 
   public void setArgument(String table) {
   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;
 package org.apache.hadoop.hbase.shell;
 
 
 import java.io.IOException;
 import java.io.IOException;
+import java.util.List;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseAdmin;
 import org.apache.hadoop.hbase.HBaseAdmin;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 
 
+/**
+ * Drops tables.
+ */
 public class DropCommand extends BasicCommand {
 public class DropCommand extends BasicCommand {
-  
-  private Text table;
+  private List<String> tableList;
 
 
   public ReturnMsg execute(Configuration conf) {
   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 {
     try {
       HBaseAdmin admin = new HBaseAdmin(conf);
       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) {
     } 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 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);
     System.exit(1);
     return null;
     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.fs.FsShell;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.hadoop.util.ToolRunner;
 
 
+/**
+ * Run hadoop filesystem commands.
+ */
 public class FsCommand extends BasicCommand {
 public class FsCommand extends BasicCommand {
   private List<String> query;
   private List<String> query;
 
 
-  public ReturnMsg execute(Configuration conf) {
+  public ReturnMsg execute(@SuppressWarnings("unused") Configuration conf) {
     FsShell shell = new FsShell();
     FsShell shell = new FsShell();
     try {
     try {
       ToolRunner.run(shell, getQuery());
       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)
 PARSER_BEGIN(Parser)
+package org.apache.hadoop.hbase.shell.generated;
+
 /**
 /**
  * Copyright 2007 The Apache Software Foundation
  * Copyright 2007 The Apache Software Foundation
  *
  *
@@ -23,7 +25,6 @@ PARSER_BEGIN(Parser)
  * See the License for the specific language governing permissions and
  * See the License for the specific language governing permissions and
  * limitations under the License.
  * limitations under the License.
  */
  */
-package org.apache.hadoop.hbase.shell.generated;
 
 
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.List;
@@ -60,43 +61,72 @@ SKIP :
  | "\n"
  | "\n"
 }
 }
 
 
-TOKEN:
+TOKEN: /** for HQL statements */
 {
 {
    <HELP: "help">
    <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: "'" (~["'"])* ( "''" (~["'"])* )* "'" >
  | <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;
        return cmd;
@@ -182,6 +215,7 @@ HelpCommand helpCommand() :
        | t=<INSERT>
        | t=<INSERT>
        | t=<DELETE>
        | t=<DELETE>
        | t=<SELECT>
        | t=<SELECT>
+       | t=<ALTER>
        | t=<CLEAR>
        | t=<CLEAR>
        | t=<FS>
        | t=<FS>
        | t=<ID>
        | t=<ID>
@@ -202,9 +236,8 @@ ShowCommand showCommand() :
 {
 {
    <SHOW>
    <SHOW>
    [
    [
-       argument = getString()
+       argument = Identifier()
    ]
    ]
-   
   { 
   { 
       show.setArgument(argument);
       show.setArgument(argument);
       return show;
       return show;
@@ -217,265 +250,352 @@ DescCommand descCommand() :
   String argument = null;
   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{
       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> getLiteralValues() :
 {
 {
-    List<String> values = new ArrayList<String>();
-    String literal = null;
+  List<String> values = new ArrayList<String>();
+  String literal = null;
 }
 }
 {    
 {    
 <LPAREN>
 <LPAREN>
@@ -490,10 +610,9 @@ List<String> getLiteralValues() :
 | ( 
 | ( 
        <ID>
        <ID>
      | <STRING_LITERAL>
      | <STRING_LITERAL>
-     | <QUOTED_STRING>
-     | <STRING> 
+     | <QUOTED_IDENTIFIER>
   )  { values.removeAll(values); }
   )  { values.removeAll(values); }
-	)*
+        )*
 <RPAREN>
 <RPAREN>
    { 
    { 
      return values;
      return values;
@@ -502,10 +621,105 @@ List<String> getLiteralValues() :
 
 
 String getStringLiteral() :
 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;
 import org.apache.hadoop.conf.Configuration;
 
 
 public class HelpCommand extends BasicCommand {
 public class HelpCommand extends BasicCommand {
-  
   private String argument;
   private String argument;
 
 
-  public ReturnMsg execute(Configuration conf) {
+  public ReturnMsg execute(@SuppressWarnings("unused") Configuration conf) {
     HelpManager.printHelp(this.argument);
     HelpManager.printHelp(this.argument);
     return null;
     return null;
   }
   }
@@ -33,5 +32,4 @@ public class HelpCommand extends BasicCommand {
   public void setArgument(String argument) {
   public void setArgument(String argument) {
     this.argument = 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() {
   public static Map<? extends String, ? extends String[]> Load() {
     Map<String, String[]> load = new HashMap<String, String[]>();
     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.",
     load.put("FS", new String[] { "Hadoop FsShell operations.",
       "FS -copyFromLocal /home/user/backup.dat fs/user/backup;" });
       "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[] {
     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[] {
     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[] {
     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[] {
     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",
     load.put("SELECT",
         new String[] {
         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;" });
     load.put("EXIT", new String[] { "Exit shell", "EXIT;" });
 
 
     return load;
     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 {
 public class HelpManager {
   /** application name */
   /** application name */
-  public static final String APP_NAME = "HBase Shell";
+  public static final String APP_NAME = "Hbase Shell";
 
 
   /** version of the code */
   /** 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 */
   /** 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() {
   public HelpManager() {
     help.putAll(HelpContents.Load());
     help.putAll(HelpContents.Load());
@@ -41,7 +42,6 @@ public class HelpManager {
 
 
   /** Print out the program version. */
   /** Print out the program version. */
   public void printVersion() {
   public void printVersion() {
-    ClearCommand.clear();
     System.out.println(APP_NAME + ", " + APP_VERSION + " version.\n"
     System.out.println(APP_NAME + ", " + APP_VERSION + " version.\n"
         + "Copyright (c) 2007 by udanax, "
         + "Copyright (c) 2007 by udanax, "
         + "licensed to Apache Software Foundation.\n"
         + "licensed to Apache Software Foundation.\n"
@@ -55,12 +55,13 @@ public class HelpManager {
       for (Map.Entry<String, String[]> helpMap : help.entrySet()) {
       for (Map.Entry<String, String[]> helpMap : help.entrySet()) {
         wrapping(helpMap.getKey(), helpMap.getValue(), false);
         wrapping(helpMap.getKey(), helpMap.getValue(), false);
       }
       }
+      System.out.println();
     } else {
     } else {
       if (help.containsKey(cmd.toUpperCase())) {
       if (help.containsKey(cmd.toUpperCase())) {
         String[] msg = help.get(cmd.toUpperCase());
         String[] msg = help.get(cmd.toUpperCase());
         wrapping(cmd.toUpperCase(), msg, true);
         wrapping(cmd.toUpperCase(), msg, true);
       } else {
       } 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)
     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.io.IOException;
 import java.util.List;
 import java.util.List;
-import java.util.Map;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HTable;
 import org.apache.hadoop.hbase.HTable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 
 
+/**
+ * Inserts values into tables.
+ */
 public class InsertCommand extends BasicCommand {
 public class InsertCommand extends BasicCommand {
   
   
-  private Text table;
+  private Text tableName;
   private List<String> columnfamilies;
   private List<String> columnfamilies;
   private List<String> values;
   private List<String> values;
-  private Map<String, List<String>> condition;
+  private String rowKey;
 
 
   public ReturnMsg execute(Configuration conf) {
   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.");
       return new ReturnMsg(0, "Syntax error : Please check 'Insert' syntax.");
 
 
     if (this.columnfamilies.size() != this.values.size())
     if (this.columnfamilies.size() != this.values.size())
@@ -43,12 +45,16 @@ public class InsertCommand extends BasicCommand {
           "Mismatch between values list and columnfamilies list");
           "Mismatch between values list and columnfamilies list");
 
 
     try {
     try {
-      HTable table = new HTable(conf, this.table);
+      HTable table = new HTable(conf, this.tableName);
       long lockId = table.startUpdate(getRow());
       long lockId = table.startUpdate(getRow());
 
 
       for (int i = 0; i < this.values.size(); i++) {
       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);
       table.commit(lockId);
 
 
@@ -60,7 +66,7 @@ public class InsertCommand extends BasicCommand {
   }
   }
 
 
   public void setTable(String table) {
   public void setTable(String table) {
-    this.table = new Text(table);
+    this.tableName = new Text(table);
   }
   }
 
 
   public void setColumnfamilies(List<String> columnfamilies) {
   public void setColumnfamilies(List<String> columnfamilies) {
@@ -71,12 +77,12 @@ public class InsertCommand extends BasicCommand {
     this.values = values;
     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() {
   public Text getRow() {
-    return new Text(this.condition.get("row").get(1));
+    return new Text(this.rowKey);
   }
   }
 
 
   public Text getColumn(int i) {
   public Text getColumn(int i) {
@@ -85,5 +91,5 @@ public class InsertCommand extends BasicCommand {
 
 
   public byte[] getValue(int i) {
   public byte[] getValue(int i) {
     return this.values.get(i).getBytes();
     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;
 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 {
 public class ReturnMsg {
-  
-  private String msg;
-  private int type;
+  private final String msg;
+  private final int type;
 
 
   public ReturnMsg(int i, String string) {
   public ReturnMsg(int i, String string) {
     this.type = i;
     this.type = i;
@@ -41,5 +44,4 @@ public class ReturnMsg {
   public int getType() {
   public int getType() {
     return this.type;
     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;
 package org.apache.hadoop.hbase.shell;
 
 
 import java.io.IOException;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
 import java.util.TreeMap;
 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.DataInputBuffer;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 
 
+/**
+ * Selects values from tables.
+ * 
+ * TODO: INTO FILE is not yet implemented.
+ */
 public class SelectCommand extends BasicCommand {
 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 int limit;
-  private Map<String, List<String>> condition;
+  private int version;
+  private boolean whereClause = false;
 
 
   public ReturnMsg execute(Configuration conf) {
   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.");
       return new ReturnMsg(0, "Syntax error : Please check 'Select' syntax.");
-
+    } 
+    
     try {
     try {
-      HTable table = new HTable(conf, this.table);
+      HTable table = new HTable(conf, this.tableName);
       HBaseAdmin admin = new HBaseAdmin(conf);
       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();
         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();
           byte[] value = entry.getValue();
           String cellData = new String(value, HConstants.UTF8_ENCODING);
           String cellData = new String(value, HConstants.UTF8_ENCODING);
 
 
@@ -112,138 +104,121 @@ public class SelectCommand extends BasicCommand {
             HRegionInfo info = new HRegionInfo();
             HRegionInfo info = new HRegionInfo();
             inbuf.reset(value, value.length);
             inbuf.reset(value, value.length);
             info.readFields(inbuf);
             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++;
             count++;
           }
           }
-          results = new TreeMap<Text, byte[]>();
         }
         }
         ConsoleTable.selectFoot();
         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) {
     } 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) {
   public void setTable(String table) {
-    this.table = new Text(table);
+    this.tableName = new Text(table);
   }
   }
 
 
   public void setLimit(int limit) {
   public void setLimit(int limit) {
     this.limit = 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.HBaseAdmin;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 
 
+/**
+ * Shows all available tables.
+ */
 public class ShowCommand extends BasicCommand {
 public class ShowCommand extends BasicCommand {
 
 
   private String command;
   private String command;
@@ -62,5 +65,4 @@ public class ShowCommand extends BasicCommand {
   public void setArgument(String argument) {
   public void setArgument(String argument) {
     this.command = argument;
     this.command = argument;
   }
   }
-
 }
 }

文件差異過大導致無法顯示
+ 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 */
 /* 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;
 package org.apache.hadoop.hbase.shell.generated;
 
 
 public interface ParserConstants {
 public interface ParserConstants {
 
 
   int EOF = 0;
   int EOF = 0;
   int HELP = 5;
   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;
   int DEFAULT = 0;
 
 
@@ -65,22 +70,30 @@ public interface ParserConstants {
     "\"\\r\"",
     "\"\\r\"",
     "\"\\n\"",
     "\"\\n\"",
     "\"help\"",
     "\"help\"",
+    "\"alter\"",
     "\"clear\"",
     "\"clear\"",
     "\"show\"",
     "\"show\"",
     "\"describe\"",
     "\"describe\"",
+    "\"desc\"",
     "\"create\"",
     "\"create\"",
     "\"drop\"",
     "\"drop\"",
     "\"fs\"",
     "\"fs\"",
     "\"exit\"",
     "\"exit\"",
     "\"insert\"",
     "\"insert\"",
+    "\"into\"",
+    "\"table\"",
     "\"delete\"",
     "\"delete\"",
     "\"select\"",
     "\"select\"",
+    "\"enable\"",
+    "\"disable\"",
+    "\"starting\"",
+    "\"where\"",
+    "\"from\"",
     "\"row\"",
     "\"row\"",
-    "\"column\"",
-    "\"time\"",
     "\"values\"",
     "\"values\"",
     "\"columnfamilies\"",
     "\"columnfamilies\"",
-    "\"where\"",
+    "\"timestamp\"",
+    "\"num_versions\"",
     "\"limit\"",
     "\"limit\"",
     "\"and\"",
     "\"and\"",
     "\"or\"",
     "\"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>",
     "<ID>",
-    "<NUM>",
-    "<STRING>",
-    "<QUOTED_STRING>",
+    "<INTEGER_LITERAL>",
+    "<FLOATING_POINT_LITERAL>",
+    "<EXPONENT>",
+    "<QUOTED_IDENTIFIER>",
     "<STRING_LITERAL>",
     "<STRING_LITERAL>",
     "\";\"",
     "\";\"",
   };
   };

文件差異過大導致無法顯示
+ 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");
     HTableDescriptor htd = new HTableDescriptor("hank");
     htd.addFamily(new HColumnDescriptor("hankfamily:"));
     htd.addFamily(new HColumnDescriptor("hankfamily:"));
     htd.addFamily(new HColumnDescriptor(new Text("hankotherfamily:"), 10,
     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());
     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"));
     HRegionInfo hri = new HRegionInfo(-1, htd, new Text(), new Text("10"));
     System.out.println(hri.toString());
     System.out.println(hri.toString());
     assertEquals("HRegionInfo", 
     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;
 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.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();
+  }
+}

部分文件因文件數量過多而無法顯示