You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by zs...@apache.org on 2010/01/21 11:38:15 UTC

svn commit: r901644 [19/37] - in /hadoop/hive/trunk: ./ ql/src/java/org/apache/hadoop/hive/ql/ ql/src/java/org/apache/hadoop/hive/ql/exec/ ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/ ql/src/java/org/apache/hadoop/hive/ql/history/ ql/src/jav...

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QBParseInfo.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QBParseInfo.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QBParseInfo.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QBParseInfo.java Thu Jan 21 10:37:58 2010
@@ -18,167 +18,181 @@
 
 package org.apache.hadoop.hive.ql.parse;
 
-import java.util.*;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
-import org.antlr.runtime.tree.*;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
 /**
  * Implementation of the parse information related to a query block
- *
+ * 
  **/
 
 public class QBParseInfo {
 
-  private boolean isSubQ;
-  private String alias;
+  private final boolean isSubQ;
+  private final String alias;
   private ASTNode joinExpr;
   private ASTNode hints;
-  private HashMap<String, ASTNode> aliasToSrc;
-  private HashMap<String, ASTNode> nameToDest;
-  private HashMap<String, TableSample> nameToSample;
-  private HashMap<String, ASTNode> destToSelExpr;
-  private HashMap<String, ASTNode> destToWhereExpr;
-  private HashMap<String, ASTNode> destToGroupby;
+  private final HashMap<String, ASTNode> aliasToSrc;
+  private final HashMap<String, ASTNode> nameToDest;
+  private final HashMap<String, TableSample> nameToSample;
+  private final HashMap<String, ASTNode> destToSelExpr;
+  private final HashMap<String, ASTNode> destToWhereExpr;
+  private final HashMap<String, ASTNode> destToGroupby;
   /**
-   * ClusterBy is a short name for both DistributeBy and SortBy.  
+   * ClusterBy is a short name for both DistributeBy and SortBy.
    */
-  private HashMap<String, ASTNode> destToClusterby;
+  private final HashMap<String, ASTNode> destToClusterby;
   /**
-   * DistributeBy controls the hashcode of the row, which determines which reducer
-   * the rows will go to. 
+   * DistributeBy controls the hashcode of the row, which determines which
+   * reducer the rows will go to.
    */
-  private HashMap<String, ASTNode> destToDistributeby;
+  private final HashMap<String, ASTNode> destToDistributeby;
   /**
-   * SortBy controls the reduce keys, which affects the order of rows 
-   * that the reducer receives. 
+   * SortBy controls the reduce keys, which affects the order of rows that the
+   * reducer receives.
    */
 
-  private HashMap<String, ASTNode> destToSortby;
+  private final HashMap<String, ASTNode> destToSortby;
 
   /**
-   * Maping from table/subquery aliases to all the associated lateral view
-   * nodes
+   * Maping from table/subquery aliases to all the associated lateral view nodes
    */
-  private HashMap<String, ArrayList<ASTNode>> aliasToLateralViews;
-  
+  private final HashMap<String, ArrayList<ASTNode>> aliasToLateralViews;
+
   /* Order by clause */
-  private HashMap<String, ASTNode> destToOrderby;
-  private HashMap<String, Integer>    destToLimit;
+  private final HashMap<String, ASTNode> destToOrderby;
+  private final HashMap<String, Integer> destToLimit;
   private int outerQueryLimit;
 
   // used by GroupBy
-  private LinkedHashMap<String, LinkedHashMap<String, ASTNode> > destToAggregationExprs;
-  private HashMap<String, ASTNode> destToDistinctFuncExpr;
+  private final LinkedHashMap<String, LinkedHashMap<String, ASTNode>> destToAggregationExprs;
+  private final HashMap<String, ASTNode> destToDistinctFuncExpr;
 
   @SuppressWarnings("unused")
   private static final Log LOG = LogFactory.getLog(QBParseInfo.class.getName());
-  
+
   public QBParseInfo(String alias, boolean isSubQ) {
-    this.aliasToSrc = new HashMap<String, ASTNode>();
-    this.nameToDest = new HashMap<String, ASTNode>();
-    this.nameToSample = new HashMap<String, TableSample>();
-    this.destToSelExpr = new HashMap<String, ASTNode>();
-    this.destToWhereExpr = new HashMap<String, ASTNode>();
-    this.destToGroupby = new HashMap<String, ASTNode>();
-    this.destToClusterby = new HashMap<String, ASTNode>();
-    this.destToDistributeby = new HashMap<String, ASTNode>();
-    this.destToSortby = new HashMap<String, ASTNode>();
-    this.destToOrderby = new HashMap<String, ASTNode>();
-    this.destToLimit = new HashMap<String, Integer>();
-    
-    this.destToAggregationExprs = new LinkedHashMap<String, LinkedHashMap<String, ASTNode> >();
-    this.destToDistinctFuncExpr = new HashMap<String, ASTNode>();
-    
+    aliasToSrc = new HashMap<String, ASTNode>();
+    nameToDest = new HashMap<String, ASTNode>();
+    nameToSample = new HashMap<String, TableSample>();
+    destToSelExpr = new HashMap<String, ASTNode>();
+    destToWhereExpr = new HashMap<String, ASTNode>();
+    destToGroupby = new HashMap<String, ASTNode>();
+    destToClusterby = new HashMap<String, ASTNode>();
+    destToDistributeby = new HashMap<String, ASTNode>();
+    destToSortby = new HashMap<String, ASTNode>();
+    destToOrderby = new HashMap<String, ASTNode>();
+    destToLimit = new HashMap<String, Integer>();
+
+    destToAggregationExprs = new LinkedHashMap<String, LinkedHashMap<String, ASTNode>>();
+    destToDistinctFuncExpr = new HashMap<String, ASTNode>();
+
     this.alias = alias;
     this.isSubQ = isSubQ;
-    this.outerQueryLimit = -1;
-    
-    this.aliasToLateralViews = new HashMap<String, ArrayList<ASTNode>>();
+    outerQueryLimit = -1;
+
+    aliasToLateralViews = new HashMap<String, ArrayList<ASTNode>>();
   }
 
-  public void setAggregationExprsForClause(String clause, LinkedHashMap<String, ASTNode> aggregationTrees) {
-    this.destToAggregationExprs.put(clause, aggregationTrees);
+  public void setAggregationExprsForClause(String clause,
+      LinkedHashMap<String, ASTNode> aggregationTrees) {
+    destToAggregationExprs.put(clause, aggregationTrees);
   }
 
   public HashMap<String, ASTNode> getAggregationExprsForClause(String clause) {
-    return this.destToAggregationExprs.get(clause);
+    return destToAggregationExprs.get(clause);
   }
 
   public void setDistinctFuncExprForClause(String clause, ASTNode ast) {
-    this.destToDistinctFuncExpr.put(clause, ast);
+    destToDistinctFuncExpr.put(clause, ast);
   }
-  
+
   public ASTNode getDistinctFuncExprForClause(String clause) {
-    return this.destToDistinctFuncExpr.get(clause);
+    return destToDistinctFuncExpr.get(clause);
   }
-  
+
   public void setSelExprForClause(String clause, ASTNode ast) {
-    this.destToSelExpr.put(clause, ast);
+    destToSelExpr.put(clause, ast);
   }
 
   public void setWhrExprForClause(String clause, ASTNode ast) {
-    this.destToWhereExpr.put(clause, ast);
+    destToWhereExpr.put(clause, ast);
   }
 
   public void setGroupByExprForClause(String clause, ASTNode ast) {
-    this.destToGroupby.put(clause, ast);
+    destToGroupby.put(clause, ast);
   }
 
   public void setDestForClause(String clause, ASTNode ast) {
-    this.nameToDest.put(clause, ast);
+    nameToDest.put(clause, ast);
   }
 
   /**
-   * Set the Cluster By AST for the clause.  
-   * @param clause the name of the clause
-   * @param ast the abstract syntax tree
+   * Set the Cluster By AST for the clause.
+   * 
+   * @param clause
+   *          the name of the clause
+   * @param ast
+   *          the abstract syntax tree
    */
   public void setClusterByExprForClause(String clause, ASTNode ast) {
-    this.destToClusterby.put(clause, ast);
+    destToClusterby.put(clause, ast);
   }
 
   /**
-   * Set the Distribute By AST for the clause.  
-   * @param clause the name of the clause
-   * @param ast the abstract syntax tree
+   * Set the Distribute By AST for the clause.
+   * 
+   * @param clause
+   *          the name of the clause
+   * @param ast
+   *          the abstract syntax tree
    */
   public void setDistributeByExprForClause(String clause, ASTNode ast) {
-    this.destToDistributeby.put(clause, ast);
+    destToDistributeby.put(clause, ast);
   }
 
   /**
-   * Set the Sort By AST for the clause.  
-   * @param clause the name of the clause
-   * @param ast the abstract syntax tree
+   * Set the Sort By AST for the clause.
+   * 
+   * @param clause
+   *          the name of the clause
+   * @param ast
+   *          the abstract syntax tree
    */
   public void setSortByExprForClause(String clause, ASTNode ast) {
-    this.destToSortby.put(clause, ast);
+    destToSortby.put(clause, ast);
   }
 
   public void setOrderByExprForClause(String clause, ASTNode ast) {
-    this.destToOrderby.put(clause, ast);
+    destToOrderby.put(clause, ast);
   }
 
   public void setSrcForAlias(String alias, ASTNode ast) {
-    this.aliasToSrc.put(alias.toLowerCase(), ast);
+    aliasToSrc.put(alias.toLowerCase(), ast);
   }
 
   public Set<String> getClauseNames() {
-    return this.destToSelExpr.keySet();
+    return destToSelExpr.keySet();
   }
 
   public Set<String> getClauseNamesForDest() {
-    return this.nameToDest.keySet();
+    return nameToDest.keySet();
   }
 
   public ASTNode getDestForClause(String clause) {
-    return this.nameToDest.get(clause);
+    return nameToDest.get(clause);
   }
 
   public ASTNode getWhrForClause(String clause) {
-    return this.destToWhereExpr.get(clause);
+    return destToWhereExpr.get(clause);
   }
 
   public HashMap<String, ASTNode> getDestToWhereExpr() {
@@ -186,77 +200,84 @@
   }
 
   public ASTNode getGroupByForClause(String clause) {
-    return this.destToGroupby.get(clause);
+    return destToGroupby.get(clause);
   }
+
   public HashMap<String, ASTNode> getDestToGroupBy() {
-    return this.destToGroupby;
+    return destToGroupby;
   }
-  
+
   public ASTNode getSelForClause(String clause) {
-    return this.destToSelExpr.get(clause);
+    return destToSelExpr.get(clause);
   }
 
   /**
-   * Get the Cluster By AST for the clause.  
-   * @param clause the name of the clause
+   * Get the Cluster By AST for the clause.
+   * 
+   * @param clause
+   *          the name of the clause
    * @return the abstract syntax tree
    */
   public ASTNode getClusterByForClause(String clause) {
-    return this.destToClusterby.get(clause);
+    return destToClusterby.get(clause);
   }
 
   public HashMap<String, ASTNode> getDestToClusterBy() {
     return destToClusterby;
   }
-  
+
   /**
-   * Get the Distribute By AST for the clause.  
-   * @param clause the name of the clause
+   * Get the Distribute By AST for the clause.
+   * 
+   * @param clause
+   *          the name of the clause
    * @return the abstract syntax tree
    */
   public ASTNode getDistributeByForClause(String clause) {
-    return this.destToDistributeby.get(clause);
+    return destToDistributeby.get(clause);
   }
 
   public HashMap<String, ASTNode> getDestToDistributeBy() {
     return destToDistributeby;
   }
-  
+
   /**
-   * Get the Sort By AST for the clause.  
-   * @param clause the name of the clause
+   * Get the Sort By AST for the clause.
+   * 
+   * @param clause
+   *          the name of the clause
    * @return the abstract syntax tree
    */
   public ASTNode getSortByForClause(String clause) {
-    return this.destToSortby.get(clause);
+    return destToSortby.get(clause);
   }
 
   public ASTNode getOrderByForClause(String clause) {
-    return this.destToOrderby.get(clause);
+    return destToOrderby.get(clause);
   }
 
   public HashMap<String, ASTNode> getDestToSortBy() {
     return destToSortby;
   }
-  
+
   public HashMap<String, ASTNode> getDestToOrderBy() {
     return destToOrderby;
   }
-  
+
   public ASTNode getSrcForAlias(String alias) {
-    return this.aliasToSrc.get(alias.toLowerCase());
+    return aliasToSrc.get(alias.toLowerCase());
   }
 
   public String getAlias() {
-    return this.alias;
+    return alias;
   }
 
   public boolean getIsSubQ() {
-    return this.isSubQ;
+    return isSubQ;
   }
 
   public ASTNode getJoinExpr() {
-    return this.joinExpr;
+    return joinExpr;
   }
 
   public void setJoinExpr(ASTNode joinExpr) {
@@ -264,81 +285,87 @@
   }
 
   public TableSample getTabSample(String alias) {
-    return this.nameToSample.get(alias.toLowerCase());
+    return nameToSample.get(alias.toLowerCase());
   }
-  
+
   public void setTabSample(String alias, TableSample tableSample) {
-    this.nameToSample.put(alias.toLowerCase(), tableSample);
+    nameToSample.put(alias.toLowerCase(), tableSample);
   }
 
   public void setDestLimit(String dest, Integer limit) {
-    this.destToLimit.put(dest, limit);
+    destToLimit.put(dest, limit);
   }
 
   public Integer getDestLimit(String dest) {
-    return this.destToLimit.get(dest);
+    return destToLimit.get(dest);
   }
 
-	/**
-	 * @return the outerQueryLimit
-	 */
-	public int getOuterQueryLimit() {
-		return outerQueryLimit;
-	}
-
-	/**
-	 * @param outerQueryLimit the outerQueryLimit to set
-	 */
-	public void setOuterQueryLimit(int outerQueryLimit) {
-		this.outerQueryLimit = outerQueryLimit;
-	}
+  /**
+   * @return the outerQueryLimit
+   */
+  public int getOuterQueryLimit() {
+    return outerQueryLimit;
+  }
+
+  /**
+   * @param outerQueryLimit
+   *          the outerQueryLimit to set
+   */
+  public void setOuterQueryLimit(int outerQueryLimit) {
+    this.outerQueryLimit = outerQueryLimit;
+  }
 
   public boolean isSelectStarQuery() {
-    if (isSubQ || 
-       (joinExpr != null) ||
-       (!nameToSample.isEmpty()) ||
-       (!destToGroupby.isEmpty()) ||
-       (!destToClusterby.isEmpty()) ||
-       (!aliasToLateralViews.isEmpty()))
+    if (isSubQ || (joinExpr != null) || (!nameToSample.isEmpty())
+        || (!destToGroupby.isEmpty()) || (!destToClusterby.isEmpty())
+        || (!aliasToLateralViews.isEmpty())) {
       return false;
-    
-    Iterator<Map.Entry<String, LinkedHashMap<String, ASTNode>>> aggrIter = destToAggregationExprs.entrySet().iterator();
+    }
+
+    Iterator<Map.Entry<String, LinkedHashMap<String, ASTNode>>> aggrIter = destToAggregationExprs
+        .entrySet().iterator();
     while (aggrIter.hasNext()) {
       HashMap<String, ASTNode> h = aggrIter.next().getValue();
-      if ((h != null) && (!h.isEmpty()))
+      if ((h != null) && (!h.isEmpty())) {
         return false;
+      }
     }
-      	
+
     if (!destToDistinctFuncExpr.isEmpty()) {
-      Iterator<Map.Entry<String, ASTNode>> distn = destToDistinctFuncExpr.entrySet().iterator();
+      Iterator<Map.Entry<String, ASTNode>> distn = destToDistinctFuncExpr
+          .entrySet().iterator();
       while (distn.hasNext()) {
         ASTNode ct = distn.next().getValue();
-        if (ct != null) 
+        if (ct != null) {
           return false;
+        }
       }
     }
-        
-    Iterator<Map.Entry<String, ASTNode>> iter = nameToDest.entrySet().iterator();
+
+    Iterator<Map.Entry<String, ASTNode>> iter = nameToDest.entrySet()
+        .iterator();
     while (iter.hasNext()) {
       Map.Entry<String, ASTNode> entry = iter.next();
       ASTNode v = entry.getValue();
-      if (!(((ASTNode)v.getChild(0)).getToken().getType() == HiveParser.TOK_TMP_FILE))
+      if (!(((ASTNode) v.getChild(0)).getToken().getType() == HiveParser.TOK_TMP_FILE)) {
         return false;
+      }
     }
-      	
+
     iter = destToSelExpr.entrySet().iterator();
     while (iter.hasNext()) {
       Map.Entry<String, ASTNode> entry = iter.next();
       ASTNode selExprList = entry.getValue();
       // Iterate over the selects
       for (int i = 0; i < selExprList.getChildCount(); ++i) {
-        
+
         // list of the columns
         ASTNode selExpr = (ASTNode) selExprList.getChild(i);
-        ASTNode sel = (ASTNode)selExpr.getChild(0);
-        
-        if (sel.getToken().getType() != HiveParser.TOK_ALLCOLREF)
+        ASTNode sel = (ASTNode) selExpr.getChild(0);
+
+        if (sel.getToken().getType() != HiveParser.TOK_ALLCOLREF) {
           return false;
+        }
       }
     }
 
@@ -346,20 +373,21 @@
   }
 
   public void setHints(ASTNode hint) {
-    this.hints = hint;
+    hints = hint;
   }
 
   public ASTNode getHints() {
     return hints;
   }
-  
+
   public Map<String, ArrayList<ASTNode>> getAliasToLateralViews() {
-    return this.aliasToLateralViews;
+    return aliasToLateralViews;
   }
+
   public List<ASTNode> getLateralViewsForAlias(String alias) {
     return aliasToLateralViews.get(alias.toLowerCase());
   }
-  
+
   public void addLateralViewForAlias(String alias, ASTNode lateralView) {
     String lowerAlias = alias.toLowerCase();
     ArrayList<ASTNode> lateralViews = aliasToLateralViews.get(lowerAlias);

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/RowResolver.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/RowResolver.java?rev=901644&r1=901643&r2=901644&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/RowResolver.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/RowResolver.java Thu Jan 21 10:37:58 2010
@@ -18,7 +18,12 @@
 
 package org.apache.hadoop.hive.ql.parse;
 
-import java.util.*;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.Vector;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.ql.exec.ColumnInfo;
@@ -26,29 +31,29 @@
 
 /**
  * Implementation of the Row Resolver
- *
+ * 
  **/
 
 public class RowResolver {
 
-  private RowSchema rowSchema;
-  private HashMap<String, LinkedHashMap<String, ColumnInfo>> rslvMap;
+  private final RowSchema rowSchema;
+  private final HashMap<String, LinkedHashMap<String, ColumnInfo>> rslvMap;
 
-  private HashMap<String, String[]> invRslvMap;
+  private final HashMap<String, String[]> invRslvMap;
 
   // TODO: Refactor this and do in a more object oriented manner
   private boolean isExprResolver;
 
   @SuppressWarnings("unused")
   private static final Log LOG = LogFactory.getLog(RowResolver.class.getName());
-  
+
   public RowResolver() {
     rowSchema = new RowSchema();
     rslvMap = new HashMap<String, LinkedHashMap<String, ColumnInfo>>();
     invRslvMap = new HashMap<String, String[]>();
     isExprResolver = false;
   }
-  
+
   public void put(String tab_alias, String col_alias, ColumnInfo colInfo) {
     if (tab_alias != null) {
       tab_alias = tab_alias.toLowerCase();
@@ -67,7 +72,7 @@
     }
     f_map.put(col_alias, colInfo);
 
-    String [] qualifiedAlias = new String[2];
+    String[] qualifiedAlias = new String[2];
     qualifiedAlias[0] = tab_alias;
     qualifiedAlias[1] = col_alias;
     invRslvMap.put(colInfo.getInternalName(), qualifiedAlias);
@@ -78,21 +83,27 @@
   }
 
   /**
-   * Gets the column Info to tab_alias.col_alias type of a column reference. I the tab_alias is not
-   * provided as can be the case with an non aliased column, this function looks up the column in all
-   * the table aliases in  this row resolver and returns the match. It also throws an exception if 
-   * the column is found in multiple table aliases. If no match is found a null values is returned.
+   * Gets the column Info to tab_alias.col_alias type of a column reference. I
+   * the tab_alias is not provided as can be the case with an non aliased
+   * column, this function looks up the column in all the table aliases in this
+   * row resolver and returns the match. It also throws an exception if the
+   * column is found in multiple table aliases. If no match is found a null
+   * values is returned.
    * 
-   * This allows us to interpret both select t.c1 type of references and select c1 kind of refereneces.
-   * The later kind are what we call non aliased column references in the query.
+   * This allows us to interpret both select t.c1 type of references and select
+   * c1 kind of refereneces. The later kind are what we call non aliased column
+   * references in the query.
    * 
-   * @param tab_alias The table alias to match (this is null if the column reference is non aliased)
-   * @param col_alias The column name that is being searched for
+   * @param tab_alias
+   *          The table alias to match (this is null if the column reference is
+   *          non aliased)
+   * @param col_alias
+   *          The column name that is being searched for
    * @return ColumnInfo
    * @throws SemanticException
    */
-  public ColumnInfo get(String tab_alias, String col_alias) 
-    throws SemanticException {
+  public ColumnInfo get(String tab_alias, String col_alias)
+      throws SemanticException {
     col_alias = col_alias.toLowerCase();
     ColumnInfo ret = null;
 
@@ -103,29 +114,29 @@
         return null;
       }
       ret = f_map.get(col_alias);
-    }
-    else {
+    } else {
       boolean found = false;
-      for(LinkedHashMap<String, ColumnInfo> cmap: rslvMap.values()) {
-        for(Map.Entry<String, ColumnInfo> cmapEnt: cmap.entrySet()) {
-          if (col_alias.equalsIgnoreCase((String)cmapEnt.getKey())) {
+      for (LinkedHashMap<String, ColumnInfo> cmap : rslvMap.values()) {
+        for (Map.Entry<String, ColumnInfo> cmapEnt : cmap.entrySet()) {
+          if (col_alias.equalsIgnoreCase(cmapEnt.getKey())) {
             if (found) {
-              throw new SemanticException("Column " + col_alias + " Found in more than One Tables/Subqueries");
+              throw new SemanticException("Column " + col_alias
+                  + " Found in more than One Tables/Subqueries");
             }
             found = true;
-            ret = (ColumnInfo)cmapEnt.getValue();
+            ret = cmapEnt.getValue();
           }
         }
       }
     }
 
-    return ret; 
+    return ret;
   }
 
   public Vector<ColumnInfo> getColumnInfos() {
     return rowSchema.getSignature();
   }
- 
+
   public HashMap<String, ColumnInfo> getFieldMap(String tab_alias) {
     if (tab_alias == null) {
       return rslvMap.get(null);
@@ -137,10 +148,10 @@
   public int getPosition(String internalName) {
     int pos = -1;
 
-    for(ColumnInfo var: rowSchema.getSignature()) {
+    for (ColumnInfo var : rowSchema.getSignature()) {
       ++pos;
       if (var.getInternalName().equals(internalName)) {
-         return pos;
+        return pos;
       }
     }
 
@@ -154,7 +165,7 @@
   public String[] reverseLookup(String internalName) {
     return invRslvMap.get(internalName);
   }
- 
+
   public void setIsExprResolver(boolean isExprResolver) {
     this.isExprResolver = isExprResolver;
   }
@@ -163,17 +174,21 @@
     return isExprResolver;
   }
 
+  @Override
   public String toString() {
     StringBuffer sb = new StringBuffer();
-    
-    for(Map.Entry<String, LinkedHashMap<String,ColumnInfo>> e: rslvMap.entrySet()) {
-      String tab = (String)e.getKey();
+
+    for (Map.Entry<String, LinkedHashMap<String, ColumnInfo>> e : rslvMap
+        .entrySet()) {
+      String tab = e.getKey();
       sb.append(tab + "{");
-      HashMap<String, ColumnInfo> f_map = (HashMap<String, ColumnInfo>)e.getValue();
-      if (f_map != null)
-        for(Map.Entry<String, ColumnInfo> entry: f_map.entrySet()) {
-          sb.append("(" + (String)entry.getKey() + "," + entry.getValue().toString() + ")");
+      HashMap<String, ColumnInfo> f_map = e.getValue();
+      if (f_map != null) {
+        for (Map.Entry<String, ColumnInfo> entry : f_map.entrySet()) {
+          sb.append("(" + entry.getKey() + "," + entry.getValue().toString()
+              + ")");
         }
+      }
       sb.append("} ");
     }
     return sb.toString();