You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by st...@apache.org on 2007/09/28 23:10:50 UTC

svn commit: r580487 - in /lucene/hadoop/trunk/src/contrib/hbase: ./ src/java/org/apache/hadoop/hbase/ src/java/org/apache/hadoop/hbase/shell/ src/java/org/apache/hadoop/hbase/shell/generated/ src/test/org/apache/hadoop/hbase/ src/test/org/apache/hadoop...

Author: stack
Date: Fri Sep 28 14:10:49 2007
New Revision: 580487

URL: http://svn.apache.org/viewvc?rev=580487&view=rev
Log:
HADOOP-1789 output formatting

Added:
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/TableFormatter.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/TableFormatterFactory.java
Removed:
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/ConsoleTable.java
Modified:
    lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HAbstractScanner.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInterface.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStore.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HTable.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/Shell.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/BasicCommand.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/ClearCommand.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/DescCommand.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/HBaseShell.jj
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/SelectCommand.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/ShowCommand.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/Parser.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/ParserTokenManager.java
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestScanner2.java
    lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/shell/TestConsoleTable.java

Modified: lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt?rev=580487&r1=580486&r2=580487&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt Fri Sep 28 14:10:49 2007
@@ -81,7 +81,8 @@
     HADOOP-1884 Remove useless debugging log messages from hbase.mapred
     HADOOP-1856 Add Jar command to hbase shell using Hadoop RunJar util
                 (Edward Yoon via Stack)
-    HADOOP-1928 ] Have master pass the regionserver the filesystem to use
+    HADOOP-1928 Have master pass the regionserver the filesystem to use
+    HADOOP-1789 Output formatting
 
 
 Below are the list of changes before 2007-08-18

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HAbstractScanner.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HAbstractScanner.java?rev=580487&r1=580486&r2=580487&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HAbstractScanner.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HAbstractScanner.java Fri Sep 28 14:10:49 2007
@@ -55,9 +55,11 @@
   /**
    * This class provides column matching functions that are more sophisticated
    * than a simple string compare. There are three types of matching:
-   * 1. Match on the column family name only
-   * 2. Match on the column family + column key regex
-   * 3. Simple match: compare column family + column key literally
+   * <ol>
+   * <li>Match on the column family name only</li>
+   * <li>Match on the column family + column key regex</li>
+   * <li>Simple match: compare column family + column key literally</li>
+   * </ul>
    */
   private static class ColumnMatcher {
     private boolean wildCardmatch;
@@ -137,11 +139,11 @@
         matchers = new Vector<ColumnMatcher>();
       }
       ColumnMatcher matcher = new ColumnMatcher(targetCols[i]);
-      if(matcher.isWildCardMatch()) {
+      if (matcher.isWildCardMatch()) {
         this.wildcardMatch = true;
       }
       matchers.add(matcher);
-      if(matchers.size() > 1) {
+      if (matchers.size() > 1) {
         this.multipleMatchers = true;
       }
       okCols.put(family, matchers);
@@ -272,4 +274,4 @@
     }
     return insertedItem;
   }
-}
+}
\ No newline at end of file

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java?rev=580487&r1=580486&r2=580487&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java Fri Sep 28 14:10:49 2007
@@ -211,7 +211,6 @@
     return this.bloomFilter;
   }
 
-  /** {@inheritDoc} */
   @Override
   public String toString() {
     // Output a name minus ':'.

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java?rev=580487&r1=580486&r2=580487&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java Fri Sep 28 14:10:49 2007
@@ -1052,9 +1052,14 @@
 
   /**
    * Return an iterator that scans over the HRegion, returning the indicated 
-   * columns for only the rows that match the data filter.  This Iterator must be closed by the caller.
+   * columns for only the rows that match the data filter.  This Iterator must
+   * be closed by the caller.
    *
-   * @param cols columns desired in result set
+   * @param cols columns to scan. If column name is a column family, all
+   * columns of the specified column family are returned.  Its also possible
+   * to pass a regex in the column qualifier. A column qualifier is judged to
+   * be a regex if it contains at least one of the following characters:
+   * <code>\+|^&*$[]]}{)(</code>.
    * @param firstRow row which is the starting point of the scan
    * @param timestamp only return rows whose timestamp is <= this value
    * @param filter row filter
@@ -1070,7 +1075,6 @@
       for(int i = 0; i < cols.length; i++) {
         families.add(HStoreKey.extractFamily(cols[i]));
       }
-
       List<HStore> storelist = new ArrayList<HStore>();
       for (Text family: families) {
         HStore s = stores.get(family);
@@ -1540,8 +1544,8 @@
     private HInternalScannerInterface[] scanners;
     private TreeMap<Text, byte []>[] resultSets;
     private HStoreKey[] keys;
-    private boolean wildcardMatch;
-    private boolean multipleMatchers;
+    private boolean wildcardMatch = false;
+    private boolean multipleMatchers = false;
     private RowFilterInterface dataFilter;
 
     /** Create an HScanner with a handle on many HStores. */
@@ -1555,8 +1559,6 @@
       this.scanners = new HInternalScannerInterface[stores.length + 1];
       this.resultSets = new TreeMap[scanners.length];
       this.keys = new HStoreKey[scanners.length];
-      this.wildcardMatch = false;
-      this.multipleMatchers = false;
 
       // Advance to the first key in each store.
       // All results will match the required column-set and scanTime.
@@ -1601,21 +1603,14 @@
       }
     }
 
-    /**
-     * {@inheritDoc}
-     */
     public boolean isWildcardScanner() {
       return wildcardMatch;
     }
     
-    /**
-     * {@inheritDoc}
-     */
     public boolean isMultipleMatchScanner() {
       return multipleMatchers;
     }
 
-    /** {@inheritDoc} */
     public boolean next(HStoreKey key, SortedMap<Text, byte[]> results)
     throws IOException {
       // Filtered flag is set by filters.  If a cell has been 'filtered out'

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInterface.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInterface.java?rev=580487&r1=580486&r2=580487&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInterface.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInterface.java Fri Sep 28 14:10:49 2007
@@ -131,7 +131,11 @@
    * Opens a remote scanner with a RowFilter.
    * 
    * @param regionName name of region to scan
-   * @param columns columns to scan
+   * @param columns columns to scan. If column name is a column family, all
+   * columns of the specified column family are returned.  Its also possible
+   * to pass a regex for column family name. A column name is judged to be
+   * regex if it contains at least one of the following characters:
+   * <code>\+|^&*$[]]}{)(</code>.
    * @param startRow starting row to scan
    * @param timestamp only return values whose timestamp is <= this value
    * @param filter RowFilter for filtering results at the row-level.

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java?rev=580487&r1=580486&r2=580487&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java Fri Sep 28 14:10:49 2007
@@ -1093,11 +1093,9 @@
   // remote scanner interface
   //
 
-  /** {@inheritDoc} */
   public long openScanner(Text regionName, Text[] cols, Text firstRow,
-      final long timestamp, final RowFilterInterface filter)
-    throws IOException {
-    
+    final long timestamp, final RowFilterInterface filter)
+  throws IOException {
     requestCount.incrementAndGet();
     try {
       HRegion r = getRegion(regionName);

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStore.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStore.java?rev=580487&r1=580486&r2=580487&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStore.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStore.java Fri Sep 28 14:10:49 2007
@@ -1447,7 +1447,6 @@
         }
         
       } catch (Exception ex) {
-        LOG.error("Failed construction", ex);
         close();
         IOException e = new IOException("HStoreScanner failed construction");
         e.initCause(ex);

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HTable.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HTable.java?rev=580487&r1=580486&r2=580487&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HTable.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HTable.java Fri Sep 28 14:10:49 2007
@@ -322,8 +322,8 @@
   /** 
    * Get all the data for the specified row
    * 
-   * @param row         - row key
-   * @return            - map of colums to values
+   * @param row row key
+   * @return map of colums to values
    * @throws IOException
    */
   public SortedMap<Text, byte[]> getRow(Text row) throws IOException {
@@ -373,7 +373,11 @@
    * Get a scanner on the current table starting at the specified row.
    * Return the specified columns.
    *
-   * @param columns array of columns to return
+   * @param columns columns to scan. If column name is a column family, all
+   * columns of the specified column family are returned.  Its also possible
+   * to pass a regex in the column qualifier. A column qualifier is judged to
+   * be a regex if it contains at least one of the following characters:
+   * <code>\+|^&*$[]]}{)(</code>.
    * @param startRow starting row in table to scan
    * @return scanner
    * @throws IOException
@@ -388,7 +392,11 @@
    * Get a scanner on the current table starting at the specified row.
    * Return the specified columns.
    *
-   * @param columns array of columns to return
+   * @param columns columns to scan. If column name is a column family, all
+   * columns of the specified column family are returned.  Its also possible
+   * to pass a regex in the column qualifier. A column qualifier is judged to
+   * be a regex if it contains at least one of the following characters:
+   * <code>\+|^&*$[]]}{)(</code>.
    * @param startRow starting row in table to scan
    * @param timestamp only return results whose timestamp <= this value
    * @return scanner
@@ -404,7 +412,11 @@
    * Get a scanner on the current table starting at the specified row.
    * Return the specified columns.
    *
-   * @param columns array of columns to return
+   * @param columns columns to scan. If column name is a column family, all
+   * columns of the specified column family are returned.  Its also possible
+   * to pass a regex in the column qualifier. A column qualifier is judged to
+   * be a regex if it contains at least one of the following characters:
+   * <code>\+|^&*$[]]}{)(</code>.
    * @param startRow starting row in table to scan
    * @param filter a row filter using row-key regexp and/or column data filter.
    * @return scanner
@@ -420,7 +432,11 @@
    * Get a scanner on the current table starting at the specified row.
    * Return the specified columns.
    *
-   * @param columns array of columns to return
+   * @param columns columns to scan. If column name is a column family, all
+   * columns of the specified column family are returned.  Its also possible
+   * to pass a regex in the column qualifier. A column qualifier is judged to
+   * be a regex if it contains at least one of the following characters:
+   * <code>\+|^&*$[]]}{)(</code>.
    * @param startRow starting row in table to scan
    * @param timestamp only return results whose timestamp <= this value
    * @param filter a row filter using row-key regexp and/or column data filter.

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/Shell.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/Shell.java?rev=580487&r1=580486&r2=580487&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/Shell.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/Shell.java Fri Sep 28 14:10:49 2007
@@ -93,7 +93,7 @@
 
   /** Return the string of prompt start string */
   private static String getPrompt(final StringBuilder queryStr) {
-    return (queryStr.toString().equals("")) ? "Hbase > " : "    --> ";
+    return (queryStr.toString().equals("")) ? "Hbase> " : "   --> ";
   }
 
   /**
@@ -102,8 +102,9 @@
    * @param end end of time interval
    * @return a string of code execution time. */
   public static String executeTime(boolean watch, long start, long end) {
-    return (watch) ?
-        "(" + String.format("%.2f", (end - start) * 0.001) + " sec)" :
-          "";
+    return watch?
+      " (" + String.format("%.2f", Double.valueOf((end - start) * 0.001)) +
+        " sec)":
+      "";
   }
 }

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/BasicCommand.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/BasicCommand.java?rev=580487&r1=580486&r2=580487&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/BasicCommand.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/BasicCommand.java Fri Sep 28 14:10:49 2007
@@ -51,4 +51,4 @@
     return (!column.endsWith(FAMILY_INDICATOR))?
       column + FAMILY_INDICATOR: column;
   } 
