瀏覽代碼

HADOOP-2506 Remove the algebra package

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk@608289 13f79535-47bb-0310-9956-ffa450edef68
Michael Stack 17 年之前
父節點
當前提交
1f636e4c1f
共有 36 個文件被更改,包括 297 次插入5828 次删除
  1. 2 1
      src/contrib/hbase/CHANGES.txt
  2. 3 247
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/HBaseShell.jj
  3. 0 33
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/HelpCommand.java
  4. 0 114
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/SaveCommand.java
  5. 13 40
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/ShowCommand.java
  6. 0 79
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/SubstituteCommand.java
  7. 0 52
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/VariableRef.java
  8. 0 52
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/VariablesPool.java
  9. 0 36
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/Constants.java
  10. 0 50
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/DuplicateTable.java
  11. 0 559
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/ExpressionParser.jj
  12. 0 83
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/GroupingFilterMap.java
  13. 0 78
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/IdentityFilterMap.java
  14. 0 76
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/IndexJoin.java
  15. 0 108
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/IndexJoinMap.java
  16. 0 49
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/IndexJoinReduce.java
  17. 0 41
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/Operation.java
  18. 0 88
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/OperationEvaluator.java
  19. 0 54
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/Projection.java
  20. 0 175
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/RelationalOperation.java
  21. 0 74
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/Selection.java
  22. 0 774
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/generated/ExpressionParser.java
  23. 0 56
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/generated/ExpressionParserConstants.java
  24. 0 658
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/generated/ExpressionParserTokenManager.java
  25. 0 192
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/generated/ParseException.java
  26. 0 439
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/generated/SimpleCharStream.java
  27. 0 81
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/generated/Token.java
  28. 0 133
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/generated/TokenMgrError.java
  29. 44 417
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/Parser.java
  30. 30 42
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/ParserConstants.java
  31. 205 244
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/ParserTokenManager.java
  32. 0 74
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/shell/TestSubstitutionVariables.java
  33. 0 84
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/shell/algebra/TestBooleanCondition.java
  34. 0 250
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/shell/algebra/TestBooleanTermFilter.java
  35. 0 45
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/shell/algebra/TestJoinCondition.java
  36. 0 250
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/shell/algebra/TestTableJoinMapReduce.java

+ 2 - 1
src/contrib/hbase/CHANGES.txt

@@ -5,6 +5,8 @@ Trunk (unreleased changes)
   INCOMPATIBLE CHANGES
    HADOOP-2056 A table with row keys containing colon fails to split regions
    HADOOP-2079 Fix generated HLog, HRegion names
+   HADOOP-2495 Minor performance improvements: Slim-down BatchOperation, etc. 
+   HADOOP-2506 Remove the algebra package
 
   NEW FEATURES
     HADOOP-2061 Add new Base64 dialects
@@ -22,7 +24,6 @@ Trunk (unreleased changes)
    HADOOP-2479 Save on number of Text object creations
    HADOOP-2485 Make mapfile index interval configurable (Set default to 32
                instead of 128)
-   HADOOP-2495 Minor performance improvements: Slim-down BatchOperation, etc. 
 
   BUG FIXES
    HADOOP-2059 In tests, exceptions in min dfs shutdown should not fail test

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

@@ -101,7 +101,6 @@ TOKEN: /** for HQL statements */
    | <AND: "and">
    | <OR:  "or">
    | <COMMA: ",">
-   | <DOT: ".">
    | <LPAREN: "(">
    | <RPAREN: ")">
    | <EQUALS: "=">
@@ -126,20 +125,11 @@ TOKEN: /** for HQL statements */
    | <NUM_ENTRIES: "num_entries">
    | <ADD: "add">
    | <CHANGE: "change">
-   | <META_TABLE: ".META.">
-   | <ROOT_TABLE: "-ROOT-">
-}
-
-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>)?
@@ -190,8 +180,6 @@ Command cmdStatement() :
       | cmd = clearCommand()
       | cmd = fsCommand()
       | cmd = jarCommand()
