Browse Source

HADOOP-1608 Relational Algrebra Operators

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk@599162 13f79535-47bb-0310-9956-ffa450edef68
Michael Stack 17 years ago
parent
commit
566ab120dc
39 changed files with 5965 additions and 439 deletions
  1. 5 0
      src/contrib/hbase/CHANGES.txt
  2. 6 0
      src/contrib/hbase/build.xml
  3. 1 1
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/mapred/GroupingTableMap.java
  4. 1 1
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/BasicCommand.java
  5. 302 44
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/HBaseShell.jj
  6. 101 71
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/HelpCommand.java
  7. 114 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/SaveCommand.java
  8. 32 6
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/ShowCommand.java
  9. 78 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/SubstituteCommand.java
  10. 50 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/VariableRef.java
  11. 52 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/VariablesPool.java
  12. 36 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/Constants.java
  13. 50 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/DuplicateTable.java
  14. 559 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/ExpressionParser.jj
  15. 83 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/GroupingFilterMap.java
  16. 78 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/IdentityFilterMap.java
  17. 76 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/IndexJoin.java
  18. 108 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/IndexJoinMap.java
  19. 49 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/IndexJoinReduce.java
  20. 41 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/Operation.java
  21. 88 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/OperationEvaluator.java
  22. 54 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/Projection.java
  23. 175 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/RelationalOperation.java
  24. 74 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/Selection.java
  25. 774 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/generated/ExpressionParser.java
  26. 56 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/generated/ExpressionParserConstants.java
  27. 659 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/generated/ExpressionParserTokenManager.java
  28. 192 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/generated/ParseException.java
  29. 439 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/generated/SimpleCharStream.java
  30. 81 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/generated/Token.java
  31. 133 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/generated/TokenMgrError.java
  32. 454 91
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/Parser.java
  33. 38 24
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/ParserConstants.java
  34. 235 201
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/ParserTokenManager.java
  35. 74 0
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/shell/TestSubstitutionVariables.java
  36. 84 0
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/shell/algebra/TestBooleanCondition.java
  37. 250 0
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/shell/algebra/TestBooleanTermFilter.java
  38. 45 0
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/shell/algebra/TestJoinCondition.java
  39. 238 0
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/shell/algebra/TestTableJoinMapReduce.java

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

@@ -48,13 +48,18 @@ Trunk (unreleased changes)
     HADOOP-2176 Htable.deleteAll documentation is ambiguous
     HADOOP-2139 (phase 1) Increase parallelism in region servers.
     HADOOP-2267 [Hbase Shell] Change the prompt's title from 'hbase' to 'hql'.
+                (Edward Yoon via Stack)
     HADOOP-2139 (phase 2) Make region server more event driven
     HADOOP-2289 Useless efforts of looking for the non-existant table in select
                 command.
+                (Edward Yoon via Stack)
     HADOOP-2257 Show a total of all requests and regions on the web ui
+                (Paul Saab via Stack)
     HADOOP-2261 HTable.abort no longer throws exception if there is no active update.
     HADOOP-2287 Make hbase unit tests take less time to complete.
     HADOOP-2262 Retry n times instead of n**2 times.
+    HADOOP-1608 Relational Algrebra Operators
+                (Edward Yoon via Stack)
 
 Release 0.15.1
 Branch 0.15

+ 6 - 0
src/contrib/hbase/build.xml

@@ -69,11 +69,17 @@
   <property name="hbaseshell.src.dir" 
        value="${src.dir}/org/apache/hadoop/hbase/shell" /> 
      <mkdir dir="${hbaseshell.src.dir}/generated" />
+     <mkdir dir="${hbaseshell.src.dir}/algebra/generated"/>
      <javacc
        target="${hbaseshell.src.dir}/HBaseShell.jj"
        outputdirectory="${hbaseshell.src.dir}/generated"
        javacchome="${javacc.home}"
      />
+     <javacc
+       target="${hbaseshell.src.dir}/algebra/ExpressionParser.jj"
+       outputdirectory="${hbaseshell.src.dir}/algebra/generated"
+       javacchome="${javacc.home}"
+     />
   </target>
 
   <target name="compile" depends="init,javacc">

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