-}
+}
\ No newline at end of file

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/ClearCommand.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/ClearCommand.java?rev=580487&r1=580486&r2=580487&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/ClearCommand.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/ClearCommand.java Fri Sep 28 14:10:49 2007
@@ -44,4 +44,4 @@
       System.out.print("\033c");
     }
   }
-}
+}
\ No newline at end of file

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/DescCommand.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/DescCommand.java?rev=580487&r1=580486&r2=580487&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/DescCommand.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/DescCommand.java Fri Sep 28 14:10:49 2007
@@ -32,16 +32,17 @@
  * Prints information about tables.
  */
 public class DescCommand extends BasicCommand {
-  
+  private static final String [] HEADER =
+    new String [] {"Column Family Descriptor"};
   private Text tableName;
 
   public ReturnMsg execute(Configuration conf) {
     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 {
       HConnection conn = HConnectionManager.getConnection(conf);
       if (!conn.tableExists(this.tableName)) {
-        return new ReturnMsg(0, "Table not found.");
+        return new ReturnMsg(0, "Table not found");
       }
       HTableDescriptor [] tables = conn.listTables();
       HColumnDescriptor [] columns = null;
@@ -52,13 +53,21 @@
           break;
         }
       }
-      ConsoleTable.printHead("ColumnFamily");
-      for (int ii = 0; ii < columns.length; ii++) {
-        String tmp = columns[ii].toString();
-        ConsoleTable.printTable(ii, tmp.substring(1, tmp.length() - 1));
+      TableFormatter formatter = TableFormatterFactory.get();
+      formatter.header(HEADER);
+      // Do a toString on the HColumnDescriptors
+      String [] columnStrs = new String[columns.length];
+      for (int i = 0; i < columns.length; i++) {
+        String tmp = columns[i].toString();
+        // Strip the curly-brackets if present.
+        if (tmp.length() > 2 && tmp.startsWith("{") && tmp.endsWith("}")) {
+          tmp = tmp.substring(1, tmp.length() - 1);
+        }
+        columnStrs[i] = tmp;
       }
-      ConsoleTable.printFoot();
-      return new ReturnMsg(1, columns.length + " columnfamilie(s) found.");
+      formatter.row(columnStrs);
+      formatter.footer();
+      return new ReturnMsg(1, columns.length + " columnfamily(s) in set");
     } catch (IOException e) {
       return new ReturnMsg(0, "error msg : " + e.toString());
     }

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/HBaseShell.jj
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/HBaseShell.jj?rev=580487&r1=580486&r2=580487&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/HBaseShell.jj (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/HBaseShell.jj Fri Sep 28 14:10:49 2007
@@ -118,7 +118,7 @@
 
 TOKEN : /** Literals */
 {
-   <ID: (["A"-"Z","a"-"z","0"-"9","_","-",".",":","/"])+ >
+   <ID: ["A"-"Z","a"-"z","_","-",".",":","/"] (["A"-"Z","a"-"z","0"-"9","_","-",".",":","/"])* >
  | <INTEGER_LITERAL: (["0"-"9"])+ >
  | <FLOATING_POINT_LITERAL:
    (["0"-"9"])+ "." (["0"-"9"])+ (<EXPONENT>)?
@@ -555,8 +555,7 @@
   ]
    
   [
-     <NUM_VERSIONS>
-      numVersion = Number()
+     <NUM_VERSIONS><EQUALS> numVersion = Number()
       {
         select.setVersion(numVersion);
       }

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/SelectCommand.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/SelectCommand.java?rev=580487&r1=580486&r2=580487&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/SelectCommand.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/SelectCommand.java Fri Sep 28 14:10:49 2007
@@ -21,19 +21,22 @@
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseAdmin;
+import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HScannerInterface;
 import org.apache.hadoop.hbase.HStoreKey;
 import org.apache.hadoop.hbase.HTable;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.hbase.shell.generated.ParseException;
+import org.apache.hadoop.hbase.shell.generated.Parser;
+import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.io.Text;
 
 /**
@@ -48,122 +51,190 @@
   private List<String> columns;
   private long timestamp;
   private int limit;
+  // Count of versions to return.
   private int version;
   private boolean whereClause = false;
+  private static final String [] HEADER_ROW_CELL =
+    new String [] {"Row", "Cell"};
+  private static final String [] HEADER_COLUMN_CELL =
+    new String [] {"Column", "Cell"};
+  private static final String [] HEADER =
+    new String [] {"Row", "Column", "Cell"};
 
   public ReturnMsg execute(Configuration conf) {
     if (this.tableName.equals("") || this.rowKey == null ||
         this.columns.size() == 0) {
       return new ReturnMsg(0, "Syntax error : Please check 'Select' syntax.");
     } 
-    
     try {
       HTable table = new HTable(conf, this.tableName);
       HBaseAdmin admin = new HBaseAdmin(conf);
+      int count = 0;
       if (this.whereClause) {
-        compoundWherePrint(table, admin);
+        count = compoundWherePrint(table, admin);
       } else {
-        scanPrint(table, admin);
+        count = scanPrint(table, admin);
       }
-      return new ReturnMsg(1, "Successfully print out the selected data.");
+      return new ReturnMsg(1, Integer.toString(count) + " row(s) in set");
     } catch (IOException e) {
       String[] msg = e.getMessage().split("[,]");
       return new ReturnMsg(0, msg[0]);
     }
   }
 
-  private void compoundWherePrint(HTable table, HBaseAdmin admin) {
+  private int compoundWherePrint(HTable table, HBaseAdmin admin) {
+    int count = 0;
+    TableFormatter formatter = TableFormatterFactory.get();
     try {
       if (this.version != 0) {
+        // A number of versions has been specified.
         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);
+        ParsedColumns parsedColumns = getColumns(admin, false);
+        boolean multiple = parsedColumns.isMultiple() || this.version > 1;
+        formatter.header(multiple? HEADER_COLUMN_CELL: null);
+        for (Text column: parsedColumns.getColumns()) {
+          if (this.timestamp != 0) {
+            result = table.get(this.rowKey, column, this.timestamp,
+              this.version);
           } else {
-            result = table.get(this.rowKey, cols[i], this.version);
+            result = table.get(this.rowKey, column, this.version);
           }
-
-          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));
+          for (int ii = 0; result != null && ii < result.length; ii++) {
+            if (multiple) {
+              formatter.row(new String [] {column.toString(),
+                toString(column, result[ii])});
+            } else {
+              formatter.row(new String [] {toString(column, result[ii])});
+            }
+            count++;
           }
-          ConsoleTable.selectFoot();
         }
       } else {
-        int count = 0;
-        ConsoleTable.selectHead();
-        
-        for (Map.Entry<Text, byte[]> entry : table.getRow(this.rowKey).entrySet()) {
-          byte[] value = entry.getValue();
-          String cellData = new String(value, HConstants.UTF8_ENCODING);
-
-          if (entry.getKey().equals(HConstants.COL_REGIONINFO)) {
-            DataInputBuffer inbuf = new DataInputBuffer();
-            HRegionInfo info = new HRegionInfo();
-            inbuf.reset(value, value.length);
-            info.readFields(inbuf);
-            cellData = String.valueOf(info.getRegionId());
-          }
-
-          if (columns.contains(entry.getKey().toString()) || columns.contains("*")) {
-            ConsoleTable.printLine(count, this.rowKey.toString(), entry.getKey()
-                .toString(), cellData);
-            count++;
+        formatter.header(isMultiple()? HEADER_COLUMN_CELL: null);
+        for (Map.Entry<Text, byte[]> e: table.getRow(this.rowKey).entrySet()) {
+          Text key = e.getKey();
+          if (!this.columns.contains(key.toString())) {
+            continue;
+          }
+          String cellData = toString(key, e.getValue());
+          if (isMultiple()) {
+            formatter.row(new String [] {key.toString(), cellData});
+          } else {
+            formatter.row(new String [] {cellData});
           }
+          count++;
         }
-        ConsoleTable.selectFoot();
       }
+      formatter.footer();
     } catch (IOException e) {
       e.printStackTrace();
     }
+    return count;
   }
-
-  private void scanPrint(HTable table, HBaseAdmin admin) {
+  
+  private String toString(final Text columnName, final byte [] cell)
+  throws IOException {
+    String result = null;
+    if (columnName.equals(HConstants.COL_REGIONINFO) ||
+        columnName.equals(HConstants.COL_SPLITA) ||
+        columnName.equals(HConstants.COL_SPLITA)) {
+      result = Writables.getHRegionInfoOrNull(cell).toString();
+    } else if (columnName.equals(HConstants.COL_STARTCODE)) {
+      result = Long.toString(Writables.bytesToLong(cell));
+    } else {
+      result = Writables.bytesToString(cell);
+    }
+    return result;
+  }
+  
+  /**
+   * Data structure with columns to use scanning and whether or not the
+   * scan could return more than one column.
+   */
+  class ParsedColumns {
+    private final List<Text> cols;
+    private final boolean isMultiple;
+    
+    ParsedColumns(final List<Text> columns) {
+      this(columns, true);
+    }
+    
+    ParsedColumns(final List<Text> columns, final boolean isMultiple) {
+      this.cols = columns;
+      this.isMultiple = isMultiple;
+    }
+    
+    public List<Text> getColumns() {
+      return this.cols;
+    }
+    
+    public boolean isMultiple() {
+      return this.isMultiple;
+    }
+  }
+  
+  private int scanPrint(HTable table, HBaseAdmin admin) {
+    int count = 0;
     HScannerInterface scan = null;
     try {
+      ParsedColumns parsedColumns = getColumns(admin, true);
+      Text [] cols = parsedColumns.getColumns().toArray(new Text [] {});
       if (this.timestamp == 0) {
-        scan = table.obtainScanner(getColumns(admin), this.rowKey);
+        scan = table.obtainScanner(cols, this.rowKey);
       } else {
-        scan = table.obtainScanner(getColumns(admin), this.rowKey, this.timestamp);
+        scan = table.obtainScanner(cols, this.rowKey, this.timestamp);
       }
-
       HStoreKey key = new HStoreKey();
       TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
-
-      ConsoleTable.selectHead();
-      int count = 0;
-
+      TableFormatter formatter = TableFormatterFactory.get();
+      // If only one column in query, then don't print out the column.
+      formatter.header((parsedColumns.isMultiple())? HEADER: HEADER_ROW_CELL);
       while (scan.next(key, results) && checkLimit(count)) {
-        Text rowKey = key.getRow();
-
+        Text r = key.getRow();
         for (Text columnKey : results.keySet()) {
-          String cellData = new String(results.get(columnKey), HConstants.UTF8_ENCODING);
-          ConsoleTable.printLine(count, rowKey.toString(), columnKey.toString(), cellData);
+          String cellData = toString(columnKey, results.get(columnKey));
+          if (parsedColumns.isMultiple()) {
+            formatter.row(new String [] {r.toString(), columnKey.toString(),
+              cellData});
+          } else {
+            // Don't print out the column since only one specified in query.
+            formatter.row(new String [] {r.toString(), cellData});
+          }
+          count++;
+          if (this.limit > 0 && count >= this.limit) {
+            break;
+          }
         }
-        count++;
       }
-      ConsoleTable.selectFoot();
+      formatter.footer();
       scan.close();
     } catch (IOException e) {
       e.printStackTrace();
     }
+    return count;
   }
 
-  public Text[] getColumns(HBaseAdmin admin) {
-    Text[] cols = null;
-
+  /**
+   * Make sense of the supplied list of columns.
+   * @param admin Admin to use.
+   * @return Interpretation of supplied list of columns.
+   */
+  public ParsedColumns getColumns(final HBaseAdmin admin,
+      final boolean scanning) {
+    ParsedColumns result = null;
     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;
+          result =
+            new ParsedColumns(Arrays.asList(HConstants.COLUMN_FAMILY_ARRAY));
         } else {
+          HTableDescriptor[] tables = admin.listTables();
           for (int i = 0; i < tables.length; i++) {
             if (tables[i].getName().equals(this.tableName)) {
-              cols = tables[i].families().keySet().toArray(new Text[] {});
+              result = new ParsedColumns(new ArrayList<Text>(tables[i].
+                families().keySet()));
+              break;
             }
           }
         }
@@ -171,19 +242,30 @@
         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) + ":");
-          
+          // Add '$' to column name if we are scanning.  Scanners support
+          // regex column names.  Adding '$', the column becomes a
+          // regex that does an explicit match on the supplied column name.
+          // Otherwise, if the specified column is a column family, then
+          // default behavior is to fetch all columns that have a matching
+          // column family.
+          column = (this.columns.get(i).contains(":"))?
+            new Text(this.columns.get(i) + (scanning? "$": "")):
+            new Text(this.columns.get(i) + ":" + (scanning? "$": ""));
           tmpList.add(column);
         }
-        cols = tmpList.toArray(new Text[] {});
+        result = new ParsedColumns(tmpList, tmpList.size() > 1);
       }
     } catch (IOException e) {
       e.printStackTrace();
     }