-      | cmd = substituteCommand()
-      | cmd = saveCommand()
   )
    {
        return cmd;
@@ -235,7 +223,7 @@ JarCommand jarCommand() :
 {
   <JAR>
   (
-      ( t=<ID> | t=<INTEGER_LITERAL> | t=<DOT> | t=<FLOATING_POINT_LITERAL> )
+      ( t=<ID> | t=<INTEGER_LITERAL> | t=<FLOATING_POINT_LITERAL> )
       { path += t.image.toString(); }
   )*
   t=<JAR>
@@ -274,8 +262,6 @@ HelpCommand helpCommand() :
        | t=<CLEAR>
        | t=<FS>
        | t=<JAR>
-       | t=<GROUP>
-       | t=<SAVE>
        | t=<ID>
      ) { argument = t.image.toString(); }
    ]
@@ -650,96 +636,6 @@ ClearCommand clearCommand() :
    }
 }
 
-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() :
 {
   List<String> values = new ArrayList<String>();
@@ -872,153 +768,13 @@ String identifier() :
 }
 {
   (
-     ( t=<ID> | t=<META_TABLE> | t=<ROOT_TABLE> )
+     t=<ID>
      { return t.image.toString(); }
    | ( 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;

+ 0 - 33
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/HelpCommand.java

@@ -121,39 +121,6 @@ public class HelpCommand extends BasicCommand {
 
     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;
   }
 

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

@@ -1,114 +0,0 @@
-/**
- * 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;
-  }
-}

+ 13 - 40
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/ShowCommand.java

@@ -45,7 +45,8 @@ public class ShowCommand extends BasicCommand {
     this(o, f, null);
   }
 
-  public ShowCommand(final Writer o, final TableFormatter f, final String argument) {
+  public ShowCommand(final Writer o, final TableFormatter f,
+      final String argument) {
     super(o);
     this.formatter = f;
     this.command = argument;
@@ -58,46 +59,18 @@ public class ShowCommand extends BasicCommand {
     try {
       HBaseAdmin admin = new HBaseAdmin(conf);
       int tableLength = 0;
-      if ("tables".equals(this.command)) {
-        HTableDescriptor[] tables = admin.listTables();
-        tableLength = tables.length;
-        if (tableLength == 0) {
-          return new ReturnMsg(0, "No tables found.");
-        }
-        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.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);
+      HTableDescriptor[] tables = admin.listTables();
+      tableLength = tables.length;
+      if (tableLength == 0) {
+        return new ReturnMsg(0, "No tables found.");
+      }
+      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.footer();
+      return new ReturnMsg(1, tableLength + " table(s) in set.");
     } catch (IOException e) {
       return new ReturnMsg(0, "error msg : " + e.toString());
     }

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

@@ -1,79 +0,0 @@
-/**
- * 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;
-  }
-}

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

@@ -1,52 +0,0 @@
-/**
- * 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;
-  }
-}

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

@@ -1,52 +0,0 @@
-/**
- * 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);
-  }
-}

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

@@ -1,36 +0,0 @@
-/**
- * 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";
-}

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

@@ -1,50 +0,0 @@
-/**
- * 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;
-  }
-}

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

@@ -1,559 +0,0 @@
-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;
-  }
-}

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

@@ -1,83 +0,0 @@
-/**
- * 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);
-      }
-    }
-  }
-}

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

@@ -1,78 +0,0 @@
-/**
- * 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);
-    }
-  }
-}

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

@@ -1,76 +0,0 @@
-/**
- * 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;
-  }
-}

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

@@ -1,108 +0,0 @@
-/**
- * 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);
-    }
-  }
-}

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

@@ -1,49 +0,0 @@
-/**
- * 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);
-    }
-  }
-}

+ 0 - 41
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/algebra/Operation.java

@@ -1,41 +0,0 @@
-/**
- * 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;
-}

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

@@ -1,88 +0,0 @@
-/**
- * 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();
-  }
-}

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

@@ -1,54 +0,0 @@
-/**
- * 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;
-  }
-}

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

@@ -1,175 +0,0 @@
-/**
- * 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;
-  }
-}

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

@@ -1,74 +0,0 @@
-/**
- * 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;
-  }
-}

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

@@ -1,774 +0,0 @@
-/* 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() {
-  }
-
-}

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

@@ -1,56 +0,0 @@
-/* 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>",
-  };
-
-}

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

@@ -1,658 +0,0 @@
-/* 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.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);
-  }
-}
-
-}

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

@@ -1,192 +0,0 @@
-/* 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();
-   }
-
-}

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

@@ -1,439 +0,0 @@
-/* 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];
-  }
-
-}

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

@@ -1,81 +0,0 @@
-/* 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();
-     }
-  }
-
-}

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

@@ -1,133 +0,0 @@
-/* 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);
-   }
-}

+ 44 - 417
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/Parser.java

@@ -75,9 +75,7 @@ public class Parser implements ParserConstants {
     case SELECT:
     case ENABLE:
     case DISABLE:
-    case SAVE:
-    case ID:
-    case 71:
+    case 65:
       switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
       case HELP:
       case ALTER:
@@ -95,15 +93,13 @@ 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(71);
+      jj_consume_token(65);
       break;
     case 0:
       jj_consume_token(0);
@@ -166,12 +162,6 @@ 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);
@@ -220,7 +210,6 @@ public class Parser implements ParserConstants {
     label_2:
     while (true) {
       switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
-      case DOT:
       case ID:
       case INTEGER_LITERAL:
       case FLOATING_POINT_LITERAL:
@@ -237,9 +226,6 @@ public class Parser implements ParserConstants {
       case INTEGER_LITERAL:
         t = jj_consume_token(INTEGER_LITERAL);
         break;
-      case DOT:
-        t = jj_consume_token(DOT);
-        break;
       case FLOATING_POINT_LITERAL:
         t = jj_consume_token(FLOATING_POINT_LITERAL);
         break;
@@ -304,8 +290,6 @@ 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:
@@ -344,12 +328,6 @@ 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;
@@ -374,8 +352,6 @@ public class Parser implements ParserConstants {
   String argument = null;
     jj_consume_token(SHOW);
     switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
-    case META_TABLE:
-    case ROOT_TABLE:
     case ID:
     case QUOTED_IDENTIFIER:
     case STRING_LITERAL:
@@ -808,104 +784,6 @@ public class Parser implements ParserConstants {
     throw new Error("Missing return statement in function");
   }
 
-  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[27] = 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[28] = 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;
@@ -922,7 +800,7 @@ public class Parser implements ParserConstants {
         ;
         break;
       default:
-        jj_la1[29] = jj_gen;
+        jj_la1[27] = jj_gen;
         break label_7;
       }
       switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
@@ -945,14 +823,14 @@ public class Parser implements ParserConstants {
           jj_consume_token(QUOTED_IDENTIFIER);
           break;
         default:
-          jj_la1[30] = jj_gen;
+          jj_la1[28] = jj_gen;
           jj_consume_token(-1);
           throw new ParseException();
         }
        values.removeAll(values);
         break;
       default:
-        jj_la1[31] = jj_gen;
+        jj_la1[29] = jj_gen;
         jj_consume_token(-1);
         throw new ParseException();
       }
@@ -972,7 +850,7 @@ public class Parser implements ParserConstants {
       s = jj_consume_token(QUOTED_IDENTIFIER);
       break;
     default:
-      jj_la1[32] = jj_gen;
+      jj_la1[30] = jj_gen;
       jj_consume_token(-1);
       throw new ParseException();
     }
@@ -998,7 +876,7 @@ public class Parser implements ParserConstants {
         col = jj_consume_token(ASTERISK);
         break;
       default:
-        jj_la1[33] = jj_gen;
+        jj_la1[31] = jj_gen;
         jj_consume_token(-1);
         throw new ParseException();
       }
@@ -1014,14 +892,14 @@ public class Parser implements ParserConstants {
         col = jj_consume_token(STRING_LITERAL);
         break;
       default:
-        jj_la1[34] = jj_gen;
+        jj_la1[32] = 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[35] = jj_gen;
+      jj_la1[33] = jj_gen;
       jj_consume_token(-1);
       throw new ParseException();
     }
@@ -1041,7 +919,7 @@ public class Parser implements ParserConstants {
         ;
         break;
       default:
-        jj_la1[36] = jj_gen;
+        jj_la1[34] = jj_gen;
         break label_8;
       }
       jj_consume_token(COMMA);
@@ -1065,7 +943,7 @@ public class Parser implements ParserConstants {
         ;
         break;
       default:
-        jj_la1[37] = jj_gen;
+        jj_la1[35] = jj_gen;
         break label_9;
       }
       jj_consume_token(COMMA);
@@ -1092,7 +970,7 @@ public class Parser implements ParserConstants {
         ;
         break;
       default:
-        jj_la1[38] = jj_gen;
+        jj_la1[36] = jj_gen;
         break label_10;
       }
       jj_consume_token(COMMA);
@@ -1114,7 +992,7 @@ public class Parser implements ParserConstants {
       t = jj_consume_token(INTEGER_LITERAL);
       break;
     default:
-      jj_la1[39] = jj_gen;
+      jj_la1[37] = jj_gen;
       jj_consume_token(-1);
       throw new ParseException();
     }
@@ -1129,24 +1007,8 @@ public class Parser implements ParserConstants {
   final public String identifier() throws ParseException {
   Token t = null;
     switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
-    case META_TABLE:
-    case ROOT_TABLE:
     case ID:
-      switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
-      case ID:
-        t = jj_consume_token(ID);
-        break;
-      case META_TABLE:
-        t = jj_consume_token(META_TABLE);
-        break;
-      case ROOT_TABLE:
-        t = jj_consume_token(ROOT_TABLE);
-        break;
-      default:
-        jj_la1[40] = jj_gen;
-        jj_consume_token(-1);
-        throw new ParseException();
-      }
+      t = jj_consume_token(ID);
        {if (true) return t.image.toString();}
       break;
     case QUOTED_IDENTIFIER:
@@ -1159,244 +1021,17 @@ public class Parser implements ParserConstants {
         t = jj_consume_token(STRING_LITERAL);
         break;
       default:
-        jj_la1[41] = jj_gen;
+        jj_la1[38] = 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[42] = 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_11:
-    while (true) {
-      switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
-      case AND:
-      case OR:
-        ;
-        break;
-      default:
-        jj_la1[43] = jj_gen;
-        break label_11;
-      }
-      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[44] = 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[45] = jj_gen;
-        jj_consume_token(-1);
-        throw new ParseException();
-      }
-      break;
-    default:
-      jj_la1[46] = 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[47] = jj_gen;
-          jj_consume_token(-1);
-          throw new ParseException();
-        }
-        break;
-      default:
-        jj_la1[48] = 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[49] = 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[50] = 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[51] = 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[52] = jj_gen;
-        ;
-      }
-      switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
-      case RCOMP:
-        jj_consume_token(RCOMP);
-                 comparator = "<" + comparator;
-        break;
-      default:
-        jj_la1[53] = 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[54] = jj_gen;
+      jj_la1[39] = jj_gen;
       jj_consume_token(-1);
       throw new ParseException();
     }
-    {if (true) return comparator;}
     throw new Error("Missing return statement in function");
   }
 
@@ -1414,42 +1049,34 @@ public class Parser implements ParserConstants {
     finally { jj_save(0, xla); }
   }
 
-  final private boolean jj_3R_12() {
+  final private boolean jj_3R_13() {
     Token xsp;
     xsp = jj_scanpos;
-    if (jj_3R_13()) {
+    if (jj_scan_token(63)) {
     jj_scanpos = xsp;
-    if (jj_3R_14()) return true;
+    if (jj_scan_token(64)) return true;
     }
     return false;
   }
 
-  final private boolean jj_3R_14() {
-    Token xsp;
-    xsp = jj_scanpos;
-    if (jj_scan_token(69)) {
-    jj_scanpos = xsp;
-    if (jj_scan_token(70)) 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_13() {
+  final private boolean jj_3R_12() {
+    if (jj_scan_token(ID)) return true;
+    return false;
+  }
+
+  final private boolean jj_3R_11() {
     Token xsp;
     xsp = jj_scanpos;
-    if (jj_scan_token(65)) {
-    jj_scanpos = xsp;
-    if (jj_scan_token(60)) {
+    if (jj_3R_12()) {
     jj_scanpos = xsp;
-    if (jj_scan_token(61)) return true;
+    if (jj_3R_13()) return true;
     }
-    }
-    return false;
-  }
-
-  final private boolean jj_3_1() {
-    if (jj_scan_token(ADD)) return true;
-    if (jj_3R_12()) return true;
     return false;
   }
 
@@ -1462,7 +1089,7 @@ public class Parser implements ParserConstants {
   public boolean lookingAhead = false;
   private boolean jj_semLA;
   private int jj_gen;
-  final private int[] jj_la1 = new int[55];
+  final private int[] jj_la1 = new int[40];
   static private int[] jj_la1_0;
   static private int[] jj_la1_1;
   static private int[] jj_la1_2;
@@ -1472,13 +1099,13 @@ public class Parser implements ParserConstants {
       jj_la1_2();
    }
    private static void jj_la1_0() {
-      jj_la1_0 = new int[] {0x79ffe0,0x79ffe1,0x79ffe0,0x0,0x0,0x0,0x0,0x0,0x19fbc0,0x19fbc0,0x0,0x600,0x0,0x0,0x0,0x0,0x0,0x0,0x1000,0x0,0x0,0x1000000,0x1800000,0x1800000,0x20000000,0x40000000,0x80000000,0x0,0x40000,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x4000000,0x0,0x4000000,0x0,0x0,0x0,0x0,0x0,0x0,0x0,};
+      jj_la1_0 = new int[] {0x79ffe0,0x79ffe1,0x79ffe0,0x0,0x0,0x0,0x0,0x0,0x19fbc0,0x19fbc0,0x0,0x600,0x0,0x0,0x0,0x0,0x0,0x0,0x1000,0x0,0x0,0x1000000,0x1800000,0x1800000,0x20000000,0x40000000,0x80000000,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,};
    }
    private static void jj_la1_1() {
-      jj_la1_1 = new int[] {0x40000000,0x40000000,0x40000000,0x0,0x8,0x8,0x0,0x0,0xc0000000,0xc0000000,0x30000000,0x0,0x398e000,0x70000,0x700000,0x398e000,0x4,0x4,0xc000000,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x1,0x80000000,0x4,0x0,0x4,0x0,0x1000,0x0,0x1000,0x4,0x4,0x4,0x0,0x30000000,0x0,0x30000000,0x3,0x3,0x0,0x8,0x0,0x8,0x10,0x40,0x40,0x80,0x100,0xfc0,};
+      jj_la1_1 = new int[] {0x0,0x0,0x0,0x8000000,0x38000000,0x38000000,0x38000000,0x38000000,0x8000000,0x8000000,0x88000000,0x0,0x1cc7000,0x38000,0x380000,0x1cc7000,0x4,0x4,0x6000000,0x80000000,0x80000000,0x0,0x0,0x0,0x0,0x0,0x0,0x88000004,0x88000000,0x88000004,0x80000000,0x18000800,0x80000000,0x98000800,0x4,0x4,0x4,0x18000000,0x80000000,0x88000000,};
    }
    private static void jj_la1_2() {
-      jj_la1_2 = new int[] {0x2,0x82,0x2,0x2,0xe,0xe,0xe,0xe,0x2,0x2,0x62,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x60,0x60,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x2,0x62,0x62,0x62,0x60,0x6,0x60,0x66,0x0,0x0,0x0,0x6,0x2,0x60,0x62,0x0,0x0,0x2,0x0,0x2,0x0,0x46,0x0,0x0,0x0,0x0,0x0,};
+      jj_la1_2 = new int[] {0x0,0x2,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x1,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x0,0x1,0x1,0x0,0x0,0x0,0x0,0x0,0x0,0x1,0x1,0x1,0x1,0x0,0x1,0x1,0x0,0x0,0x0,0x0,0x1,0x1,};
    }
   final private JJCalls[] jj_2_rtns = new JJCalls[1];
   private boolean jj_rescan = false;
@@ -1493,7 +1120,7 @@ public class Parser implements ParserConstants {
     token = new Token();
     jj_ntk = -1;
     jj_gen = 0;
-    for (int i = 0; i < 55; i++) jj_la1[i] = -1;
+    for (int i = 0; i < 40; i++) jj_la1[i] = -1;
     for (int i = 0; i < jj_2_rtns.length; i++) jj_2_rtns[i] = new JJCalls();
   }
 
@@ -1506,7 +1133,7 @@ public class Parser implements ParserConstants {
     token = new Token();
     jj_ntk = -1;
     jj_gen = 0;
-    for (int i = 0; i < 55; i++) jj_la1[i] = -1;
+    for (int i = 0; i < 40; i++) jj_la1[i] = -1;
     for (int i = 0; i < jj_2_rtns.length; i++) jj_2_rtns[i] = new JJCalls();
   }
 
@@ -1516,7 +1143,7 @@ public class Parser implements ParserConstants {
     token = new Token();
     jj_ntk = -1;
     jj_gen = 0;
-    for (int i = 0; i < 55; i++) jj_la1[i] = -1;
+    for (int i = 0; i < 40; i++) jj_la1[i] = -1;
     for (int i = 0; i < jj_2_rtns.length; i++) jj_2_rtns[i] = new JJCalls();
   }
 
@@ -1526,7 +1153,7 @@ public class Parser implements ParserConstants {
     token = new Token();
     jj_ntk = -1;
     jj_gen = 0;
-    for (int i = 0; i < 55; i++) jj_la1[i] = -1;
+    for (int i = 0; i < 40; i++) jj_la1[i] = -1;
     for (int i = 0; i < jj_2_rtns.length; i++) jj_2_rtns[i] = new JJCalls();
   }
 
@@ -1535,7 +1162,7 @@ public class Parser implements ParserConstants {
     token = new Token();
     jj_ntk = -1;
     jj_gen = 0;
-    for (int i = 0; i < 55; i++) jj_la1[i] = -1;
+    for (int i = 0; i < 40; i++) jj_la1[i] = -1;
     for (int i = 0; i < jj_2_rtns.length; i++) jj_2_rtns[i] = new JJCalls();
   }
 
@@ -1544,7 +1171,7 @@ public class Parser implements ParserConstants {
     token = new Token();
     jj_ntk = -1;
     jj_gen = 0;
-    for (int i = 0; i < 55; i++) jj_la1[i] = -1;
+    for (int i = 0; i < 40; i++) jj_la1[i] = -1;
     for (int i = 0; i < jj_2_rtns.length; i++) jj_2_rtns[i] = new JJCalls();
   }
 
@@ -1655,15 +1282,15 @@ public class Parser implements ParserConstants {
 
   public ParseException generateParseException() {
     jj_expentries.removeAllElements();
-    boolean[] la1tokens = new boolean[72];
-    for (int i = 0; i < 72; i++) {
+    boolean[] la1tokens = new boolean[66];
+    for (int i = 0; i < 66; i++) {
       la1tokens[i] = false;
     }
     if (jj_kind >= 0) {
       la1tokens[jj_kind] = true;
       jj_kind = -1;
     }
-    for (int i = 0; i < 55; i++) {
+    for (int i = 0; i < 40; i++) {
       if (jj_la1[i] == jj_gen) {
         for (int j = 0; j < 32; j++) {
           if ((jj_la1_0[i] & (1<<j)) != 0) {
@@ -1678,7 +1305,7 @@ public class Parser implements ParserConstants {
         }
       }
     }
-    for (int i = 0; i < 72; i++) {
+    for (int i = 0; i < 66; i++) {
       if (la1tokens[i]) {
         jj_expentry = new int[1];
         jj_expentry[0] = i;

+ 30 - 42
src/contrib/hbase/src/java/org/apache/hadoop/hbase/shell/generated/ParserConstants.java

@@ -34,42 +34,36 @@ public interface ParserConstants {
   int AND = 32;
   int OR = 33;
   int COMMA = 34;
-  int DOT = 35;
-  int LPAREN = 36;
-  int RPAREN = 37;
-  int EQUALS = 38;
-  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 META_TABLE = 60;
-  int ROOT_TABLE = 61;
-  int SAVE = 62;
-  int GROUP = 63;
-  int BY = 64;
-  int ID = 65;
-  int INTEGER_LITERAL = 66;
-  int FLOATING_POINT_LITERAL = 67;
-  int EXPONENT = 68;
-  int QUOTED_IDENTIFIER = 69;
-  int STRING_LITERAL = 70;
+  int LPAREN = 35;
+  int RPAREN = 36;
+  int EQUALS = 37;
+  int LCOMP = 38;
+  int RCOMP = 39;
+  int NOT = 40;
+  int IN = 41;
+  int NOTEQUAL = 42;
+  int ASTERISK = 43;
+  int MAX_VERSIONS = 44;
+  int MAX_LENGTH = 45;
+  int COMPRESSION = 46;
+  int NONE = 47;
+  int BLOCK = 48;
+  int RECORD = 49;
+  int IN_MEMORY = 50;
+  int BLOOMFILTER = 51;
+  int COUNTING_BLOOMFILTER = 52;
+  int RETOUCHED_BLOOMFILTER = 53;
+  int VECTOR_SIZE = 54;
+  int NUM_HASH = 55;
+  int NUM_ENTRIES = 56;
+  int ADD = 57;
+  int CHANGE = 58;
+  int ID = 59;
+  int INTEGER_LITERAL = 60;
+  int FLOATING_POINT_LITERAL = 61;
+  int EXPONENT = 62;
+  int QUOTED_IDENTIFIER = 63;
+  int STRING_LITERAL = 64;
 
   int DEFAULT = 0;
 
@@ -109,7 +103,6 @@ public interface ParserConstants {
     "\"and\"",
     "\"or\"",
     "\",\"",
-    "\".\"",
     "\"(\"",
     "\")\"",
     "\"=\"",
@@ -134,11 +127,6 @@ public interface ParserConstants {
     "\"num_entries\"",
     "\"add\"",
     "\"change\"",
-    "\".META.\"",
-    "\"-ROOT-\"",
-    "\"save\"",
-    "\"group\"",
-    "\"by\"",
     "<ID>",
     "<INTEGER_LITERAL>",
     "<FLOATING_POINT_LITERAL>",

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


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

@@ -1,74 +0,0 @@
-/**
- * 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));
-  }
-  
-}

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

@@ -1,84 +0,0 @@
-/**
- * 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));
-  }
-
-}

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

@@ -1,250 +0,0 @@
-/**
- * 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;
-  }
-}

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

@@ -1,45 +0,0 @@
-/**
- * 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));
-  }
-}

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

@@ -1,250 +0,0 @@
-/**
- * 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.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.StaticTestEnvironment;
-import org.apache.hadoop.hbase.MasterNotRunningException;
-import org.apache.hadoop.hbase.MiniHBaseCluster;
-import org.apache.hadoop.hbase.MultiRegionTable;
-import org.apache.hadoop.hbase.mapred.TableReduce;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobClient;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.MiniMRCluster;
-
-/**
- * HBase shell join test
- */
-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();
-    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) {
-      StaticTestEnvironment.shutdownDfs(dfsCluster);
-      throw e;
-    }
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void tearDown() throws Exception {
-    super.tearDown();
-    if (hCluster != null) {
-      hCluster.shutdown();
-    }
-    StaticTestEnvironment.shutdownDfs(dfsCluster);
-  }
-
-  /**
-   * @throws Exception
-   */
-  public void testTableJoinMapReduce() throws Exception {
-    HTable table = null;
-    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
-      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();
-    } finally {
-      if (table != null) {
-        table.close();
-      }
-    }
-
-    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
-      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();
-    } finally {
-      if (table != null) {
-        table.close();
-      }
-    }
-
-    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);
-      TableReduce.initJob(OUTPUT_TABLE, IndexJoinReduce.class, jobConf);
-
-      JobClient.runJob(jobConf);
-
-    } catch (Exception e) {
-      e.printStackTrace();
-      throw e;
-    } finally {
-      if (mrCluster != null) {
-        mrCluster.shutdown();
-      }
-    }
-
-    try {
-      verify(OUTPUT_TABLE);
-    } catch (Exception e) {
-      e.printStackTrace();
-      throw e;
-    }
-  }
-
-  /**
-   * Check the result table lattice size.
-   * 
-   * @param conf
-   * @param outputTable
-   * @throws IOException
-   */
-  private void verify(String outputTable) throws IOException {
-    // Sleep before we start the verify to ensure that when the scanner takes
-    // its snapshot, all the updates have made it into the cache.
-    try {
-      Thread.sleep(conf.getLong("hbase.regionserver.optionalcacheflushinterval",
-          60L * 1000L));
-    } catch (InterruptedException e) {
-      // ignore
-    }
-
-    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)) {
-        LOG.info("result_table.column.size: " + results.keySet().size());
-        assertEquals(5, 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