@@ -47,7 +47,7 @@ public class GroupingTableMap extends TableMap {
   public static final String GROUP_COLUMNS =
     "hbase.mapred.groupingtablemap.columns";
   
-  private Text[] m_columns;
+  protected Text[] m_columns;
 
   /** default constructor */
   public GroupingTableMap() {

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

@@ -69,7 +69,7 @@ public abstract class BasicCommand implements Command, CommandFactory {
    * at the end of the column name.
    */
   protected String appendDelimiter(String column) {
-    return (!column.endsWith(FAMILY_INDICATOR))?
+    return (!column.endsWith(FAMILY_INDICATOR) && column.indexOf(FAMILY_INDICATOR) == -1)?
       column + FAMILY_INDICATOR: column;
   }
 

+ 302 - 44
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/HBaseShell.jj

@@ -33,6 +33,8 @@ import java.util.HashMap;
 import java.io.StringReader;
 import java.io.Reader;
 import java.io.Writer;
+import java.net.URLEncoder;
+import java.io.UnsupportedEncodingException;
 
 import org.apache.hadoop.hbase.shell.*;
 
@@ -43,6 +45,7 @@ public class Parser {
   private String QueryString;
   private TableFormatter formatter;
   private Writer out;
+  private String secondR;
   
   public Parser(final String query, final Writer o, final TableFormatter f) {
     this((Reader)(new StringReader(query)));
@@ -102,7 +105,11 @@ TOKEN: /** for HQL statements */
    | <LPAREN: "(">
    | <RPAREN: ")">
    | <EQUALS: "=">
-   | <NOTEQUAL: "<>">
+   | <LCOMP: ">">
+   | <RCOMP: "<">
+   | <NOT: "not">
+   | <IN: "in">
+   | <NOTEQUAL: "!=">
    | <ASTERISK: "*">
    | <MAX_VERSIONS: "max_versions">
    | <MAX_LENGTH: "max_length">
@@ -121,9 +128,16 @@ TOKEN: /** for HQL statements */
    | <CHANGE: "change">
 }
 
+TOKEN : 
+{
+     <SAVE: "save">
+   | <GROUP: "group">
+   | <BY: "by">
+}
+
 TOKEN : /** Literals */
 {
-   <ID: ["A"-"Z","a"-"z","_","-",".",":","/"] (["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>)?
@@ -144,7 +158,10 @@ Command terminatedCommand() :
     Command statement = null;
 }
 {
-    ([statement = cmdStatement()] ";" | <EOF>)
+    (
+      [statement = cmdStatement()] ";" | <EOF>
+    )
+  
     {
         return statement;
     }
@@ -171,6 +188,8 @@ Command cmdStatement() :
       | cmd = clearCommand()
       | cmd = fsCommand()
       | cmd = jarCommand()
+      | cmd = substituteCommand()
+      | cmd = saveCommand()
   )
    {
        return cmd;
@@ -245,6 +264,8 @@ HelpCommand helpCommand() :
        | t=<CLEAR>
        | t=<FS>
        | t=<JAR>
+       | t=<GROUP>
+       | t=<SAVE>
        | t=<ID>
      ) { argument = t.image.toString(); }
    ]
@@ -263,7 +284,7 @@ ShowCommand showCommand() :
 {
    <SHOW>
    [
-       argument = Identifier()
+       argument = identifier()
    ]
   { 
       show.setArgument(argument);
@@ -278,7 +299,7 @@ DescCommand descCommand() :
 }
 {
   ( <DESCRIBE> | <DESC> )
-  argument = Identifier()
+  argument = identifier()
   { 
     desc.setArgument(argument);
     return desc;
@@ -360,14 +381,14 @@ CreateCommand createCommand() :
 {
   <CREATE> 
   <TABLE> 
-  table = Identifier() 
+  table = identifier() 
   { 
     createCommand.setTable(table); 
   }
   
   <LPAREN>
   
-  column = Identifier() 
+  column = identifier() 
   columnSpec = ColumnSpec()
   { 
     createCommand.addColumnSpec(column, columnSpec); 
@@ -375,7 +396,7 @@ CreateCommand createCommand() :
   
   (
      <COMMA> 
-      column = Identifier() 
+      column = identifier() 
       columnSpec = ColumnSpec() 
       { 
         createCommand.addColumnSpec(column, columnSpec); 
@@ -395,12 +416,12 @@ AlterCommand alterCommand() :
 }
 {
   <ALTER> 
-  <TABLE> table = Identifier() 
+  <TABLE> table = identifier() 
   { alterCommand.setTable(table); }
 
   (
     LOOKAHEAD(2)
-      <ADD> column = Identifier() columnSpec = ColumnSpec() 
+      <ADD> column = identifier() columnSpec = ColumnSpec() 
       { 
          alterCommand.setOperationType(AlterCommand.OperationType.ADD);
          alterCommand.addColumnSpec(column, columnSpec); 
@@ -412,14 +433,14 @@ AlterCommand alterCommand() :
       alterCommand.setOperationType(AlterCommand.OperationType.ADD); 
     }
       
-    column = Identifier() columnSpec = ColumnSpec() 
+    column = identifier() columnSpec = ColumnSpec() 
     { 
       alterCommand.addColumnSpec(column, columnSpec); 
     }
       
     ( 
         <COMMA> 
-        column = Identifier()
+        column = identifier()
         columnSpec = ColumnSpec() 
         { 
           alterCommand.addColumnSpec(column, columnSpec); 
@@ -427,13 +448,13 @@ AlterCommand alterCommand() :
     )*
     <RPAREN>
   |
-    <DROP> column = Identifier()
+    <DROP> column = identifier()
     { 
       alterCommand.setOperationType(AlterCommand.OperationType.DROP);
       alterCommand.setColumn(column); 
     }
   |
-    <CHANGE> column = Identifier() columnSpec = ColumnSpec()  
+    <CHANGE> column = identifier() columnSpec = ColumnSpec()  
     { 
       alterCommand.setOperationType(AlterCommand.OperationType.CHANGE);
       alterCommand.addColumnSpec(column, columnSpec);
@@ -450,7 +471,7 @@ DropCommand dropCommand() :
 {
    <DROP>
    <TABLE>
-   tableList = TableList()
+   tableList = tableList()
    { 
      drop.setTableList(tableList);
      return drop;
@@ -468,16 +489,15 @@ InsertCommand insertCommand() :
 {
   <INSERT>
   <INTO>
-  table = Identifier()
+  table = identifier()
   { 
      in.setTable(table); 
   }
-    
   columnfamilies = getColumns()
   {
      in.setColumnfamilies(columnfamilies);
   }
-    
+  
   <VALUES> values = getLiteralValues()
   {
      in.setValues(values);
@@ -502,13 +522,13 @@ DeleteCommand deleteCommand() :
 }
 {
   <DELETE>
-  columnList = ColumnList()
+  columnList = columnList()
   { 
     deleteCommand.setColumnList(columnList); 
   }
 
   <FROM>
-  table = Identifier()
+  table = identifier()
   { 
     deleteCommand.setTable(table); 
   }
@@ -534,9 +554,9 @@ SelectCommand selectCommand() :
 }
 {
   <SELECT>
-  columns = ColumnList()
+  columns = columnList()
   <FROM>
-  tableName = Identifier()
+  tableName = identifier()
   { 
      select.setColumns(columns);
      select.setTable(tableName);
@@ -583,7 +603,7 @@ EnableCommand enableCommand() :
 }
 {
   <ENABLE> 
-  table = Identifier()
+  table = identifier()
   { 
     enableCommand.setTable(table);
     return enableCommand;
@@ -597,7 +617,7 @@ DisableCommand disableCommand() :
 }
 {
   <DISABLE> 
-  table = Identifier()
+  table = identifier()
   { 
     disableCommand.setTable(table); 
     return disableCommand;
@@ -615,8 +635,95 @@ ClearCommand clearCommand() :
    }
 }
 
-////////////////////////////////////////////////
-// Utility expansion units...
+SubstituteCommand substituteCommand() :
+{
+  Token key = null;
+  Token chainKey = null;
+  Token operation = null;
+  String tableName = null;
+  String condition = "";
+  List<String> notInList = new ArrayList<String>();
+  SubstituteCommand substitute = new SubstituteCommand(this.out);
+  Token extendedKey = null;
+}
+{
+  key=<ID><EQUALS>
+  { substitute.setKey(key.image.toString()); }
+  (
+      chainKey=<ID><DOT>
+    { substitute.setChainKey(chainKey.image.toString()); }
+      operation = <ID> 
+    { substitute.setOperation(operation.image.toString()); }
+    <LPAREN>
+    {
+      String operationType = operation.image.toLowerCase();
+      if(operationType.equals("projection")) {
+        List<String> columnList = columnList();
+        for (int i = 0; i < columnList.size(); i++) {
+          condition += appendIndicator(columnList.get(i)) + " ";
+        }
+    
+      } else {
+        condition = booleanTerm();
+      }
+      
+      substitute.setCondition(condition);
+    }
+    <RPAREN>
+    [
+      <AND> extendedKey=<ID>
+      {
+        String eKey = extendedKey.image.toString();
+        String cKey = chainKey.image.toString();
+        substitute.setCondition(condition);
+        if (secondR.equals(eKey)) {
+          substitute.resetVariableRelation(eKey, cKey);
+        } else {
+          substitute.resetVariableRelation(cKey, eKey);
+        }
+      }
+    ]
+  | <TABLE><LPAREN>
+    tableName = identifier() 
+    { substitute.setInput(tableName); }
+    <RPAREN>
+  | operation=<GROUP>chainKey=<ID><BY>
+    <LPAREN>
+      { 
+        List<String> columnList = columnList(); 
+        for (int i = 0; i < columnList.size(); i++) {
+          condition += appendIndicator(columnList.get(i));
+        }  
+      }
+    <RPAREN>
+    { 
+      substitute.setChainKey(chainKey.image.toString());
+      substitute.setOperation(operation.image.toString());
+      substitute.setCondition(condition);
+    }
+  )
+  
+  {
+    return substitute;
+  }
+}
+
+SaveCommand saveCommand() :
+{
+  Token t = null;
+  String tableName;
+  SaveCommand save = new SaveCommand(this.out);
+}
+{
+  <SAVE> t=<ID>
+  { save.setStatement(t.image.toString()); }
+  <INTO><TABLE><LPAREN>
+  tableName = identifier() { save.setOutput(tableName); }
+  <RPAREN>
+  {
+    return save;
+  }
+}
 
 List<String> getLiteralValues() :
 {
@@ -657,6 +764,19 @@ String getStringLiteral() :
  }
 }
 
+String getColumn() :
+{
+  Token col;
+}
+{
+  (
+    ( col=<ID> | col=<INTEGER_LITERAL> | col=<ASTERISK> )
+      { return col.image.toString(); }
+    | (col=<QUOTED_IDENTIFIER> | col=<STRING_LITERAL> )
+      { return col.image.substring(1,col.image.toString().length() - 1); }
+  )
+}
+
 List<String> getColumns() : // return parenthesized column list
 {
   List<String> values = new ArrayList<String>();
@@ -680,34 +800,21 @@ List<String> getColumns() : // return parenthesized column list
     }
 }
 
-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() :
 {
   List<String> tableList = new ArrayList<String>();
   String table = null;
 }
 {
-  table = Identifier() { tableList.add(table); }
-  ( <COMMA> table = Identifier()
+  table = identifier() { tableList.add(table); }
+  ( <COMMA> table = identifier()
     { tableList.add(table); } 
   )*
   
   { return tableList; }
 }
 
-List<String> ColumnList() :
+List<String> columnList() :
 {
   List<String> columnList = new ArrayList<String>();  
   String column = null;
@@ -737,7 +844,7 @@ int Number() :
   { return Integer.parseInt(t.image.toString()); }
 }
 
-String Identifier() :
+String identifier() :
 {
   Token t = null;
 }
@@ -748,4 +855,155 @@ String Identifier() :
    | ( t=<QUOTED_IDENTIFIER> | t=<STRING_LITERAL> )
      { return t.image.substring(1,t.image.toString().length() - 1); }
   )
+}
+
+String booleanTerm() :
+{
+  String query = "";
+  String tmp = null;
+  String joinTerm = null;
+  String connector = null;
+}
+{
+  query = booleanTerms()
+  {
+    if(query.endsWith(".ROW")) {
+      joinTerm = query;
+      query = "";
+    }
+  }
+  (
+      (
+       <AND> { connector = " AND ";  }
+       | <OR> { connector = " OR "; }
+      ) tmp = booleanTerms() { 
+     if(tmp.endsWith(".ROW")) {
+      joinTerm = tmp; 
+     } else {
+       if(!query.equals(""))
+         query += connector;
+        query += tmp; 
+     }
+      }
+  )*
+  { 
+    if(joinTerm != null) {
+      return joinTerm + " BOOL " + query;
+    } else {
+      return query;
+    }
+  }
+}
+
+String booleanTerms() : 
+{
+  Token tSearchName, tComparand;
+  List<String> inList = new ArrayList<String>();
+  String searchName=null,comparator=null,comparand=null;
+  Token joinColumn = null;
+  Token joinKey = null;
+  String result = null;
+}
+{
+  ( 
+    tSearchName=<ID> { searchName = tSearchName.image.toString(); }
+    [
+      <DOT> 
+      (  
+        joinColumn=<ID>
+        { searchName += "." + joinColumn.image.toString(); }
+        | <ROW>
+        { 
+          secondR = searchName;
+          searchName += ".ROW";
+        }
+      )
+    ]
+  )
+  comparator = getComparator()
+  (
+     tComparand=<INTEGER_LITERAL> 
+      { comparand = tComparand.image.toString(); }
+    | tComparand=<STRING_LITERAL> 
+      { comparand = tComparand.image.substring(1,tComparand.image.length() - 1); }
+    | tComparand=<ID>
+     { comparand = tComparand.image.toString(); }
+      [
+        <DOT>
+      (
+          <ROW>
+          { 
+            secondR = comparand;
+            comparand += ".ROW"; 
+            }
+          | joinColumn=<ID>
+          { comparand += "." + joinColumn.image.toString(); }
+      )
+      ]
+    | inList = getColumns()
+    { 
+      if(comparator == null) {
+        comparator = "==";
+      }
+      comparand = "";
+      try{
+        for(int i=0; i<inList.size(); i++) {     
+          comparand += URLEncoder.encode(inList.get(i), "UTF-8");
+          if(inList.size() != (i+1)) { comparand += "|"; }
+        }
+      } catch (UnsupportedEncodingException e) {
+          e.printStackTrace();
+      }
+    }
+  )
+   { 
+    if(searchName.endsWith(".ROW")) {
+      result = appendIndicator(comparand) 
+      + " " + comparator + " " + searchName;
+    } else {
+      result = appendIndicator(searchName) 
+      + " " + comparator + " " + comparand;
+    }
+    
+     return result; 
+   }
+}
+
+String getComparator() :
+{
+  Token t = null;
+  String comparator = null;
+}
+{
+  (  
+      t=<LCOMP> 
+        { comparator = t.image.toString(); } 
+      [<EQUALS> { comparator += "="; }]
+    | t=<RCOMP> 
+        { comparator = t.image.toString(); } 
+      [<EQUALS> { comparator += "="; }]
+    | t=<EQUALS> 
+        { comparator = t.image.toString(); }
+      [<LCOMP> { comparator = ">" + comparator; } ]
+      [<RCOMP> { comparator = "<" + comparator; } ]
+    | t=<NOTEQUAL>
+        { comparator = t.image.toString(); }
+    | <NOT><IN>
+        { comparator = "!!"; }
+    | <IN>
+        { comparator = "=="; }
+  )
+  
+  { return comparator; }
+}
+
+String appendIndicator(String columnName) :
+{
+  String column = columnName;
+}
+{
+  {
+     return (!column.endsWith(":") && column.indexOf(":") == -1) 
+     ? column + ":" : column;
+  }
 }

+ 101 - 71
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/HelpCommand.java

@@ -26,14 +26,13 @@ import java.io.Writer;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 
 public class HelpCommand extends BasicCommand {
   private String argument;
-  private static final String [] HEADER =
-    new String [] {"Command", "Description", "Example"};
-  
+  private static final String[] HEADER = new String[] { "Command",
+      "Description", "Example" };
+
   /** application name */
   public static final String APP_NAME = "Hbase Shell";
 
@@ -41,18 +40,18 @@ public class HelpCommand extends BasicCommand {
   public static final String APP_VERSION = "0.0.2";
 
   /** help contents map */
-  public final Map<String, String[]> help =
-    new HashMap<String, String[]>();
-    
+  public final Map<String, String[]> help = new HashMap<String, String[]>();
+
   private final TableFormatter formatter;
-  
+
   public HelpCommand(final Writer o, final TableFormatter f) {
     super(o);
     this.help.putAll(load());
     this.formatter = f;
   }
 
-  public ReturnMsg execute(@SuppressWarnings("unused") HBaseConfiguration conf) {
+  public ReturnMsg execute(@SuppressWarnings("unused")
+  HBaseConfiguration conf) {
     try {
       printHelp(this.argument);
     } catch (IOException e) {
@@ -66,73 +65,104 @@ public class HelpCommand extends BasicCommand {
   }
 
   /**
-   * add help contents 
+   * add help contents
    */
   private Map<? extends String, ? extends String[]> load() {
     Map<String, String[]> load = new HashMap<String, String[]>();
-    load.put("SHOW", new String[] {"List all user tables", "SHOW TABLES;"});
-    
-    load.put("FS", new String[] { "Hadoop FsShell; entering a lone 'FS;' " +
-      "will emit usage",
-      "FS -copyFromLocal /home/user/backup.dat fs/user/backup;"});
-    
-    load.put("JAR", new String[] { "Hadoop RunJar util", 
-      "JAR ./build/hadoop-examples.jar pi 10 10;"});
-    load.put("CLEAR", new String[] {"Clear the screen", "CLEAR;"} );
-    
+    load.put("SHOW", new String[] { "Show information about selected title",
+        "SHOW TABLES[or substitution variable name];" });
+
+    load.put("FS", new String[] {
+        "Hadoop FsShell; entering a lone 'FS;' " + "will emit usage",
+        "FS -copyFromLocal /home/user/backup.dat fs/user/backup;" });
+
+    load.put("JAR", new String[] { "Hadoop RunJar util",
+        "JAR ./build/hadoop-examples.jar pi 10 10;" });
+    load.put("CLEAR", new String[] { "Clear the screen", "CLEAR;" });
+
     load.put("DESCRIBE", new String[] { "Print table information",
-      "[DESCRIBE|DESC] table_name;"});
-    
-    load.put("CREATE", new String[] {
-        "Create tables",
-        "CREATE TABLE table_name (column_family_name [MAX_VERSIONS=n] " +
-        "[MAX_LENGTH=n] [COMPRESSION=NONE|RECORD|BLOCK] [IN_MEMORY] " +
-        "[BLOOMFILTER=NONE|BLOOM|COUNTING|RETOUCHED VECTOR_SIZE=n NUM_HASH=n], " +
-        "...)"
-    });
-    load.put("DROP", new String[] {
-        "Drop tables",
-        "DROP TABLE table_name [, table_name] ...;"});
-    
+        "[DESCRIBE|DESC] table_name;" });
+
+    load
+        .put(
+            "CREATE",
+            new String[] {
+                "Create tables",
+                "CREATE TABLE table_name (column_family_name [MAX_VERSIONS=n] "
+                    + "[MAX_LENGTH=n] [COMPRESSION=NONE|RECORD|BLOCK] [IN_MEMORY] "
+                    + "[BLOOMFILTER=NONE|BLOOM|COUNTING|RETOUCHED VECTOR_SIZE=n NUM_HASH=n], "
+                    + "...)" });
+    load.put("DROP", new String[] { "Drop tables",
+        "DROP TABLE table_name [, table_name] ...;" });
+
     load.put("INSERT", new String[] {
         "Insert values into table",
-        "INSERT INTO table_name (column_name, ...) " +
-        "VALUES ('value', ...) WHERE row='row_key';"
-    });
-    
+        "INSERT INTO table_name (column_name, ...) "
+            + "VALUES ('value', ...) WHERE row='row_key';" });
+
     load.put("DELETE", new String[] {
         "Delete table data",
-        "DELETE {column_name, [, column_name] ... | *} FROM table_name " +
-        "WHERE row='row-key';"
-    });
-    
-    load.put("SELECT",
-        new String[] {
-            "Select values from table",
-            "SELECT {column_name, [, column_name] ... | *} FROM table_name " +
-            "[WHERE row='row_key' | STARTING FROM 'row-key'] " +
-            "[NUM_VERSIONS = version_count] " +
-            "[TIMESTAMP 'timestamp'] " +
-            "[LIMIT = row_count] " +
-            "[INTO FILE 'file_name'];"
-    });
-                
-    load.put("ALTER",
-        new String[] {
-            "Alter structure of table",
-            "ALTER TABLE table_name ADD column_spec | " +
-            "ADD (column_spec, column_spec, ...) | " +
-            "DROP column_family_name | " +
-            "CHANGE column_spec;"
-    });
-
-    load.put("EXIT", new String[] { "Exit shell", "EXIT;"});
+        "DELETE {column_name, [, column_name] ... | *} FROM table_name "
+            + "WHERE row='row-key';" });
+
+    load.put("SELECT", new String[] {
+        "Select values from table",
+        "SELECT {column_name, [, column_name] ... | *} FROM table_name "
+            + "[WHERE row='row_key' | STARTING FROM 'row-key'] "
+            + "[NUM_VERSIONS = version_count] " + "[TIMESTAMP 'timestamp'] "
+            + "[LIMIT = row_count] " + "[INTO FILE 'file_name'];" });
+
+    load.put("ALTER", new String[] {
+        "Alter structure of table",
+        "ALTER TABLE table_name ADD column_spec | "
+            + "ADD (column_spec, column_spec, ...) | "
+            + "DROP column_family_name | " + "CHANGE column_spec;" });
+
+    load.put("EXIT", new String[] { "Exit shell", "EXIT;" });
+
+    // A Algebraic Query Commands
+    // this is a tentative query language based on a hbase which uses relational
+    // model of
+    // data.
+
+    load.put("TABLE",
+        new String[] { "Load a table", "A = table('table_name');" });
+    load.put("SUBSTITUTE", new String[] { "Substitute expression to [A~Z]",
+        "D = A.projection('cf_name1'[, 'cf_name2']);" });
+    load.put("SAVE", new String[] { "Save results into specified table (It runs a mapreduce job)",
+        "SAVE A INTO table('table_name');" });
+
+    // Relational Operations
+    load.put("PROJECTION", new String[] {
+        "Selects a subset of the columnfamilies of a relation",
+        "A = TABLE('table_name');"
+            + " B = A.Projection('cf_name1'[, 'cf_name2']);" });
+    load
+        .put(
+            "SELECTION",
+            new String[] {
+                "Selects a subset of the rows in a relation that satisfy a selection condition (>, <, AND, OR, etc.)",
+                "A = Table('table_name');"
+                    + " B = A.Selection(cf_name1 > 100 [AND cf_name2 = 'string_value']);" });
+
+    // Aggregation Functions
+    //TODO : and apply aggregate function independently to each group of rows 
+    load
+        .put(
+            "GROUP",
+            new String[] {
+                "Group rows by value of an attribute",
+                "A = Table('table_name');"
+                    + " B = Group A by ('cf_name1'[, 'cf_name2']);" });
 
     return load;
   }
 
-  /** Print out the program version. 
-   * @throws IOException */
+  /**
+   * Print out the program version.
+   * 
+   * @throws IOException
+   */
   public void printVersion() throws IOException {
     println(APP_NAME + ", " + APP_VERSION + " version.\n"
         + "Copyright (c) 2007 by udanax, "
@@ -146,7 +176,7 @@ public class HelpCommand extends BasicCommand {
       printHelp(this.help);
     } else {
       if (this.help.containsKey(cmd.toUpperCase())) {
-        final Map<String, String []> m = new HashMap<String, String []>();
+        final Map<String, String[]> m = new HashMap<String, String[]>();
         m.put(cmd.toUpperCase(), this.help.get(cmd.toUpperCase()));
         printHelp(m);
       } else {
@@ -154,14 +184,14 @@ public class HelpCommand extends BasicCommand {
       }
     }
   }
-  
-  private void printHelp(final Map<String, String []> m) throws IOException {
+
+  private void printHelp(final Map<String, String[]> m) throws IOException {
     this.formatter.header(HEADER);
-    for (Map.Entry<String, String []> e: m.entrySet()) {
-      String [] value = e.getValue();
+    for (Map.Entry<String, String[]> e : m.entrySet()) {
+      String[] value = e.getValue();
       if (value.length == 2) {
-        this.formatter.row(new String [] {e.getKey().toUpperCase(), value[0],
-          value[1]});
+        this.formatter.row(new String[] { e.getKey().toUpperCase(), value[0],
+            value[1] });
       } else {
         throw new IOException("Value has too many elements:" + value);
       }

+ 114 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/SaveCommand.java

@@ -0,0 +1,114 @@
+/**
+ * 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.io.Writer;
+
+import org.apache.hadoop.hbase.HBaseAdmin;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConnection;
+import org.apache.hadoop.hbase.HConnectionManager;
+import org.apache.hadoop.hbase.shell.algebra.OperationEvaluator;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RunningJob;
+
+/**
+ * Save results to specified table.
+ */
+public class SaveCommand extends BasicCommand {
+  private String chainKey;
+  private String output;
+
+  public SaveCommand(Writer o) {
+    super(o);
+  }
+
+  public ReturnMsg execute(HBaseConfiguration conf) {
+    try {
+      HConnection conn = HConnectionManager.getConnection(conf);
+
+      if (!conn.tableExists(new Text(output))) {
+        OperationEvaluator eval = new OperationEvaluator(conf, chainKey, output);
+        JobConf jobConf = eval.getJobConf();
+        if (submitJob(jobConf)) {
+          return new ReturnMsg(0, "Successfully complete.");
+        } else {
+          HBaseAdmin admin = new HBaseAdmin(conf);
+          admin.deleteTable(new Text(output));
+
+          return new ReturnMsg(0, "Job failed.");
+        }
+      } else {
+        return new ReturnMsg(0, "'" + output + "' table already exist.");
+      }
+
+    } catch (IOException e) {
+      return new ReturnMsg(0, e.toString());
+    }
+  }
+
+  /**
+   * Submit a job to job tracker.
+   * 
+   * @param job
+   * @return result
+   * @throws IOException
+   */
+  public boolean submitJob(JobConf job) throws IOException {
+    JobClient jc = new JobClient(job);
+    boolean success = true;
+    RunningJob running = null;
+    try {
+      running = jc.submitJob(job);
+      String jobId = running.getJobID();
+
+      while (!running.isComplete()) {
+        try {
+          Thread.sleep(5000);
+        } catch (InterruptedException e) {
+        }
+        running = jc.getJob(jobId);
+      }
+      success = running.isSuccessful();
+    } finally {
+      if (!success && (running != null)) {
+        running.killJob();
+      }
+      jc.close();
+    }
+    return success;
+  }
+
+  public void setOutput(String output) {
+    this.output = output;
+  }
+
+  public void setStatement(String chainKey) {
+    this.chainKey = chainKey;
+  }
+
+  @Override
+  public CommandType getCommandType() {
+    return CommandType.SHELL;
+  }
+}

+ 32 - 6
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/ShowCommand.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.shell;
 
 import java.io.IOException;
 import java.io.Writer;
+import java.util.Map;
 
 import org.apache.hadoop.hbase.HBaseAdmin;
 import org.apache.hadoop.hbase.HBaseConfiguration;
@@ -30,16 +31,16 @@ import org.apache.hadoop.hbase.HTableDescriptor;
  * Shows all available tables.
  */
 public class ShowCommand extends BasicCommand {
-  private static final String [] HEADER = new String [] {"Name", "Descriptor"};
+  private static final String[] HEADER = new String[] { "Name", "Descriptor" };
   private String command;
   private final TableFormatter formatter;
-  
+
   // Not instantiable
   @SuppressWarnings("unused")
   private ShowCommand() {
     this(null, null);
   }
-  
+
   public ShowCommand(final Writer o, final TableFormatter f) {
     this(o, f, null);
   }
@@ -67,12 +68,37 @@ public class ShowCommand extends BasicCommand {
         formatter.header(HEADER);
         for (int i = 0; i < tableLength; i++) {
           String tableName = tables[i].getName().toString();
-          formatter.row(new String [] {tableName, tables[i].toString()});
+          formatter.row(new String[] { tableName, tables[i].toString() });
         }
         formatter.footer();
         return new ReturnMsg(1, tableLength + " table(s) in set");
+      } else {
+        Map<String, VariableRef> refer = VariablesPool.get(command);
+        if (refer == null) {
+          return new ReturnMsg(0, "Unknown arguments.");
+        }
+
+        String msg = null;
+        for (Map.Entry<String, VariableRef> e : refer.entrySet()) {
+          msg = command + " = ";
+          if (e.getKey() != null) {
+            msg += e.getKey() + ".";
+          }
+          msg += e.getValue().getOperation() + "(";
+          if (e.getValue().getOperation().equals("projection")) {
+            String[] proj = e.getValue().getArgument().split(" ");
+            for (int i = 0; i < proj.length; i++) {
+              msg += "'" + proj[i] + "'";
+              if (i + 1 != proj.length)
+                msg += ", ";
+            }
+          } else {
+            msg += e.getValue().getArgument().replace(" BOOL ", " and ");
+          }
+          msg += ");\n";
+        }
+        return new ReturnMsg(0, msg);
       }
-      return new ReturnMsg(0, "Missing parameters. Please check 'Show' syntax");
     } catch (IOException e) {
       return new ReturnMsg(0, "error msg : " + e.toString());
     }
@@ -81,4 +107,4 @@ public class ShowCommand extends BasicCommand {
   public void setArgument(String argument) {
     this.command = argument;
   }
-}
+}

+ 78 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/SubstituteCommand.java

@@ -0,0 +1,78 @@
+/**
+ * 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.Writer;
+
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.shell.algebra.Constants;
+
+/**
+ * This class represents a substitute command.
+ */
+public class SubstituteCommand extends BasicCommand {
+  private String key;
+  private String chainKey;
+  private String operation;
+  private String condition;
+
+  public SubstituteCommand(Writer o) {
+    super(o);
+  }
+
+  public ReturnMsg execute(HBaseConfiguration conf) {
+    VariableRef formula = new VariableRef(operation, condition);
+    VariablesPool.put(key, chainKey, formula);
+    return null;
+  }
+
+  public void setInput(String input) {
+    this.operation = "table";
+    this.condition = input;
+  }
+
+  public void setKey(String key) {
+    this.key = key;
+  }
+
+  public void setChainKey(String chainKey) {
+    this.chainKey = chainKey;
+  }
+
+  public void setOperation(String operation) {
+    this.operation = operation;
+  }
+
+  public void setCondition(String condition) {
+    this.condition = condition;
+  }
+
+  public void resetVariableRelation(String r1, String r2) {
+    setChainKey(r1);
+    String tableName = VariablesPool.get(r1).get(null).getArgument();
+    VariableRef formula = new VariableRef(Constants.JOIN_SECOND_RELATION, tableName);
+    VariablesPool.put(r1, r2, formula);
+  }
+
+  @Override
+  public CommandType getCommandType() {
+    return CommandType.SHELL;
+  }
+}

+ 50 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/VariableRef.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;
+
+/**
+ * VariableRef is used to reference declared Variables.
+ */
+public class VariableRef {
+  String operation;
+  String argument;
+
+  /** Constructor */
+  public VariableRef(String operation, String condition) {
+    this.operation = operation;
+    this.argument = condition;
+  }
+
+  /**
+   * Return argument of an operation
+   * @return argument
+   */
+  public String getArgument() {
+    return argument;
+  }
+
+  /**
+   * Return operation
+   * @return operation
+   */
+  public String getOperation() {
+    return operation;
+  }
+}

+ 52 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/VariablesPool.java

@@ -0,0 +1,52 @@
+/**
+ * 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.HashMap;
+
+/**
+ * Variable pool is a collection of substitution variables.
+ */
+public class VariablesPool {
+  static HashMap<String, HashMap<String, VariableRef>> variables = new HashMap<String, HashMap<String, VariableRef>>();
+
+  /**
+   * puts the date in the substitution variable.
+   * 
+   * @param key
+   * @param parentKey
+   * @param statement
+   */
+  public static void put(String key, String parentKey, VariableRef statement) {
+    HashMap<String, VariableRef> value = new HashMap<String, VariableRef>();
+    value.put(parentKey, statement);
+    variables.put(key, value);
+  }
+
+  /**
+   * returns the substitution variable's value.
+   * 
+   * @param key
+   * @return HashMap<String, VariableRef>
+   */
+  public static HashMap<String, VariableRef> get(String key) {
+    return variables.get(key);
+  }
+}

+ 36 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/Constants.java

@@ -0,0 +1,36 @@
+/**
+ * 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.algebra;
+
+/**
+ * List of access control algebraic operations constants.
+ */
+public class Constants {
+  public static final String OUTPUT_TABLE_EXIST = "job.config.output.table";
+  public static final String CONFIG_INPUT = "input";
+  public static final String CONFIG_OUTPUT = "output";
+  public static final String EXPRESSION_FILTER_LIST = "expression.filter.list";
+  
+  public static final String RELATIONAL_PROJECTION = "projection";
+  public static final String RELATIONAL_SELECTION = "selection";
+  public static final String RELATIONAL_GROUP = "group";
+  public static final String RELATIONAL_JOIN = "join";
+  public static final String JOIN_SECOND_RELATION = "secondR";
+}

+ 50 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/DuplicateTable.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.algebra;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.mapred.IdentityTableMap;
+import org.apache.hadoop.hbase.mapred.IdentityTableReduce;
+import org.apache.hadoop.mapred.JobConf;
+
+/**
+ * Duplicates Table. R1 to R3 in O(N)
+ */
+public class DuplicateTable extends RelationalOperation {
+  public DuplicateTable(HBaseConfiguration conf, Map<String, String> condition) {
+    super(conf, condition);
+  }
+
+  @Override
+  public JobConf getConf() throws IOException, RuntimeException {
+    HColumnDescriptor[] columns = getInputColumnDescriptor();
+    outputTableCreate(columns, null);
+
+    IdentityTableMap.initJob(input, getColumnStringArray(columns),
+        IdentityTableMap.class, jobConf);
+    IdentityTableReduce.initJob(output, IdentityTableReduce.class, jobConf);
+
+    return jobConf;
+  }
+}

+ 559 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/ExpressionParser.jj

@@ -0,0 +1,559 @@
+options {
+  STATIC = false;
+  IGNORE_CASE = true;
+}
+
+PARSER_BEGIN(ExpressionParser)
+package org.apache.hadoop.hbase.shell.algebra.generated;
+
+/**
+ * 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.
+ */
+
+import java.io.IOException;
+import java.io.Reader;
+import java.io.StringReader;
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HTable;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.MapWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Parsing command line.
+ */
+public class ExpressionParser {
+  HBaseConfiguration conf = new HBaseConfiguration();
+  private String expression;
+  private String joinKeyColumn;
+  private String secondRelation;
+  private Map<String, List<String>> unionSet = new HashMap<String, List<String>>();
+  private Map<String, List<String>> intersectionSet = new HashMap<String, List<String>>();
+  private SortedMap<Text, byte[]> secondValue = null;
+
+  public ExpressionParser(final String expression) {
+    this((Reader)(new StringReader(expression)));
+    this.expression = expression;
+  }
+  
+  public String getExpression() {
+    return this.expression;
+  }
+}
+
+PARSER_END(ExpressionParser)
+
+SKIP :                             
+{
+    " "
+ | "\t"
+ | "\r"
+ | "\n"
+}
+
+TOKEN : /** Literals */
+{
+   <ROW: "row">
+ | <AND: "and">
+ | <OR: "or">
+ | <DOT: ".">
+ | <EQUALS: "=">
+ | <LCOMP: ">">
+ | <RCOMP: "<">
+ | <LCOMPE: ">=">
+ | <RCOMPE: "<=">
+ | <IN: "==">
+ | <NOTIN: "!!">
+ | <BAR: "|">
+ | <BOOL: "bool">
+ | <ID: ["A"-"Z","a"-"z","_","-",":","/"] (["A"-"Z","a"-"z","0"-"9","_","-",":","/"])* >
+ | <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: "'" (~["'"])* ( "''" (~["'"])* )* "'" >
+}
+
+void booleanExpressionParse() :
+{
+  Map<String, List<String>> temp = new HashMap<String, List<String>>();
+}
+{
+  temp = booleanTerm()
+  (
+    (
+        <AND>
+        { 
+          if(temp != null) {
+            for(Map.Entry<String, List<String>> e : temp.entrySet()) {
+              List<String> newList = intersectionSet.get(e.getKey());
+              if(newList != null) {
+                newList.addAll(e.getValue());
+              } else {
+                newList = e.getValue();
+              }
+              intersectionSet.put(e.getKey(), newList);
+            }
+          }
+          temp = booleanTerm();
+          for(Map.Entry<String, List<String>> e : temp.entrySet()) {
+            List<String> newList = intersectionSet.get(e.getKey());
+            if(newList != null) {
+              newList.addAll(e.getValue());
+            } else {
+              newList = e.getValue();
+            }
+            intersectionSet.put(e.getKey(), newList);
+          }
+          temp = null;
+          }
+      | <OR>
+        {
+        if(temp != null) {
+          for(Map.Entry<String, List<String>> e : temp.entrySet()) {
+            List<String> newList = unionSet.get(e.getKey());
+            if(newList != null) {
+              newList.addAll(e.getValue());
+            } else {
+              newList = e.getValue();
+            }
+            unionSet.put(e.getKey(), newList);
+          }    
+        }
+        temp = booleanTerm();
+        for(Map.Entry<String, List<String>> e : temp.entrySet()) {
+          List<String> newList = unionSet.get(e.getKey());
+          if(newList != null) {
+            newList.addAll(e.getValue());
+          } else {
+            newList = e.getValue();
+          }
+          unionSet.put(e.getKey(), newList);
+        }
+        temp = null;
+        }
+      ) 
+  )* 
+  {
+    if(temp != null) {
+      for(Map.Entry<String, List<String>> e : temp.entrySet()) {
+        List<String> newList = unionSet.get(e.getKey());
+        if(newList != null) {
+          newList.addAll(e.getValue());
+        } else {
+          newList = e.getValue();
+        }
+        unionSet.put(e.getKey(), newList);
+      }
+    }
+  }
+}
+
+void joinExpressionParse() :
+{
+  Map<String, List<String>> temp = new HashMap<String, List<String>>();
+}
+{
+  indexJoinCondition()
+  <BOOL>
+  [
+    temp = booleanTerm()
+      (
+      (
+       <AND>
+       { 
+         if(temp != null) {
+           for(Map.Entry<String, List<String>> e : temp.entrySet()) {
+             List<String> newList = intersectionSet.get(e.getKey());
+             if(newList != null) {
+               newList.addAll(e.getValue());
+             } else {
+               newList = e.getValue();
+             }
+             intersectionSet.put(e.getKey(), newList);
+           }
+         }
+         temp = booleanTerm();
+         for(Map.Entry<String, List<String>> e : temp.entrySet()) {
+           List<String> newList = intersectionSet.get(e.getKey());
+           if(newList != null) {
+             newList.addAll(e.getValue());
+           } else {
+             newList = e.getValue();
+           }
+           intersectionSet.put(e.getKey(), newList);
+         }
+         temp = null;
+         }
+      | <OR>
+        {
+        if(temp != null) {
+          for(Map.Entry<String, List<String>> e : temp.entrySet()) {
+            List<String> newList = unionSet.get(e.getKey());
+            if(newList != null) {
+              newList.addAll(e.getValue());
+            } else {
+              newList = e.getValue();
+            }
+            unionSet.put(e.getKey(), newList);
+          }
+        }
+        temp = booleanTerm();
+        for(Map.Entry<String, List<String>> e : temp.entrySet()) {
+          List<String> newList = unionSet.get(e.getKey());
+          if(newList != null) {
+            newList.addAll(e.getValue());
+          } else {
+            newList = e.getValue();
+          }
+          unionSet.put(e.getKey(), newList);
+        }
+        temp = null;
+        }
+      ) 
+    )* 
+    {
+      if(temp != null) {
+        for(Map.Entry<String, List<String>> e : temp.entrySet()) {
+          List<String> newList = unionSet.get(e.getKey());
+          if(newList != null) {
+            newList.addAll(e.getValue());
+          } else {
+            newList = e.getValue();
+          }
+          unionSet.put(e.getKey(), newList);
+        }
+        }
+    }
+  ]
+}
+
+void indexJoinCondition() :
+{
+ Token firstR = null; 
+  Token joinKey = null;
+  Token secondR = null;
+}
+{
+  firstR=<ID><DOT>joinKey=<ID>
+  <EQUALS>
+  secondR=<ID><DOT><ROW>
+  {
+    joinKeyColumn = joinKey.image.toString();
+    secondRelation = secondR.image.toString();
+  }
+}
+
+Map<String, List<String>> booleanTerm() :
+{
+  Token tSearchName = null;
+  Token tComparator = null;
+  Token tFirst = null;
+  Token tSecond = null;
+  Map<String, List<String>> result = new HashMap<String, List<String>>();
+  List<String> valueList = new ArrayList<String>();
+  String comparand = null;
+}
+{
+  (
+      tSearchName=<ID>
+    | tSearchName=<INTEGER_LITERAL>
+  ) [<DOT> ( tFirst=<ID> | tFirst=<INTEGER_LITERAL> ) ]
+  ( tComparator=<EQUALS> | tComparator=<LCOMP> | tComparator=<LCOMPE> | tComparator=<RCOMPE>
+  | tComparator=<RCOMP> | tComparator=<IN> | tComparator=<NOTIN> )
+  comparand = getValueList()
+  [<DOT>  ( tSecond=<ID> | tSecond=<INTEGER_LITERAL> )]
+
+  {
+    if(tFirst == null && tSecond == null) {
+      valueList.add(tComparator.image.toString() + " " +comparand);
+      result.put(tSearchName.image.toString(), valueList);
+    } else if (tFirst != null && tSecond != null ){
+      if(tSearchName.image.toString().equals(secondRelation)) {
+        valueList.add(tComparator.image.toString() + " " 
+            + secondValue.get(new Text(tSearchName.image.toString())));
+        result.put(tFirst.image.toString(), valueList);
+      } else {
+        valueList.add(tComparator.image.toString() + " " + tSecond.image.toString());
+        result.put(tFirst.image.toString(), valueList);
+      }
+    }
+    return result;
+  }
+}
+
+String getValueList() :
+{
+  Token tComparand = null;
+  Token tList = null;
+  String result = "";
+}
+{
+  ( tComparand=<ID> | tComparand=<INTEGER_LITERAL> )
+  {
+    result = tComparand.image.toString();
+  }
+  [
+    <BAR>
+    (
+      (tList=<ID> | tList=<INTEGER_LITERAL> )
+      {
+        result += "|" + tList.image.toString();
+      }
+      [ <BAR> ]
+    )*
+  ]
+  { return result; }
+}
+
+boolean checkConstraints(MapWritable data) :
+{
+  Map<String, String> record = getComparisonObject(data);
+  boolean result = false;
+  if(intersectionSet.size() == 0 && unionSet.size() == 0) {
+    return true;
+  }
+  if (data == null || record.size() == 0) {
+    return result; // return false if data is null.
+  } 
+}
+{
+  {
+      if (intersectionSet.size() == record.size()) {
+        result = booleanCompare(intersectionSet, record, true);
+      } else if (unionSet.size() == record.size()) {
+        result = booleanCompare(unionSet, record, false);
+      } else {
+        result = getCompareResult(record);
+      }
+    return result;
+  }
+}
+
+Map<String, String> getComparisonObject(MapWritable data) :
+{
+  Map<String, String> result = new HashMap<String, String>();
+}
+{
+  {
+    for (Map.Entry<Writable, Writable> e : data.entrySet()) {
+      String cKey = e.getKey().toString();
+      String val = new String(((ImmutableBytesWritable) e.getValue()).get());
+
+      if (intersectionSet.containsKey(cKey) || unionSet.containsKey(cKey)) {
+        result.put(cKey, val);
+      }
+    }
+
+    return result;
+  }
+}
+
+MapWritable getJoinColumns(MapWritable value, int numCols, String secondRelation) :
+{
+  MapWritable appendValue = new MapWritable();
+  String joinKey = null;
+  this.secondRelation = secondRelation;
+}
+{
+  {
+    if (numCols > 0) {
+      for (Map.Entry<Writable, Writable> e : value.entrySet()) {
+        Text column = (Text) e.getKey();
+        for (int i = 0; i < numCols; i++) {
+          if (column.equals(new Text(joinKeyColumn))) {
+            joinKey = new String(((ImmutableBytesWritable) e.getValue()).get());
+            break;
+          }
+        }
+      }
+    }
+    
+    //If joinKey is null, just return.
+    if(joinKey == null) {
+      return appendValue;
+    }
+    
+    try {
+      HTable table = new HTable(conf, new Text(secondRelation));
+      secondValue = table.getRow(new Text(joinKey));
+      for (Map.Entry<Text, byte[]> e : secondValue.entrySet()) {
+        appendValue.put(e.getKey(), new ImmutableBytesWritable(e.getValue()));
+      }
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+
+    return appendValue;
+  }
+}
+
+boolean getCompareResult(Map<String, String> record) :
+{
+}
+{
+  {
+      return (booleanCompare(intersectionSet, record, true) && 
+          booleanCompare(unionSet, record, false)) ? true : false;
+  }
+}
+
+boolean booleanCompare(Map<String, List<String>> expression,
+    Map<String, String> record, boolean isIntersection) :
+{
+  boolean negative = true;
+  boolean positive = false;
+}
+{
+  {
+    try{
+      for (Map.Entry<String, List<String>> e : expression.entrySet()) {
+        String key = e.getKey();
+        List<String> valueList = e.getValue();
+        String recordValueList = record.get(key);
+
+        for (int i = 0; i < valueList.size(); i++) {
+          String[] term = valueList.get(i).split(" ");
+          String comparator = term[0];
+          String comparand = term[1];
+
+          switch (comparator.charAt(0)) {
+            case '>':
+              if (isSecond(comparator, "=")) {
+                if (Integer.parseInt(comparand) > Integer
+                    .parseInt(recordValueList)) {
+                  negative = false;
+                } else {
+                  positive = true;
+                }
+              } else {
+                if (Integer.parseInt(comparand) > Integer
+                    .parseInt(recordValueList)
+                    || comparand.equals(recordValueList)) {
+                  negative = false;
+                } else {
+                  positive = true;
+                }
+              }
+              break;
+            case '<':
+              if (isSecond(comparator, "=")) {
+                if (Integer.parseInt(comparand) < Integer
+                    .parseInt(recordValueList))
+                  negative = false;
+                else
+                  positive = true;
+              } else {
+                if (Integer.parseInt(comparand) < Integer
+                    .parseInt(recordValueList)
+                    || comparand.equals(recordValueList))
+                  negative = false;
+                else
+                  positive = true;
+              }
+              break;
+
+            case '!':
+              if (isSecond(comparator, "!")) {
+                boolean checkBool = true;
+                String[] coms = comparand.split("[|]");
+                for (int j = 0; j < coms.length; j++) {
+                  if (URLDecoder.decode(coms[j], "UTF-8").equals(recordValueList)) {
+                    checkBool = false;
+                  }
+                }
+
+                if (!checkBool) {
+                  negative = false;
+                } else {
+                  positive = true;
+                }
+
+              } else {
+                if (comparand.equals(recordValueList))
+                  negative = false;
+                else
+                  positive = true;
+              }
+              break;
+            case '=':
+              if (isSecond(comparator, "=")) {
+
+                boolean checkBool = true;
+                String[] coms = comparand.split("[|]");
+                for (int j = 0; j < coms.length; j++) {
+                  if (URLDecoder.decode(coms[j], "UTF-8").equals(recordValueList)) {
+                    checkBool = false;
+                  }
+                }
+
+                if (checkBool) {
+                  negative = false;
+                } else {
+                  positive = true;
+                }
+
+              } else {
+                if (!comparand.equals(recordValueList))
+                  negative = false;
+                else
+                  positive = true;
+              }
+              break;
+          }
+        }
+      }
+    } catch (UnsupportedEncodingException e) {
+      e.printStackTrace();
+    }
+
+    boolean result = false;
+    if (isIntersection) {
+      result = negative;
+    } else {
+      result = positive;
+    }
+
+    return result;
+  }
+}
+
+boolean isSecond(String comparator, String string) :
+{
+}
+{
+  {
+    return (comparator.length() == 2 && string.charAt(0) == comparator.charAt(1)) 
+    ? true : false;
+  }
+}

+ 83 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/GroupingFilterMap.java

@@ -0,0 +1,83 @@
+/**
+ * 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.algebra;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.HStoreKey;
+import org.apache.hadoop.hbase.mapred.GroupingTableMap;
+import org.apache.hadoop.hbase.mapred.TableMap;
+import org.apache.hadoop.hbase.mapred.TableOutputCollector;
+import org.apache.hadoop.hbase.shell.algebra.generated.ExpressionParser;
+import org.apache.hadoop.hbase.shell.algebra.generated.ParseException;
+import org.apache.hadoop.io.MapWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Reporter;
+
+/**
+ * Extract grouping columns from filtered records.
+ */
+public class GroupingFilterMap extends GroupingTableMap {
+  ExpressionParser expressionParser;
+  public static final String EXPRESSION = "shell.mapred.filtertablemap.exps";
+
+  public static void initJob(String table, String columns, String groupColumns,
+      String expression, Class<? extends TableMap> mapper, JobConf job) {
+    initJob(table, columns, mapper, job);
+    job.set(GROUP_COLUMNS, groupColumns);
+    job.set(EXPRESSION, expression);
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public void configure(JobConf job) {
+    super.configure(job);
+    String[] cols = job.get(GROUP_COLUMNS, "").split(" ");
+    m_columns = new Text[cols.length];
+    for (int i = 0; i < cols.length; i++) {
+      m_columns[i] = new Text(cols[i]);
+    }
+    expressionParser = new ExpressionParser(job.get(EXPRESSION, ""));
+    try {
+      expressionParser.booleanExpressionParse();
+    } catch (ParseException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public void map(@SuppressWarnings("unused")
+  HStoreKey key, MapWritable value, TableOutputCollector output,
+      @SuppressWarnings("unused")
+      Reporter reporter) throws IOException {
+    byte[][] keyVals = extractKeyValues(value);
+    if (keyVals != null) {
+      Text tKey = createGroupKey(keyVals);
+
+      try {
+        if (expressionParser.checkConstraints(value)) {
+          output.collect(tKey, value);
+        }
+      } catch (ParseException e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+}

+ 78 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/IdentityFilterMap.java

@@ -0,0 +1,78 @@
+/**
+ * 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.algebra;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.HStoreKey;
+import org.apache.hadoop.hbase.mapred.IdentityTableMap;
+import org.apache.hadoop.hbase.mapred.TableMap;
+import org.apache.hadoop.hbase.mapred.TableOutputCollector;
+import org.apache.hadoop.hbase.shell.algebra.generated.ExpressionParser;
+import org.apache.hadoop.hbase.shell.algebra.generated.ParseException;
+import org.apache.hadoop.io.MapWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Reporter;
+
+/**
+ * Extract filtered records.
+ */
+public class IdentityFilterMap extends IdentityTableMap {
+  ExpressionParser expressionParser;
+  public static final String EXPRESSION = "shell.mapred.filtertablemap.exps";
+
+  @SuppressWarnings("deprecation")
+  public static void initJob(String table, String columns, String expression,
+      Class<? extends TableMap> mapper, JobConf job) {
+    initJob(table, columns, mapper, job);
+    job.set(EXPRESSION, expression);
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see org.apache.hadoop.hbase.mapred.TableMap#configure(org.apache.hadoop.mapred.JobConf)
+   */
+  public void configure(JobConf job) {
+    super.configure(job);
+    expressionParser = new ExpressionParser(job.get(EXPRESSION, ""));
+    try {
+      expressionParser.booleanExpressionParse();
+    } catch (ParseException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Filter the value for each specified column family.
+   */
+  public void map(HStoreKey key, MapWritable value,
+      TableOutputCollector output, Reporter reporter) throws IOException {
+    Text tKey = key.getRow();
+    try {
+      if (expressionParser.checkConstraints(value)) {
+        output.collect(tKey, value);
+      }
+    } catch (ParseException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

+ 76 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/IndexJoin.java

@@ -0,0 +1,76 @@
+/**
+ * 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.algebra;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobConf;
+
+/**
+ * Perform a index join using MapReduce.
+ */
+public class IndexJoin extends RelationalOperation {
+  public IndexJoin(HBaseConfiguration conf, Map<String, String> condition) {
+    super(conf, condition);
+  }
+
+  @Override
+  public JobConf getConf() throws IOException, RuntimeException {
+    String secondRelation = condition.get(Constants.JOIN_SECOND_RELATION);
+
+    HColumnDescriptor[] firstColumns = null;
+    HColumnDescriptor[] secondColumns = null;
+    for (int i = 0; i < tables.length; i++) {
+      if (tables[i].getName().equals(new Text(input))) {
+        firstColumns = tables[i].getFamilies().values().toArray(
+            new HColumnDescriptor[] {});
+      } else if (tables[i].getName().equals(new Text(secondRelation))) {
+        secondColumns = tables[i].getFamilies().values().toArray(
+            new HColumnDescriptor[] {});
+      }
+    }
+
+    String firstColumnsStr = "";
+    String secondColumnsStr = "";
+
+    for (int i = 0; i < firstColumns.length; i++) {
+      desc.addFamily(firstColumns[i]);
+      firstColumnsStr += firstColumns[i].getName() + " ";
+    }
+
+    for (int i = 0; i < secondColumns.length; i++) {
+      desc.addFamily(secondColumns[i]);
+      secondColumnsStr += secondColumns[i].getName() + " ";
+    }
+
+    admin.createTable(desc); // create output table.
+
+    IndexJoinMap.initJob(input, secondRelation, firstColumnsStr,
+        secondColumnsStr, condition.get(Constants.RELATIONAL_JOIN),
+        IndexJoinMap.class, jobConf);
+    IndexJoinReduce.initJob(output, IndexJoinReduce.class, jobConf);
+
+    return jobConf;
+  }
+}

+ 108 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/IndexJoinMap.java

@@ -0,0 +1,108 @@
+/**
+ * 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.algebra;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.HStoreKey;
+import org.apache.hadoop.hbase.mapred.TableMap;
+import org.apache.hadoop.hbase.mapred.TableOutputCollector;
+import org.apache.hadoop.hbase.shell.algebra.generated.ExpressionParser;
+import org.apache.hadoop.hbase.shell.algebra.generated.ParseException;
+import org.apache.hadoop.io.MapWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Reporter;
+
+/**
+ * An index join exploits the existence of an row index for one of the relations
+ * used in the join to find matching rows more quickly.
+ * 
+ * Index join (using R2 row index) takes time O(i+m)/map function number.
+ */
+public class IndexJoinMap extends TableMap {
+  ExpressionParser expressionParser;
+  private String secondRelation;
+  public static final String JOIN_EXPRESSION = "shell.mapred.join.expression";
+  public static final String SECOND_RELATION = "shell.mapred.join.second.relation";
+  public static final String FIRST_COLUMNS = "shell.mapred.first.columns";
+  private Text[] first_columns;
+
+  /** constructor */
+  public IndexJoinMap() {
+    super();
+  }
+
+  /**
+   * @param firstRelation R1
+   * @param secondRelation R2
+   * @param firstColumns (A 1,A 2,...,A n)
+   * @param secondColumns (B~1~,B~2~,...,B~m~)
+   * @param joinExpression join condition expression
+   * @param mapper mapper class
+   * @param job jobConf
+   */
+  public static void initJob(String firstRelation, String secondRelation,
+      String firstColumns, String secondColumns, String joinExpression,
+      Class<? extends TableMap> mapper, JobConf job) {
+    initJob(firstRelation, firstColumns, mapper, job);
+    job.set(JOIN_EXPRESSION, joinExpression);
+    job.set(SECOND_RELATION, secondRelation);
+    job.set(FIRST_COLUMNS, firstColumns);
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public void configure(JobConf job) {
+    super.configure(job);
+    secondRelation = job.get(SECOND_RELATION, "");
+    String[] cols = job.get(FIRST_COLUMNS, "").split(" ");
+    first_columns = new Text[cols.length];
+    for (int i = 0; i < cols.length; i++) {
+      first_columns[i] = new Text(cols[i]);
+    }
+
+    expressionParser = new ExpressionParser(job.get(JOIN_EXPRESSION, ""));
+    try {
+      expressionParser.joinExpressionParse();
+    } catch (ParseException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public void map(HStoreKey key, MapWritable value,
+      TableOutputCollector output, Reporter reporter) throws IOException {
+    Text tKey = key.getRow();
+    try {
+      MapWritable appendValue = expressionParser.getJoinColumns(value,
+          first_columns.length, secondRelation);
+
+      if (appendValue.size() != 0) {
+        value.putAll(appendValue);
+        if (expressionParser.checkConstraints(value)) {
+          output.collect(tKey, value);
+        }
+      }
+    } catch (ParseException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

+ 49 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/IndexJoinReduce.java

@@ -0,0 +1,49 @@
+/**
+ * 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.algebra;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.hadoop.hbase.mapred.TableOutputCollector;
+import org.apache.hadoop.hbase.mapred.TableReduce;
+import org.apache.hadoop.io.MapWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.Reporter;
+
+/**
+ * Table join Reduce class
+ */
+public class IndexJoinReduce extends TableReduce {
+  /** constructor */
+  public IndexJoinReduce() {
+    super();
+  }
+
+  @Override
+  public void reduce(Text key, @SuppressWarnings("unchecked")
+  Iterator values, TableOutputCollector output, Reporter reporter)
+      throws IOException {
+    while (values.hasNext()) {
+      MapWritable r = (MapWritable) values.next();
+      output.collect(key, r);
+    }
+  }
+}

+ 41 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/Operation.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.algebra;
+
+import java.io.IOException;
+
+import org.apache.hadoop.mapred.JobConf;
+
+/**
+ * Operation interface for one of algebra operations like relational algebra
+ * operations, matrix algebra operations, linear algebra operations, topological
+ * algebra operations, etc.
+ */
+public interface Operation {
+
+  /**
+   * return the Map/Reduce job configuration for performing operations.
+   * 
+   * @return JobConf
+   * @throws IOException
+   * @throws RuntimeException
+   */
+  JobConf getConf() throws IOException, RuntimeException;
+}

+ 88 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/OperationEvaluator.java

@@ -0,0 +1,88 @@
+/**
+ * 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.algebra;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.shell.VariableRef;
+import org.apache.hadoop.hbase.shell.VariablesPool;
+import org.apache.hadoop.mapred.JobConf;
+
+/**
+ * Each algebra operation can be evaluated one of several different algorithms.
+ * 
+ * So, It should be query executor/optimizer later. And It will become the core
+ * module that regulates the query-performance of Hbase Shell.
+ * 
+ * @see <a
+ *      href="http://wiki.apache.org/lucene-hadoop/Hbase/HbaseShell/Executor">Intergrated
+ *      query executor architecture</a>
+ */
+public class OperationEvaluator {
+  private HBaseConfiguration conf;
+  Map<String, String> condition = new HashMap<String, String>();
+
+  /** Constructor */
+  public OperationEvaluator(HBaseConfiguration conf, String chainKey, String output) {
+    this.conf = conf;
+    String chain = chainKey;
+    String input = null;
+
+    while (chain != null) {
+      for (Map.Entry<String, VariableRef> e : VariablesPool.get(chain).entrySet()) {
+        if (e.getKey() == null) {
+          input = e.getValue().getArgument();
+        } else {
+          condition.put(e.getValue().getOperation(), e.getValue()
+              .getArgument());
+        }
+        chain = e.getKey();
+      }
+    }
+    condition.put(Constants.CONFIG_INPUT, input);
+    condition.put(Constants.CONFIG_OUTPUT, output);
+  }
+
+
+  /**
+   * Returns the job configuration object for statements type
+   * 
+   * @return JobConf
+   * @throws IOException
+   * @throws RuntimeException
+   */
+  public JobConf getJobConf() throws IOException, RuntimeException {
+    RelationalOperation operation;
+    if (condition.containsKey(Constants.RELATIONAL_SELECTION)) {
+      operation = new Selection(conf, condition);
+    } else if (condition.containsKey(Constants.RELATIONAL_PROJECTION)) {
+      operation = new Projection(conf, condition);
+    } else if (condition.containsKey(Constants.RELATIONAL_JOIN)) {
+      operation = new IndexJoin(conf, condition);
+    } else {
+      operation = new DuplicateTable(conf, condition);
+    }
+
+    return operation.getOperation().getConf();
+  }
+}

+ 54 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/Projection.java

@@ -0,0 +1,54 @@
+/**
+ * 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.algebra;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.mapred.GroupingTableMap;
+import org.apache.hadoop.hbase.mapred.IdentityTableMap;
+import org.apache.hadoop.hbase.mapred.IdentityTableReduce;
+import org.apache.hadoop.mapred.JobConf;
+
+/**
+ * Perform a relational projection using MapReduce.
+ */
+public class Projection extends RelationalOperation {
+  public Projection(HBaseConfiguration conf, Map<String, String> condition) {
+    super(conf, condition);
+  }
+
+  @Override
+  public JobConf getConf() throws IOException, RuntimeException {
+    outputTableCreate(null, condition.get(Constants.RELATIONAL_PROJECTION));
+
+    if (condition.containsKey(Constants.RELATIONAL_GROUP)) {
+      GroupingTableMap.initJob(input, getProjColumns(), getGroupColumns(),
+          GroupingTableMap.class, jobConf);
+    } else {
+      IdentityTableMap.initJob(input, getProjColumns(), IdentityTableMap.class,
+          jobConf);
+    }
+
+    IdentityTableReduce.initJob(output, IdentityTableReduce.class, jobConf);
+    return jobConf;
+  }
+}

+ 175 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/RelationalOperation.java

@@ -0,0 +1,175 @@
+/**
+ * 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.algebra;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.HBaseAdmin;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConnection;
+import org.apache.hadoop.hbase.HConnectionManager;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.ClusterStatus;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+
+/**
+ * Represents the interface to an relational algebra operation like projection,
+ * selection, join, group.
+ */
+public abstract class RelationalOperation implements Operation {
+  protected JobConf jobConf;
+  protected HConnection conn;
+  protected HBaseAdmin admin;
+  protected JobClient jobClient;
+  protected HTableDescriptor desc;
+  protected String input;
+  protected String output;
+  protected Map<String, String> condition;
+  protected HTableDescriptor[] tables;
+  protected Set<String> projSet = new HashSet<String>();
+
+  /**
+   * Constructor
+   * 
+   * @param conf
+   * @param statements
+   */
+  public RelationalOperation(HBaseConfiguration conf,
+      Map<String, String> statements) {
+    this.jobConf = new JobConf(conf);
+    this.conn = HConnectionManager.getConnection(conf);
+    this.condition = statements;
+    this.input = statements.get(Constants.CONFIG_INPUT);
+    this.output = statements.get(Constants.CONFIG_OUTPUT);
+    jobConf.setJobName("shell.mapred-" + +System.currentTimeMillis());
+    desc = new HTableDescriptor(output);
+
+    try {
+      this.admin = new HBaseAdmin(conf);
+      this.jobClient = new JobClient(jobConf);
+      tables = conn.listTables();
+
+      ClusterStatus cluster = jobClient.getClusterStatus();
+      jobConf.setNumMapTasks(cluster.getMapTasks());
+      jobConf.setNumReduceTasks(1);
+    } catch (Exception e) {
+      e.printStackTrace();
+    }
+  }
+
+  /**
+   * Gets the input table column descriptor[]
+   * 
+   * @return columns
+   */
+  public HColumnDescriptor[] getInputColumnDescriptor() {
+    HColumnDescriptor[] columns = null;
+    for (int i = 0; i < tables.length; i++) {
+      if (tables[i].getName().equals(new Text(input))) {
+        columns = tables[i].getFamilies().values().toArray(
+            new HColumnDescriptor[] {});
+        break;
+      }
+    }
+    return columns;
+  }
+
+  /**
+   * Convert HColumnDescriptor[] to String
+   * 
+   * @param columns
+   * @return columns string
+   */
+  public String getColumnStringArray(HColumnDescriptor[] columns) {
+    String result = "";
+    for (int i = 0; i < columns.length; i++) {
+      desc.addFamily(columns[i]);
+      result += columns[i].getName() + " ";
+    }
+    return result;
+  }
+
+  /**
+   * Creates the output table
+   * 
+   * @param columns
+   * @param columnString
+   * @throws IOException
+   */
+  public void outputTableCreate(HColumnDescriptor[] columns, String columnString)
+      throws IOException {
+    if (columnString == null) {
+      for (int i = 0; i < columns.length; i++) {
+        if (projSet.size() > 0) {
+          desc.addFamily(columns[i]);
+        } else {
+          if (projSet.contains(columns[i].getName().toString())) {
+            desc.addFamily(columns[i]);
+          }
+        }
+      }
+    } else {
+      String[] cols = columnString.split(" ");
+      for (int i = 0; i < cols.length; i++) {
+        desc.addFamily(new HColumnDescriptor(cols[i]));
+      }
+    }
+
+    admin.createTable(desc);
+  }
+
+  /**
+   * Return the jobConf
+   */
+  public JobConf getConf() throws IOException, RuntimeException {
+    return jobConf;
+  }
+
+  /**
+   * @return projection conditions
+   */
+  public String getProjColumns() {
+    return condition.get(Constants.RELATIONAL_PROJECTION);
+  }
+
+  /**
+   * @return selection conditions
+   */
+  public String getExpression() {
+    return condition.get(Constants.RELATIONAL_SELECTION);
+  }
+
+  /**
+   * @return group conditions
+   */
+  public String getGroupColumns() {
+    return condition.get(Constants.RELATIONAL_GROUP);
+  }
+
+  public Operation getOperation() {
+    return this;
+  }
+}

+ 74 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/Selection.java

@@ -0,0 +1,74 @@
+/**
+ * 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.algebra;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.mapred.IdentityTableReduce;
+import org.apache.hadoop.mapred.JobConf;
+
+/**
+ * Perform a relational selection by linear search algorithm on each Map
+ * Functions.
+ * 
+ * Scan each file block an test all records to see whether they satisfy the
+ * selection condition.
+ */
+public class Selection extends RelationalOperation {
+  public Selection(HBaseConfiguration conf, Map<String, String> condition) {
+    super(conf, condition);
+  }
+
+  @Override
+  public JobConf getConf() throws IOException, RuntimeException {
+    HColumnDescriptor[] columns = getInputColumnDescriptor();
+    String groupColumns = getGroupColumns(columns);
+    outputTableCreate(columns, null);
+
+    if (condition.containsKey(Constants.RELATIONAL_GROUP)) {
+      GroupingFilterMap.initJob(input, groupColumns, getGroupColumns(),
+          getExpression(), GroupingFilterMap.class, jobConf);
+    } else {
+      IdentityFilterMap.initJob(input, groupColumns, getExpression(),
+          IdentityFilterMap.class, jobConf);
+    }
+
+    IdentityTableReduce.initJob(output, IdentityTableReduce.class, jobConf);
+    return jobConf;
+  }
+
+  private String getGroupColumns(HColumnDescriptor[] columns) {
+    String result = null;
+    if (condition.containsKey(Constants.RELATIONAL_PROJECTION)) {
+      result = getProjColumns();
+    } else {
+      result = getColumnStringArray(columns);
+    }
+
+    String[] groups = result.split(" ");
+    for (int i = 0; i < groups.length; i++) {
+      projSet.add(groups[i]);
+    }
+    return result;
+  }
+}

+ 774 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/generated/ExpressionParser.java

@@ -0,0 +1,774 @@
+/* Generated By:JavaCC: Do not edit this line. ExpressionParser.java */
+package org.apache.hadoop.hbase.shell.algebra.generated;
+
+/**
+ * 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.
+ */
+
+import java.io.IOException;
+import java.io.Reader;
+import java.io.StringReader;
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HTable;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.MapWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Parsing command line.
+ */
+public class ExpressionParser implements ExpressionParserConstants {
+  HBaseConfiguration conf = new HBaseConfiguration();
+  private String expression;
+  private String joinKeyColumn;
+  private String secondRelation;
+  private Map<String, List<String>> unionSet = new HashMap<String, List<String>>();
+  private Map<String, List<String>> intersectionSet = new HashMap<String, List<String>>();
+  private SortedMap<Text, byte[]> secondValue = null;
+
+  public ExpressionParser(final String expression) {
+    this((Reader)(new StringReader(expression)));
+    this.expression = expression;
+  }
+
+  public String getExpression() {
+    return this.expression;
+  }
+
+  final public void booleanExpressionParse() throws ParseException {
+  Map<String, List<String>> temp = new HashMap<String, List<String>>();
+    temp = booleanTerm();
+    label_1:
+    while (true) {
+      switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+      case AND:
+      case OR:
+        ;
+        break;
+      default:
+        jj_la1[0] = jj_gen;
+        break label_1;
+      }
+      switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+      case AND:
+        jj_consume_token(AND);
+          if(temp != null) {
+            for(Map.Entry<String, List<String>> e : temp.entrySet()) {
+              List<String> newList = intersectionSet.get(e.getKey());
+              if(newList != null) {
+                newList.addAll(e.getValue());
+              } else {
+                newList = e.getValue();
+              }
+              intersectionSet.put(e.getKey(), newList);
+            }
+          }
+          temp = booleanTerm();
+          for(Map.Entry<String, List<String>> e : temp.entrySet()) {
+            List<String> newList = intersectionSet.get(e.getKey());
+            if(newList != null) {
+              newList.addAll(e.getValue());
+            } else {
+              newList = e.getValue();
+            }
+            intersectionSet.put(e.getKey(), newList);
+          }
+          temp = null;
+        break;
+      case OR:
+        jj_consume_token(OR);
+        if(temp != null) {
+          for(Map.Entry<String, List<String>> e : temp.entrySet()) {
+            List<String> newList = unionSet.get(e.getKey());
+            if(newList != null) {
+              newList.addAll(e.getValue());
+            } else {
+              newList = e.getValue();
+            }
+            unionSet.put(e.getKey(), newList);
+          }
+        }
+        temp = booleanTerm();
+        for(Map.Entry<String, List<String>> e : temp.entrySet()) {
+          List<String> newList = unionSet.get(e.getKey());
+          if(newList != null) {
+            newList.addAll(e.getValue());
+          } else {
+            newList = e.getValue();
+          }
+          unionSet.put(e.getKey(), newList);
+        }
+        temp = null;
+        break;
+      default:
+        jj_la1[1] = jj_gen;
+        jj_consume_token(-1);
+        throw new ParseException();
+      }
+    }
+    if(temp != null) {
+      for(Map.Entry<String, List<String>> e : temp.entrySet()) {
+        List<String> newList = unionSet.get(e.getKey());
+        if(newList != null) {
+          newList.addAll(e.getValue());
+        } else {
+          newList = e.getValue();
+        }
+        unionSet.put(e.getKey(), newList);
+      }
+    }
+  }
+
+  final public void joinExpressionParse() throws ParseException {
+  Map<String, List<String>> temp = new HashMap<String, List<String>>();
+    indexJoinCondition();
+    jj_consume_token(BOOL);
+    switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+    case ID:
+    case INTEGER_LITERAL:
+      temp = booleanTerm();
+      label_2:
+      while (true) {
+        switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+        case AND:
+        case OR:
+          ;
+          break;
+        default:
+          jj_la1[2] = jj_gen;
+          break label_2;
+        }
+        switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+        case AND:
+          jj_consume_token(AND);
+         if(temp != null) {
+           for(Map.Entry<String, List<String>> e : temp.entrySet()) {
+             List<String> newList = intersectionSet.get(e.getKey());
+             if(newList != null) {
+               newList.addAll(e.getValue());
+             } else {
+               newList = e.getValue();
+             }
+             intersectionSet.put(e.getKey(), newList);
+           }
+         }
+         temp = booleanTerm();
+         for(Map.Entry<String, List<String>> e : temp.entrySet()) {
+           List<String> newList = intersectionSet.get(e.getKey());
+           if(newList != null) {
+             newList.addAll(e.getValue());
+           } else {
+             newList = e.getValue();
+           }
+           intersectionSet.put(e.getKey(), newList);
+         }
+         temp = null;
+          break;
+        case OR:
+          jj_consume_token(OR);
+        if(temp != null) {
+          for(Map.Entry<String, List<String>> e : temp.entrySet()) {
+            List<String> newList = unionSet.get(e.getKey());
+            if(newList != null) {
+              newList.addAll(e.getValue());
+            } else {
+              newList = e.getValue();
+            }
+            unionSet.put(e.getKey(), newList);
+          }
+        }
+        temp = booleanTerm();
+        for(Map.Entry<String, List<String>> e : temp.entrySet()) {
+          List<String> newList = unionSet.get(e.getKey());
+          if(newList != null) {
+            newList.addAll(e.getValue());
+          } else {
+            newList = e.getValue();
+          }
+          unionSet.put(e.getKey(), newList);
+        }
+        temp = null;
+          break;
+        default:
+          jj_la1[3] = jj_gen;
+          jj_consume_token(-1);
+          throw new ParseException();
+        }
+      }
+      if(temp != null) {
+        for(Map.Entry<String, List<String>> e : temp.entrySet()) {
+          List<String> newList = unionSet.get(e.getKey());
+          if(newList != null) {
+            newList.addAll(e.getValue());
+          } else {
+            newList = e.getValue();
+          }
+          unionSet.put(e.getKey(), newList);
+        }
+        }
+      break;
+    default:
+      jj_la1[4] = jj_gen;
+      ;
+    }
+  }
+
+  final public void indexJoinCondition() throws ParseException {
+ Token firstR = null;
+  Token joinKey = null;
+  Token secondR = null;
+    firstR = jj_consume_token(ID);
+    jj_consume_token(DOT);
+    joinKey = jj_consume_token(ID);
+    jj_consume_token(EQUALS);
+    secondR = jj_consume_token(ID);
+    jj_consume_token(DOT);
+    jj_consume_token(ROW);
+    joinKeyColumn = joinKey.image.toString();
+    secondRelation = secondR.image.toString();
+  }
+
+  final public Map<String, List<String>> booleanTerm() throws ParseException {
+  Token tSearchName = null;
+  Token tComparator = null;
+  Token tFirst = null;
+  Token tSecond = null;
+  Map<String, List<String>> result = new HashMap<String, List<String>>();
+  List<String> valueList = new ArrayList<String>();
+  String comparand = null;
+    switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+    case ID:
+      tSearchName = jj_consume_token(ID);
+      break;
+    case INTEGER_LITERAL:
+      tSearchName = jj_consume_token(INTEGER_LITERAL);
+      break;
+    default:
+      jj_la1[5] = jj_gen;
+      jj_consume_token(-1);
+      throw new ParseException();
+    }
+    switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+    case DOT:
+      jj_consume_token(DOT);
+      switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+      case ID:
+        tFirst = jj_consume_token(ID);
+        break;
+      case INTEGER_LITERAL:
+        tFirst = jj_consume_token(INTEGER_LITERAL);
+        break;
+      default:
+        jj_la1[6] = jj_gen;
+        jj_consume_token(-1);
+        throw new ParseException();
+      }
+      break;
+    default:
+      jj_la1[7] = jj_gen;
+      ;
+    }
+    switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+    case EQUALS:
+      tComparator = jj_consume_token(EQUALS);
+      break;
+    case LCOMP:
+      tComparator = jj_consume_token(LCOMP);
+      break;
+    case LCOMPE:
+      tComparator = jj_consume_token(LCOMPE);
+      break;
+    case RCOMPE:
+      tComparator = jj_consume_token(RCOMPE);
+      break;
+    case RCOMP:
+      tComparator = jj_consume_token(RCOMP);
+      break;
+    case IN:
+      tComparator = jj_consume_token(IN);
+      break;
+    case NOTIN:
+      tComparator = jj_consume_token(NOTIN);
+      break;
+    default:
+      jj_la1[8] = jj_gen;
+      jj_consume_token(-1);
+      throw new ParseException();
+    }
+    comparand = getValueList();
+    switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+    case DOT:
+      jj_consume_token(DOT);
+      switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+      case ID:
+        tSecond = jj_consume_token(ID);
+        break;
+      case INTEGER_LITERAL:
+        tSecond = jj_consume_token(INTEGER_LITERAL);
+        break;
+      default:
+        jj_la1[9] = jj_gen;
+        jj_consume_token(-1);
+        throw new ParseException();
+      }
+      break;
+    default:
+      jj_la1[10] = jj_gen;
+      ;
+    }
+    if(tFirst == null && tSecond == null) {
+      valueList.add(tComparator.image.toString() + " " +comparand);
+      result.put(tSearchName.image.toString(), valueList);
+    } else if (tFirst != null && tSecond != null ){
+      if(tSearchName.image.toString().equals(secondRelation)) {
+        valueList.add(tComparator.image.toString() + " "
+            + secondValue.get(new Text(tSearchName.image.toString())));
+        result.put(tFirst.image.toString(), valueList);
+      } else {
+        valueList.add(tComparator.image.toString() + " " + tSecond.image.toString());
+        result.put(tFirst.image.toString(), valueList);
+      }
+    }
+    {if (true) return result;}
+    throw new Error("Missing return statement in function");
+  }
+
+  final public String getValueList() throws ParseException {
+  Token tComparand = null;
+  Token tList = null;
+  String result = "";
+    switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+    case ID:
+      tComparand = jj_consume_token(ID);
+      break;
+    case INTEGER_LITERAL:
+      tComparand = jj_consume_token(INTEGER_LITERAL);
+      break;
+    default:
+      jj_la1[11] = jj_gen;
+      jj_consume_token(-1);
+      throw new ParseException();
+    }
+    result = tComparand.image.toString();
+    switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+    case BAR:
+      jj_consume_token(BAR);
+      label_3:
+      while (true) {
+        switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+        case ID:
+        case INTEGER_LITERAL:
+          ;
+          break;
+        default:
+          jj_la1[12] = jj_gen;
+          break label_3;
+        }
+        switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+        case ID:
+          tList = jj_consume_token(ID);
+          break;
+        case INTEGER_LITERAL:
+          tList = jj_consume_token(INTEGER_LITERAL);
+          break;
+        default:
+          jj_la1[13] = jj_gen;
+          jj_consume_token(-1);
+          throw new ParseException();
+        }
+        result += "|" + tList.image.toString();
+        switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+        case BAR:
+          jj_consume_token(BAR);
+          break;
+        default:
+          jj_la1[14] = jj_gen;
+          ;
+        }
+      }
+      break;
+    default:
+      jj_la1[15] = jj_gen;
+      ;
+    }
+    {if (true) return result;}
+    throw new Error("Missing return statement in function");
+  }
+
+  final public boolean checkConstraints(MapWritable data) throws ParseException {
+  Map<String, String> record = getComparisonObject(data);
+  boolean result = false;
+  if(intersectionSet.size() == 0 && unionSet.size() == 0) {
+    return true;
+  }
+  if (data == null || record.size() == 0) {
+    return result; // return false if data is null.
+  }
+      if (intersectionSet.size() == record.size()) {
+        result = booleanCompare(intersectionSet, record, true);
+      } else if (unionSet.size() == record.size()) {
+        result = booleanCompare(unionSet, record, false);
+      } else {
+        result = getCompareResult(record);
+      }
+    {if (true) return result;}
+    throw new Error("Missing return statement in function");
+  }
+
+  final public Map<String, String> getComparisonObject(MapWritable data) throws ParseException {
+  Map<String, String> result = new HashMap<String, String>();
+    for (Map.Entry<Writable, Writable> e : data.entrySet()) {
+      String cKey = e.getKey().toString();
+      String val = new String(((ImmutableBytesWritable) e.getValue()).get());
+
+      if (intersectionSet.containsKey(cKey) || unionSet.containsKey(cKey)) {
+        result.put(cKey, val);
+      }
+    }
+
+    {if (true) return result;}
+    throw new Error("Missing return statement in function");
+  }
+
+  final public MapWritable getJoinColumns(MapWritable value, int numCols, String secondRelation) throws ParseException {
+  MapWritable appendValue = new MapWritable();
+  String joinKey = null;
+  this.secondRelation = secondRelation;
+    if (numCols > 0) {
+      for (Map.Entry<Writable, Writable> e : value.entrySet()) {
+        Text column = (Text) e.getKey();
+        for (int i = 0; i < numCols; i++) {
+          if (column.equals(new Text(joinKeyColumn))) {
+            joinKey = new String(((ImmutableBytesWritable) e.getValue()).get());
+            break;
+          }
+        }
+      }
+    }
+
+    //If joinKey is null, just return.
+    if(joinKey == null) {
+      {if (true) return appendValue;}
+    }
+
+    try {
+      HTable table = new HTable(conf, new Text(secondRelation));
+      secondValue = table.getRow(new Text(joinKey));
+      for (Map.Entry<Text, byte[]> e : secondValue.entrySet()) {
+        appendValue.put(e.getKey(), new ImmutableBytesWritable(e.getValue()));
+      }
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+
+    {if (true) return appendValue;}
+    throw new Error("Missing return statement in function");
+  }
+
+  final public boolean getCompareResult(Map<String, String> record) throws ParseException {
+      {if (true) return (booleanCompare(intersectionSet, record, true) &&
+          booleanCompare(unionSet, record, false)) ? true : false;}
+    throw new Error("Missing return statement in function");
+  }
+
+  final public boolean booleanCompare(Map<String, List<String>> expression,
+    Map<String, String> record, boolean isIntersection) throws ParseException {
+  boolean negative = true;
+  boolean positive = false;
+    try{
+      for (Map.Entry<String, List<String>> e : expression.entrySet()) {
+        String key = e.getKey();
+        List<String> valueList = e.getValue();
+        String recordValueList = record.get(key);
+
+        for (int i = 0; i < valueList.size(); i++) {
+          String[] term = valueList.get(i).split(" ");
+          String comparator = term[0];
+          String comparand = term[1];
+
+          switch (comparator.charAt(0)) {
+            case '>':
+              if (isSecond(comparator, "=")) {
+                if (Integer.parseInt(comparand) > Integer
+                    .parseInt(recordValueList)) {
+                  negative = false;
+                } else {
+                  positive = true;
+                }
+              } else {
+                if (Integer.parseInt(comparand) > Integer
+                    .parseInt(recordValueList)
+                    || comparand.equals(recordValueList)) {
+                  negative = false;
+                } else {
+                  positive = true;
+                }
+              }
+              break;
+            case '<':
+              if (isSecond(comparator, "=")) {
+                if (Integer.parseInt(comparand) < Integer
+                    .parseInt(recordValueList))
+                  negative = false;
+                else
+                  positive = true;
+              } else {
+                if (Integer.parseInt(comparand) < Integer
+                    .parseInt(recordValueList)
+                    || comparand.equals(recordValueList))
+                  negative = false;
+                else
+                  positive = true;
+              }
+              break;
+
+            case '!':
+              if (isSecond(comparator, "!")) {
+                boolean checkBool = true;
+                String[] coms = comparand.split("[|]");
+                for (int j = 0; j < coms.length; j++) {
+                  if (URLDecoder.decode(coms[j], "UTF-8").equals(recordValueList)) {
+                    checkBool = false;
+                  }
+                }
+
+                if (!checkBool) {
+                  negative = false;
+                } else {
+                  positive = true;
+                }
+
+              } else {
+                if (comparand.equals(recordValueList))
+                  negative = false;
+                else
+                  positive = true;
+              }
+              break;
+            case '=':
+              if (isSecond(comparator, "=")) {
+
+                boolean checkBool = true;
+                String[] coms = comparand.split("[|]");
+                for (int j = 0; j < coms.length; j++) {
+                  if (URLDecoder.decode(coms[j], "UTF-8").equals(recordValueList)) {
+                    checkBool = false;
+                  }
+                }
+
+                if (checkBool) {
+                  negative = false;
+                } else {
+                  positive = true;
+                }
+
+              } else {
+                if (!comparand.equals(recordValueList))
+                  negative = false;
+                else
+                  positive = true;
+              }
+              break;
+          }
+        }
+      }
+    } catch (UnsupportedEncodingException e) {
+      e.printStackTrace();
+    }
+
+    boolean result = false;
+    if (isIntersection) {
+      result = negative;
+    } else {
+      result = positive;
+    }
+
+    {if (true) return result;}
+    throw new Error("Missing return statement in function");
+  }
+
+  final public boolean isSecond(String comparator, String string) throws ParseException {
+    {if (true) return (comparator.length() == 2 && string.charAt(0) == comparator.charAt(1))
+    ? true : false;}
+    throw new Error("Missing return statement in function");
+  }
+
+  public ExpressionParserTokenManager token_source;
+  SimpleCharStream jj_input_stream;
+  public Token token, jj_nt;
+  private int jj_ntk;
+  private int jj_gen;
+  final private int[] jj_la1 = new int[16];
+  static private int[] jj_la1_0;
+  static {
+      jj_la1_0();
+   }
+   private static void jj_la1_0() {
+      jj_la1_0 = new int[] {0xc0,0xc0,0xc0,0xc0,0xc0000,0xc0000,0xc0000,0x100,0xfe00,0xc0000,0x100,0xc0000,0xc0000,0xc0000,0x10000,0x10000,};
+   }
+
+  public ExpressionParser(java.io.InputStream stream) {
+     this(stream, null);
+  }
+  public ExpressionParser(java.io.InputStream stream, String encoding) {
+    try { jj_input_stream = new SimpleCharStream(stream, encoding, 1, 1); } catch(java.io.UnsupportedEncodingException e) { throw new RuntimeException(e); }
+    token_source = new ExpressionParserTokenManager(jj_input_stream);
+    token = new Token();
+    jj_ntk = -1;
+    jj_gen = 0;
+    for (int i = 0; i < 16; i++) jj_la1[i] = -1;
+  }
+
+  public void ReInit(java.io.InputStream stream) {
+     ReInit(stream, null);
+  }
+  public void ReInit(java.io.InputStream stream, String encoding) {
+    try { jj_input_stream.ReInit(stream, encoding, 1, 1); } catch(java.io.UnsupportedEncodingException e) { throw new RuntimeException(e); }
+    token_source.ReInit(jj_input_stream);
+    token = new Token();
+    jj_ntk = -1;
+    jj_gen = 0;
+    for (int i = 0; i < 16; i++) jj_la1[i] = -1;
+  }
+
+  public ExpressionParser(java.io.Reader stream) {
+    jj_input_stream = new SimpleCharStream(stream, 1, 1);
+    token_source = new ExpressionParserTokenManager(jj_input_stream);
+    token = new Token();
+    jj_ntk = -1;
+    jj_gen = 0;
+    for (int i = 0; i < 16; i++) jj_la1[i] = -1;
+  }
+
+  public void ReInit(java.io.Reader stream) {
+    jj_input_stream.ReInit(stream, 1, 1);
+    token_source.ReInit(jj_input_stream);
+    token = new Token();
+    jj_ntk = -1;
+    jj_gen = 0;
+    for (int i = 0; i < 16; i++) jj_la1[i] = -1;
+  }
+
+  public ExpressionParser(ExpressionParserTokenManager tm) {
+    token_source = tm;
+    token = new Token();
+    jj_ntk = -1;
+    jj_gen = 0;
+    for (int i = 0; i < 16; i++) jj_la1[i] = -1;
+  }
+
+  public void ReInit(ExpressionParserTokenManager tm) {
+    token_source = tm;
+    token = new Token();
+    jj_ntk = -1;
+    jj_gen = 0;
+    for (int i = 0; i < 16; i++) jj_la1[i] = -1;
+  }
+
+  final private Token jj_consume_token(int kind) throws ParseException {
+    Token oldToken;
+    if ((oldToken = token).next != null) token = token.next;
+    else token = token.next = token_source.getNextToken();
+    jj_ntk = -1;
+    if (token.kind == kind) {
+      jj_gen++;
+      return token;
+    }
+    token = oldToken;
+    jj_kind = kind;
+    throw generateParseException();
+  }
+
+  final public Token getNextToken() {
+    if (token.next != null) token = token.next;
+    else token = token.next = token_source.getNextToken();
+    jj_ntk = -1;
+    jj_gen++;
+    return token;
+  }
+
+  final public Token getToken(int index) {
+    Token t = token;
+    for (int i = 0; i < index; i++) {
+      if (t.next != null) t = t.next;
+      else t = t.next = token_source.getNextToken();
+    }
+    return t;
+  }
+
+  final private int jj_ntk() {
+    if ((jj_nt=token.next) == null)
+      return (jj_ntk = (token.next=token_source.getNextToken()).kind);
+    else
+      return (jj_ntk = jj_nt.kind);
+  }
+
+  private java.util.Vector jj_expentries = new java.util.Vector();
+  private int[] jj_expentry;
+  private int jj_kind = -1;
+
+  public ParseException generateParseException() {
+    jj_expentries.removeAllElements();
+    boolean[] la1tokens = new boolean[24];
+    for (int i = 0; i < 24; i++) {
+      la1tokens[i] = false;
+    }
+    if (jj_kind >= 0) {
+      la1tokens[jj_kind] = true;
+      jj_kind = -1;
+    }
+    for (int i = 0; i < 16; i++) {
+      if (jj_la1[i] == jj_gen) {
+        for (int j = 0; j < 32; j++) {
+          if ((jj_la1_0[i] & (1<<j)) != 0) {
+            la1tokens[j] = true;
+          }
+        }
+      }
+    }
+    for (int i = 0; i < 24; i++) {
+      if (la1tokens[i]) {
+        jj_expentry = new int[1];
+        jj_expentry[0] = i;
+        jj_expentries.addElement(jj_expentry);
+      }
+    }
+    int[][] exptokseq = new int[jj_expentries.size()][];
+    for (int i = 0; i < jj_expentries.size(); i++) {
+      exptokseq[i] = (int[])jj_expentries.elementAt(i);
+    }
+    return new ParseException(token, exptokseq, tokenImage);
+  }
+
+  final public void enable_tracing() {
+  }
+
+  final public void disable_tracing() {
+  }
+
+}

+ 56 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/generated/ExpressionParserConstants.java

@@ -0,0 +1,56 @@
+/* Generated By:JavaCC: Do not edit this line. ExpressionParserConstants.java */
+package org.apache.hadoop.hbase.shell.algebra.generated;
+
+public interface ExpressionParserConstants {
+
+  int EOF = 0;
+  int ROW = 5;
+  int AND = 6;
+  int OR = 7;
+  int DOT = 8;
+  int EQUALS = 9;
+  int LCOMP = 10;
+  int RCOMP = 11;
+  int LCOMPE = 12;
+  int RCOMPE = 13;
+  int IN = 14;
+  int NOTIN = 15;
+  int BAR = 16;
+  int BOOL = 17;
+  int ID = 18;
+  int INTEGER_LITERAL = 19;
+  int FLOATING_POINT_LITERAL = 20;
+  int EXPONENT = 21;
+  int QUOTED_IDENTIFIER = 22;
+  int STRING_LITERAL = 23;
+
+  int DEFAULT = 0;
+
+  String[] tokenImage = {
+    "<EOF>",
+    "\" \"",
+    "\"\\t\"",
+    "\"\\r\"",
+    "\"\\n\"",
+    "\"row\"",
+    "\"and\"",
+    "\"or\"",
+    "\".\"",
+    "\"=\"",
+    "\">\"",
+    "\"<\"",
+    "\">=\"",
+    "\"<=\"",
+    "\"==\"",
+    "\"!!\"",
+    "\"|\"",
+    "\"bool\"",
+    "<ID>",
+    "<INTEGER_LITERAL>",
+    "<FLOATING_POINT_LITERAL>",
+    "<EXPONENT>",
+    "<QUOTED_IDENTIFIER>",
+    "<STRING_LITERAL>",
+  };
+
+}

+ 659 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/generated/ExpressionParserTokenManager.java

@@ -0,0 +1,659 @@
+/* Generated By:JavaCC: Do not edit this line. ExpressionParserTokenManager.java */
+package org.apache.hadoop.hbase.shell.algebra.generated;
+/**
+ * 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.
+ */
+
+import java.io.IOException;
+import java.io.Reader;
+import java.io.StringReader;
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HTable;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.MapWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+
+public class ExpressionParserTokenManager implements ExpressionParserConstants
+{
+  public  java.io.PrintStream debugStream = System.out;
+  public  void setDebugStream(java.io.PrintStream ds) { debugStream = ds; }
+private final int jjStopStringLiteralDfa_0(int pos, long active0)
+{
+   switch (pos)
+   {
+      case 0:
+         if ((active0 & 0x200e0L) != 0L)
+         {
+            jjmatchedKind = 18;
+            return 1;
+         }
+         if ((active0 & 0x100L) != 0L)
+            return 3;
+         return -1;
+      case 1:
+         if ((active0 & 0x20060L) != 0L)
+         {
+            jjmatchedKind = 18;
+            jjmatchedPos = 1;
+            return 1;
+         }
+         if ((active0 & 0x80L) != 0L)
+            return 1;
+         return -1;
+      case 2:
+         if ((active0 & 0x20000L) != 0L)
+         {
+            jjmatchedKind = 18;
+            jjmatchedPos = 2;
+            return 1;
+         }
+         if ((active0 & 0x60L) != 0L)
+            return 1;
+         return -1;
+      default :
+         return -1;
+   }
+}
+private final int jjStartNfa_0(int pos, long active0)
+{
+   return jjMoveNfa_0(jjStopStringLiteralDfa_0(pos, active0), pos + 1);
+}
+private final int jjStopAtPos(int pos, int kind)
+{
+   jjmatchedKind = kind;
+   jjmatchedPos = pos;
+   return pos + 1;
+}
+private final int jjStartNfaWithStates_0(int pos, int kind, int state)
+{
+   jjmatchedKind = kind;
+   jjmatchedPos = pos;
+   try { curChar = input_stream.readChar(); }
+   catch(java.io.IOException e) { return pos + 1; }
+   return jjMoveNfa_0(state, pos + 1);
+}
+private final int jjMoveStringLiteralDfa0_0()
+{
+   switch(curChar)
+   {
+      case 33:
+         return jjMoveStringLiteralDfa1_0(0x8000L);
+      case 46:
+         return jjStartNfaWithStates_0(0, 8, 3);
+      case 60:
+         jjmatchedKind = 11;
+         return jjMoveStringLiteralDfa1_0(0x2000L);
+      case 61:
+         jjmatchedKind = 9;
+         return jjMoveStringLiteralDfa1_0(0x4000L);
+      case 62:
+         jjmatchedKind = 10;
+         return jjMoveStringLiteralDfa1_0(0x1000L);
+      case 65:
+      case 97:
+         return jjMoveStringLiteralDfa1_0(0x40L);
+      case 66:
+      case 98:
+         return jjMoveStringLiteralDfa1_0(0x20000L);
+      case 79:
+      case 111:
+         return jjMoveStringLiteralDfa1_0(0x80L);
+      case 82:
+      case 114:
+         return jjMoveStringLiteralDfa1_0(0x20L);
+      case 124:
+         return jjStopAtPos(0, 16);
+      default :
+         return jjMoveNfa_0(0, 0);
+   }
+}
+private final int jjMoveStringLiteralDfa1_0(long active0)
+{
+   try { curChar = input_stream.readChar(); }
+   catch(java.io.IOException e) {
+      jjStopStringLiteralDfa_0(0, active0);
+      return 1;
+   }
+   switch(curChar)
+   {
+      case 33:
+         if ((active0 & 0x8000L) != 0L)
+            return jjStopAtPos(1, 15);
+         break;
+      case 61:
+         if ((active0 & 0x1000L) != 0L)
+            return jjStopAtPos(1, 12);
+         else if ((active0 & 0x2000L) != 0L)
+            return jjStopAtPos(1, 13);
+         else if ((active0 & 0x4000L) != 0L)
+            return jjStopAtPos(1, 14);
+         break;
+      case 78:
+      case 110:
+         return jjMoveStringLiteralDfa2_0(active0, 0x40L);
+      case 79:
+      case 111:
+         return jjMoveStringLiteralDfa2_0(active0, 0x20020L);
+      case 82:
+      case 114:
+         if ((active0 & 0x80L) != 0L)
+            return jjStartNfaWithStates_0(1, 7, 1);
+         break;
+      default :
+         break;
+   }
+   return jjStartNfa_0(0, active0);
+}
+private final int jjMoveStringLiteralDfa2_0(long old0, long active0)
+{
+   if (((active0 &= old0)) == 0L)
+      return jjStartNfa_0(0, old0); 
+   try { curChar = input_stream.readChar(); }
+   catch(java.io.IOException e) {
+      jjStopStringLiteralDfa_0(1, active0);
+      return 2;
+   }
+   switch(curChar)
+   {
+      case 68:
+      case 100:
+         if ((active0 & 0x40L) != 0L)
+            return jjStartNfaWithStates_0(2, 6, 1);
+         break;
+      case 79:
+      case 111:
+         return jjMoveStringLiteralDfa3_0(active0, 0x20000L);
+      case 87:
+      case 119:
+         if ((active0 & 0x20L) != 0L)
+            return jjStartNfaWithStates_0(2, 5, 1);
+         break;
+      default :
+         break;
+   }
+   return jjStartNfa_0(1, active0);
+}
+private final int jjMoveStringLiteralDfa3_0(long old0, long active0)
+{
+   if (((active0 &= old0)) == 0L)
+      return jjStartNfa_0(1, old0); 
+   try { curChar = input_stream.readChar(); }
+   catch(java.io.IOException e) {
+      jjStopStringLiteralDfa_0(2, active0);
+      return 3;
+   }
+   switch(curChar)
+   {
+      case 76:
+      case 108:
+         if ((active0 & 0x20000L) != 0L)
+            return jjStartNfaWithStates_0(3, 17, 1);
+         break;
+      default :
+         break;
+   }
+   return jjStartNfa_0(2, active0);
+}
+private final void jjCheckNAdd(int state)
+{
+   if (jjrounds[state] != jjround)
+   {
+      jjstateSet[jjnewStateCnt++] = state;
+      jjrounds[state] = jjround;
+   }
+}
+private final void jjAddStates(int start, int end)
+{
+   do {
+      jjstateSet[jjnewStateCnt++] = jjnextStates[start];
+   } while (start++ != end);
+}
+private final void jjCheckNAddTwoStates(int state1, int state2)
+{
+   jjCheckNAdd(state1);
+   jjCheckNAdd(state2);
+}
+private final void jjCheckNAddStates(int start, int end)
+{
+   do {
+      jjCheckNAdd(jjnextStates[start]);
+   } while (start++ != end);
+}
+private final void jjCheckNAddStates(int start)
+{
+   jjCheckNAdd(jjnextStates[start]);
+   jjCheckNAdd(jjnextStates[start + 1]);
+}
+static final long[] jjbitVec0 = {
+   0x0L, 0x0L, 0xffffffffffffffffL, 0xffffffffffffffffL
+};
+private final int jjMoveNfa_0(int startState, int curPos)
+{
+   int[] nextStates;
+   int startsAt = 0;
+   jjnewStateCnt = 32;
+   int i = 1;
+   jjstateSet[0] = startState;
+   int j, kind = 0x7fffffff;
+   for (;;)
+   {
+      if (++jjround == 0x7fffffff)
+         ReInitRounds();
+      if (curChar < 64)
+      {
+         long l = 1L << curChar;
+         MatchLoop: do
+         {
+            switch(jjstateSet[--i])
+            {
+               case 0:
+                  if ((0x3ff000000000000L & l) != 0L)
+                  {
+                     if (kind > 19)
+                        kind = 19;
+                     jjCheckNAddStates(0, 6);
+                  }
+                  else if ((0x400a00000000000L & l) != 0L)
+                  {
+                     if (kind > 18)
+                        kind = 18;
+                     jjCheckNAdd(1);
+                  }
+                  else if (curChar == 39)
+                     jjCheckNAddStates(7, 9);
+                  else if (curChar == 34)
+                     jjCheckNAdd(8);
+                  else if (curChar == 46)
+                     jjCheckNAdd(3);
+                  break;
+               case 1:
+                  if ((0x7ffa00000000000L & l) == 0L)
+                     break;
+                  if (kind > 18)
+                     kind = 18;
+                  jjCheckNAdd(1);
+                  break;
+               case 2:
+                  if (curChar == 46)
+                     jjCheckNAdd(3);
+                  break;
+               case 3:
+                  if ((0x3ff000000000000L & l) == 0L)
+                     break;
+                  if (kind > 20)
+                     kind = 20;
+                  jjCheckNAddTwoStates(3, 4);
+                  break;
+               case 5:
+                  if ((0x280000000000L & l) != 0L)
+                     jjCheckNAdd(6);
+                  break;
+               case 6:
+                  if ((0x3ff000000000000L & l) == 0L)
+                     break;
+                  if (kind > 20)
+                     kind = 20;
+                  jjCheckNAdd(6);
+                  break;
+               case 7:
+                  if (curChar == 34)
+                     jjCheckNAdd(8);
+                  break;
+               case 8:
+                  if ((0xfffffffbffffffffL & l) != 0L)
+                     jjCheckNAddTwoStates(8, 9);
+                  break;
+               case 9:
+                  if (curChar == 34 && kind > 22)
+                     kind = 22;
+                  break;
+               case 10:
+                  if (curChar == 39)
+                     jjCheckNAddStates(7, 9);
+                  break;
+               case 11:
+                  if ((0xffffff7fffffffffL & l) != 0L)
+                     jjCheckNAddStates(7, 9);
+                  break;
+               case 12:
+                  if (curChar == 39)
+                     jjCheckNAddStates(10, 12);
+                  break;
+               case 13:
+                  if (curChar == 39)
+                     jjstateSet[jjnewStateCnt++] = 12;
+                  break;
+               case 14:
+                  if ((0xffffff7fffffffffL & l) != 0L)
+                     jjCheckNAddStates(10, 12);
+                  break;
+               case 15:
+                  if (curChar == 39 && kind > 23)
+                     kind = 23;
+                  break;
+               case 16:
+                  if ((0x3ff000000000000L & l) == 0L)
+                     break;
+                  if (kind > 19)
+                     kind = 19;
+                  jjCheckNAddStates(0, 6);
+                  break;
+               case 17:
+                  if ((0x3ff000000000000L & l) == 0L)
+                     break;
+                  if (kind > 19)
+                     kind = 19;
+                  jjCheckNAdd(17);
+                  break;
+               case 18:
+                  if ((0x3ff000000000000L & l) != 0L)
+                     jjCheckNAddTwoStates(18, 19);
+                  break;
+               case 19:
+                  if (curChar == 46)
+                     jjCheckNAdd(20);
+                  break;
+               case 20:
+                  if ((0x3ff000000000000L & l) == 0L)
+                     break;
+                  if (kind > 20)
+                     kind = 20;
+                  jjCheckNAddTwoStates(20, 21);
+                  break;
+               case 22:
+                  if ((0x280000000000L & l) != 0L)
+                     jjCheckNAdd(23);
+                  break;
+               case 23:
+                  if ((0x3ff000000000000L & l) == 0L)
+                     break;
+                  if (kind > 20)
+                     kind = 20;
+                  jjCheckNAdd(23);
+                  break;
+               case 24:
+                  if ((0x3ff000000000000L & l) != 0L)
+                     jjCheckNAddTwoStates(24, 25);
+                  break;
+               case 26:
+                  if ((0x280000000000L & l) != 0L)
+                     jjCheckNAdd(27);
+                  break;
+               case 27:
+                  if ((0x3ff000000000000L & l) == 0L)
+                     break;
+                  if (kind > 20)
+                     kind = 20;
+                  jjCheckNAdd(27);
+                  break;
+               case 28:
+                  if ((0x3ff000000000000L & l) == 0L)
+                     break;
+                  if (kind > 20)
+                     kind = 20;
+                  jjCheckNAddTwoStates(28, 29);
+                  break;
+               case 30:
+                  if ((0x280000000000L & l) != 0L)
+                     jjCheckNAdd(31);
+                  break;
+               case 31:
+                  if ((0x3ff000000000000L & l) == 0L)
+                     break;
+                  if (kind > 20)
+                     kind = 20;
+                  jjCheckNAdd(31);
+                  break;
+               default : break;
+            }
+         } while(i != startsAt);
+      }
+      else if (curChar < 128)
+      {
+         long l = 1L << (curChar & 077);
+         MatchLoop: do
+         {
+            switch(jjstateSet[--i])
+            {
+               case 0:
+               case 1:
+                  if ((0x7fffffe87fffffeL & l) == 0L)
+                     break;
+                  if (kind > 18)
+                     kind = 18;
+                  jjCheckNAdd(1);
+                  break;
+               case 4:
+                  if ((0x2000000020L & l) != 0L)
+                     jjAddStates(13, 14);
+                  break;
+               case 8:
+                  jjAddStates(15, 16);
+                  break;
+               case 11:
+                  jjCheckNAddStates(7, 9);
+                  break;
+               case 14:
+                  jjCheckNAddStates(10, 12);
+                  break;
+               case 21:
+                  if ((0x2000000020L & l) != 0L)
+                     jjAddStates(17, 18);
+                  break;
+               case 25:
+                  if ((0x2000000020L & l) != 0L)
+                     jjAddStates(19, 20);
+                  break;
+               case 29:
+                  if ((0x2000000020L & l) != 0L)
+                     jjAddStates(21, 22);
+                  break;
+               default : break;
+            }
+         } while(i != startsAt);
+      }
+      else
+      {
+         int i2 = (curChar & 0xff) >> 6;
+         long l2 = 1L << (curChar & 077);
+         MatchLoop: do
+         {
+            switch(jjstateSet[--i])
+            {
+               case 8:
+                  if ((jjbitVec0[i2] & l2) != 0L)
+                     jjAddStates(15, 16);
+                  break;
+               case 11:
+                  if ((jjbitVec0[i2] & l2) != 0L)
+                     jjCheckNAddStates(7, 9);
+                  break;
+               case 14:
+                  if ((jjbitVec0[i2] & l2) != 0L)
+                     jjCheckNAddStates(10, 12);
+                  break;
+               default : break;
+            }
+         } while(i != startsAt);
+      }
+      if (kind != 0x7fffffff)
+      {
+         jjmatchedKind = kind;
+         jjmatchedPos = curPos;
+         kind = 0x7fffffff;
+      }
+      ++curPos;
+      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 = {
+   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, "\56", "\75", "\76", "\74", 
+"\76\75", "\74\75", "\75\75", "\41\41", "\174", null, null, null, null, null, null, null, };
+public static final String[] lexStateNames = {
+   "DEFAULT", 
+};
+static final long[] jjtoToken = {
+   0xdfffe1L, 
+};
+static final long[] jjtoSkip = {
+   0x1eL, 
+};
+protected SimpleCharStream input_stream;
+private final int[] jjrounds = new int[32];
+private final int[] jjstateSet = new int[64];
+protected char curChar;
+public ExpressionParserTokenManager(SimpleCharStream stream){
+   if (SimpleCharStream.staticFlag)
+      throw new Error("ERROR: Cannot use a static CharStream class with a non-static lexical analyzer.");
+   input_stream = stream;
+}
+public ExpressionParserTokenManager(SimpleCharStream stream, int lexState){
+   this(stream);
+   SwitchTo(lexState);
+}
+public void ReInit(SimpleCharStream stream)
+{
+   jjmatchedPos = jjnewStateCnt = 0;
+   curLexState = defaultLexState;
+   input_stream = stream;
+   ReInitRounds();
+}
+private final void ReInitRounds()
+{
+   int i;
+   jjround = 0x80000001;
+   for (i = 32; i-- > 0;)
+      jjrounds[i] = 0x80000000;
+}
+public void ReInit(SimpleCharStream stream, int lexState)
+{
+   ReInit(stream);
+   SwitchTo(lexState);
+}
+public void SwitchTo(int lexState)
+{
+   if (lexState >= 1 || lexState < 0)
+      throw new TokenMgrError("Error: Ignoring invalid lexical state : " + lexState + ". State unchanged.", TokenMgrError.INVALID_LEXICAL_STATE);
+   else
+      curLexState = lexState;
+}
+
+protected Token jjFillToken()
+{
+   Token t = Token.newToken(jjmatchedKind);
+   t.kind = jjmatchedKind;
+   String im = jjstrLiteralImages[jjmatchedKind];
+   t.image = (im == null) ? input_stream.GetImage() : im;
+   t.beginLine = input_stream.getBeginLine();
+   t.beginColumn = input_stream.getBeginColumn();
+   t.endLine = input_stream.getEndLine();
+   t.endColumn = input_stream.getEndColumn();
+   return t;
+}
+
+int curLexState = 0;
+int defaultLexState = 0;
+int jjnewStateCnt;
+int jjround;
+int jjmatchedPos;
+int jjmatchedKind;
+
+public Token getNextToken() 
+{
+  int kind;
+  Token specialToken = null;
+  Token matchedToken;
+  int curPos = 0;
+
+  EOFLoop :
+  for (;;)
+  {   
+   try   
+   {     
+      curChar = input_stream.BeginToken();
+   }     
+   catch(java.io.IOException e)
+   {        
+      jjmatchedKind = 0;
+      matchedToken = jjFillToken();
+      return matchedToken;
+   }
+
+   try { input_stream.backup(0);
+      while (curChar <= 32 && (0x100002600L & (1L << curChar)) != 0L)
+         curChar = input_stream.BeginToken();
+   }
+   catch (java.io.IOException e1) { continue EOFLoop; }
+   jjmatchedKind = 0x7fffffff;
+   jjmatchedPos = 0;
+   curPos = jjMoveStringLiteralDfa0_0();
+   if (jjmatchedKind != 0x7fffffff)
+   {
+      if (jjmatchedPos + 1 < curPos)
+         input_stream.backup(curPos - jjmatchedPos - 1);
+      if ((jjtoToken[jjmatchedKind >> 6] & (1L << (jjmatchedKind & 077))) != 0L)
+      {
+         matchedToken = jjFillToken();
+         return matchedToken;
+      }
+      else
+      {
+         continue EOFLoop;
+      }
+   }
+   int error_line = input_stream.getEndLine();
+   int error_column = input_stream.getEndColumn();
+   String error_after = null;
+   boolean EOFSeen = false;
+   try { input_stream.readChar(); input_stream.backup(1); }
+   catch (java.io.IOException e1) {
+      EOFSeen = true;
+      error_after = curPos <= 1 ? "" : input_stream.GetImage();
+      if (curChar == '\n' || curChar == '\r') {
+         error_line++;
+         error_column = 0;
+      }
+      else
+         error_column++;
+   }
+   if (!EOFSeen) {
+      input_stream.backup(1);
+      error_after = curPos <= 1 ? "" : input_stream.GetImage();
+   }
+   throw new TokenMgrError(EOFSeen, curLexState, error_line, error_column, error_after, curChar, TokenMgrError.LEXICAL_ERROR);
+  }
+}
+
+}

+ 192 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/generated/ParseException.java

@@ -0,0 +1,192 @@
+/* Generated By:JavaCC: Do not edit this line. ParseException.java Version 3.0 */
+package org.apache.hadoop.hbase.shell.algebra.generated;
+
+/**
+ * This exception is thrown when parse errors are encountered.
+ * You can explicitly create objects of this exception type by
+ * calling the method generateParseException in the generated
+ * parser.
+ *
+ * You can modify this class to customize your error reporting
+ * mechanisms so long as you retain the public fields.
+ */
+public class ParseException extends Exception {
+
+  /**
+   * This constructor is used by the method "generateParseException"
+   * in the generated parser.  Calling this constructor generates
+   * a new object of this type with the fields "currentToken",
+   * "expectedTokenSequences", and "tokenImage" set.  The boolean
+   * flag "specialConstructor" is also set to true to indicate that
+   * this constructor was used to create this object.
+   * This constructor calls its super class with the empty string
+   * to force the "toString" method of parent class "Throwable" to
+   * print the error message in the form:
+   *     ParseException: <result of getMessage>
+   */
+  public ParseException(Token currentTokenVal,
+                        int[][] expectedTokenSequencesVal,
+                        String[] tokenImageVal
+                       )
+  {
+    super("");
+    specialConstructor = true;
+    currentToken = currentTokenVal;
+    expectedTokenSequences = expectedTokenSequencesVal;
+    tokenImage = tokenImageVal;
+  }
+
+  /**
+   * The following constructors are for use by you for whatever
+   * purpose you can think of.  Constructing the exception in this
+   * manner makes the exception behave in the normal way - i.e., as
+   * documented in the class "Throwable".  The fields "errorToken",
+   * "expectedTokenSequences", and "tokenImage" do not contain
+   * relevant information.  The JavaCC generated code does not use
+   * these constructors.
+   */
+
+  public ParseException() {
+    super();
+    specialConstructor = false;
+  }
+
+  public ParseException(String message) {
+    super(message);
+    specialConstructor = false;
+  }
+
+  /**
+   * This variable determines which constructor was used to create
+   * this object and thereby affects the semantics of the
+   * "getMessage" method (see below).
+   */
+  protected boolean specialConstructor;
+
+  /**
+   * This is the last token that has been consumed successfully.  If
+   * this object has been created due to a parse error, the token
+   * followng this token will (therefore) be the first error token.
+   */
+  public Token currentToken;
+
+  /**
+   * Each entry in this array is an array of integers.  Each array
+   * of integers represents a sequence of tokens (by their ordinal
+   * values) that is expected at this point of the parse.
+   */
+  public int[][] expectedTokenSequences;
+
+  /**
+   * This is a reference to the "tokenImage" array of the generated
+   * parser within which the parse error occurred.  This array is
+   * defined in the generated ...Constants interface.
+   */
+  public String[] tokenImage;
+
+  /**
+   * This method has the standard behavior when this object has been
+   * created using the standard constructors.  Otherwise, it uses
+   * "currentToken" and "expectedTokenSequences" to generate a parse
+   * error message and returns it.  If this object has been created
+   * due to a parse error, and you do not catch it (it gets thrown
+   * from the parser), then this method is called during the printing
+   * of the final stack trace, and hence the correct error message
+   * gets displayed.
+   */
+  public String getMessage() {
+    if (!specialConstructor) {
+      return super.getMessage();
+    }
+    StringBuffer expected = new StringBuffer();
+    int maxSize = 0;
+    for (int i = 0; i < expectedTokenSequences.length; i++) {
+      if (maxSize < expectedTokenSequences[i].length) {
+        maxSize = expectedTokenSequences[i].length;
+      }
+      for (int j = 0; j < expectedTokenSequences[i].length; j++) {
+        expected.append(tokenImage[expectedTokenSequences[i][j]]).append(" ");
+      }
+      if (expectedTokenSequences[i][expectedTokenSequences[i].length - 1] != 0) {
+        expected.append("...");
+      }
+      expected.append(eol).append("    ");
+    }
+    String retval = "Encountered \"";
+    Token tok = currentToken.next;
+    for (int i = 0; i < maxSize; i++) {
+      if (i != 0) retval += " ";
+      if (tok.kind == 0) {
+        retval += tokenImage[0];
+        break;
+      }
+      retval += add_escapes(tok.image);
+      tok = tok.next; 
+    }
+    retval += "\" at line " + currentToken.next.beginLine + ", column " + currentToken.next.beginColumn;
+    retval += "." + eol;
+    if (expectedTokenSequences.length == 1) {
+      retval += "Was expecting:" + eol + "    ";
+    } else {
+      retval += "Was expecting one of:" + eol + "    ";
+    }
+    retval += expected.toString();
+    return retval;
+  }
+
+  /**
+   * The end of line string for this machine.
+   */
+  protected String eol = System.getProperty("line.separator", "\n");
+ 
+  /**
+   * Used to convert raw characters to their escaped version
+   * when these raw version cannot be used as part of an ASCII
+   * string literal.
+   */
+  protected String add_escapes(String str) {
+      StringBuffer retval = new StringBuffer();
+      char ch;
+      for (int i = 0; i < str.length(); i++) {
+        switch (str.charAt(i))
+        {
+           case 0 :
+              continue;
+           case '\b':
+              retval.append("\\b");
+              continue;
+           case '\t':
+              retval.append("\\t");
+              continue;
+           case '\n':
+              retval.append("\\n");
+              continue;
+           case '\f':
+              retval.append("\\f");
+              continue;
+           case '\r':
+              retval.append("\\r");
+              continue;
+           case '\"':
+              retval.append("\\\"");
+              continue;
+           case '\'':
+              retval.append("\\\'");
+              continue;
+           case '\\':
+              retval.append("\\\\");
+              continue;
+           default:
+              if ((ch = str.charAt(i)) < 0x20 || ch > 0x7e) {
+                 String s = "0000" + Integer.toString(ch, 16);
+                 retval.append("\\u" + s.substring(s.length() - 4, s.length()));
+              } else {
+                 retval.append(ch);
+              }
+              continue;
+        }
+      }
+      return retval.toString();
+   }
+
+}

+ 439 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/generated/SimpleCharStream.java

@@ -0,0 +1,439 @@
+/* Generated By:JavaCC: Do not edit this line. SimpleCharStream.java Version 4.0 */
+package org.apache.hadoop.hbase.shell.algebra.generated;
+
+/**
+ * An implementation of interface CharStream, where the stream is assumed to
+ * contain only ASCII characters (without unicode processing).
+ */
+
+public class SimpleCharStream
+{
+  public static final boolean staticFlag = false;
+  int bufsize;
+  int available;
+  int tokenBegin;
+  public int bufpos = -1;
+  protected int bufline[];
+  protected int bufcolumn[];
+
+  protected int column = 0;
+  protected int line = 1;
+
+  protected boolean prevCharIsCR = false;
+  protected boolean prevCharIsLF = false;
+
+  protected java.io.Reader inputStream;
+
+  protected char[] buffer;
+  protected int maxNextCharInd = 0;
+  protected int inBuf = 0;
+  protected int tabSize = 8;
+
+  protected void setTabSize(int i) { tabSize = i; }
+  protected int getTabSize(int i) { return tabSize; }
+
+
+  protected void ExpandBuff(boolean wrapAround)
+  {
+     char[] newbuffer = new char[bufsize + 2048];
+     int newbufline[] = new int[bufsize + 2048];
+     int newbufcolumn[] = new int[bufsize + 2048];
+
+     try
+     {
+        if (wrapAround)
+        {
+           System.arraycopy(buffer, tokenBegin, newbuffer, 0, bufsize - tokenBegin);
+           System.arraycopy(buffer, 0, newbuffer,
+                                             bufsize - tokenBegin, bufpos);
+           buffer = newbuffer;
+
+           System.arraycopy(bufline, tokenBegin, newbufline, 0, bufsize - tokenBegin);
+           System.arraycopy(bufline, 0, newbufline, bufsize - tokenBegin, bufpos);
+           bufline = newbufline;
+
+           System.arraycopy(bufcolumn, tokenBegin, newbufcolumn, 0, bufsize - tokenBegin);
+           System.arraycopy(bufcolumn, 0, newbufcolumn, bufsize - tokenBegin, bufpos);
+           bufcolumn = newbufcolumn;
+
+           maxNextCharInd = (bufpos += (bufsize - tokenBegin));
+        }
+        else
+        {
+           System.arraycopy(buffer, tokenBegin, newbuffer, 0, bufsize - tokenBegin);
+           buffer = newbuffer;
+
+           System.arraycopy(bufline, tokenBegin, newbufline, 0, bufsize - tokenBegin);
+           bufline = newbufline;
+
+           System.arraycopy(bufcolumn, tokenBegin, newbufcolumn, 0, bufsize - tokenBegin);
+           bufcolumn = newbufcolumn;
+
+           maxNextCharInd = (bufpos -= tokenBegin);
+        }
+     }
+     catch (Throwable t)
+     {
+        throw new Error(t.getMessage());
+     }
+
+
+     bufsize += 2048;
+     available = bufsize;
+     tokenBegin = 0;
+  }
+
+  protected void FillBuff() throws java.io.IOException
+  {
+     if (maxNextCharInd == available)
+     {
+        if (available == bufsize)
+        {
+           if (tokenBegin > 2048)
+           {
+              bufpos = maxNextCharInd = 0;
+              available = tokenBegin;
+           }
+           else if (tokenBegin < 0)
+              bufpos = maxNextCharInd = 0;
+           else
+              ExpandBuff(false);
+        }
+        else if (available > tokenBegin)
+           available = bufsize;
+        else if ((tokenBegin - available) < 2048)
+           ExpandBuff(true);
+        else
+           available = tokenBegin;
+     }
+
+     int i;
+     try {
+        if ((i = inputStream.read(buffer, maxNextCharInd,
+                                    available - maxNextCharInd)) == -1)
+        {
+           inputStream.close();
+           throw new java.io.IOException();
+        }
+        else
+           maxNextCharInd += i;
+        return;
+     }
+     catch(java.io.IOException e) {
+        --bufpos;
+        backup(0);
+        if (tokenBegin == -1)
+           tokenBegin = bufpos;
+        throw e;
+     }
+  }
+
+  public char BeginToken() throws java.io.IOException
+  {
+     tokenBegin = -1;
+     char c = readChar();
+     tokenBegin = bufpos;
+
+     return c;
+  }
+
+  protected void UpdateLineColumn(char c)
+  {
+     column++;
+
+     if (prevCharIsLF)
+     {
+        prevCharIsLF = false;
+        line += (column = 1);
+     }
+     else if (prevCharIsCR)
+     {
+        prevCharIsCR = false;
+        if (c == '\n')
+        {
+           prevCharIsLF = true;
+        }
+        else
+           line += (column = 1);
+     }
+
+     switch (c)
+     {
+        case '\r' :
+           prevCharIsCR = true;
+           break;
+        case '\n' :
+           prevCharIsLF = true;
+           break;
+        case '\t' :
+           column--;
+           column += (tabSize - (column % tabSize));
+           break;
+        default :
+           break;
+     }
+
+     bufline[bufpos] = line;
+     bufcolumn[bufpos] = column;
+  }
+
+  public char readChar() throws java.io.IOException
+  {
+     if (inBuf > 0)
+     {
+        --inBuf;
+
+        if (++bufpos == bufsize)
+           bufpos = 0;
+
+        return buffer[bufpos];
+     }
+
+     if (++bufpos >= maxNextCharInd)
+        FillBuff();
+
+     char c = buffer[bufpos];
+
+     UpdateLineColumn(c);
+     return (c);
+  }
+
+  /**
+   * @deprecated 
+   * @see #getEndColumn
+   */
+
+  public int getColumn() {
+     return bufcolumn[bufpos];
+  }
+
+  /**
+   * @deprecated 
+   * @see #getEndLine
+   */
+
+  public int getLine() {
+     return bufline[bufpos];
+  }
+
+  public int getEndColumn() {
+     return bufcolumn[bufpos];
+  }
+
+  public int getEndLine() {
+     return bufline[bufpos];
+  }
+
+  public int getBeginColumn() {
+     return bufcolumn[tokenBegin];
+  }
+
+  public int getBeginLine() {
+     return bufline[tokenBegin];
+  }
+
+  public void backup(int amount) {
+
+    inBuf += amount;
+    if ((bufpos -= amount) < 0)
+       bufpos += bufsize;
+  }
+
+  public SimpleCharStream(java.io.Reader dstream, int startline,
+  int startcolumn, int buffersize)
+  {
+    inputStream = dstream;
+    line = startline;
+    column = startcolumn - 1;
+
+    available = bufsize = buffersize;
+    buffer = new char[buffersize];
+    bufline = new int[buffersize];
+    bufcolumn = new int[buffersize];
+  }
+
+  public SimpleCharStream(java.io.Reader dstream, int startline,
+                          int startcolumn)
+  {
+     this(dstream, startline, startcolumn, 4096);
+  }
+
+  public SimpleCharStream(java.io.Reader dstream)
+  {
+     this(dstream, 1, 1, 4096);
+  }
+  public void ReInit(java.io.Reader dstream, int startline,
+  int startcolumn, int buffersize)
+  {
+    inputStream = dstream;
+    line = startline;
+    column = startcolumn - 1;
+
+    if (buffer == null || buffersize != buffer.length)
+    {
+      available = bufsize = buffersize;
+      buffer = new char[buffersize];
+      bufline = new int[buffersize];
+      bufcolumn = new int[buffersize];
+    }
+    prevCharIsLF = prevCharIsCR = false;
+    tokenBegin = inBuf = maxNextCharInd = 0;
+    bufpos = -1;
+  }
+
+  public void ReInit(java.io.Reader dstream, int startline,
+                     int startcolumn)
+  {
+     ReInit(dstream, startline, startcolumn, 4096);
+  }
+
+  public void ReInit(java.io.Reader dstream)
+  {
+     ReInit(dstream, 1, 1, 4096);
+  }
+  public SimpleCharStream(java.io.InputStream dstream, String encoding, int startline,
+  int startcolumn, int buffersize) throws java.io.UnsupportedEncodingException
+  {
+     this(encoding == null ? new java.io.InputStreamReader(dstream) : new java.io.InputStreamReader(dstream, encoding), startline, startcolumn, buffersize);
+  }
+
+  public SimpleCharStream(java.io.InputStream dstream, int startline,
+  int startcolumn, int buffersize)
+  {
+     this(new java.io.InputStreamReader(dstream), startline, startcolumn, buffersize);
+  }
+
+  public SimpleCharStream(java.io.InputStream dstream, String encoding, int startline,
+                          int startcolumn) throws java.io.UnsupportedEncodingException
+  {
+     this(dstream, encoding, startline, startcolumn, 4096);
+  }
+
+  public SimpleCharStream(java.io.InputStream dstream, int startline,
+                          int startcolumn)
+  {
+     this(dstream, startline, startcolumn, 4096);
+  }
+
+  public SimpleCharStream(java.io.InputStream dstream, String encoding) throws java.io.UnsupportedEncodingException
+  {
+     this(dstream, encoding, 1, 1, 4096);
+  }
+
+  public SimpleCharStream(java.io.InputStream dstream)
+  {
+     this(dstream, 1, 1, 4096);
+  }
+
+  public void ReInit(java.io.InputStream dstream, String encoding, int startline,
+                          int startcolumn, int buffersize) throws java.io.UnsupportedEncodingException
+  {
+     ReInit(encoding == null ? new java.io.InputStreamReader(dstream) : new java.io.InputStreamReader(dstream, encoding), startline, startcolumn, buffersize);
+  }
+
+  public void ReInit(java.io.InputStream dstream, int startline,
+                          int startcolumn, int buffersize)
+  {
+     ReInit(new java.io.InputStreamReader(dstream), startline, startcolumn, buffersize);
+  }
+
+  public void ReInit(java.io.InputStream dstream, String encoding) throws java.io.UnsupportedEncodingException
+  {
+     ReInit(dstream, encoding, 1, 1, 4096);
+  }
+
+  public void ReInit(java.io.InputStream dstream)
+  {
+     ReInit(dstream, 1, 1, 4096);
+  }
+  public void ReInit(java.io.InputStream dstream, String encoding, int startline,
+                     int startcolumn) throws java.io.UnsupportedEncodingException
+  {
+     ReInit(dstream, encoding, startline, startcolumn, 4096);
+  }
+  public void ReInit(java.io.InputStream dstream, int startline,
+                     int startcolumn)
+  {
+     ReInit(dstream, startline, startcolumn, 4096);
+  }
+  public String GetImage()
+  {
+     if (bufpos >= tokenBegin)
+        return new String(buffer, tokenBegin, bufpos - tokenBegin + 1);
+     else
+        return new String(buffer, tokenBegin, bufsize - tokenBegin) +
+                              new String(buffer, 0, bufpos + 1);
+  }
+
+  public char[] GetSuffix(int len)
+  {
+     char[] ret = new char[len];
+
+     if ((bufpos + 1) >= len)
+        System.arraycopy(buffer, bufpos - len + 1, ret, 0, len);
+     else
+     {
+        System.arraycopy(buffer, bufsize - (len - bufpos - 1), ret, 0,
+                                                          len - bufpos - 1);
+        System.arraycopy(buffer, 0, ret, len - bufpos - 1, bufpos + 1);
+     }
+
+     return ret;
+  }
+
+  public void Done()
+  {
+     buffer = null;
+     bufline = null;
+     bufcolumn = null;
+  }
+
+  /**
+   * Method to adjust line and column numbers for the start of a token.
+   */
+  public void adjustBeginLineColumn(int newLine, int newCol)
+  {
+     int start = tokenBegin;
+     int len;
+
+     if (bufpos >= tokenBegin)
+     {
+        len = bufpos - tokenBegin + inBuf + 1;
+     }
+     else
+     {
+        len = bufsize - tokenBegin + bufpos + 1 + inBuf;
+     }
+
+     int i = 0, j = 0, k = 0;
+     int nextColDiff = 0, columnDiff = 0;
+
+     while (i < len &&
+            bufline[j = start % bufsize] == bufline[k = ++start % bufsize])
+     {
+        bufline[j] = newLine;
+        nextColDiff = columnDiff + bufcolumn[k] - bufcolumn[j];
+        bufcolumn[j] = newCol + columnDiff;
+        columnDiff = nextColDiff;
+        i++;
+     } 
+
+     if (i < len)
+     {
+        bufline[j] = newLine++;
+        bufcolumn[j] = newCol + columnDiff;
+
+        while (i++ < len)
+        {
+           if (bufline[j = start % bufsize] != bufline[++start % bufsize])
+              bufline[j] = newLine++;
+           else
+              bufline[j] = newLine;
+        }
+     }
+
+     line = bufline[j];
+     column = bufcolumn[j];
+  }
+
+}

+ 81 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/generated/Token.java

@@ -0,0 +1,81 @@
+/* Generated By:JavaCC: Do not edit this line. Token.java Version 3.0 */
+package org.apache.hadoop.hbase.shell.algebra.generated;
+
+/**
+ * Describes the input token stream.
+ */
+
+public class Token {
+
+  /**
+   * An integer that describes the kind of this token.  This numbering
+   * system is determined by JavaCCParser, and a table of these numbers is
+   * stored in the file ...Constants.java.
+   */
+  public int kind;
+
+  /**
+   * beginLine and beginColumn describe the position of the first character
+   * of this token; endLine and endColumn describe the position of the
+   * last character of this token.
+   */
+  public int beginLine, beginColumn, endLine, endColumn;
+
+  /**
+   * The string image of the token.
+   */
+  public String image;
+
+  /**
+   * A reference to the next regular (non-special) token from the input
+   * stream.  If this is the last token from the input stream, or if the
+   * token manager has not read tokens beyond this one, this field is
+   * set to null.  This is true only if this token is also a regular
+   * token.  Otherwise, see below for a description of the contents of
+   * this field.
+   */
+  public Token next;
+
+  /**
+   * This field is used to access special tokens that occur prior to this
+   * token, but after the immediately preceding regular (non-special) token.
+   * If there are no such special tokens, this field is set to null.
+   * When there are more than one such special token, this field refers
+   * to the last of these special tokens, which in turn refers to the next
+   * previous special token through its specialToken field, and so on
+   * until the first special token (whose specialToken field is null).
+   * The next fields of special tokens refer to other special tokens that
+   * immediately follow it (without an intervening regular token).  If there
+   * is no such token, this field is null.
+   */
+  public Token specialToken;
+
+  /**
+   * Returns the image.
+   */
+  public String toString()
+  {
+     return image;
+  }
+
+  /**
+   * Returns a new Token object, by default. However, if you want, you
+   * can create and return subclass objects based on the value of ofKind.
+   * Simply add the cases to the switch for all those special cases.
+   * For example, if you have a subclass of Token called IDToken that
+   * you want to create if ofKind is ID, simlpy add something like :
+   *
+   *    case MyParserConstants.ID : return new IDToken();
+   *
+   * to the following switch statement. Then you can cast matchedToken
+   * variable to the appropriate type and use it in your lexical actions.
+   */
+  public static final Token newToken(int ofKind)
+  {
+     switch(ofKind)
+     {
+       default : return new Token();
+     }
+  }
+
+}

+ 133 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/generated/TokenMgrError.java

@@ -0,0 +1,133 @@
+/* Generated By:JavaCC: Do not edit this line. TokenMgrError.java Version 3.0 */
+package org.apache.hadoop.hbase.shell.algebra.generated;
+
+public class TokenMgrError extends Error
+{
+   /*
+    * Ordinals for various reasons why an Error of this type can be thrown.
+    */
+
+   /**
+    * Lexical error occured.
+    */
+   static final int LEXICAL_ERROR = 0;
+
+   /**
+    * An attempt wass made to create a second instance of a static token manager.
+    */
+   static final int STATIC_LEXER_ERROR = 1;
+
+   /**
+    * Tried to change to an invalid lexical state.
+    */
+   static final int INVALID_LEXICAL_STATE = 2;
+
+   /**
+    * Detected (and bailed out of) an infinite loop in the token manager.
+    */
+   static final int LOOP_DETECTED = 3;
+
+   /**
+    * Indicates the reason why the exception is thrown. It will have
+    * one of the above 4 values.
+    */
+   int errorCode;
+
+   /**
+    * Replaces unprintable characters by their espaced (or unicode escaped)
+    * equivalents in the given string
+    */
+   protected static final String addEscapes(String str) {
+      StringBuffer retval = new StringBuffer();
+      char ch;
+      for (int i = 0; i < str.length(); i++) {
+        switch (str.charAt(i))
+        {
+           case 0 :
+              continue;
+           case '\b':
+              retval.append("\\b");
+              continue;
+           case '\t':
+              retval.append("\\t");
+              continue;
+           case '\n':
+              retval.append("\\n");
+              continue;
+           case '\f':
+              retval.append("\\f");
+              continue;
+           case '\r':
+              retval.append("\\r");
+              continue;
+           case '\"':
+              retval.append("\\\"");
+              continue;
+           case '\'':
+              retval.append("\\\'");
+              continue;
+           case '\\':
+              retval.append("\\\\");
+              continue;
+           default:
+              if ((ch = str.charAt(i)) < 0x20 || ch > 0x7e) {
+                 String s = "0000" + Integer.toString(ch, 16);
+                 retval.append("\\u" + s.substring(s.length() - 4, s.length()));
+              } else {
+                 retval.append(ch);
+              }
+              continue;
+        }
+      }
+      return retval.toString();
+   }
+
+   /**
+    * Returns a detailed message for the Error when it is thrown by the
+    * token manager to indicate a lexical error.
+    * Parameters : 
+    *    EOFSeen     : indicates if EOF caused the lexicl error
+    *    curLexState : lexical state in which this error occured
+    *    errorLine   : line number when the error occured
+    *    errorColumn : column number when the error occured
+    *    errorAfter  : prefix that was seen before this error occured
+    *    curchar     : the offending character
+    * Note: You can customize the lexical error message by modifying this method.
+    */
+   protected static String LexicalError(boolean EOFSeen, int lexState, int errorLine, int errorColumn, String errorAfter, char curChar) {
+      return("Lexical error at line " +
+           errorLine + ", column " +
+           errorColumn + ".  Encountered: " +
+           (EOFSeen ? "<EOF> " : ("\"" + addEscapes(String.valueOf(curChar)) + "\"") + " (" + (int)curChar + "), ") +
+           "after : \"" + addEscapes(errorAfter) + "\"");
+   }
+
+   /**
+    * You can also modify the body of this method to customize your error messages.
+    * For example, cases like LOOP_DETECTED and INVALID_LEXICAL_STATE are not
+    * of end-users concern, so you can return something like : 
+    *
+    *     "Internal Error : Please file a bug report .... "
+    *
+    * from this method for such cases in the release version of your parser.
+    */
+   public String getMessage() {
+      return super.getMessage();
+   }
+
+   /*
+    * Constructors of various flavors follow.
+    */
+
+   public TokenMgrError() {
+   }
+
+   public TokenMgrError(String message, int reason) {
+      super(message);
+      errorCode = reason;
+   }
+
+   public TokenMgrError(boolean EOFSeen, int lexState, int errorLine, int errorColumn, String errorAfter, char curChar, int reason) {
+      this(LexicalError(EOFSeen, lexState, errorLine, errorColumn, errorAfter, curChar), reason);
+   }
+}

+ 454 - 91
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/Parser.java

@@ -28,6 +28,8 @@ import java.util.HashMap;
 import java.io.StringReader;
 import java.io.Reader;
 import java.io.Writer;
+import java.net.URLEncoder;
+import java.io.UnsupportedEncodingException;
 
 import org.apache.hadoop.hbase.shell.*;
 
@@ -38,6 +40,7 @@ public class Parser implements ParserConstants {
   private String QueryString;
   private TableFormatter formatter;
   private Writer out;
+  private String secondR;
 
   public Parser(final String query, final Writer o, final TableFormatter f) {
     this((Reader)(new StringReader(query)));
@@ -72,7 +75,9 @@ public class Parser implements ParserConstants {
     case SELECT:
     case ENABLE:
     case DISABLE:
-    case 62:
+    case SAVE:
+    case ID:
+    case 69:
       switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
       case HELP:
       case ALTER:
@@ -90,13 +95,15 @@ public class Parser implements ParserConstants {
       case SELECT:
       case ENABLE:
       case DISABLE:
+      case SAVE:
+      case ID:
         statement = cmdStatement();
         break;
       default:
         jj_la1[0] = jj_gen;
         ;
       }
-      jj_consume_token(62);
+      jj_consume_token(69);
       break;
     case 0:
       jj_consume_token(0);
@@ -159,6 +166,12 @@ public class Parser implements ParserConstants {
     case JAR:
       cmd = jarCommand();
       break;
+    case ID:
+      cmd = substituteCommand();
+      break;
+    case SAVE:
+      cmd = saveCommand();
+      break;
     default:
       jj_la1[2] = jj_gen;
       jj_consume_token(-1);
@@ -251,6 +264,8 @@ public class Parser implements ParserConstants {
     case INSERT:
     case DELETE:
     case SELECT:
+    case SAVE:
+    case GROUP:
     case ID:
       switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
       case SHOW:
@@ -289,6 +304,12 @@ public class Parser implements ParserConstants {
       case JAR:
         t = jj_consume_token(JAR);
         break;
+      case GROUP:
+        t = jj_consume_token(GROUP);
+        break;
+      case SAVE:
+        t = jj_consume_token(SAVE);
+        break;
       case ID:
         t = jj_consume_token(ID);
         break;
@@ -316,7 +337,7 @@ public class Parser implements ParserConstants {
     case ID:
     case QUOTED_IDENTIFIER:
     case STRING_LITERAL:
-      argument = Identifier();
+      argument = identifier();
       break;
     default:
       jj_la1[8] = jj_gen;
@@ -342,7 +363,7 @@ public class Parser implements ParserConstants {
       jj_consume_token(-1);
       throw new ParseException();
     }
-    argument = Identifier();
+    argument = identifier();
     desc.setArgument(argument);
     {if (true) return desc;}
     throw new Error("Missing return statement in function");
@@ -461,10 +482,10 @@ public class Parser implements ParserConstants {
   String column = null;
     jj_consume_token(CREATE);
     jj_consume_token(TABLE);
-    table = Identifier();
+    table = identifier();
     createCommand.setTable(table);
     jj_consume_token(LPAREN);
-    column = Identifier();
+    column = identifier();
     columnSpec = ColumnSpec();
     createCommand.addColumnSpec(column, columnSpec);
     label_4:
@@ -478,7 +499,7 @@ public class Parser implements ParserConstants {
         break label_4;
       }
       jj_consume_token(COMMA);
-      column = Identifier();
+      column = identifier();
       columnSpec = ColumnSpec();
         createCommand.addColumnSpec(column, columnSpec);
     }
@@ -494,11 +515,11 @@ public class Parser implements ParserConstants {
   Map<String, Object> columnSpec = null;
     jj_consume_token(ALTER);
     jj_consume_token(TABLE);
-    table = Identifier();
+    table = identifier();
     alterCommand.setTable(table);
     if (jj_2_1(2)) {
       jj_consume_token(ADD);
-      column = Identifier();
+      column = identifier();
       columnSpec = ColumnSpec();
          alterCommand.setOperationType(AlterCommand.OperationType.ADD);
          alterCommand.addColumnSpec(column, columnSpec);
@@ -508,7 +529,7 @@ public class Parser implements ParserConstants {
         jj_consume_token(ADD);
         jj_consume_token(LPAREN);
       alterCommand.setOperationType(AlterCommand.OperationType.ADD);
-        column = Identifier();
+        column = identifier();
         columnSpec = ColumnSpec();
       alterCommand.addColumnSpec(column, columnSpec);
         label_5:
@@ -522,7 +543,7 @@ public class Parser implements ParserConstants {
             break label_5;
           }
           jj_consume_token(COMMA);
-          column = Identifier();
+          column = identifier();
           columnSpec = ColumnSpec();
           alterCommand.addColumnSpec(column, columnSpec);
         }
@@ -530,13 +551,13 @@ public class Parser implements ParserConstants {
         break;
       case DROP:
         jj_consume_token(DROP);
-        column = Identifier();
+        column = identifier();
       alterCommand.setOperationType(AlterCommand.OperationType.DROP);
       alterCommand.setColumn(column);
         break;
       case CHANGE:
         jj_consume_token(CHANGE);
-        column = Identifier();
+        column = identifier();
         columnSpec = ColumnSpec();
       alterCommand.setOperationType(AlterCommand.OperationType.CHANGE);
       alterCommand.addColumnSpec(column, columnSpec);
@@ -556,7 +577,7 @@ public class Parser implements ParserConstants {
   List<String> tableList = null;
     jj_consume_token(DROP);
     jj_consume_token(TABLE);
-    tableList = TableList();
+    tableList = tableList();
      drop.setTableList(tableList);
      {if (true) return drop;}
     throw new Error("Missing return statement in function");
@@ -570,7 +591,7 @@ public class Parser implements ParserConstants {
   Token t = null;
     jj_consume_token(INSERT);
     jj_consume_token(INTO);
-    table = Identifier();
+    table = identifier();
      in.setTable(table);
     columnfamilies = getColumns();
      in.setColumnfamilies(columnfamilies);
@@ -603,10 +624,10 @@ public class Parser implements ParserConstants {
   Token t = null;
   String table = null;
     jj_consume_token(DELETE);
-    columnList = ColumnList();
+    columnList = columnList();
     deleteCommand.setColumnList(columnList);
     jj_consume_token(FROM);
-    table = Identifier();
+    table = identifier();
     deleteCommand.setTable(table);
     jj_consume_token(WHERE);
     jj_consume_token(ROW);
@@ -637,9 +658,9 @@ public class Parser implements ParserConstants {
   String tableName = null;
   int limit;
     jj_consume_token(SELECT);
-    columns = ColumnList();
+    columns = columnList();
     jj_consume_token(FROM);
-    tableName = Identifier();
+    tableName = identifier();
      select.setColumns(columns);
      select.setTable(tableName);
     switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
@@ -712,7 +733,7 @@ public class Parser implements ParserConstants {
   EnableCommand enableCommand = new EnableCommand(this.out);
   String table = null;
     jj_consume_token(ENABLE);
-    table = Identifier();
+    table = identifier();
     enableCommand.setTable(table);
     {if (true) return enableCommand;}
     throw new Error("Missing return statement in function");
@@ -722,7 +743,7 @@ public class Parser implements ParserConstants {
   DisableCommand disableCommand = new DisableCommand(this.out);
   String table = null;
     jj_consume_token(DISABLE);
-    table = Identifier();
+    table = identifier();
     disableCommand.setTable(table);
     {if (true) return disableCommand;}
     throw new Error("Missing return statement in function");
@@ -735,8 +756,104 @@ public class Parser implements ParserConstants {
     throw new Error("Missing return statement in function");
   }
 
-////////////////////////////////////////////////
-// Utility expansion units...
+  final public SubstituteCommand substituteCommand() throws ParseException {
+  Token key = null;
+  Token chainKey = null;
+  Token operation = null;
+  String tableName = null;
+  String condition = "";
+  List<String> notInList = new ArrayList<String>();
+  SubstituteCommand substitute = new SubstituteCommand(this.out);
+  Token extendedKey = null;
+    key = jj_consume_token(ID);
+    jj_consume_token(EQUALS);
+    substitute.setKey(key.image.toString());
+    switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+    case ID:
+      chainKey = jj_consume_token(ID);
+      jj_consume_token(DOT);
+      substitute.setChainKey(chainKey.image.toString());
+      operation = jj_consume_token(ID);
+      substitute.setOperation(operation.image.toString());
+      jj_consume_token(LPAREN);
+      String operationType = operation.image.toLowerCase();
+      if(operationType.equals("projection")) {
+        List<String> columnList = columnList();
+        for (int i = 0; i < columnList.size(); i++) {
+          condition += appendIndicator(columnList.get(i)) + " ";
+        }
+
+      } else {
+        condition = booleanTerm();
+      }
+
+      substitute.setCondition(condition);
+      jj_consume_token(RPAREN);
+      switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+      case AND:
+        jj_consume_token(AND);
+        extendedKey = jj_consume_token(ID);
+        String eKey = extendedKey.image.toString();
+        String cKey = chainKey.image.toString();
+        substitute.setCondition(condition);
+        if (secondR.equals(eKey)) {
+          substitute.resetVariableRelation(eKey, cKey);
+        } else {
+          substitute.resetVariableRelation(cKey, eKey);
+        }
+        break;
+      default:
+        jj_la1[24] = jj_gen;
+        ;
+      }
+      break;
+    case TABLE:
+      jj_consume_token(TABLE);
+      jj_consume_token(LPAREN);
+      tableName = identifier();
+      substitute.setInput(tableName);
+      jj_consume_token(RPAREN);
+      break;
+    case GROUP:
+      operation = jj_consume_token(GROUP);
+      chainKey = jj_consume_token(ID);
+      jj_consume_token(BY);
+      jj_consume_token(LPAREN);
+        List<String> columnList = columnList();
+        for (int i = 0; i < columnList.size(); i++) {
+          condition += appendIndicator(columnList.get(i));
+        }
+      jj_consume_token(RPAREN);
+      substitute.setChainKey(chainKey.image.toString());
+      substitute.setOperation(operation.image.toString());
+      substitute.setCondition(condition);
+      break;
+    default:
+      jj_la1[25] = jj_gen;
+      jj_consume_token(-1);
+      throw new ParseException();
+    }
+    {if (true) return substitute;}
+    throw new Error("Missing return statement in function");
+  }
+
+  final public SaveCommand saveCommand() throws ParseException {
+  Token t = null;
+  String tableName;
+  SaveCommand save = new SaveCommand(this.out);
+    jj_consume_token(SAVE);
+    t = jj_consume_token(ID);
+    save.setStatement(t.image.toString());
+    jj_consume_token(INTO);
+    jj_consume_token(TABLE);
+    jj_consume_token(LPAREN);
+    tableName = identifier();
+                             save.setOutput(tableName);
+    jj_consume_token(RPAREN);
+    {if (true) return save;}
+    throw new Error("Missing return statement in function");
+  }
+
   final public List<String> getLiteralValues() throws ParseException {
   List<String> values = new ArrayList<String>();
   String literal = null;
@@ -753,7 +870,7 @@ public class Parser implements ParserConstants {
         ;
         break;
       default:
-        jj_la1[24] = jj_gen;
+        jj_la1[26] = jj_gen;
         break label_6;
       }
       switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
@@ -776,14 +893,14 @@ public class Parser implements ParserConstants {
           jj_consume_token(QUOTED_IDENTIFIER);
           break;
         default:
-          jj_la1[25] = jj_gen;
+          jj_la1[27] = jj_gen;
           jj_consume_token(-1);
           throw new ParseException();
         }
        values.removeAll(values);
         break;
       default:
-        jj_la1[26] = jj_gen;
+        jj_la1[28] = jj_gen;
         jj_consume_token(-1);
         throw new ParseException();
       }
@@ -803,7 +920,7 @@ public class Parser implements ParserConstants {
       s = jj_consume_token(QUOTED_IDENTIFIER);
       break;
     default:
-      jj_la1[27] = jj_gen;
+      jj_la1[29] = jj_gen;
       jj_consume_token(-1);
       throw new ParseException();
     }
@@ -812,45 +929,24 @@ public class Parser implements ParserConstants {
     throw new Error("Missing return statement in function");
   }
 
-  final public List<String> getColumns() throws ParseException {
-  List<String> values = new ArrayList<String>();
-  String literal = null;
-    jj_consume_token(LPAREN);
-   literal = getColumn();
- if(literal != null) values.add(literal);
-    label_7:
-    while (true) {
-      switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
-      case COMMA:
-        ;
-        break;
-      default:
-        jj_la1[28] = jj_gen;
-        break label_7;
-      }
-      jj_consume_token(COMMA);
-        literal = getColumn();
-        if(literal != null) values.add(literal);
-    }
-    jj_consume_token(RPAREN);
-     {if (true) return values;}
-    throw new Error("Missing return statement in function");
-  }
-
   final public String getColumn() throws ParseException {
   Token col;
     switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
     case ASTERISK:
     case ID:
+    case INTEGER_LITERAL:
       switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
       case ID:
         col = jj_consume_token(ID);
         break;
+      case INTEGER_LITERAL:
+        col = jj_consume_token(INTEGER_LITERAL);
+        break;
       case ASTERISK:
         col = jj_consume_token(ASTERISK);
         break;
       default:
-        jj_la1[29] = jj_gen;
+        jj_la1[30] = jj_gen;
         jj_consume_token(-1);
         throw new ParseException();
       }
@@ -866,24 +962,49 @@ public class Parser implements ParserConstants {
         col = jj_consume_token(STRING_LITERAL);
         break;
       default:
-        jj_la1[30] = jj_gen;
+        jj_la1[31] = jj_gen;
         jj_consume_token(-1);
         throw new ParseException();
       }
         {if (true) return col.image.substring(1,col.image.toString().length() - 1);}
       break;
     default:
-      jj_la1[31] = jj_gen;
+      jj_la1[32] = jj_gen;
       jj_consume_token(-1);
       throw new ParseException();
     }
     throw new Error("Missing return statement in function");
   }
 
-  final public List<String> TableList() throws ParseException {
+  final public List<String> getColumns() throws ParseException {
+  List<String> values = new ArrayList<String>();
+  String literal = null;
+    jj_consume_token(LPAREN);
+   literal = getColumn();
+ if(literal != null) values.add(literal);
+    label_7:
+    while (true) {
+      switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+      case COMMA:
+        ;
+        break;
+      default:
+        jj_la1[33] = jj_gen;
+        break label_7;
+      }
+      jj_consume_token(COMMA);
+        literal = getColumn();
+        if(literal != null) values.add(literal);
+    }
+    jj_consume_token(RPAREN);
+     {if (true) return values;}
+    throw new Error("Missing return statement in function");
+  }
+
+  final public List<String> tableList() throws ParseException {
   List<String> tableList = new ArrayList<String>();
   String table = null;
-    table = Identifier();
+    table = identifier();
                          tableList.add(table);
     label_8:
     while (true) {
@@ -892,18 +1013,18 @@ public class Parser implements ParserConstants {
         ;
         break;
       default:
-        jj_la1[32] = jj_gen;
+        jj_la1[34] = jj_gen;
         break label_8;
       }
       jj_consume_token(COMMA);
-      table = Identifier();
+      table = identifier();
       tableList.add(table);
     }
     {if (true) return tableList;}
     throw new Error("Missing return statement in function");
   }
 
-  final public List<String> ColumnList() throws ParseException {
+  final public List<String> columnList() throws ParseException {
   List<String> columnList = new ArrayList<String>();
   String column = null;
     column = getColumn();
@@ -919,7 +1040,7 @@ public class Parser implements ParserConstants {
         ;
         break;
       default:
-        jj_la1[33] = jj_gen;
+        jj_la1[35] = jj_gen;
         break label_9;
       }
       jj_consume_token(COMMA);
@@ -937,7 +1058,7 @@ public class Parser implements ParserConstants {
     throw new Error("Missing return statement in function");
   }
 
-  final public String Identifier() throws ParseException {
+  final public String identifier() throws ParseException {
   Token t = null;
     switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
     case ID:
@@ -954,20 +1075,254 @@ public class Parser implements ParserConstants {
         t = jj_consume_token(STRING_LITERAL);
         break;
       default:
-        jj_la1[34] = jj_gen;
+        jj_la1[36] = jj_gen;
         jj_consume_token(-1);
         throw new ParseException();
       }
        {if (true) return t.image.substring(1,t.image.toString().length() - 1);}
       break;
     default:
-      jj_la1[35] = jj_gen;
+      jj_la1[37] = jj_gen;
       jj_consume_token(-1);
       throw new ParseException();
     }
     throw new Error("Missing return statement in function");
   }
 
+  final public String booleanTerm() throws ParseException {
+  String query = "";
+  String tmp = null;
+  String joinTerm = null;
+  String connector = null;
+    query = booleanTerms();
+    if(query.endsWith(".ROW")) {
+      joinTerm = query;
+      query = "";
+    }
+    label_10:
+    while (true) {
+      switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+      case AND:
+      case OR:
+        ;
+        break;
+      default:
+        jj_la1[38] = jj_gen;
+        break label_10;
+      }
+      switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+      case AND:
+        jj_consume_token(AND);
+               connector = " AND ";
+        break;
+      case OR:
+        jj_consume_token(OR);
+                connector = " OR ";
+        break;
+      default:
+        jj_la1[39] = jj_gen;
+        jj_consume_token(-1);
+        throw new ParseException();
+      }
+      tmp = booleanTerms();
+     if(tmp.endsWith(".ROW")) {
+      joinTerm = tmp;
+     } else {
+       if(!query.equals(""))
+         query += connector;
+        query += tmp;
+     }
+    }
+    if(joinTerm != null) {
+      {if (true) return joinTerm + " BOOL " + query;}
+    } else {
+      {if (true) return query;}
+    }
+    throw new Error("Missing return statement in function");
+  }
+
+  final public String booleanTerms() throws ParseException {
+  Token tSearchName, tComparand;
+  List<String> inList = new ArrayList<String>();
+  String searchName=null,comparator=null,comparand=null;
+  Token joinColumn = null;
+  Token joinKey = null;
+  String result = null;
+    tSearchName = jj_consume_token(ID);
+                       searchName = tSearchName.image.toString();
+    switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+    case DOT:
+      jj_consume_token(DOT);
+      switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+      case ID:
+        joinColumn = jj_consume_token(ID);
+          searchName += "." + joinColumn.image.toString();
+        break;
+      case ROW:
+        jj_consume_token(ROW);
+          secondR = searchName;
+          searchName += ".ROW";
+        break;
+      default:
+        jj_la1[40] = jj_gen;
+        jj_consume_token(-1);
+        throw new ParseException();
+      }
+      break;
+    default:
+      jj_la1[41] = jj_gen;
+      ;
+    }
+    comparator = getComparator();
+    switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+    case INTEGER_LITERAL:
+      tComparand = jj_consume_token(INTEGER_LITERAL);
+        comparand = tComparand.image.toString();
+      break;
+    case STRING_LITERAL:
+      tComparand = jj_consume_token(STRING_LITERAL);
+        comparand = tComparand.image.substring(1,tComparand.image.length() - 1);
+      break;
+    case ID:
+      tComparand = jj_consume_token(ID);
+       comparand = tComparand.image.toString();
+      switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+      case DOT:
+        jj_consume_token(DOT);
+        switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+        case ROW:
+          jj_consume_token(ROW);
+            secondR = comparand;
+            comparand += ".ROW";
+          break;
+        case ID:
+          joinColumn = jj_consume_token(ID);
+            comparand += "." + joinColumn.image.toString();
+          break;
+        default:
+          jj_la1[42] = jj_gen;
+          jj_consume_token(-1);
+          throw new ParseException();
+        }
+        break;
+      default:
+        jj_la1[43] = jj_gen;
+        ;
+      }
+      break;
+    case LPAREN:
+      inList = getColumns();
+      if(comparator == null) {
+        comparator = "==";
+      }
+      comparand = "";
+      try{
+        for(int i=0; i<inList.size(); i++) {
+          comparand += URLEncoder.encode(inList.get(i), "UTF-8");
+          if(inList.size() != (i+1)) { comparand += "|"; }
+        }
+      } catch (UnsupportedEncodingException e) {
+          e.printStackTrace();
+      }
+      break;
+    default:
+      jj_la1[44] = jj_gen;
+      jj_consume_token(-1);
+      throw new ParseException();
+    }
+    if(searchName.endsWith(".ROW")) {
+      result = appendIndicator(comparand)
+      + " " + comparator + " " + searchName;
+    } else {
+      result = appendIndicator(searchName)
+      + " " + comparator + " " + comparand;
+    }
+
+     {if (true) return result;}
+    throw new Error("Missing return statement in function");
+  }
+
+  final public String getComparator() throws ParseException {
+  Token t = null;
+  String comparator = null;
+    switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+    case LCOMP:
+      t = jj_consume_token(LCOMP);
+          comparator = t.image.toString();
+      switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+      case EQUALS:
+        jj_consume_token(EQUALS);
+                  comparator += "=";
+        break;
+      default:
+        jj_la1[45] = jj_gen;
+        ;
+      }
+      break;
+    case RCOMP:
+      t = jj_consume_token(RCOMP);
+          comparator = t.image.toString();
+      switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+      case EQUALS:
+        jj_consume_token(EQUALS);
+                  comparator += "=";
+        break;
+      default:
+        jj_la1[46] = jj_gen;
+        ;
+      }
+      break;
+    case EQUALS:
+      t = jj_consume_token(EQUALS);
+          comparator = t.image.toString();
+      switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+      case LCOMP:
+        jj_consume_token(LCOMP);
+                 comparator = ">" + comparator;
+        break;
+      default:
+        jj_la1[47] = jj_gen;
+        ;
+      }
+      switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
+      case RCOMP:
+        jj_consume_token(RCOMP);
+                 comparator = "<" + comparator;
+        break;
+      default:
+        jj_la1[48] = jj_gen;
+        ;
+      }
+      break;
+    case NOTEQUAL:
+      t = jj_consume_token(NOTEQUAL);
+          comparator = t.image.toString();
+      break;
+    case NOT:
+      jj_consume_token(NOT);
+      jj_consume_token(IN);
+          comparator = "!!";
+      break;
+    case IN:
+      jj_consume_token(IN);
+          comparator = "==";
+      break;
+    default:
+      jj_la1[49] = jj_gen;
+      jj_consume_token(-1);
+      throw new ParseException();
+    }
+    {if (true) return comparator;}
+    throw new Error("Missing return statement in function");
+  }
+
+  final public String appendIndicator(String columnName) throws ParseException {
+  String column = columnName;
+     {if (true) return (!column.endsWith(":") && column.indexOf(":") == -1)
+     ? column + ":" : column;}
+    throw new Error("Missing return statement in function");
+  }
+
   final private boolean jj_2_1(int xla) {
     jj_la = xla; jj_lastpos = jj_scanpos = token;
     try { return !jj_3_1(); }
@@ -975,34 +1330,34 @@ public class Parser implements ParserConstants {
     finally { jj_save(0, xla); }
   }
 
-  final private boolean jj_3R_12() {
-    Token xsp;
-    xsp = jj_scanpos;
-    if (jj_scan_token(60)) {
-    jj_scanpos = xsp;
-    if (jj_scan_token(61)) return true;
-    }
+  final private boolean jj_3_1() {
+    if (jj_scan_token(ADD)) return true;
+    if (jj_3R_11()) return true;
     return false;
   }
 
   final private boolean jj_3R_11() {
-    if (jj_scan_token(ID)) return true;
+    Token xsp;
+    xsp = jj_scanpos;
+    if (jj_3R_12()) {
+    jj_scanpos = xsp;
+    if (jj_3R_13()) return true;
+    }
     return false;
   }
 
-  final private boolean jj_3R_10() {
+  final private boolean jj_3R_13() {
     Token xsp;
     xsp = jj_scanpos;
-    if (jj_3R_11()) {
+    if (jj_scan_token(67)) {
     jj_scanpos = xsp;
-    if (jj_3R_12()) return true;
+    if (jj_scan_token(68)) return true;
     }
     return false;
   }
 
-  final private boolean jj_3_1() {
-    if (jj_scan_token(ADD)) return true;
-    if (jj_3R_10()) return true;
+  final private boolean jj_3R_12() {
+    if (jj_scan_token(ID)) return true;
     return false;
   }
 
@@ -1015,18 +1370,23 @@ public class Parser implements ParserConstants {
   public boolean lookingAhead = false;
   private boolean jj_semLA;
   private int jj_gen;
-  final private int[] jj_la1 = new int[36];
+  final private int[] jj_la1 = new int[50];
   static private int[] jj_la1_0;
   static private int[] jj_la1_1;
+  static private int[] jj_la1_2;
   static {
       jj_la1_0();
       jj_la1_1();
+      jj_la1_2();
    }
    private static void jj_la1_0() {
-      jj_la1_0 = new int[] {0x79ffe0,0x79ffe1,0x79ffe0,0x0,0x0,0x0,0x19fbc0,0x19fbc0,0x0,0x600,0x0,0x0,0x0,0x0,0x0,0x0,0x1000,0x0,0x0,0x1800000,0x1800000,0x20000000,0x40000000,0x80000000,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,};
+      jj_la1_0 = new int[] {0x79ffe0,0x79ffe1,0x79ffe0,0x0,0x0,0x0,0x19fbc0,0x19fbc0,0x0,0x600,0x0,0x0,0x0,0x0,0x0,0x0,0x1000,0x0,0x0,0x1800000,0x1800000,0x20000000,0x40000000,0x80000000,0x0,0x40000,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x4000000,0x0,0x4000000,0x0,0x0,0x0,0x0,0x0,0x0,0x0,};
    }
    private static void jj_la1_1() {
-      jj_la1_1 = new int[] {0x0,0x40000000,0x0,0x1000000,0x3000000,0x3000000,0x1000000,0x1000000,0x31000000,0x0,0x398e00,0x7000,0x70000,0x398e00,0x4,0x4,0xc00000,0x30000000,0x30000000,0x0,0x0,0x0,0x0,0x0,0x31000004,0x31000000,0x31000004,0x30000000,0x4,0x1000100,0x30000000,0x31000100,0x4,0x4,0x30000000,0x31000000,};
+      jj_la1_1 = new int[] {0x90000000,0x90000000,0x90000000,0x80000000,0x80000000,0x80000000,0xb0000000,0xb0000000,0x80000000,0x0,0x398e000,0x70000,0x700000,0x398e000,0x4,0x4,0xc000000,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x1,0xa0000000,0x80000004,0x80000000,0x80000004,0x0,0x80001000,0x0,0x80001000,0x4,0x4,0x4,0x0,0x80000000,0x3,0x3,0x80000000,0x8,0x80000000,0x8,0x80000010,0x40,0x40,0x80,0x100,0xfc0,};
+   }
+   private static void jj_la1_2() {
+      jj_la1_2 = new int[] {0x0,0x20,0x0,0x0,0x1,0x1,0x0,0x0,0x18,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x18,0x18,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x18,0x18,0x18,0x18,0x1,0x18,0x19,0x0,0x0,0x0,0x18,0x18,0x0,0x0,0x0,0x0,0x0,0x0,0x11,0x0,0x0,0x0,0x0,0x0,};
    }
   final private JJCalls[] jj_2_rtns = new JJCalls[1];
   private boolean jj_rescan = false;
@@ -1041,7 +1401,7 @@ public class Parser implements ParserConstants {
     token = new Token();
     jj_ntk = -1;
     jj_gen = 0;
-    for (int i = 0; i < 36; i++) jj_la1[i] = -1;
+    for (int i = 0; i < 50; i++) jj_la1[i] = -1;
     for (int i = 0; i < jj_2_rtns.length; i++) jj_2_rtns[i] = new JJCalls();
   }
 
@@ -1054,7 +1414,7 @@ public class Parser implements ParserConstants {
     token = new Token();
     jj_ntk = -1;
     jj_gen = 0;
-    for (int i = 0; i < 36; i++) jj_la1[i] = -1;
+    for (int i = 0; i < 50; i++) jj_la1[i] = -1;
     for (int i = 0; i < jj_2_rtns.length; i++) jj_2_rtns[i] = new JJCalls();
   }
 
@@ -1064,7 +1424,7 @@ public class Parser implements ParserConstants {
     token = new Token();
     jj_ntk = -1;
     jj_gen = 0;
-    for (int i = 0; i < 36; i++) jj_la1[i] = -1;
+    for (int i = 0; i < 50; i++) jj_la1[i] = -1;
     for (int i = 0; i < jj_2_rtns.length; i++) jj_2_rtns[i] = new JJCalls();
   }
 
@@ -1074,7 +1434,7 @@ public class Parser implements ParserConstants {
     token = new Token();
     jj_ntk = -1;
     jj_gen = 0;
-    for (int i = 0; i < 36; i++) jj_la1[i] = -1;
+    for (int i = 0; i < 50; i++) jj_la1[i] = -1;
     for (int i = 0; i < jj_2_rtns.length; i++) jj_2_rtns[i] = new JJCalls();
   }
 
@@ -1083,7 +1443,7 @@ public class Parser implements ParserConstants {
     token = new Token();
     jj_ntk = -1;
     jj_gen = 0;
-    for (int i = 0; i < 36; i++) jj_la1[i] = -1;
+    for (int i = 0; i < 50; i++) jj_la1[i] = -1;
     for (int i = 0; i < jj_2_rtns.length; i++) jj_2_rtns[i] = new JJCalls();
   }
 
@@ -1092,7 +1452,7 @@ public class Parser implements ParserConstants {
     token = new Token();
     jj_ntk = -1;
     jj_gen = 0;
-    for (int i = 0; i < 36; i++) jj_la1[i] = -1;
+    for (int i = 0; i < 50; i++) jj_la1[i] = -1;
     for (int i = 0; i < jj_2_rtns.length; i++) jj_2_rtns[i] = new JJCalls();
   }
 
@@ -1203,15 +1563,15 @@ public class Parser implements ParserConstants {
 
   public ParseException generateParseException() {
     jj_expentries.removeAllElements();
-    boolean[] la1tokens = new boolean[63];
-    for (int i = 0; i < 63; i++) {
+    boolean[] la1tokens = new boolean[70];
+    for (int i = 0; i < 70; i++) {
       la1tokens[i] = false;
     }
     if (jj_kind >= 0) {
       la1tokens[jj_kind] = true;
       jj_kind = -1;
     }
-    for (int i = 0; i < 36; i++) {
+    for (int i = 0; i < 50; i++) {
       if (jj_la1[i] == jj_gen) {
         for (int j = 0; j < 32; j++) {
           if ((jj_la1_0[i] & (1<<j)) != 0) {
@@ -1220,10 +1580,13 @@ public class Parser implements ParserConstants {
           if ((jj_la1_1[i] & (1<<j)) != 0) {
             la1tokens[32+j] = true;
           }
+          if ((jj_la1_2[i] & (1<<j)) != 0) {
+            la1tokens[64+j] = true;
+          }
         }
       }
     }
-    for (int i = 0; i < 63; i++) {
+    for (int i = 0; i < 70; i++) {
       if (la1tokens[i]) {
         jj_expentry = new int[1];
         jj_expentry[0] = i;

+ 38 - 24
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/ParserConstants.java

@@ -38,29 +38,36 @@ public interface ParserConstants {
   int LPAREN = 36;
   int RPAREN = 37;
   int EQUALS = 38;
-  int NOTEQUAL = 39;
-  int ASTERISK = 40;
-  int MAX_VERSIONS = 41;
-  int MAX_LENGTH = 42;
-  int COMPRESSION = 43;
-  int NONE = 44;
-  int BLOCK = 45;
-  int RECORD = 46;
-  int IN_MEMORY = 47;
-  int BLOOMFILTER = 48;
-  int COUNTING_BLOOMFILTER = 49;
-  int RETOUCHED_BLOOMFILTER = 50;
-  int VECTOR_SIZE = 51;
-  int NUM_HASH = 52;
-  int NUM_ENTRIES = 53;
-  int ADD = 54;
-  int CHANGE = 55;
-  int ID = 56;
-  int INTEGER_LITERAL = 57;
-  int FLOATING_POINT_LITERAL = 58;
-  int EXPONENT = 59;
-  int QUOTED_IDENTIFIER = 60;
-  int STRING_LITERAL = 61;
+  int LCOMP = 39;
+  int RCOMP = 40;
+  int NOT = 41;
+  int IN = 42;
+  int NOTEQUAL = 43;
+  int ASTERISK = 44;
+  int MAX_VERSIONS = 45;
+  int MAX_LENGTH = 46;
+  int COMPRESSION = 47;
+  int NONE = 48;
+  int BLOCK = 49;
+  int RECORD = 50;
+  int IN_MEMORY = 51;
+  int BLOOMFILTER = 52;
+  int COUNTING_BLOOMFILTER = 53;
+  int RETOUCHED_BLOOMFILTER = 54;
+  int VECTOR_SIZE = 55;
+  int NUM_HASH = 56;
+  int NUM_ENTRIES = 57;
+  int ADD = 58;
+  int CHANGE = 59;
+  int SAVE = 60;
+  int GROUP = 61;
+  int BY = 62;
+  int ID = 63;
+  int INTEGER_LITERAL = 64;
+  int FLOATING_POINT_LITERAL = 65;
+  int EXPONENT = 66;
+  int QUOTED_IDENTIFIER = 67;
+  int STRING_LITERAL = 68;
 
   int DEFAULT = 0;
 
@@ -104,7 +111,11 @@ public interface ParserConstants {
     "\"(\"",
     "\")\"",
     "\"=\"",
-    "\"<>\"",
+    "\">\"",
+    "\"<\"",
+    "\"not\"",
+    "\"in\"",
+    "\"!=\"",
     "\"*\"",
     "\"max_versions\"",
     "\"max_length\"",
@@ -121,6 +132,9 @@ public interface ParserConstants {
     "\"num_entries\"",
     "\"add\"",
     "\"change\"",
+    "\"save\"",
+    "\"group\"",
+    "\"by\"",
     "<ID>",
     "<INTEGER_LITERAL>",
     "<FLOATING_POINT_LITERAL>",

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


+ 74 - 0
src/contrib/hbase/src/test/org/apache/hadoop/hbase/shell/TestSubstitutionVariables.java

@@ -0,0 +1,74 @@
+/**
+ * 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.OutputStreamWriter;
+import java.io.UnsupportedEncodingException;
+import java.io.Writer;
+
+import junit.framework.TestCase;
+import junit.framework.TestSuite;
+import junit.textui.TestRunner;
+
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.shell.algebra.Constants;
+import org.apache.hadoop.hbase.shell.algebra.TestBooleanCondition;
+
+/**
+ * Binding variables, substitution variables test
+ */
+public class TestSubstitutionVariables extends TestCase {
+  
+  private String TABLE_NAME = "table_name";
+  private String SUBSTITUTION_VARIABLE = "A";
+  static HBaseConfiguration conf = new HBaseConfiguration();
+  
+  public void testSubstitution() {
+    SubstituteCommand substitute = new SubstituteCommand(null);
+    
+    substitute.setKey(SUBSTITUTION_VARIABLE);
+    substitute.setInput(TABLE_NAME);
+    substitute.execute(conf);
+    
+    VariableRef ref = VariablesPool.get(SUBSTITUTION_VARIABLE).get(null);
+    assertTrue(ref.getArgument().equals(TABLE_NAME));
+  }
+  
+  public void testCombinedQueries() throws UnsupportedEncodingException {
+    Writer out = new OutputStreamWriter(System.out, "UTF-8");
+    SubstituteCommand substitute = new SubstituteCommand(out);
+    
+    substitute.setKey(SUBSTITUTION_VARIABLE);
+    substitute.setInput(TABLE_NAME);
+    substitute.execute(conf);
+    
+    substitute = new SubstituteCommand(out);
+    substitute.setKey("B");
+    substitute.setChainKey(SUBSTITUTION_VARIABLE);
+    substitute.setOperation(Constants.RELATIONAL_SELECTION);
+    substitute.setCondition(TestBooleanCondition.EXPRESSION_OR);
+    substitute.execute(conf);
+  }
+  
+  public static void main(String[] args) {
+    TestRunner.run(new TestSuite(TestSubstitutionVariables.class));
+  }
+  
+}

+ 84 - 0
src/contrib/hbase/src/test/org/apache/hadoop/hbase/shell/algebra/TestBooleanCondition.java

@@ -0,0 +1,84 @@
+/**
+ * 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.algebra;
+
+import java.io.UnsupportedEncodingException;
+
+import junit.framework.TestCase;
+import junit.framework.TestSuite;
+import junit.textui.TestRunner;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.shell.algebra.generated.ExpressionParser;
+import org.apache.hadoop.hbase.shell.algebra.generated.ParseException;
+import org.apache.hadoop.io.MapWritable;
+import org.apache.hadoop.io.Text;
+
+/**
+ * Test boolean expression
+ */
+public class TestBooleanCondition extends TestCase {
+  ExpressionParser expression;
+  public static String EXPRESSION_OR = "key:2 = value2 OR key:3 = value1";
+  public static String EXPRESSION_AND = "key:2 = value2 AND key:3 = value1";
+  public static String EXPRESSION = "key:3 > 1";
+  public static String TERM_FILTER = "key:4 > 100 AND key:4 !! 110|120|130|140|150";
+
+  Text[] keys = { new Text("key:1"), new Text("key:2"), new Text("key:3"), new Text("key:4") };
+
+  ImmutableBytesWritable[] values = {
+      new ImmutableBytesWritable("value1".getBytes()),
+      new ImmutableBytesWritable("value2".getBytes()),
+      new ImmutableBytesWritable("3".getBytes()),
+      new ImmutableBytesWritable("150".getBytes())
+  };
+
+  public void testCheckConstraints() throws UnsupportedEncodingException {
+    MapWritable data = new MapWritable();
+    for (int i = 0; i < keys.length; i++) {
+      data.put(keys[i], values[i]);
+    }
+
+    try {
+      expression = new ExpressionParser(EXPRESSION_OR);
+      expression.booleanExpressionParse();
+      assertTrue(expression.checkConstraints(data));
+
+      expression = new ExpressionParser(EXPRESSION_AND);
+      expression.booleanExpressionParse();
+      assertFalse(expression.checkConstraints(data));
+
+      expression = new ExpressionParser(EXPRESSION);
+      expression.booleanExpressionParse();
+      assertTrue(expression.checkConstraints(data));
+      
+      expression = new ExpressionParser(TERM_FILTER);
+      expression.booleanExpressionParse();
+      assertFalse(expression.checkConstraints(data));
+    } catch (ParseException e) {
+      e.printStackTrace();
+    }
+  }
+
+  public static void main(String[] args) {
+    TestRunner.run(new TestSuite(TestBooleanCondition.class));
+  }
+
+}

+ 250 - 0
src/contrib/hbase/src/test/org/apache/hadoop/hbase/shell/algebra/TestBooleanTermFilter.java

@@ -0,0 +1,250 @@
+/**
+ * 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.algebra;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.Random;
+import java.util.TreeMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.dfs.MiniDFSCluster;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseAdmin;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+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.hbase.MasterNotRunningException;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.MultiRegionTable;
+import org.apache.hadoop.hbase.mapred.IdentityTableReduce;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.MiniMRCluster;
+
+public class TestBooleanTermFilter extends MultiRegionTable {
+  @SuppressWarnings("hiding")
+  private static final Log LOG = LogFactory.getLog(TestBooleanTermFilter.class
+      .getName());
+
+  static final String INPUT_TABLE = "test_table";
+  static final String OUTPUT_TABLE = "result_table";
+  static final Text COUNT_COLUMNFAMILY = new Text("count:");
+  static final Text RANDOMINT_COLUMNFAMILY = new Text("randomInt:");
+  static final String GROUP_COLUMN_FAMILIES = "count: randomInt:";
+  static final String BOOLEAN_TERM = "randomInt: > 100 AND count: <= 100 AND randomInt: !! 110|120|130|140|150";
+  private MiniDFSCluster dfsCluster = null;
+  private FileSystem fs;
+  private Path dir;
+  private MiniHBaseCluster hCluster = null;
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    conf.setLong("hbase.hregion.max.filesize", 256 * 1024);
+    dfsCluster = new MiniDFSCluster(conf, 1, true, (String[]) null);
+    try {
+      fs = dfsCluster.getFileSystem();
+      dir = new Path("/hbase");
+      fs.mkdirs(dir);
+      // Start up HBase cluster
+      hCluster = new MiniHBaseCluster(conf, 1, dfsCluster);
+    } catch (Exception e) {
+      if (dfsCluster != null) {
+        dfsCluster.shutdown();
+        dfsCluster = null;
+      }
+      throw e;
+    }
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void tearDown() throws Exception {
+    super.tearDown();
+    if (hCluster != null) {
+      hCluster.shutdown();
+    }
+
+    if (dfsCluster != null) {
+      dfsCluster.shutdown();
+    }
+
+    if (fs != null) {
+      try {
+        fs.close();
+      } catch (IOException e) {
+        LOG.info("During tear down got a " + e.getMessage());
+      }
+    }
+  }
+
+  public void testBooleanFilterMapReduce() {
+    try {
+      HTableDescriptor desc = new HTableDescriptor(INPUT_TABLE);
+      String[] columns = GROUP_COLUMN_FAMILIES.split(" ");
+      for (int i = 0; i < columns.length; i++) {
+        desc.addFamily(new HColumnDescriptor(columns[i]));
+      }
+      HBaseAdmin admin = new HBaseAdmin(this.conf);
+      admin.createTable(desc);
+
+      // insert random data into the input table
+      HTable table = new HTable(conf, new Text(INPUT_TABLE));
+      Random oRandom = new Random();
+
+      for (int j = 0; j < 200; j++) {
+        int i = oRandom.nextInt(200) + 1;
+
+        long lockid = table.startUpdate(new Text("rowKey" + j));
+        table.put(lockid, COUNT_COLUMNFAMILY, Integer.toString(j).getBytes(
+            HConstants.UTF8_ENCODING));
+        table.put(lockid, RANDOMINT_COLUMNFAMILY, Integer.toString(i).getBytes(
+            HConstants.UTF8_ENCODING));
+        table.commit(lockid, System.currentTimeMillis());
+      }
+
+      long lockid = table.startUpdate(new Text("rowKey2001"));
+      table.put(lockid, COUNT_COLUMNFAMILY, "12"
+          .getBytes(HConstants.UTF8_ENCODING));
+      table.put(lockid, RANDOMINT_COLUMNFAMILY, "120"
+          .getBytes(HConstants.UTF8_ENCODING));
+      table.commit(lockid, System.currentTimeMillis());
+
+    } catch (MasterNotRunningException e) {
+      e.printStackTrace();
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+
+    try {
+      HTableDescriptor output = new HTableDescriptor(OUTPUT_TABLE);
+      String[] columns = GROUP_COLUMN_FAMILIES.split(" ");
+      for (int i = 0; i < columns.length; i++) {
+        output.addFamily(new HColumnDescriptor(columns[i]));
+      }
+      // create output table
+      HBaseAdmin admin = new HBaseAdmin(this.conf);
+      admin.createTable(output);
+    } catch (MasterNotRunningException e) {
+      e.printStackTrace();
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+
+    MiniMRCluster mrCluster = null;
+    try {
+      mrCluster = new MiniMRCluster(2, fs.getUri().toString(), 1);
+
+      JobConf jobConf = new JobConf(conf, TestBooleanTermFilter.class);
+      jobConf.setJobName("process boolean term filter mapreduce");
+      jobConf.setNumMapTasks(2);
+      jobConf.setNumReduceTasks(1);
+
+      IdentityFilterMap.initJob(INPUT_TABLE, GROUP_COLUMN_FAMILIES,
+          BOOLEAN_TERM, IdentityFilterMap.class, jobConf);
+
+      IdentityTableReduce.initJob(OUTPUT_TABLE, IdentityTableReduce.class,
+          jobConf);
+
+      JobClient.runJob(jobConf);
+
+    } catch (IOException e) {
+      e.printStackTrace();
+    } finally {
+      mrCluster.shutdown();
+    }
+
+    try {
+      verify(conf, OUTPUT_TABLE);
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+  }
+
+  /**
+   * Check the filtered value
+   * 
+   * @param conf
+   * @param outputTable
+   * @throws IOException
+   */
+  private void verify(HBaseConfiguration conf, String outputTable)
+      throws IOException {
+    HTable table = new HTable(conf, new Text(outputTable));
+    Text[] columns = { COUNT_COLUMNFAMILY, RANDOMINT_COLUMNFAMILY };
+    HScannerInterface scanner = table.obtainScanner(columns,
+        HConstants.EMPTY_START_ROW);
+
+    try {
+      HStoreKey key = new HStoreKey();
+      TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
+
+      int i = 0;
+      while (scanner.next(key, results)) {
+        for (Map.Entry<Text, byte[]> e : results.entrySet()) {
+          if (e.getKey().equals(COUNT_COLUMNFAMILY)) {
+            if (Integer.parseInt(new String(e.getValue())) > 100) {
+              LOG.info("result_table.count: " + new String(e.getValue()));
+            }
+            assertTrue((Integer.parseInt(new String(e.getValue())) <= 100));
+          } else {
+            if (Integer.parseInt(new String(e.getValue())) <= 100
+                || !checkNotInList(Integer.parseInt(new String(e.getValue())))) {
+              LOG.info("result_table.randomInt: " + new String(e.getValue()));
+            }
+            assertTrue((Integer.parseInt(new String(e.getValue())) > 100 && checkNotInList(Integer
+                .parseInt(new String(e.getValue())))));
+          }
+          i++;
+        }
+      }
+
+      assertTrue(i > 0);
+      if (i <= 0) {
+        LOG.info("result_table.rowNumber: " + i);
+      }
+
+    } finally {
+      scanner.close();
+    }
+
+  }
+
+  /**
+   * Check 'NOT IN' filter-list
+   */
+  private boolean checkNotInList(int parseInt) {
+    return (parseInt != 110 && parseInt != 120 && parseInt != 130
+        && parseInt != 140 && parseInt != 150) ? true : false;
+  }
+}

+ 45 - 0
src/contrib/hbase/src/test/org/apache/hadoop/hbase/shell/algebra/TestJoinCondition.java

@@ -0,0 +1,45 @@
+/**
+ * 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.algebra;
+
+import junit.framework.TestCase;
+import junit.framework.TestSuite;
+import junit.textui.TestRunner;
+
+import org.apache.hadoop.hbase.shell.algebra.generated.ExpressionParser;
+import org.apache.hadoop.hbase.shell.algebra.generated.ParseException;
+
+public class TestJoinCondition extends TestCase {
+  ExpressionParser expressionParser;
+  public static String EXPRESSION = "a.ROW = b.size BOOL a.length = b.length AND a.name = edward";
+  
+  public void testJoinCondition() {
+    expressionParser = new ExpressionParser(EXPRESSION);
+    try {
+      expressionParser.joinExpressionParse();
+    } catch (ParseException e) {
+      e.printStackTrace();
+    }
+  }
+
+  public static void main(String[] args) {
+    TestRunner.run(new TestSuite(TestJoinCondition.class));
+  }
+}

+ 238 - 0
src/contrib/hbase/src/test/org/apache/hadoop/hbase/shell/algebra/TestTableJoinMapReduce.java

@@ -0,0 +1,238 @@
+/**
+ * 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.algebra;
+
+import java.io.IOException;
+import java.util.TreeMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.dfs.MiniDFSCluster;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseAdmin;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+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.hbase.MasterNotRunningException;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.MultiRegionTable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.MiniMRCluster;
+
+public class TestTableJoinMapReduce extends MultiRegionTable {
+  @SuppressWarnings("hiding")
+  private static final Log LOG = LogFactory.getLog(TestTableJoinMapReduce.class
+      .getName());
+  static final String FIRST_RELATION = "r1";
+  static final String SECOND_RELATION = "r2";
+  static final String JOIN_EXPRESSION = "r1.c: = r2.ROW BOOL ";
+  static final String FIRST_COLUMNS = "a: b: c:";
+  static final String SECOND_COLUMNS = "d: e:";
+  static final String OUTPUT_TABLE = "result_table";
+  private MiniDFSCluster dfsCluster = null;
+  private FileSystem fs;
+  private Path dir;
+  private MiniHBaseCluster hCluster = null;
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    conf.setLong("hbase.hregion.max.filesize", 256 * 1024);
+    dfsCluster = new MiniDFSCluster(conf, 1, true, (String[]) null);
+    try {
+      fs = dfsCluster.getFileSystem();
+      dir = new Path("/hbase");
+      fs.mkdirs(dir);
+      // Start up HBase cluster
+      hCluster = new MiniHBaseCluster(conf, 1, dfsCluster);
+    } catch (Exception e) {
+      if (dfsCluster != null) {
+        dfsCluster.shutdown();
+        dfsCluster = null;
+      }
+      throw e;
+    }
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void tearDown() throws Exception {
+    super.tearDown();
+    if (hCluster != null) {
+      hCluster.shutdown();
+    }
+
+    if (dfsCluster != null) {
+      dfsCluster.shutdown();
+    }
+
+    if (fs != null) {
+      try {
+        fs.close();
+      } catch (IOException e) {
+        LOG.info("During tear down got a " + e.getMessage());
+      }
+    }
+  }
+
+  public void testTableJoinMapReduce() {
+    try {
+      HTableDescriptor desc = new HTableDescriptor(FIRST_RELATION);
+      String[] columns = FIRST_COLUMNS.split(" ");
+      for (int i = 0; i < columns.length; i++) {
+        desc.addFamily(new HColumnDescriptor(columns[i]));
+      }
+      HBaseAdmin admin = new HBaseAdmin(this.conf);
+      admin.createTable(desc);
+
+      // insert random data into the input table
+      HTable table = new HTable(conf, new Text(FIRST_RELATION));
+      for (int j = 0; j < 5; j++) {
+        long lockid = table.startUpdate(new Text("rowKey" + j));
+        table.put(lockid, new Text("a:"), Integer.toString(j).getBytes(
+            HConstants.UTF8_ENCODING));
+        table.put(lockid, new Text("b:"), Integer.toString(j).getBytes(
+            HConstants.UTF8_ENCODING));
+        table.put(lockid, new Text("c:"), ("joinKey-" + Integer.toString(j))
+            .getBytes(HConstants.UTF8_ENCODING));
+        table.commit(lockid, System.currentTimeMillis());
+      }
+
+    } catch (MasterNotRunningException e) {
+      e.printStackTrace();
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+
+    try {
+      HTableDescriptor desc = new HTableDescriptor(SECOND_RELATION);
+      String[] columns = SECOND_COLUMNS.split(" ");
+      for (int i = 0; i < columns.length; i++) {
+        desc.addFamily(new HColumnDescriptor(columns[i]));
+      }
+      HBaseAdmin admin = new HBaseAdmin(this.conf);
+      admin.createTable(desc);
+
+      // insert random data into the input table
+      HTable table = new HTable(conf, new Text(SECOND_RELATION));
+
+      for (int j = 0; j < 3; j++) {
+        long lockid = table.startUpdate(new Text("joinKey-" + j));
+        table.put(lockid, new Text("d:"), ("s-" + Integer.toString(j))
+            .getBytes(HConstants.UTF8_ENCODING));
+        table.put(lockid, new Text("e:"), ("s-" + Integer.toString(j))
+            .getBytes(HConstants.UTF8_ENCODING));
+        table.commit(lockid, System.currentTimeMillis());
+      }
+
+    } catch (MasterNotRunningException e) {
+      e.printStackTrace();
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+
+    try {
+      HTableDescriptor output = new HTableDescriptor(OUTPUT_TABLE);
+      String[] columns = (FIRST_COLUMNS + " " + SECOND_COLUMNS).split(" ");
+      for (int i = 0; i < columns.length; i++) {
+        output.addFamily(new HColumnDescriptor(columns[i]));
+      }
+      // create output table
+      HBaseAdmin admin = new HBaseAdmin(this.conf);
+      admin.createTable(output);
+    } catch (MasterNotRunningException e) {
+      e.printStackTrace();
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+
+    MiniMRCluster mrCluster = null;
+    try {
+      mrCluster = new MiniMRCluster(2, fs.getUri().toString(), 1);
+
+      JobConf jobConf = new JobConf(conf, TestTableJoinMapReduce.class);
+      jobConf.setJobName("process table join mapreduce");
+      jobConf.setNumMapTasks(2);
+      jobConf.setNumReduceTasks(1);
+
+      IndexJoinMap.initJob(FIRST_RELATION, SECOND_RELATION, FIRST_COLUMNS,
+          SECOND_COLUMNS, JOIN_EXPRESSION, IndexJoinMap.class, jobConf);
+      IndexJoinReduce.initJob(OUTPUT_TABLE, IndexJoinReduce.class, jobConf);
+
+      JobClient.runJob(jobConf);
+
+    } catch (IOException e) {
+      e.printStackTrace();
+    } finally {
+      mrCluster.shutdown();
+    }
+
+    try {
+      verify(conf, OUTPUT_TABLE);
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+  }
+
+  /**
+   * Check the result table lattice size.
+   * 
+   * @param conf
+   * @param outputTable
+   * @throws IOException
+   */
+  private void verify(HBaseConfiguration conf, String outputTable)
+      throws IOException {
+    HTable table = new HTable(conf, new Text(outputTable));
+    Text[] columns = { new Text("a:"), new Text("b:"), new Text("c:"),
+        new Text("d:"), new Text("e:") };
+    HScannerInterface scanner = table.obtainScanner(columns,
+        HConstants.EMPTY_START_ROW);
+
+    try {
+      HStoreKey key = new HStoreKey();
+      TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
+
+      int i = 0;
+      while (scanner.next(key, results)) {
+        assertTrue(results.keySet().size() == 5);
+        LOG.info("result_table.column.size: " + results.keySet().size());
+        i++;
+      }
+      assertTrue(i == 3);
+      LOG.info("result_table.row.count: " + i);
+    } finally {
+      scanner.close();
+    }
+
+  }
+}

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