-    return cols;
+    return result;
+  }
+  
+  /*
+   * @return True if query contains multiple columns.
+   */
+  private boolean isMultiple() {
+    return this.columns.size() > 1 || this.columns.contains("*");
   }
 
   private boolean checkLimit(int count) {
@@ -218,7 +300,17 @@
       this.rowKey = new Text(rowKey);
   }
 
+  /**
+   * @param version Set maximum versions for this selection
+   */
   public void setVersion(int version) {
     this.version = version;
+  }
+  
+  public static void main(String[] args) throws ParseException {
+    // For debugging
+    Parser parser = new Parser("select * from -ROOT-;");
+    Command cmd = parser.terminatedCommand();
+    ReturnMsg rm = cmd.execute(new HBaseConfiguration());
   }
 }

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/ShowCommand.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/ShowCommand.java?rev=580487&r1=580486&r2=580487&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/ShowCommand.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/ShowCommand.java Fri Sep 28 14:10:49 2007
@@ -29,34 +29,32 @@
  * Shows all available tables.
  */
 public class ShowCommand extends BasicCommand {
-
+  private static final String [] HEADER = new String [] {"Table Name"};
   private String command;
 
   public ReturnMsg execute(Configuration conf) {
-    if (this.command == null)
-      return new ReturnMsg(0, "Syntax error : Please check 'Show' syntax.");
-
+    if (this.command == null) {
+      return new ReturnMsg(0, "Syntax error : Please check 'Show' syntax");
+    }
     try {
       HBaseAdmin admin = new HBaseAdmin(conf);
-
       int tableLength = 0;
       if ("tables".equals(this.command)) {
         HTableDescriptor[] tables = admin.listTables();
         tableLength = tables.length;
         if (tableLength == 0) {
-          return new ReturnMsg(0, "Table not found.");
+          return new ReturnMsg(0, "Table not found");
         }
-
-        ConsoleTable.printHead("Table Name");
+        TableFormatter formatter = TableFormatterFactory.get();
+        formatter.header(HEADER);
         for (int i = 0; i < tableLength; i++) {
           String tableName = tables[i].getName().toString();
-          ConsoleTable.printTable(i, tableName);
+          formatter.row(new String [] {tableName});
         }
-        ConsoleTable.printFoot();
-
-        return new ReturnMsg(1, tableLength + " table(s) found.");
+        formatter.footer();
+        return new ReturnMsg(1, tableLength + " table(s) in set");
       }
-      return new ReturnMsg(0, "Missing parameters. Please check 'Show' syntax.");
+      return new ReturnMsg(0, "Missing parameters. Please check 'Show' syntax");
     } catch (IOException e) {
       return new ReturnMsg(0, "error msg : " + e.toString());
     }
@@ -65,4 +63,4 @@
   public void setArgument(String argument) {
     this.command = argument;
   }
-}
+}
\ No newline at end of file

