You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lens.apache.org by am...@apache.org on 2014/12/31 05:50:37 UTC

incubator-lens git commit: LENS-175 : Allow multiple expressions in ExprColumn - cube metadata changes (Jaideep Dhok via amareshwari)

Repository: incubator-lens
Updated Branches:
  refs/heads/master 9e5bbcf5b -> dab0978ac


LENS-175 : Allow multiple expressions in ExprColumn - cube metadata changes (Jaideep Dhok via amareshwari)


Project: http://git-wip-us.apache.org/repos/asf/incubator-lens/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-lens/commit/dab0978a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-lens/tree/dab0978a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-lens/diff/dab0978a

Branch: refs/heads/master
Commit: dab0978acc6ed526891d236317f78a0e0875e92c
Parents: 9e5bbcf
Author: Amareshwari Sriramdasu <am...@inmobi.com>
Authored: Wed Dec 31 10:20:26 2014 +0530
Committer: Amareshwari Sriramdasu <am...@inmobi.com>
Committed: Wed Dec 31 10:20:26 2014 +0530

----------------------------------------------------------------------
 .../apache/lens/cube/metadata/ExprColumn.java   | 168 ++++++++++++++++---
 .../lens/cube/metadata/MetastoreConstants.java  |   1 +
 .../lens/cube/metadata/MetastoreUtil.java       |   4 +
 .../org/apache/lens/cube/parse/HQLParser.java   |  37 ++++
 .../cube/metadata/TestCubeMetastoreClient.java  |  45 ++++-
 .../lens/cube/metadata/TestExprColumn.java      |  51 ++++++
 .../apache/lens/cube/parse/TestHQLParser.java   |  18 ++
 7 files changed, 297 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/dab0978a/lens-cube/src/main/java/org/apache/lens/cube/metadata/ExprColumn.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/metadata/ExprColumn.java b/lens-cube/src/main/java/org/apache/lens/cube/metadata/ExprColumn.java
index 352fef9..00452ba 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/metadata/ExprColumn.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/metadata/ExprColumn.java
@@ -19,28 +19,64 @@
 
 package org.apache.lens.cube.metadata;
 
-import java.util.Map;
+import java.io.UnsupportedEncodingException;
+import java.util.*;
 