Added: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/TableFormatter.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/TableFormatter.java?rev=580487&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/TableFormatter.java (added)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/TableFormatter.java Fri Sep 28 14:10:49 2007
@@ -0,0 +1,42 @@
+/**
+ * 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;
+
+/**
+ * Interface implemented by table formatters outputting select results.
+ */
+public interface TableFormatter {
+  /**
+   * Output header.
+   * @param titles Titles to emit.
+   */
+  public void header(final String [] titles);
+  
+  /**
+   * Output footer.
+   */
+  public void footer();
+  
+  /**
+   * Output a row.
+   * @param cells
+   */
+  public void row(final String [] cells);
+}
\ No newline at end of file

Added: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/TableFormatterFactory.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/TableFormatterFactory.java?rev=580487&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/TableFormatterFactory.java (added)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/TableFormatterFactory.java Fri Sep 28 14:10:49 2007
@@ -0,0 +1,171 @@
+/**
+ * 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.PrintStream;
+import java.io.UnsupportedEncodingException;
+
+/**
+ * Table formatter.
+ * TODO: Make a factory that chooses the formatter to use based off
+ * configuration.  Read a property from hbase-site or from System properties.
+ * For now, default is the internal AsciiTableFormatter.
+ * TODO: Mysql has --skip-column-names and --silent which inserts a tab as
+ * separator.  Also has --html and --xml.
+ */
+public class TableFormatterFactory {
+  private static final TableFormatterFactory factory =
+    new TableFormatterFactory();
+  private final TableFormatter formatter;
+  
+  private TableFormatterFactory() {
+    this.formatter = new AsciiTableFormatter();
+  }
+  
+  /**
+   * @return Configured table formatter.
+   */
+  public static TableFormatter get() {
+    return factory.formatter;
+  }
+  
+  /*
+   * Formmatter that outputs data in UTF-8 inside an ASCII table on STDOUT.
+   * If only a single cell result, then no formatting is done.  Presumption is
+   * that client manages serial access outputting tables.
+   */
+  private class AsciiTableFormatter implements TableFormatter {
+    private PrintStream out;
+    private static final String COLUMN_DELIMITER = "| ";
+    private static final String COLUMN_CLOSER = "|";
+    private static final int DEFAULT_COLUMN_WIDTH = 26;
+    // Width is a line of content + delimiter
+    private int columnWidth = DEFAULT_COLUMN_WIDTH;
+    // Amount of width to use for a line of content.
+    private int columnContentWidth =
+      DEFAULT_COLUMN_WIDTH - COLUMN_DELIMITER.length();
+    // COLUMN_LINE is put at head and foot of a column and per column, is drawn
+    // as row delimiter
+    private String columnHorizLine;
+    private final String COLUMN_HORIZ_LINE_CLOSER = "+";
+    // Used padding content to fill column
+    private final String PADDING_CHAR = " ";
+    // True if we are to output no formatting.
+    private boolean noFormatting = false;
+    
+    /*
+     * Constructor. 
+     */
+    protected AsciiTableFormatter() {
+      try {
+        this.out = new PrintStream(System.out, true, "UTF-8");
+      } catch (UnsupportedEncodingException e) {
+        throw new RuntimeException("Failed setting output to UTF-8", e);
+      }
+    }
+    
+    /**
+     * @param titles List of titles.  Pass null if no formatting (i.e.
+     * no header, no footer, etc.
+     */
+    public void header(String[] titles) {
+      if (titles == null) {
+        // print nothing.
+        this.noFormatting = true;
+        return;
+      }
+      // Calculate width of columns.
+      this.columnWidth = titles.length == 1? 3 * DEFAULT_COLUMN_WIDTH:
+        titles.length == 2? 39: DEFAULT_COLUMN_WIDTH;
+      this.columnContentWidth = this.columnWidth - COLUMN_DELIMITER.length();
+      // Create the horizontal line to draw across the top of each column.
+      this.columnHorizLine = calculateColumnHorizLine(this.columnWidth);
+      // Print out a column topper per column.
+      printRowDelimiter(titles.length);
+      row(titles);
+    }
+
+    public void row(String [] cells) {
+      if (this.noFormatting) {
+        this.out.print(cells[0]);
+        return;
+      }
+      // Ok.  Output cells a line at a time w/ delimiters between cells.
+      int [] indexes = new int[cells.length];
+      for (int i = 0; i < indexes.length; i++) {
+        indexes[i] = 0;
+      }
+      int allFinished = 0;
+      while (allFinished < indexes.length) {
+        StringBuffer sb = new StringBuffer();
+        for (int i = 0; i < cells.length; i++) {
+          sb.append(COLUMN_DELIMITER);
+          int offset = indexes[i];
+          if (offset + this.columnContentWidth >= cells[i].length()) {
+            String substr = cells[i].substring(offset);
+            if (substr.length() > 0) {
+              // This column is finished
+              allFinished++;
+              sb.append(substr);
+            }
+            for (int j = 0; j < this.columnContentWidth - substr.length(); j++) {
+              sb.append(PADDING_CHAR);
+            }
+            indexes[i] = cells[i].length();
+          } else {
+            String substr = cells[i].substring(indexes[i],
+              indexes[i] + this.columnContentWidth);
+            indexes[i] += this.columnContentWidth;
+            sb.append(substr);
+          }
+        }
+        sb.append(COLUMN_CLOSER);
+        this.out.println(sb.toString());
+      }
+      printRowDelimiter(cells.length);
+    }
+
+    public void footer() {
+      if (this.noFormatting) {
+        // If no formatting, output a newline to delimit cell and the
+        // result summary output at end of every command.
+        this.out.println();
+      }
+      // We're done. Clear flag.
+      this.noFormatting = false;
+    }
+    
+    private void printRowDelimiter(final int columnCount) {
+      for (int i = 0; i < columnCount; i++) {
+        this.out.print(this.columnHorizLine);
+      }
+      this.out.println(COLUMN_HORIZ_LINE_CLOSER);
+    }
+    
+    private String calculateColumnHorizLine(final int width) {
+      StringBuffer sb = new StringBuffer();
+      sb.append("+");
+      for (int i = 1; i < width; i++) {
+        sb.append("-");
+      }
+      return sb.toString();
+    }
+  }
+}
\ No newline at end of file

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/Parser.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/Parser.java?rev=580487&r1=580486&r2=580487&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/Parser.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/Parser.java Fri Sep 28 14:10:49 2007
@@ -676,6 +676,7 @@
     switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
     case NUM_VERSIONS:
       jj_consume_token(NUM_VERSIONS);
+      jj_consume_token(EQUALS);
       numVersion = Number();
         select.setVersion(numVersion);
       break;
@@ -969,6 +970,11 @@
     finally { jj_save(0, xla); }
   }
 
+  final private boolean jj_3R_11() {
+    if (jj_scan_token(ID)) return true;
+    return false;
+  }
+
   final private boolean jj_3R_10() {
     Token xsp;
     xsp = jj_scanpos;
@@ -992,11 +998,6 @@
     jj_scanpos = xsp;
     if (jj_scan_token(61)) return true;
     }
-    return false;
-  }
-
-  final private boolean jj_3R_11() {
-    if (jj_scan_token(ID)) return true;
     return false;
   }
 

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/ParserTokenManager.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/ParserTokenManager.java?rev=580487&r1=580486&r2=580487&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/ParserTokenManager.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/ParserTokenManager.java Fri Sep 28 14:10:49 2007
@@ -38,36 +38,36 @@
    {
       case 0:
          if ((active0 & 0x800000000L) != 0L)
-            return 31;
+            return 32;
          if ((active0 & 0xfffe03ffffffe0L) != 0L)
          {
             jjmatchedKind = 56;
-            return 0;
+            return 1;
          }
          return -1;
       case 1:
          if ((active0 & 0x200002000L) != 0L)
-            return 0;
+            return 1;
          if ((active0 & 0xfffe01ffffdfe0L) != 0L)
          {
             jjmatchedKind = 56;
             jjmatchedPos = 1;
-            return 0;
+            return 1;
          }
          return -1;
       case 2:
          if ((active0 & 0x40000104004000L) != 0L)
-            return 0;
+            return 1;
          if ((active0 & 0xbffe00fbff9fe0L) != 0L)
          {
             jjmatchedKind = 56;
             jjmatchedPos = 2;
-            return 0;
+            return 1;
          }
          return -1;
       case 3:
          if ((active0 & 0x100002029720L) != 0L)
-            return 0;
+            return 1;
          if ((active0 & 0xbfee00f9fd08c0L) != 0L)
          {
             if (jjmatchedPos != 3)
@@ -75,113 +75,113 @@
                jjmatchedKind = 56;
                jjmatchedPos = 3;
             }
-            return 0;
+            return 1;
          }
          return -1;
       case 4:
-         if ((active0 & 0x2000810400c0L) != 0L)
-            return 0;
          if ((active0 & 0xbfce0078f90a00L) != 0L)
          {
             jjmatchedKind = 56;
             jjmatchedPos = 4;
-            return 0;
+            return 1;
          }
+         if ((active0 & 0x2000810400c0L) != 0L)
+            return 1;
          return -1;
       case 5:
-         if ((active0 & 0x80400008390800L) != 0L)
-            return 0;
          if ((active0 & 0x3f8e0070c00200L) != 0L)
          {
             jjmatchedKind = 56;
             jjmatchedPos = 5;
-            return 0;
+            return 1;
          }
+         if ((active0 & 0x80400008390800L) != 0L)
+            return 1;
          return -1;
       case 6:
-         if ((active0 & 0x400000L) != 0L)
-            return 0;
          if ((active0 & 0x3f8e0070800200L) != 0L)
          {
             jjmatchedKind = 56;
             jjmatchedPos = 6;
-            return 0;
+            return 1;
          }
+         if ((active0 & 0x400000L) != 0L)
+            return 1;
          return -1;
       case 7:
-         if ((active0 & 0x10000000800200L) != 0L)
-            return 0;
          if ((active0 & 0x2f8e0070000000L) != 0L)
          {
             jjmatchedKind = 56;
             jjmatchedPos = 7;
-            return 0;
+            return 1;
          }
+         if ((active0 & 0x10000000800200L) != 0L)
+            return 1;
          return -1;
       case 8:
-         if ((active0 & 0x800020000000L) != 0L)
-            return 0;
          if ((active0 & 0x2f0e0050000000L) != 0L)
          {
             jjmatchedKind = 56;
             jjmatchedPos = 8;
-            return 0;
+            return 1;
          }
+         if ((active0 & 0x800020000000L) != 0L)
+            return 1;
          return -1;
       case 9:
-         if ((active0 & 0x40000000000L) != 0L)
-            return 0;
          if ((active0 & 0x2f0a0050000000L) != 0L)
          {
             jjmatchedKind = 56;
             jjmatchedPos = 9;
-            return 0;
+            return 1;
          }
+         if ((active0 & 0x40000000000L) != 0L)
+            return 1;
          return -1;
       case 10:
          if ((active0 & 0x29080000000000L) != 0L)
-            return 0;
+            return 1;
          if ((active0 & 0x6020050000000L) != 0L)
          {
             jjmatchedKind = 56;
             jjmatchedPos = 10;
-            return 0;
+            return 1;
          }
          return -1;
       case 11:
-         if ((active0 & 0x20040000000L) != 0L)
-            return 0;
          if ((active0 & 0x6000010000000L) != 0L)
          {
             jjmatchedKind = 56;
             jjmatchedPos = 11;
-            return 0;
+            return 1;
          }
+         if ((active0 & 0x20040000000L) != 0L)
+            return 1;
          return -1;
       case 12:
          if ((active0 & 0x6000010000000L) != 0L)
          {
             jjmatchedKind = 56;
             jjmatchedPos = 12;
-            return 0;
+            return 1;
          }
          return -1;
       case 13:
-         if ((active0 & 0x10000000L) != 0L)
-            return 0;
          if ((active0 & 0x6000000000000L) != 0L)
          {
             jjmatchedKind = 56;
             jjmatchedPos = 13;
-            return 0;
+            return 1;
          }
+         if ((active0 & 0x10000000L) != 0L)
+            return 1;
          return -1;
       case 14:
          if ((active0 & 0x6000000000000L) != 0L)
          {
             jjmatchedKind = 56;
             jjmatchedPos = 14;
-            return 0;
+            return 1;
          }
          return -1;
       case 15:
@@ -189,7 +189,7 @@
          {
             jjmatchedKind = 56;
             jjmatchedPos = 15;
-            return 0;
+            return 1;
          }
          return -1;
       case 16:
@@ -197,7 +197,7 @@
          {
             jjmatchedKind = 56;
             jjmatchedPos = 16;
-            return 0;
+            return 1;
          }
          return -1;
       case 17:
@@ -205,7 +205,7 @@
          {
             jjmatchedKind = 56;
             jjmatchedPos = 17;
-            return 0;
+            return 1;
          }
          return -1;
       case 18:
@@ -213,7 +213,7 @@
          {
             jjmatchedKind = 56;
             jjmatchedPos = 18;
-            return 0;
+            return 1;
          }
          return -1;
       case 19:
@@ -221,10 +221,10 @@
          {
             jjmatchedKind = 56;
             jjmatchedPos = 19;
-            return 0;
+            return 1;
          }
          if ((active0 & 0x2000000000000L) != 0L)
-            return 0;
+            return 1;
          return -1;
       default :
          return -1;
@@ -261,7 +261,7 @@
       case 44:
          return jjStopAtPos(0, 34);
       case 46:
-         return jjStartNfaWithStates_0(0, 35, 31);
+         return jjStartNfaWithStates_0(0, 35, 32);
       case 59:
          return jjStopAtPos(0, 62);
       case 60:
@@ -323,7 +323,7 @@
       case 119:
          return jjMoveStringLiteralDfa1_0(0x1000000L);
       default :
-         return jjMoveNfa_0(1, 0);
+         return jjMoveNfa_0(0, 0);
    }
 }
 private final int jjMoveStringLiteralDfa1_0(long active0)
@@ -366,12 +366,12 @@
       case 82:
       case 114:
          if ((active0 & 0x200000000L) != 0L)
-            return jjStartNfaWithStates_0(1, 33, 0);
+            return jjStartNfaWithStates_0(1, 33, 1);
          return jjMoveStringLiteralDfa2_0(active0, 0x2001800L);
       case 83:
       case 115:
          if ((active0 & 0x2000L) != 0L)
-            return jjStartNfaWithStates_0(1, 13, 0);
+            return jjStartNfaWithStates_0(1, 13, 1);
          break;
       case 84:
       case 116:
@@ -412,9 +412,9 @@
       case 68:
       case 100:
          if ((active0 & 0x100000000L) != 0L)
-            return jjStartNfaWithStates_0(2, 32, 0);
+            return jjStartNfaWithStates_0(2, 32, 1);
          else if ((active0 & 0x40000000000000L) != 0L)
-            return jjStartNfaWithStates_0(2, 54, 0);
+            return jjStartNfaWithStates_0(2, 54, 1);
          break;
       case 69:
       case 101:
@@ -437,7 +437,7 @@
       case 82:
       case 114:
          if ((active0 & 0x4000L) != 0L)
-            return jjStartNfaWithStates_0(2, 14, 0);
+            return jjStartNfaWithStates_0(2, 14, 1);
          break;
       case 83:
       case 115:
@@ -451,7 +451,7 @@
       case 87:
       case 119:
          if ((active0 & 0x4000000L) != 0L)
-            return jjStartNfaWithStates_0(2, 26, 0);
+            return jjStartNfaWithStates_0(2, 26, 1);
          break;
       case 88:
       case 120:
@@ -491,7 +491,7 @@
       case 69:
       case 101:
          if ((active0 & 0x100000000000L) != 0L)
-            return jjStartNfaWithStates_0(3, 44, 0);
+            return jjStartNfaWithStates_0(3, 44, 1);
          return jjMoveStringLiteralDfa4_0(active0, 0x20190040L);
       case 73:
       case 105:
@@ -502,7 +502,7 @@
       case 77:
       case 109:
          if ((active0 & 0x2000000L) != 0L)