+import org.apache.commons.codec.binary.Base64;
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.ql.parse.ASTNode;
 import org.apache.hadoop.hive.ql.parse.ParseException;
 import org.apache.lens.cube.parse.HQLParser;
 
 public class ExprColumn extends CubeColumn {
-  private final String expr;
+  public static final char EXPRESSION_DELIMITER = '|';
+  private static final String EXPRESSION_ENCODED = "true";
+  private final Set<String> expressionSet = new LinkedHashSet<String>();
+  private List<ASTNode> astNodeList;
   private final String type;
-  private ASTNode ast;
+  private boolean hasHashCode = false;
+  private int hashCode;
 
-  public ExprColumn(FieldSchema column, String displayString, String expr) throws ParseException {
+  public ExprColumn(FieldSchema column, String displayString, String ... expressions) throws ParseException {
     super(column.getName(), column.getComment(), displayString, null, null, 0.0);
-    this.expr = expr;
+
+    if (expressions == null || expressions.length == 0) {
+      throw new IllegalArgumentException("No expression specified for column " + column.getName());
+    }
+
+    for (String e : expressions) {
+      expressionSet.add(e);
+    }
+
     this.type = column.getType();
     assert (getAst() != null);
   }
 
   public ExprColumn(String name, Map<String, String> props) {
     super(name, props);
-    this.expr = props.get(MetastoreUtil.getExprColumnKey(getName()));
+
+    String serializedExpressions = props.get(MetastoreUtil.getExprColumnKey(getName()));
+    String[] expressions = StringUtils.split(serializedExpressions, EXPRESSION_DELIMITER);
+
+    if (expressions.length == 0) {
+      throw new IllegalArgumentException("No expressions found for column "
+        + name + " property val=" + serializedExpressions);
+    }
+
+    boolean isExpressionBase64Encoded =
+      EXPRESSION_ENCODED.equals(props.get(MetastoreUtil.getExprEncodingPropertyKey(getName())));
+
+    for (String e : expressions) {
+      try {
+        String decodedExpr = isExpressionBase64Encoded ? new String(Base64.decodeBase64(e), "UTF-8") : e;
+        expressionSet.add(decodedExpr);
+      } catch (UnsupportedEncodingException e1) {
+        throw new IllegalArgumentException("Error decoding expression for expression column "
+          + name + " encoded value=" + e);
+      }
+    }
+
     this.type = props.get(MetastoreUtil.getExprTypePropertyKey(getName()));
   }
 
@@ -48,7 +84,7 @@ public class ExprColumn extends CubeColumn {
    * @return the expression
    */
   public String getExpr() {
-    return expr;
+    return expressionSet.iterator().next();
   }
 
   public String getType() {
@@ -58,17 +94,39 @@ public class ExprColumn extends CubeColumn {
   @Override
   public void addProperties(Map<String, String> props) {
     super.addProperties(props);
-    props.put(MetastoreUtil.getExprColumnKey(getName()), expr);
+
+    String[] encodedExpressions = expressionSet.toArray(new String[expressionSet.size()]);
+    for (int i = 0; i < encodedExpressions.length; i++) {
+      String expression = encodedExpressions[i];
+      try {
+        encodedExpressions[i] = Base64.encodeBase64String(expression.getBytes("UTF-8"));
+      } catch (UnsupportedEncodingException e) {
+        throw new IllegalArgumentException("Failed to encode expression " + expression);
+      }
+    }
+
+    String serializedExpressions = StringUtils.join(encodedExpressions, EXPRESSION_DELIMITER);
+    props.put(MetastoreUtil.getExprColumnKey(getName()) + ".base64", EXPRESSION_ENCODED);
+    props.put(MetastoreUtil.getExprColumnKey(getName()), serializedExpressions);
     props.put(MetastoreUtil.getExprTypePropertyKey(getName()), type);
   }
 
   @Override
   public int hashCode() {
-    final int prime = 31;
-    int result = super.hashCode();
-    result = prime * result + ((getType() == null) ? 0 : getType().toLowerCase().hashCode());
-    result = prime * result + ((getExpr() == null) ? 0 : getExpr().toLowerCase().hashCode());
-    return result;
+    if (!hasHashCode) {
+      final int prime = 31;
+      int result = super.hashCode();
+      result = prime * result + ((getType() == null) ? 0 : getType().toLowerCase().hashCode());
+
+      for (ASTNode exprNode : getExpressionASTList()) {
+        String exprNormalized = HQLParser.getString(exprNode);
+        result = prime * result + exprNormalized.hashCode();
+      }
+
+      hashCode = result;
+      hasHashCode = true;
+    }
+    return hashCode;
   }
 
   @Override
@@ -84,13 +142,24 @@ public class ExprColumn extends CubeColumn {
     } else if (!this.getType().equalsIgnoreCase(other.getType())) {
       return false;
     }
-    if (this.getExpr() == null) {
-      if (other.getExpr() != null) {
+    if (this.getAllExpressions() == null) {
+      if (other.getAllExpressions() != null) {
         return false;
       }
-    } else if (!this.getExpr().equalsIgnoreCase(other.getExpr())) {
+    }
+
+    if (expressionSet.size() != other.expressionSet.size()) {
       return false;
     }
+    // Compare expressions for both - compare ASTs
+    List<ASTNode> myExpressions = getExpressionASTList();
+    List<ASTNode> otherExpressions = other.getExpressionASTList();
+
+    for (int i = 0; i < myExpressions.size(); i++) {
+      if (!HQLParser.equalsAST(myExpressions.get(i), otherExpressions.get(i))) {
+        return false;
+      }
+    }
     return true;
   }
 
@@ -98,7 +167,7 @@ public class ExprColumn extends CubeColumn {
   public String toString() {
     String str = super.toString();
     str += "#type:" + type;
-    str += "#expr:" + expr;
+    str += "#expr:" + expressionSet.toString();
     return str;
   }
 
@@ -109,9 +178,68 @@ public class ExprColumn extends CubeColumn {
    * @throws ParseException
    */
   public ASTNode getAst() throws ParseException {
-    if (ast == null) {
-      this.ast = HQLParser.parseExpr(expr);
+    return getExpressionASTList().get(0);
+  }
+
+  public List<ASTNode> getExpressionASTList() {
+    if (astNodeList == null) {
+      astNodeList = new ArrayList<ASTNode>(expressionSet.size());
+      for (String expr : expressionSet) {
+        try {
+          astNodeList.add(HQLParser.parseExpr(expr));
+        } catch (ParseException e) {
+          // Should not throw exception since expr should have been validated when it was added
+          throw new IllegalStateException("Expression can't be parsed: " + expr, e);
+        }
+      }
     }
-    return ast;
+    return astNodeList;
+  }
+
+  private Set<String> getAllExpressions() {
+    return expressionSet;
   }
+
+  /**
+   * Get immutable view of this column's expressions
+   * @return
+   */
+  public Collection<String> getExpressions() {
+    return Collections.unmodifiableSet(expressionSet);
+  }
+
+  /**
+   * Add an expression to existing set of expressions for this column
+   * @param expression
+   * @throws ParseException
+   */
+  public void addExpression(String expression) throws ParseException {
+    if (expression == null || expression.isEmpty()) {
+      throw new IllegalArgumentException("Empty expression not allowed");
+    }
+
+    // Validate if expression can be correctly parsed
+    HQLParser.parseExpr(expression);
+    expressionSet.add(expression);
+    astNodeList = null;
+    hasHashCode = false;
+  }
+
+  /**
+   * Remove an expression from the set of expressions of this column
+   * @param expression
+   * @return
+   */
+  public boolean removeExpression(String expression) {
+    if (expression == null || expression.isEmpty()) {
+      throw new IllegalArgumentException("Empty expression not allowed");
+    }
+    boolean removed = expressionSet.remove(expression);
+    if (removed) {
+      astNodeList = null;
+      hasHashCode = false;
+    }
+    return removed;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/dab0978a/lens-cube/src/main/java/org/apache/lens/cube/metadata/MetastoreConstants.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/metadata/MetastoreConstants.java b/lens-cube/src/main/java/org/apache/lens/cube/metadata/MetastoreConstants.java
index c51f0ba..6b81cae 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/metadata/MetastoreConstants.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/metadata/MetastoreConstants.java
@@ -56,6 +56,7 @@ public interface MetastoreConstants {
   // column constants
   public static final String COL_PFX = "cube.col.";
   public static final String TYPE_SFX = ".type";
+  public static final String BASE64_SFX = ".base64";
   public static final String START_TIME_SFX = ".starttime";
   public static final String END_TIME_SFX = ".endtime";
   public static final String COST_SFX = ".cost";

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/dab0978a/lens-cube/src/main/java/org/apache/lens/cube/metadata/MetastoreUtil.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/metadata/MetastoreUtil.java b/lens-cube/src/main/java/org/apache/lens/cube/metadata/MetastoreUtil.java
index 48c5fbb..4f283ad 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/metadata/MetastoreUtil.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/metadata/MetastoreUtil.java
@@ -168,6 +168,10 @@ public class MetastoreUtil implements MetastoreConstants {
     return getColumnKeyPrefix(colName) + TYPE_SFX;
   }
 
+  public static final String getExprEncodingPropertyKey(String colName) {
+    return getExprColumnKey(colName) + BASE64_SFX;
+  }
+
   ////////////////////////////
   // Join chain properties  //
   ////////////////////////////

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/dab0978a/lens-cube/src/main/java/org/apache/lens/cube/parse/HQLParser.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/HQLParser.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/HQLParser.java
index 7347b3a..3c551c2 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/HQLParser.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/HQLParser.java
@@ -707,4 +707,41 @@ public class HQLParser {
 
     return astNodeType;
   }
+
+  public static boolean equalsAST(ASTNode n1, ASTNode n2) {
+    if (n1 == null && n2 != null) {
+      return false;
+    }
+
+    if (n1 != null && n2 == null) {
+      return false;
+    }
+
+    if (n1 == null) {
+      return true;
+    }
+
+    if (n1.getToken().getType() != n2.getToken().getType()) {
+      return false;
+    }
+
+    // Compare text. For literals, comparison is case sensitive
+    if ((n1.getToken().getType() == StringLiteral && !n1.getText().equals(n2.getText()))
+      || !n1.getText().equalsIgnoreCase(n2.getText())) {
+      return false;
+    }
+
+    // Compare children
+    if (n1.getChildCount() != n2.getChildCount()) {
+      return false;
+    }
+
+    for (int i = 0; i < n1.getChildCount(); i++) {
+      if (!equalsAST((ASTNode) n1.getChild(i), (ASTNode)n2.getChild(i))) {
+        return false;
+      }
+    }
+
+    return true;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/dab0978a/lens-cube/src/test/java/org/apache/lens/cube/metadata/TestCubeMetastoreClient.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/java/org/apache/lens/cube/metadata/TestCubeMetastoreClient.java b/lens-cube/src/test/java/org/apache/lens/cube/metadata/TestCubeMetastoreClient.java
index 5889506..f9e9f3d 100644
--- a/lens-cube/src/test/java/org/apache/lens/cube/metadata/TestCubeMetastoreClient.java
+++ b/lens-cube/src/test/java/org/apache/lens/cube/metadata/TestCubeMetastoreClient.java
@@ -166,13 +166,13 @@ public class TestCubeMetastoreClient {
         new TableReference("testdim2", "id")));
 
     cubeExpressions.add(new ExprColumn(new FieldSchema("msr5", "double", "fifth measure"), "Avg msr5",
-        "avg(msr1 + msr2)"));
+        "avg(msr1 + msr2)", "avg(msr2 + msr1)", "avg(msr1 + msr2 - msr1 + msr1)" ));
     cubeExpressions.add(new ExprColumn(new FieldSchema("msr5start", "double", "expr measure with start and end times"),
         "AVG of SUM", "avg(msr1 + msr2)"));
     cubeExpressions.add(new ExprColumn(new FieldSchema("booleancut", "boolean", "a boolean expression"), "Boolean Cut",
-        "dim1 != 'x' AND dim2 != 10 "));
+        "dim1 != 'x' AND dim2 != 10 ", "dim1 | dim2 AND dim2 = 'XYZ'"));
     cubeExpressions.add(new ExprColumn(new FieldSchema("substrexpr", "string", "a subt string expression"),
-        "SUBSTR EXPR", "substr(dim1, 3)"));
+        "SUBSTR EXPR", "substr(dim1, 3)", "substr(dim2, 3)"));
 
     List<CubeDimAttribute> locationHierarchyWithStartTime = new ArrayList<CubeDimAttribute>();
     locationHierarchyWithStartTime.add(new ReferencedDimAtrribute(new FieldSchema("zipcode2", "int", "zip"),
@@ -268,7 +268,7 @@ public class TestCubeMetastoreClient {
     cityAttrs.add(new ReferencedDimAtrribute(new FieldSchema("stateid", "int", "state id"), "State refer",
         new TableReference("statedim", "id")));
     dimExpressions.add(new ExprColumn(new FieldSchema("stateAndCountry", "String", "state and country together"),
-        "State and Country", "concat(statedim.name, \":\", countrydim.name)"));
+        "State and Country", "concat(statedim.name, \":\", countrydim.name)", "state_and_country"));
     dimExpressions.add(new ExprColumn(new FieldSchema("CityAddress", "string", "city with state and city and zip"),
         "City Address", "concat(citydim.name, \":\", statedim.name, \":\", countrydim.name, \":\", zipcode.code)"));
     Map<String, String> dimProps = new HashMap<String, String>();
@@ -335,16 +335,43 @@ public class TestCubeMetastoreClient {
     Assert.assertNotNull(city.getExpressionByName("cityaddress"));
     Assert.assertEquals(city.getExpressionByName("cityaddress").getDescription(), "city with state and city and zip");
     Assert.assertEquals(city.getExpressionByName("cityaddress").getDisplayString(), "City Address");
-    city.alterExpression(new ExprColumn(new FieldSchema("stateAndCountry", "String",
-        "state and country together with hiphen as separator"), "State and Country",
-        "concat(statedim.name, \"-\", countrydim.name)"));
+
+    ExprColumn stateCountryExpr = new ExprColumn(new FieldSchema("stateAndCountry", "String",
+      "state and country together with hiphen as separator"), "State and Country",
+      "concat(statedim.name, \"-\", countrydim.name)");
+    stateCountryExpr.addExpression("concat(countrydim.name, \"-\", countrydim.name)");
+
+    // Assert expression validation
+    try {
+      stateCountryExpr.addExpression("contact(countrydim.name");
+      Assert.fail("Expected add expression to fail because of syntax error");
+    } catch (ParseException exc) {
+      // Pass
+    }
+    city.alterExpression(stateCountryExpr);
+
+
     city.removeExpression("cityAddress");
     city = client.getDimension(cityDim.getName());
     Assert.assertEquals(1, city.getExpressions().size());
+
+    ExprColumn stateAndCountryActual = city.getExpressionByName("stateAndCountry");
+    Assert.assertNotNull(stateAndCountryActual.getExpressions());
+    Assert.assertEquals(2, stateAndCountryActual.getExpressions().size());
+    Assert.assertTrue(stateAndCountryActual.getExpressions().contains("concat(statedim.name, \"-\", countrydim.name)"));
+    Assert.assertTrue(stateAndCountryActual.getExpressions().contains("concat(countrydim.name, \"-\", countrydim.name)"));
+
     Assert.assertNotNull(city.getExpressionByName("stateAndCountry"));
     Assert.assertEquals(city.getExpressionByName("stateAndCountry").getExpr(),
         "concat(statedim.name, \"-\", countrydim.name)");
 
+    stateAndCountryActual.removeExpression("concat(countrydim.name, \"-\", countrydim.name)");
+    city.alterExpression(stateAndCountryActual);
+    client.alterDimension(city.getName(), city);
+    Dimension cityAltered = client.getDimension(city.getName());
+    Assert.assertEquals(1, cityAltered.getExpressionByName("stateAndCountry").getExpressions().size());
+
+
     List<TableReference> chain = new ArrayList<TableReference>();
     chain.add(new TableReference("zipdim", "cityid"));
     chain.add(new TableReference("citydim", "id"));
@@ -476,6 +503,10 @@ public class TestCubeMetastoreClient {
     Assert.assertNotNull(cube2.getExpressionByName("booleancut"));
     Assert.assertEquals(cube2.getExpressionByName("booleancut").getDescription(), "a boolean expression");
     Assert.assertEquals(cube2.getExpressionByName("booleancut").getDisplayString(), "Boolean Cut");
+    Assert.assertEquals(cube2.getExpressionByName("booleancut").getExpressions().size(), 2);
+    // Validate expression can contain delimiter character
+    List<String> booleanCutExprs = new ArrayList<String>(cube2.getExpressionByName("booleancut").getExpressions());
+    Assert.assertTrue(booleanCutExprs.contains("dim1 | dim2 AND dim2 = 'XYZ'"));
     Assert.assertTrue(cube2.allFieldsQueriable());
 
     Assert.assertTrue(cube2.getJoinChainNames().contains("cityfromzip"));

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/dab0978a/lens-cube/src/test/java/org/apache/lens/cube/metadata/TestExprColumn.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/java/org/apache/lens/cube/metadata/TestExprColumn.java b/lens-cube/src/test/java/org/apache/lens/cube/metadata/TestExprColumn.java
new file mode 100644
index 0000000..f21361e
--- /dev/null
+++ b/lens-cube/src/test/java/org/apache/lens/cube/metadata/TestExprColumn.java
@@ -0,0 +1,51 @@
+/**
+ * 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.lens.cube.metadata;
+
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotEquals;
+
+public class TestExprColumn {
+  @Test
+  public void testExprColumnEquality() throws Exception {
+    FieldSchema colSchema = new FieldSchema("someExprCol", "double", "some exprcol");
+
+    ExprColumn col1 = new ExprColumn(colSchema, "someExprDisplayString", "AVG(msr1) + AVG(msr2)");
+    ExprColumn col2 = new ExprColumn(colSchema, "someExprDisplayString", "avg(MSR1) + avg(MSR2)");
+    assertEquals(col1, col2);
+    assertEquals(col1.hashCode(), col2.hashCode());
+
+    ExprColumn col3 = new ExprColumn(colSchema, "someExprDisplayString", "AVG(msr1)");
+    assertNotEquals(col1, col3);
+    assertNotEquals(col1.hashCode(), col3.hashCode());
+
+    ExprColumn col4 = new ExprColumn(colSchema, "someExprDisplayString", "dim1 = 'FooBar' AND dim2 = 'BarFoo'");
+    ExprColumn col5 = new ExprColumn(colSchema, "someExprDisplayString", "dim1 = 'FOOBAR' AND dim2 = 'BarFoo'");
+    assertNotEquals(col4.hashCode(), col5.hashCode());
+    assertNotEquals(col4, col5);
+
+    ExprColumn col6 = new ExprColumn(colSchema, "someExprDisplayString", "DIM1 = 'FooBar' AND DIM2 = 'BarFoo'");
+    assertEquals(col4, col6);
+    assertEquals(col4.hashCode(), col6.hashCode());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/dab0978a/lens-cube/src/test/java/org/apache/lens/cube/parse/TestHQLParser.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/java/org/apache/lens/cube/parse/TestHQLParser.java b/lens-cube/src/test/java/org/apache/lens/cube/parse/TestHQLParser.java
index 760aaab..fcb973c 100644
--- a/lens-cube/src/test/java/org/apache/lens/cube/parse/TestHQLParser.java
+++ b/lens-cube/src/test/java/org/apache/lens/cube/parse/TestHQLParser.java
@@ -291,4 +291,22 @@ public class TestHQLParser {
         TOK_SELECT));
   }
 
+  @Test
+  public void testEqualsAST() throws Exception {
+    ASTNode expr1 = HQLParser.parseExpr("T1.a + T2.b - T2.c");
+    ASTNode expr2 = HQLParser.parseExpr("t1.A + t2.B - t2.C");
+
+    Assert.assertTrue(HQLParser.equalsAST(expr1, expr2));
+
+    ASTNode literalExpr1 = HQLParser.parseExpr("A = 'FooBar'");
+    ASTNode literalExpr2 = HQLParser.parseExpr("a = 'FooBar'");
+    Assert.assertTrue(HQLParser.equalsAST(literalExpr1, literalExpr2));
+
+    ASTNode literalExpr3 = HQLParser.parseExpr("A = 'fOObAR'");
+    Assert.assertFalse(HQLParser.equalsAST(literalExpr1, literalExpr3));
+
+    ASTNode literalExpr4 = HQLParser.parseExpr("A <> 'FooBar'");
+    Assert.assertFalse(HQLParser.equalsAST(literalExpr1, literalExpr4));
+  }
+
 }