-            return jjStartNfaWithStates_0(3, 25, 0);
+            return jjStartNfaWithStates_0(3, 25, 1);
          return jjMoveStringLiteralDfa4_0(active0, 0x800000000000L);
       case 78:
       case 110:
@@ -510,14 +510,14 @@
       case 79:
       case 111:
          if ((active0 & 0x20000L) != 0L)
-            return jjStartNfaWithStates_0(3, 17, 0);
+            return jjStartNfaWithStates_0(3, 17, 1);
          return jjMoveStringLiteralDfa4_0(active0, 0x5400000000000L);
       case 80:
       case 112:
          if ((active0 & 0x20L) != 0L)
-            return jjStartNfaWithStates_0(3, 5, 0);
+            return jjStartNfaWithStates_0(3, 5, 1);
          else if ((active0 & 0x1000L) != 0L)
-            return jjStartNfaWithStates_0(3, 12, 0);
+            return jjStartNfaWithStates_0(3, 12, 1);
          return jjMoveStringLiteralDfa4_0(active0, 0x80000000000L);
       case 82:
       case 114:
@@ -525,7 +525,7 @@
       case 84:
       case 116:
          if ((active0 & 0x8000L) != 0L)
-            return jjStartNfaWithStates_0(3, 15, 0);
+            return jjStartNfaWithStates_0(3, 15, 1);
          return jjMoveStringLiteralDfa4_0(active0, 0x8000000000000L);
       case 85:
       case 117:
@@ -533,7 +533,7 @@
       case 87:
       case 119:
          if ((active0 & 0x100L) != 0L)
-            return jjStartNfaWithStates_0(3, 8, 0);
+            return jjStartNfaWithStates_0(3, 8, 1);
          break;
       default :
          break;
@@ -560,9 +560,9 @@
       case 69:
       case 101:
          if ((active0 & 0x40000L) != 0L)
-            return jjStartNfaWithStates_0(4, 18, 0);
+            return jjStartNfaWithStates_0(4, 18, 1);
          else if ((active0 & 0x1000000L) != 0L)
-            return jjStartNfaWithStates_0(4, 24, 0);
+            return jjStartNfaWithStates_0(4, 24, 1);
          return jjMoveStringLiteralDfa5_0(active0, 0x20800008000000L);
       case 71:
       case 103:
@@ -573,7 +573,7 @@
       case 75:
       case 107:
          if ((active0 & 0x200000000000L) != 0L)
-            return jjStartNfaWithStates_0(4, 45, 0);
+            return jjStartNfaWithStates_0(4, 45, 1);
          break;
       case 76:
       case 108:
@@ -587,9 +587,9 @@
       case 82:
       case 114:
          if ((active0 & 0x40L) != 0L)
-            return jjStartNfaWithStates_0(4, 6, 0);
+            return jjStartNfaWithStates_0(4, 6, 1);
          else if ((active0 & 0x80L) != 0L)
-            return jjStartNfaWithStates_0(4, 7, 0);
+            return jjStartNfaWithStates_0(4, 7, 1);
          return jjMoveStringLiteralDfa5_0(active0, 0x480000010200L);
       case 83:
       case 115:
@@ -597,7 +597,7 @@
       case 84:
       case 116:
          if ((active0 & 0x80000000L) != 0L)
-            return jjStartNfaWithStates_0(4, 31, 0);
+            return jjStartNfaWithStates_0(4, 31, 1);
          return jjMoveStringLiteralDfa5_0(active0, 0x2000000880800L);
       case 85:
       case 117:
@@ -630,18 +630,18 @@
       case 68:
       case 100:
          if ((active0 & 0x400000000000L) != 0L)
-            return jjStartNfaWithStates_0(5, 46, 0);
+            return jjStartNfaWithStates_0(5, 46, 1);
          break;
       case 69:
       case 101:
          if ((active0 & 0x800L) != 0L)
-            return jjStartNfaWithStates_0(5, 11, 0);
+            return jjStartNfaWithStates_0(5, 11, 1);
          else if ((active0 & 0x80000L) != 0L)
-            return jjStartNfaWithStates_0(5, 19, 0);
+            return jjStartNfaWithStates_0(5, 19, 1);
          else if ((active0 & 0x200000L) != 0L)
-            return jjStartNfaWithStates_0(5, 21, 0);
+            return jjStartNfaWithStates_0(5, 21, 1);
          else if ((active0 & 0x80000000000000L) != 0L)
-            return jjStartNfaWithStates_0(5, 55, 0);
+            return jjStartNfaWithStates_0(5, 55, 1);
          return jjMoveStringLiteralDfa6_0(active0, 0xe0040000000L);
       case 70:
       case 102:
@@ -664,14 +664,14 @@
       case 83:
       case 115:
          if ((active0 & 0x8000000L) != 0L)
-            return jjStartNfaWithStates_0(5, 27, 0);
+            return jjStartNfaWithStates_0(5, 27, 1);
          break;
       case 84:
       case 116:
          if ((active0 & 0x10000L) != 0L)
-            return jjStartNfaWithStates_0(5, 16, 0);
+            return jjStartNfaWithStates_0(5, 16, 1);
          else if ((active0 & 0x100000L) != 0L)
-            return jjStartNfaWithStates_0(5, 20, 0);
+            return jjStartNfaWithStates_0(5, 20, 1);
          return jjMoveStringLiteralDfa6_0(active0, 0x20000000L);
       default :
          break;
@@ -700,7 +700,7 @@
       case 69:
       case 101:
          if ((active0 & 0x400000L) != 0L)
-            return jjStartNfaWithStates_0(6, 22, 0);
+            return jjStartNfaWithStates_0(6, 22, 1);
          break;
       case 70:
       case 102:
@@ -748,17 +748,17 @@
       case 69:
       case 101:
          if ((active0 & 0x200L) != 0L)
-            return jjStartNfaWithStates_0(7, 9, 0);
+            return jjStartNfaWithStates_0(7, 9, 1);
          return jjMoveStringLiteralDfa8_0(active0, 0x4000000000000L);
       case 71:
       case 103:
          if ((active0 & 0x800000L) != 0L)
-            return jjStartNfaWithStates_0(7, 23, 0);
+            return jjStartNfaWithStates_0(7, 23, 1);
          return jjMoveStringLiteralDfa8_0(active0, 0x2040000000000L);
       case 72:
       case 104:
          if ((active0 & 0x10000000000000L) != 0L)
-            return jjStartNfaWithStates_0(7, 52, 0);
+            return jjStartNfaWithStates_0(7, 52, 1);
          break;
       case 76:
       case 108:
@@ -802,7 +802,7 @@
       case 80:
       case 112:
          if ((active0 & 0x20000000L) != 0L)
-            return jjStartNfaWithStates_0(8, 29, 0);
+            return jjStartNfaWithStates_0(8, 29, 1);
          break;
       case 84:
       case 116:
@@ -810,7 +810,7 @@
       case 89:
       case 121:
          if ((active0 & 0x800000000000L) != 0L)
-            return jjStartNfaWithStates_0(8, 47, 0);
+            return jjStartNfaWithStates_0(8, 47, 1);
          break;
       default :
          break;
@@ -839,7 +839,7 @@
       case 72:
       case 104:
          if ((active0 & 0x40000000000L) != 0L)
-            return jjStartNfaWithStates_0(9, 42, 0);
+            return jjStartNfaWithStates_0(9, 42, 1);
          break;
       case 73:
       case 105:
@@ -872,7 +872,7 @@
       case 69:
       case 101:
          if ((active0 & 0x8000000000000L) != 0L)
-            return jjStartNfaWithStates_0(10, 51, 0);
+            return jjStartNfaWithStates_0(10, 51, 1);
          break;
       case 76:
       case 108:
@@ -880,17 +880,17 @@
       case 78:
       case 110:
          if ((active0 & 0x80000000000L) != 0L)
-            return jjStartNfaWithStates_0(10, 43, 0);
+            return jjStartNfaWithStates_0(10, 43, 1);
          return jjMoveStringLiteralDfa11_0(active0, 0x20040000000L);
       case 82:
       case 114:
          if ((active0 & 0x1000000000000L) != 0L)
-            return jjStartNfaWithStates_0(10, 48, 0);
+            return jjStartNfaWithStates_0(10, 48, 1);
          break;
       case 83:
       case 115:
          if ((active0 & 0x20000000000000L) != 0L)
-            return jjStartNfaWithStates_0(10, 53, 0);
+            return jjStartNfaWithStates_0(10, 53, 1);
          break;
       default :
          break;
@@ -920,9 +920,9 @@
       case 83:
       case 115:
          if ((active0 & 0x40000000L) != 0L)
-            return jjStartNfaWithStates_0(11, 30, 0);
+            return jjStartNfaWithStates_0(11, 30, 1);
          else if ((active0 & 0x20000000000L) != 0L)
-            return jjStartNfaWithStates_0(11, 41, 0);
+            return jjStartNfaWithStates_0(11, 41, 1);
          break;
       default :
          break;
@@ -971,7 +971,7 @@
       case 83:
       case 115:
          if ((active0 & 0x10000000L) != 0L)
-            return jjStartNfaWithStates_0(13, 28, 0);
+            return jjStartNfaWithStates_0(13, 28, 1);
          break;
       default :
          break;
@@ -1105,7 +1105,7 @@
       case 82:
       case 114:
          if ((active0 & 0x2000000000000L) != 0L)
-            return jjStartNfaWithStates_0(19, 49, 0);
+            return jjStartNfaWithStates_0(19, 49, 1);
          break;
       default :
          break;
@@ -1126,7 +1126,7 @@
       case 82:
       case 114:
          if ((active0 & 0x4000000000000L) != 0L)
-            return jjStartNfaWithStates_0(20, 50, 0);
+            return jjStartNfaWithStates_0(20, 50, 1);
          break;
       default :
          break;
@@ -1170,7 +1170,7 @@
 {
    int[] nextStates;
    int startsAt = 0;
-   jjnewStateCnt = 31;
+   jjnewStateCnt = 32;
    int i = 1;
    jjstateSet[0] = startState;
    int j, kind = 0x7fffffff;
@@ -1185,173 +1185,177 @@
          {
             switch(jjstateSet[--i])
             {
-               case 31:
-                  if ((0x7ffe00000000000L & l) != 0L)
+               case 0:
+                  if ((0x3ff000000000000L & l) != 0L)
                   {
-                     if (kind > 56)
-                        kind = 56;
-                     jjCheckNAdd(0);
+                     if (kind > 57)
+                        kind = 57;
+                     jjCheckNAddStates(0, 6);
                   }
-                  if ((0x3ff000000000000L & l) != 0L)
+                  else if ((0x400e00000000000L & l) != 0L)
                   {
-                     if (kind > 58)
-                        kind = 58;
-                     jjCheckNAddTwoStates(2, 3);
+                     if (kind > 56)
+                        kind = 56;
+                     jjCheckNAdd(1);
                   }
+                  else if (curChar == 39)
+                     jjCheckNAddStates(7, 9);
+                  else if (curChar == 34)
+                     jjCheckNAdd(8);
+                  if (curChar == 46)
+                     jjCheckNAdd(3);
                   break;
-               case 1:
+               case 32:
                   if ((0x7ffe00000000000L & l) != 0L)
                   {
                      if (kind > 56)
                         kind = 56;
-                     jjCheckNAdd(0);
+                     jjCheckNAdd(1);
                   }
-                  else if (curChar == 39)
-                     jjCheckNAddStates(0, 2);
-                  else if (curChar == 34)
-                     jjCheckNAdd(7);
                   if ((0x3ff000000000000L & l) != 0L)
                   {
-                     if (kind > 57)
-                        kind = 57;
-                     jjCheckNAddStates(3, 9);
+                     if (kind > 58)
+                        kind = 58;
+                     jjCheckNAddTwoStates(3, 4);
                   }
-                  else if (curChar == 46)
-                     jjCheckNAdd(2);
                   break;
-               case 0:
+               case 1:
                   if ((0x7ffe00000000000L & l) == 0L)
                      break;
                   if (kind > 56)
                      kind = 56;
-                  jjCheckNAdd(0);
+                  jjCheckNAdd(1);
                   break;
                case 2:
+                  if (curChar == 46)
+                     jjCheckNAdd(3);
+                  break;
+               case 3:
                   if ((0x3ff000000000000L & l) == 0L)
                      break;
                   if (kind > 58)
                      kind = 58;
-                  jjCheckNAddTwoStates(2, 3);
+                  jjCheckNAddTwoStates(3, 4);
                   break;
-               case 4:
+               case 5:
                   if ((0x280000000000L & l) != 0L)
-                     jjCheckNAdd(5);
+                     jjCheckNAdd(6);
                   break;
-               case 5:
+               case 6:
                   if ((0x3ff000000000000L & l) == 0L)
                      break;
                   if (kind > 58)
                      kind = 58;
-                  jjCheckNAdd(5);
+                  jjCheckNAdd(6);
                   break;
-               case 6:
+               case 7:
                   if (curChar == 34)
-                     jjCheckNAdd(7);
+                     jjCheckNAdd(8);
                   break;
-               case 7:
+               case 8:
                   if ((0xfffffffbffffffffL & l) != 0L)
-                     jjCheckNAddTwoStates(7, 8);
+                     jjCheckNAddTwoStates(8, 9);
                   break;
-               case 8:
+               case 9:
                   if (curChar == 34 && kind > 60)
                      kind = 60;
                   break;
-               case 9:
+               case 10:
                   if (curChar == 39)
-                     jjCheckNAddStates(0, 2);
+                     jjCheckNAddStates(7, 9);
                   break;
-               case 10:
+               case 11:
                   if ((0xffffff7fffffffffL & l) != 0L)
-                     jjCheckNAddStates(0, 2);
+                     jjCheckNAddStates(7, 9);
                   break;
-               case 11:
+               case 12:
                   if (curChar == 39)
                      jjCheckNAddStates(10, 12);
                   break;
-               case 12:
+               case 13:
                   if (curChar == 39)
-                     jjstateSet[jjnewStateCnt++] = 11;
+                     jjstateSet[jjnewStateCnt++] = 12;
                   break;
-               case 13:
+               case 14:
                   if ((0xffffff7fffffffffL & l) != 0L)
                      jjCheckNAddStates(10, 12);
                   break;
-               case 14:
+               case 15:
                   if (curChar == 39 && kind > 61)
                      kind = 61;
                   break;
-               case 15:
+               case 16:
                   if ((0x3ff000000000000L & l) == 0L)
                      break;
                   if (kind > 57)
                      kind = 57;
-                  jjCheckNAddStates(3, 9);
+                  jjCheckNAddStates(0, 6);
                   break;
-               case 16:
+               case 17:
                   if ((0x3ff000000000000L & l) == 0L)
                      break;
                   if (kind > 57)
                      kind = 57;
-                  jjCheckNAdd(16);
+                  jjCheckNAdd(17);
                   break;
-               case 17:
+               case 18:
                   if ((0x3ff000000000000L & l) != 0L)
-                     jjCheckNAddTwoStates(17, 18);
+                     jjCheckNAddTwoStates(18, 19);
                   break;
-               case 18:
+               case 19:
                   if (curChar == 46)
-                     jjCheckNAdd(19);
+                     jjCheckNAdd(20);
                   break;
-               case 19:
+               case 20:
                   if ((0x3ff000000000000L & l) == 0L)
                      break;
                   if (kind > 58)
                      kind = 58;
-                  jjCheckNAddTwoStates(19, 20);
+                  jjCheckNAddTwoStates(20, 21);
                   break;
-               case 21:
+               case 22:
                   if ((0x280000000000L & l) != 0L)
-                     jjCheckNAdd(22);
+                     jjCheckNAdd(23);
                   break;
-               case 22:
+               case 23:
                   if ((0x3ff000000000000L & l) == 0L)
                      break;
                   if (kind > 58)
                      kind = 58;
-                  jjCheckNAdd(22);
+                  jjCheckNAdd(23);
                   break;
-               case 23:
+               case 24:
                   if ((0x3ff000000000000L & l) != 0L)
-                     jjCheckNAddTwoStates(23, 24);
+                     jjCheckNAddTwoStates(24, 25);
                   break;
-               case 25:
+               case 26:
                   if ((0x280000000000L & l) != 0L)
-                     jjCheckNAdd(26);
+                     jjCheckNAdd(27);
                   break;
-               case 26:
+               case 27:
                   if ((0x3ff000000000000L & l) == 0L)
                      break;
                   if (kind > 58)
                      kind = 58;
-                  jjCheckNAdd(26);
+                  jjCheckNAdd(27);
                   break;
-               case 27:
+               case 28:
                   if ((0x3ff000000000000L & l) == 0L)
                      break;
                   if (kind > 58)
                      kind = 58;
-                  jjCheckNAddTwoStates(27, 28);
+                  jjCheckNAddTwoStates(28, 29);
                   break;
-               case 29:
+               case 30:
                   if ((0x280000000000L & l) != 0L)
-                     jjCheckNAdd(30);
+                     jjCheckNAdd(31);
                   break;
-               case 30:
+               case 31:
                   if ((0x3ff000000000000L & l) == 0L)
                      break;
                   if (kind > 58)
                      kind = 58;
-                  jjCheckNAdd(30);
+                  jjCheckNAdd(31);
                   break;
                default : break;
             }
@@ -1364,43 +1368,43 @@
          {
             switch(jjstateSet[--i])
             {
-               case 31:
                case 0:
                   if ((0x7fffffe87fffffeL & l) == 0L)
                      break;
                   if (kind > 56)
                      kind = 56;
-                  jjCheckNAdd(0);
+                  jjCheckNAdd(1);
                   break;
+               case 32:
                case 1:
                   if ((0x7fffffe87fffffeL & l) == 0L)
                      break;
                   if (kind > 56)
                      kind = 56;
-                  jjCheckNAdd(0);
+                  jjCheckNAdd(1);
                   break;
-               case 3:
+               case 4:
                   if ((0x2000000020L & l) != 0L)
                      jjAddStates(13, 14);
                   break;
-               case 7:
+               case 8:
                   jjAddStates(15, 16);
                   break;
-               case 10:
-                  jjCheckNAddStates(0, 2);
+               case 11:
+                  jjCheckNAddStates(7, 9);
                   break;
-               case 13:
+               case 14:
                   jjCheckNAddStates(10, 12);
                   break;
-               case 20:
+               case 21:
                   if ((0x2000000020L & l) != 0L)
                      jjAddStates(17, 18);
                   break;
-               case 24:
+               case 25:
                   if ((0x2000000020L & l) != 0L)
                      jjAddStates(19, 20);
                   break;
-               case 28:
+               case 29:
                   if ((0x2000000020L & l) != 0L)
                      jjAddStates(21, 22);
                   break;
@@ -1416,15 +1420,15 @@
          {
             switch(jjstateSet[--i])
             {
-               case 7:
+               case 8:
                   if ((jjbitVec0[i2] & l2) != 0L)
                      jjAddStates(15, 16);
                   break;
-               case 10:
+               case 11:
                   if ((jjbitVec0[i2] & l2) != 0L)
-                     jjCheckNAddStates(0, 2);
+                     jjCheckNAddStates(7, 9);
                   break;
-               case 13:
+               case 14:
                   if ((jjbitVec0[i2] & l2) != 0L)
                      jjCheckNAddStates(10, 12);
                   break;
@@ -1439,15 +1443,15 @@
          kind = 0x7fffffff;
       }
       ++curPos;
-      if ((i = jjnewStateCnt) == (startsAt = 31 - (jjnewStateCnt = startsAt)))
+      if ((i = jjnewStateCnt) == (startsAt = 32 - (jjnewStateCnt = startsAt)))
          return curPos;
       try { curChar = input_stream.readChar(); }
       catch(java.io.IOException e) { return curPos; }
    }
 }
 static final int[] jjnextStates = {
-   10, 12, 14, 16, 17, 18, 23, 24, 27, 28, 12, 13, 14, 4, 5, 7, 
-   8, 21, 22, 25, 26, 29, 30, 
+   17, 18, 19, 24, 25, 28, 29, 11, 13, 15, 13, 14, 15, 5, 6, 8, 
+   9, 22, 23, 26, 27, 30, 31, 
 };
 public static final String[] jjstrLiteralImages = {
 "", null, null, null, null, null, null, null, null, null, null, null, null, 
@@ -1465,8 +1469,8 @@
    0x1eL, 
 };
 protected SimpleCharStream input_stream;
-private final int[] jjrounds = new int[31];
-private final int[] jjstateSet = new int[62];
+private final int[] jjrounds = new int[32];
+private final int[] jjstateSet = new int[64];
 protected char curChar;
 public ParserTokenManager(SimpleCharStream stream){
    if (SimpleCharStream.staticFlag)
@@ -1488,7 +1492,7 @@
 {
    int i;
    jjround = 0x80000001;
-   for (i = 31; i-- > 0;)
+   for (i = 32; i-- > 0;)
       jjrounds[i] = 0x80000000;
 }
 public void ReInit(SimpleCharStream stream, int lexState)

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestScanner2.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestScanner2.java?rev=580487&r1=580486&r2=580487&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestScanner2.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestScanner2.java Fri Sep 28 14:10:49 2007
@@ -71,6 +71,57 @@
   }
   
   /**
+   * Test getting scanners with regexes for column names.
+   * @throws IOException 
+   */
+  public void testRegexForColumnName() throws IOException {
+    // Setup HClient, ensure that it is running correctly
+    HBaseAdmin admin = new HBaseAdmin(conf);
+    
+    // Setup colkeys to be inserted
+    Text tableName = new Text(getName());
+    createTable(admin, tableName);
+    HTable table = new HTable(this.conf, tableName);
+    // Add a row to columns without qualifiers and then two with.  Make one
+    // numbers only so easy to find w/ a regex.
+    long id = table.startUpdate(new Text(getName()));
+    final String firstColkeyFamily = Character.toString(FIRST_COLKEY) + ":";
+    table.put(id, new Text(firstColkeyFamily + getName()), GOOD_BYTES);
+    table.put(id, new Text(firstColkeyFamily + "22222"), GOOD_BYTES);
+    table.put(id, new Text(firstColkeyFamily), GOOD_BYTES);
+    table.commit(id);
+    // Now do a scan using a regex for a column name.
+    checkRegexingScanner(table, firstColkeyFamily + "\\d+");
+    // Do a new scan that only matches on column family.
+    checkRegexingScanner(table, firstColkeyFamily + "$");
+  }
+  
+  /*
+   * Create a scanner w/ passed in column name regex.  Assert we only get
+   * back one column that matches.
+   * @param table
+   * @param regexColumnname
+   * @throws IOException
+   */
+  private void checkRegexingScanner(final HTable table,
+      final String regexColumnname) throws IOException {
+    Text [] regexCol = new Text [] {new Text(regexColumnname)};
+    HScannerInterface scanner =
+      table.obtainScanner(regexCol, HConstants.EMPTY_START_ROW);
+    HStoreKey key = new HStoreKey();
+    TreeMap<Text, byte []> results = new TreeMap<Text, byte []>();
+    int count = 0;
+    while (scanner.next(key, results)) {
+      for (Text c: results.keySet()) {
+        assertTrue(c.toString().matches(regexColumnname));
+        count++;
+      }
+    }
+    assertTrue(count == 1);
+    scanner.close();
+  }
+
+  /**
    * Test the scanner's handling of various filters.  
    * 
    * @throws Exception
@@ -80,17 +131,8 @@
     HBaseAdmin admin = new HBaseAdmin(conf);
     
     // Setup colkeys to be inserted
-    HTableDescriptor htd = new HTableDescriptor(getName());
     Text tableName = new Text(getName());
-    Text[] colKeys = new Text[(LAST_COLKEY - FIRST_COLKEY) + 1];
-    for (char i = 0; i < colKeys.length; i++) {
-      colKeys[i] = new Text(new String(new char[] { 
-        (char)(FIRST_COLKEY + i), ':' }));
-      htd.addFamily(new HColumnDescriptor(colKeys[i].toString()));
-    }
-    admin.createTable(htd);
-    assertTrue("Table with name " + tableName + " created successfully.", 
-        admin.tableExists(tableName));
+    Text [] colKeys = createTable(admin, tableName);
     assertTrue("Master is running.", admin.isMasterRunning());
     
     // Enter data
@@ -107,6 +149,28 @@
     
     regExpFilterTest(table, colKeys);
     rowFilterSetTest(table, colKeys);
+  }
+  
+  /**
+   * @param admin
+   * @param tableName
+   * @return Returns column keys used making table.
+   * @throws IOException
+   */
+  private Text [] createTable(final HBaseAdmin admin, final Text tableName)
+  throws IOException {
+    // Setup colkeys to be inserted
+    HTableDescriptor htd = new HTableDescriptor(getName());
+    Text[] colKeys = new Text[(LAST_COLKEY - FIRST_COLKEY) + 1];
+    for (char i = 0; i < colKeys.length; i++) {
+      colKeys[i] = new Text(new String(new char[] { 
+        (char)(FIRST_COLKEY + i), ':' }));
+      htd.addFamily(new HColumnDescriptor(colKeys[i].toString()));
+    }
+    admin.createTable(htd);
+    assertTrue("Table with name " + tableName + " created successfully.", 
+      admin.tableExists(tableName));
+    return colKeys;
   }
   
   private void regExpFilterTest(HTable table, Text[] colKeys) 

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/shell/TestConsoleTable.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/shell/TestConsoleTable.java?rev=580487&r1=580486&r2=580487&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/shell/TestConsoleTable.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/shell/TestConsoleTable.java Fri Sep 28 14:10:49 2007
@@ -25,14 +25,10 @@
 
 /**
  * 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 void testPrintTable() {
+    
   }
   
   public static void main(String[] args) {