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 2015/05/07 07:49:02 UTC

incubator-lens git commit: LENS-176 : Lens server should support API to retrieve and update expression columns with multiple expressions (amareshwari)

Repository: incubator-lens
Updated Branches:
  refs/heads/master 10917c25e -> 365f5f72b


LENS-176 : Lens server should support API to retrieve and update expression columns with multiple expressions (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/365f5f72
Tree: http://git-wip-us.apache.org/repos/asf/incubator-lens/tree/365f5f72
Diff: http://git-wip-us.apache.org/repos/asf/incubator-lens/diff/365f5f72

Branch: refs/heads/master
Commit: 365f5f72bf3b06c964241a4b2a8c16cd2b4af50e
Parents: 10917c2
Author: Amareshwari Sriramadasu <am...@apache.org>
Authored: Thu May 7 11:18:49 2015 +0530
Committer: Amareshwari Sriramadasu <am...@apache.org>
Committed: Thu May 7 11:18:49 2015 +0530

----------------------------------------------------------------------
 lens-api/src/main/resources/cube-0.1.xsd        |  34 +++
 lens-cli/src/test/resources/sample-cube.xml     |   9 +-
 .../apache/lens/cube/metadata/CubeColumn.java   |  15 +-
 .../apache/lens/cube/metadata/ExprColumn.java   | 229 ++++++++++++++++---
 .../lens/cube/parse/ExpressionResolver.java     |  13 +-
 .../cube/metadata/TestCubeMetastoreClient.java  |  36 ++-
 .../lens/cube/metadata/TestExprColumn.java      | 159 ++++++++++++-
 lens-examples/src/main/resources/customer.xml   |   2 +-
 lens-examples/src/main/resources/product.xml    |   6 +
 lens-examples/src/main/resources/sales-cube.xml |  15 +-
 .../src/main/resources/sample-cube.xml          |   9 +-
 .../main/resources/sample-db-only-dimension.xml |   4 +-
 .../src/main/resources/sample-dimension.xml     |   4 +-
 .../src/main/resources/sample-dimension2.xml    |   2 +-
 .../apache/lens/server/metastore/JAXBUtils.java |  43 +++-
 .../lens/server/ui/MetastoreUIResource.java     |   2 +-
 .../server/metastore/TestMetastoreService.java  |  54 ++++-
 .../lens/server/query/TestQueryService.java     |   3 +-
 18 files changed, 560 insertions(+), 79 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/365f5f72/lens-api/src/main/resources/cube-0.1.xsd
----------------------------------------------------------------------
diff --git a/lens-api/src/main/resources/cube-0.1.xsd b/lens-api/src/main/resources/cube-0.1.xsd
index 24d0d64..10b1dfa 100644
--- a/lens-api/src/main/resources/cube-0.1.xsd
+++ b/lens-api/src/main/resources/cube-0.1.xsd
@@ -248,6 +248,14 @@
         An expression column
       </xs:documentation>
     </xs:annotation>
+    <xs:sequence>
+      <xs:annotation>
+        <xs:documentation>
+          All the expressions associated with expression column.
+        </xs:documentation>
+      </xs:annotation>
+      <xs:element type="x_expr_spec" name="expr_spec" maxOccurs="unbounded" minOccurs="1"/>
+    </xs:sequence>
     <xs:attribute type="xs:string" name="name" use="required" />
     <xs:attribute type="xs:string" name="type" use="required">
       <xs:annotation>
@@ -269,6 +277,18 @@
         </xs:documentation>
       </xs:annotation>
     </xs:attribute>
+  </xs:complexType>
+
+  <xs:complexType name="x_expr_spec">
+    <xs:annotation>
+      <xs:documentation>
+        Specification of single expression. Specification consists of required expression field, and optional start and
+        end times associated with expression. Expression is valid between start time and end time, if both are
+        specified. If only start_time is specified, is valid from start_time to till date. If only end_time is
+        specified, is valid upto date specified, from any time. If no start time, end time are specified, the
+        expression is always valid.
+      </xs:documentation>
+    </xs:annotation>
     <xs:attribute type="xs:string" name="expr" use="required">
       <xs:annotation>
         <xs:documentation>
@@ -276,6 +296,20 @@
         </xs:documentation>
       </xs:annotation>
     </xs:attribute>
+    <xs:attribute type="xs:dateTime" name="start_time">
+      <xs:annotation>
+        <xs:documentation>
+          The start time from when the expression is valid.
+        </xs:documentation>
+      </xs:annotation>
+    </xs:attribute>
+    <xs:attribute type="xs:dateTime" name="end_time">
+      <xs:annotation>
+        <xs:documentation>
+          The end time till when the expression is valid.
+        </xs:documentation>
+      </xs:annotation>
+    </xs:attribute>
   </xs:complexType>
 
   <xs:complexType name="x_expressions">

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/365f5f72/lens-cli/src/test/resources/sample-cube.xml
----------------------------------------------------------------------
diff --git a/lens-cli/src/test/resources/sample-cube.xml b/lens-cli/src/test/resources/sample-cube.xml
index 4908f85..d54bcfb 100644
--- a/lens-cli/src/test/resources/sample-cube.xml
+++ b/lens-cli/src/test/resources/sample-cube.xml
@@ -24,15 +24,18 @@
   <measures>
     <measure name="measure1" type="BIGINT" />
     <measure name="measure2" type="INT" default_aggr="SUM" />
-    <measure name="measure3" type="FLOAT" default_aggr="MAX" start_time='2013-12-12-00:00' />
+    <measure name="measure3" type="FLOAT" default_aggr="MAX" start_time='2013-12-12T00:00:00' />
     <measure name="measure4" type="DOUBLE" default_aggr="MIN" />
   </measures>
   <expressions>
-    <expression name="expr_msr5" type="DOUBLE" expr="measure3 + measure4" />
+    <expression name="expr_msr5" type="DOUBLE">
+      <expr_spec expr = "measure3 + measure4" end_time='2013-12-12T00:00:00'/>
+      <expr_spec expr = "measure3 + measure4 + 0.01" start_time='2013-12-12T00:00:00'/>
+    </expression>
   </expressions>
   <dim_attributes>
     <dim_attribute name="dim1" type="INT" />
-    <dim_attribute name="dim2" type="INT" start_time='2013-12-01-00:00' />
+    <dim_attribute name="dim2" type="INT" start_time='2013-12-01T00:00:00' />
     <dim_attribute name="dim3" type="INT">
       <ref_spec>
         <table_references>

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/365f5f72/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeColumn.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeColumn.java b/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeColumn.java
index cccb888..a2a00d2 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeColumn.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeColumn.java
@@ -27,8 +27,11 @@ import java.util.TimeZone;
 import org.apache.lens.cube.parse.TimeRange;
 
 import com.google.common.base.Optional;
+
 import lombok.NonNull;
+import lombok.extern.slf4j.Slf4j;
 
+@Slf4j
 public abstract class CubeColumn implements Named {
 
   private final String name;
@@ -59,12 +62,17 @@ public abstract class CubeColumn implements Named {
   }
 
   private Date getDate(String propKey, Map<String, String> props) {
-    String timeKey = props.get(propKey);
-    if (timeKey != null) {
+    String timeStr = props.get(propKey);
+    return getDate(timeStr);
+  }
+
+  protected Date getDate(String timeStr) {
+    if (timeStr != null) {
       try {
-        return COLUMN_TIME_FORMAT.get().parse(timeKey);
+        return COLUMN_TIME_FORMAT.get().parse(timeStr);
       } catch (Exception e) {
         // ignore and return null
+        log.warn("Column time passed:{} is not parsable, its ignored", timeStr, e);
       }
     }
     return null;
@@ -77,6 +85,7 @@ public abstract class CubeColumn implements Named {
         return Double.parseDouble(doubleStr);
       } catch (Exception e) {
         // ignore and return null
+        log.warn("Property {} value {} is not parsable, its ignored", propKey, doubleStr, e);
       }
     }
     return null;

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/365f5f72/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 df4cf25..a7f711f 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
@@ -30,26 +30,47 @@ 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 lombok.Getter;
+import lombok.NoArgsConstructor;
+import lombok.Setter;
+import lombok.ToString;
+
 public class ExprColumn extends CubeColumn {
   public static final char EXPRESSION_DELIMITER = '|';
+  public static final char EXPRESSION_SPEC_DELIMITER = ':';
   private static final String EXPRESSION_ENCODED = "true";
-  private final Set<String> expressionSet = new LinkedHashSet<String>();
-  private List<ASTNode> astNodeList;
+  private final Set<ExprSpec> expressionSet = new LinkedHashSet<ExprSpec>();
+  private List<ASTNode> astNodeList = new ArrayList<ASTNode>();
   private final String type;
   private boolean hasHashCode = false;
   private int hashCode;
 
-  public ExprColumn(FieldSchema column, String displayString, String... expressions) throws ParseException {
+  // for backward compatibility
+  public ExprColumn(FieldSchema column, String displayString, String expression) {
+    this(column, displayString, new ExprSpec(expression, null, null));
+  }
+
+  public ExprColumn(FieldSchema column, String displayString, ExprSpec... expressions) {
     super(column.getName(), column.getComment(), displayString, null, null, 0.0);
 
     if (expressions == null || expressions.length == 0) {
-      throw new IllegalArgumentException("No expression specified for column " + column.getName());
+      throw new IllegalArgumentException("No expressions specified for column " + column.getName());
     }
 
-    for (String e : expressions) {
+    for (int i = 0; i < expressions.length; i++) {
+      ExprSpec e = expressions[i];
+      if (StringUtils.isBlank(e.getExpr())) {
+        throw new IllegalArgumentException(
+          "No expression string specified for column " + column.getName() + " at index:" + i);
+      }
+      if (e.getStartTime() != null && e.getEndTime() != null) {
+        if (e.getStartTime().after(e.getEndTime())) {
+          throw new IllegalArgumentException("Start time is after end time for column " + column.getName()
+            + " for expression at index:" + i + " for " + e.getExpr());
+        }
+      }
       expressionSet.add(e);
     }
-
     this.type = column.getType();
     assert (getAst() != null);
   }
@@ -69,9 +90,26 @@ public class ExprColumn extends CubeColumn {
       EXPRESSION_ENCODED.equals(props.get(MetastoreUtil.getExprEncodingPropertyKey(getName())));
 
     for (String e : expressions) {
+      String[] exprSpecStrs = StringUtils.splitPreserveAllTokens(e, EXPRESSION_SPEC_DELIMITER);
       try {
-        String decodedExpr = isExpressionBase64Encoded ? new String(Base64.decodeBase64(e), "UTF-8") : e;
-        expressionSet.add(decodedExpr);
+        String decodedExpr =
+          isExpressionBase64Encoded ? new String(Base64.decodeBase64(exprSpecStrs[0]), "UTF-8") : exprSpecStrs[0];
+        ExprSpec exprSpec = new ExprSpec();
+        exprSpec.setExpr(decodedExpr);
+        if (exprSpecStrs.length > 1) {
+          // start time and end time serialized
+          if (StringUtils.isNotBlank(exprSpecStrs[1])) {
+            // start time available
+            exprSpec.setStartTime(getDate(exprSpecStrs[1]));
+          }
+          if (exprSpecStrs.length > 2) {
+            if (StringUtils.isNotBlank(exprSpecStrs[2])) {
+              // end time available
+              exprSpec.setEndTime(getDate(exprSpecStrs[2]));
+            }
+          }
+        }
+        expressionSet.add(exprSpec);
       } catch (UnsupportedEncodingException e1) {
         throw new IllegalArgumentException("Error decoding expression for expression column "
           + name + " encoded value=" + e);
@@ -81,11 +119,69 @@ public class ExprColumn extends CubeColumn {
     this.type = props.get(MetastoreUtil.getExprTypePropertyKey(getName()));
   }
 
+  @NoArgsConstructor
+  @ToString(exclude = {"astNode", "hasHashCode", "hashCode"})
+  public static class ExprSpec {
+    @Getter
+    @Setter
+    private String expr;
+    @Getter
+    @Setter
+    private Date startTime;
+    @Getter
+    @Setter
+    private Date endTime;
+
+    private transient ASTNode astNode;
+    private boolean hasHashCode = false;
+    private transient int hashCode;
+
+    public ExprSpec(String expr, Date startTime, Date endTime) {
+      this.expr = expr;
+      this.startTime = startTime;
+      this.endTime = endTime;
+    }
+
+    synchronized ASTNode getASTNode() {
+      if (astNode == null) {
+        try {
+          if (StringUtils.isNotBlank(expr)) {
+            astNode = HQLParser.parseExpr(getExpr());
+          }
+        } catch (ParseException e) {
+          throw new IllegalArgumentException("Expression can't be parsed: " + getExpr(), e);
+        }
+      }
+      return astNode;
+    }
+
+    @Override
+    public int hashCode() {
+      if (!hasHashCode) {
+        final int prime = 31;
+        int result = 1;
+        if (getASTNode() != null) {
+          String exprNormalized = HQLParser.getString(getASTNode());
+          result = prime * result + exprNormalized.hashCode();
+        }
+        result = prime * result + ((getStartTime() == null) ? 0 : COLUMN_TIME_FORMAT.get().format(
+          getStartTime()).hashCode());
+        result = prime * result + ((getEndTime() == null) ? 0 : COLUMN_TIME_FORMAT.get().format(
+          getEndTime()).hashCode());
+        hashCode = result;
+        hasHashCode = true;
+      }
+      return hashCode;
+    }
+  }
+
   /**
+   * Returns the first expression.
+   *
    * @return the expression
    */
   public String getExpr() {
-    return expressionSet.iterator().next();
+    return expressionSet.iterator().next().getExpr();
   }
 
   public String getType() {
@@ -96,11 +192,25 @@ public class ExprColumn extends CubeColumn {
   public void addProperties(Map<String, String> props) {
     super.addProperties(props);
 
-    String[] encodedExpressions = expressionSet.toArray(new String[expressionSet.size()]);
-    for (int i = 0; i < encodedExpressions.length; i++) {
-      String expression = encodedExpressions[i];
+    String[] encodedExpressions = new String[expressionSet.size()];
+    StringBuilder exprSpecBuilder = new StringBuilder();
+    int i = 0;
+    for (ExprSpec es : expressionSet) {
+      String expression = es.getExpr();
       try {
-        encodedExpressions[i] = Base64.encodeBase64String(expression.getBytes("UTF-8"));
+        exprSpecBuilder.append(Base64.encodeBase64String(expression.getBytes("UTF-8")));
+        exprSpecBuilder.append(EXPRESSION_SPEC_DELIMITER);
+        if (es.getStartTime() != null) {
+          exprSpecBuilder.append(COLUMN_TIME_FORMAT.get().format(es.getStartTime()));
+        }
+        exprSpecBuilder.append(EXPRESSION_SPEC_DELIMITER);
+        if (es.getEndTime() != null) {
+          exprSpecBuilder.append(COLUMN_TIME_FORMAT.get().format(es.getEndTime()));
+        }
+        // encoded expression contains the Base64 encoded expression, start time and end time.
+        encodedExpressions[i] = exprSpecBuilder.toString();
+        exprSpecBuilder.setLength(0);
+        i++;
       } catch (UnsupportedEncodingException e) {
         throw new IllegalArgumentException("Failed to encode expression " + expression);
       }
@@ -119,9 +229,8 @@ public class ExprColumn extends CubeColumn {
       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();
+      for (ExprSpec exprSpec : expressionSet) {
+        result = prime * result + exprSpec.hashCode();
       }
 
       hashCode = result;
@@ -161,6 +270,36 @@ public class ExprColumn extends CubeColumn {
         return false;
       }
     }
+    // compare start and end times for expressions
+    Iterator<ExprSpec> thisIter = this.expressionSet.iterator();
+    Iterator<ExprSpec> otherIter = other.expressionSet.iterator();
+    while (thisIter.hasNext() && otherIter.hasNext()) {
+      ExprSpec thisES = thisIter.next();
+      ExprSpec otherES = otherIter.next();
+      if (!equalDates(thisES.getStartTime(), otherES.getStartTime())) {
+        return false;
+      }
+      if (!equalDates(thisES.getEndTime(), otherES.getEndTime())) {
+        return false;
+      }
+    }
+    if (thisIter.hasNext() != otherIter.hasNext()) {
+      return false;
+    }
+    return true;
+  }
+
+  private boolean equalDates(Date d1, Date d2) {
+    if (d1 == null) {
+      if (d2 != null) {
+        return false;
+      }
+    } else if (d2 == null) {
+      return false;
+    } else if (!COLUMN_TIME_FORMAT.get().format(d1).equals(COLUMN_TIME_FORMAT.get().format(
+      d2))) {
+      return false;
+    }
     return true;
   }
 
@@ -178,35 +317,49 @@ public class ExprColumn extends CubeColumn {
    * @return the ast
    * @throws ParseException
    */
-  public ASTNode getAst() throws ParseException {
+  public ASTNode getAst() {
     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);
+    synchronized (expressionSet) {
+      if (astNodeList.isEmpty()) {
+        for (ExprSpec expr : expressionSet) {
+          astNodeList.add(expr.getASTNode());
         }
       }
     }
     return astNodeList;
   }
 
-  private Set<String> getAllExpressions() {
+  private Set<ExprSpec> getAllExpressions() {
     return expressionSet;
   }
 
+  private final Set<String> cachedExpressionStrings = new LinkedHashSet<String>();
+
   /**
-   * Get immutable view of this column's expressions
+   * Get immutable view of this column's expression strings
    *
    * @return
    */
   public Collection<String> getExpressions() {
+    if (cachedExpressionStrings.isEmpty()) {
+      synchronized (expressionSet) {
+        for (ExprSpec es : expressionSet) {
+          cachedExpressionStrings.add(es.getExpr());
+        }
+      }
+    }
+    return Collections.unmodifiableSet(cachedExpressionStrings);
+  }
+
+  /**
+   * Get immutable view of this column's expression full spec
+   *
+   * @return
+   */
+  public Collection<ExprSpec> getExpressionSpecs() {
     return Collections.unmodifiableSet(expressionSet);
   }
 
@@ -216,14 +369,16 @@ public class ExprColumn extends CubeColumn {
    * @param expression
    * @throws ParseException
    */
-  public void addExpression(String expression) throws ParseException {
-    if (expression == null || expression.isEmpty()) {
+  public void addExpression(ExprSpec expression) throws ParseException {
+    if (expression == null || expression.getExpr().isEmpty()) {
       throw new IllegalArgumentException("Empty expression not allowed");
     }
 
     // Validate if expression can be correctly parsed
-    HQLParser.parseExpr(expression);
-    expressionSet.add(expression);
+    HQLParser.parseExpr(expression.getExpr());
+    synchronized (expressionSet) {
+      expressionSet.add(expression);
+    }
     astNodeList = null;
     hasHashCode = false;
   }
@@ -238,7 +393,17 @@ public class ExprColumn extends CubeColumn {
     if (expression == null || expression.isEmpty()) {
       throw new IllegalArgumentException("Empty expression not allowed");
     }
-    boolean removed = expressionSet.remove(expression);
+    boolean removed = false;
+    synchronized (expressionSet) {
+      Iterator<ExprSpec> it = expressionSet.iterator();
+      while (it.hasNext()) {
+        if (it.next().getExpr().equals(expression)) {
+          it.remove();
+          removed = true;
+          break;
+        }
+      }
+    }
     if (removed) {
       astNodeList = null;
       hasHashCode = false;

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/365f5f72/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
index fa23b8e..c81cf34 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
@@ -32,7 +32,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.parse.ASTNode;
 import org.apache.hadoop.hive.ql.parse.HiveParser;
-import org.apache.hadoop.hive.ql.parse.ParseException;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 
 import org.antlr.runtime.CommonToken;
@@ -112,11 +111,7 @@ class ExpressionResolver implements ContextRewriter {
     if (((AbstractBaseTable) cubeql.getQueriedTable(table)).getExpressionByName(column) == null) {
       return null;
     }
-    try {
-      return ((AbstractBaseTable) cubeql.getQueriedTable(table)).getExpressionByName(column).getAst();
-    } catch (ParseException e) {
-      throw new SemanticException(e);
-    }
+    return ((AbstractBaseTable) cubeql.getQueriedTable(table)).getExpressionByName(column).getAst();
   }
 
   private ASTNode getExprAST(final CubeQueryContext cubeql, final String column) throws SemanticException {
@@ -143,11 +138,7 @@ class ExpressionResolver implements ContextRewriter {
     if (expr == null) {
       return null;
     }
-    try {
-      return expr.getAst();
-    } catch (ParseException e) {
-      throw new SemanticException(e);
-    }
+    return expr.getAst();
   }
 
   private ASTNode replaceAlias(final ASTNode expr, final CubeQueryContext cubeql) throws SemanticException {

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/365f5f72/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 8c8d578..d481f93 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
@@ -21,6 +21,7 @@ package org.apache.lens.cube.metadata;
 
 import java.util.*;
 
+import org.apache.lens.cube.metadata.ExprColumn.ExprSpec;
 import org.apache.lens.cube.metadata.timeline.EndsAndHolesPartitionTimeline;
 import org.apache.lens.cube.metadata.timeline.PartitionTimeline;
 import org.apache.lens.cube.metadata.timeline.StoreAllPartitionTimeline;
@@ -196,14 +197,30 @@ public class TestCubeMetastoreClient {
     cubeDimensions.add(new ReferencedDimAtrribute(new FieldSchema("dim2", "id", "ref dim"), "Dim2 refer",
       new TableReference("testdim2", "id")));
 
+    ExprSpec expr1 = new ExprSpec();
+    expr1.setExpr("avg(msr1 + msr2)");
+    ExprSpec expr2 = new ExprSpec();
+    expr2.setExpr("avg(msr2 + msr1)");
+    ExprSpec expr3 = new ExprSpec();
+    expr3.setExpr("avg(msr1 + msr2 - msr1 + msr1)");
     cubeExpressions.add(new ExprColumn(new FieldSchema("msr5", "double", "fifth measure"), "Avg msr5",
-      "avg(msr1 + msr2)", "avg(msr2 + msr1)", "avg(msr1 + msr2 - msr1 + msr1)"));
+      expr1, expr2, expr3));
+    expr1 = new ExprSpec();
+    expr1.setExpr("avg(msr1 + msr2)");
     cubeExpressions.add(new ExprColumn(new FieldSchema("msr5start", "double", "expr measure with start and end times"),
-      "AVG of SUM", "avg(msr1 + msr2)"));
+      "AVG of SUM", expr1));
+    expr1 = new ExprSpec();
+    expr1.setExpr("dim1 != 'x' AND dim2 != 10 ");
+    expr2 = new ExprSpec();
+    expr2.setExpr("dim1 | dim2 AND dim2 = 'XYZ'");
     cubeExpressions.add(new ExprColumn(new FieldSchema("booleancut", "boolean", "a boolean expression"), "Boolean Cut",
-      "dim1 != 'x' AND dim2 != 10 ", "dim1 | dim2 AND dim2 = 'XYZ'"));
+      expr1, expr2));
+    expr1 = new ExprSpec();
+    expr1.setExpr("substr(dim1, 3)");
+    expr2 = new ExprSpec();
+    expr2.setExpr("substr(dim2, 3)");
     cubeExpressions.add(new ExprColumn(new FieldSchema("substrexpr", "string", "a subt string expression"),
-      "SUBSTR EXPR", "substr(dim1, 3)", "substr(dim2, 3)"));
+      "SUBSTR EXPR", expr1, expr2));
 
     List<CubeDimAttribute> locationHierarchyWithStartTime = new ArrayList<CubeDimAttribute>();
     locationHierarchyWithStartTime.add(new ReferencedDimAtrribute(new FieldSchema("zipcode2", "int", "zip"),
@@ -300,7 +317,8 @@ 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"));
+      "State and Country", new ExprSpec("concat(statedim.name, \":\", countrydim.name)", null, null),
+      new ExprSpec("state_and_country", null, null)));
     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>();
@@ -376,11 +394,15 @@ public class TestCubeMetastoreClient {
     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)");
+    ExprSpec expr1 = new ExprSpec();
+    expr1.setExpr("concat(countrydim.name, \"-\", countrydim.name)");
+    stateCountryExpr.addExpression(expr1);
 
     // Assert expression validation
     try {
-      stateCountryExpr.addExpression("contact(countrydim.name");
+      expr1 = new ExprSpec();
+      expr1.setExpr("contact(countrydim.name");
+      stateCountryExpr.addExpression(expr1);
       Assert.fail("Expected add expression to fail because of syntax error");
     } catch (ParseException exc) {
       // Pass

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/365f5f72/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
index 51db143..d1cd590 100644
--- 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
@@ -19,14 +19,40 @@
 
 package org.apache.lens.cube.metadata;
 
-import static org.testng.Assert.assertEquals;
-import static org.testng.Assert.assertNotEquals;
+import static org.testng.Assert.*;
+
+import java.util.Calendar;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import org.apache.lens.cube.metadata.ExprColumn.ExprSpec;
 
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 
 import org.testng.annotations.Test;
 
 public class TestExprColumn {
+
+  private final Date now;
+  private final Date twoDaysBack;
+  private final Date nowUptoHours;
+  private final Date twoDaysBackUptoHours;
+
+  public TestExprColumn() {
+    Calendar cal = Calendar.getInstance();
+    now = cal.getTime();
+    cal.add(Calendar.DAY_OF_MONTH, -2);
+    twoDaysBack = cal.getTime();
+    cal.set(Calendar.MINUTE, 0);
+    cal.set(Calendar.SECOND, 0);
+    cal.set(Calendar.MILLISECOND, 0);
+    twoDaysBackUptoHours = cal.getTime();
+    cal.add(Calendar.DAY_OF_MONTH, +2);
+    nowUptoHours = cal.getTime();
+  }
+
   @Test
   public void testExprColumnEquality() throws Exception {
     FieldSchema colSchema = new FieldSchema("someExprCol", "double", "some exprcol");
@@ -36,6 +62,10 @@ public class TestExprColumn {
     assertEquals(col1, col2);
     assertEquals(col1.hashCode(), col2.hashCode());
 
+    col2 = new ExprColumn(colSchema, "someExprDisplayString", new ExprSpec("avg(MSR1) + avg(MSR2)", null, null));
+    assertEquals(col1, col2);
+    assertEquals(col1.hashCode(), col2.hashCode());
+
     ExprColumn col3 = new ExprColumn(colSchema, "someExprDisplayString", "AVG(msr1)");
     assertNotEquals(col1, col3);
     assertNotEquals(col1.hashCode(), col3.hashCode());
@@ -49,4 +79,129 @@ public class TestExprColumn {
     assertEquals(col4, col6);
     assertEquals(col4.hashCode(), col6.hashCode());
   }
+
+  @Test
+  public void testExpressionStartAndEndTimes() throws Exception {
+    FieldSchema colSchema = new FieldSchema("multiExprCol", "double", "multi exprcol");
+
+    ExprColumn col1 = new ExprColumn(colSchema, "multiExprColDisplayString", new ExprSpec("avg(MSR1) + avg(MSR2)",
+      twoDaysBack, null));
+    Map<String, String> props = new HashMap<String, String>();
+    col1.addProperties(props);
+    ExprColumn col1FromProps = new ExprColumn("multiExprCol", props);
+    assertEquals(col1, col1FromProps);
+    assertEquals(col1.hashCode(), col1FromProps.hashCode());
+    assertEquals(col1FromProps.getExpressionSpecs().iterator().next().getStartTime(), twoDaysBackUptoHours);
+
+    col1 = new ExprColumn(colSchema, "multiExprColDisplayString", new ExprSpec("avg(MSR1) + avg(MSR2)",
+      null, twoDaysBack));
+    props = new HashMap<String, String>();
+    col1.addProperties(props);
+    col1FromProps = new ExprColumn("multiExprCol", props);
+    assertEquals(col1, col1FromProps);
+    assertEquals(col1.hashCode(), col1FromProps.hashCode());
+    assertEquals(col1FromProps.getExpressionSpecs().iterator().next().getEndTime(), twoDaysBackUptoHours);
+
+
+    col1 = new ExprColumn(colSchema, "multiExprColDisplayString", new ExprSpec("avg(MSR1) + avg(MSR2)",
+      twoDaysBack, now));
+    props = new HashMap<String, String>();
+    col1.addProperties(props);
+    col1FromProps = new ExprColumn("multiExprCol", props);
+    assertEquals(col1, col1FromProps);
+    assertEquals(col1.hashCode(), col1FromProps.hashCode());
+    assertEquals(col1FromProps.getExpressionSpecs().iterator().next().getStartTime(), twoDaysBackUptoHours);
+    assertEquals(col1FromProps.getExpressionSpecs().iterator().next().getEndTime(), nowUptoHours);
+  }
+
+  @Test
+  public void testMultipleExpressionStartAndEndTimes() throws Exception {
+    FieldSchema colSchema = new FieldSchema("multiExprCol", "double", "multi exprcol");
+
+    ExprColumn col1 = new ExprColumn(colSchema, "multiExprColDisplayString", new ExprSpec("avg(MSR1) + avg(MSR2)",
+      twoDaysBack, null), new ExprSpec("avg(MSR1) + avg(MSR2) - m1 + m1", now, null),
+      new ExprSpec("avg(MSR1) + avg(MSR2)", null, twoDaysBack),
+      new ExprSpec("avg(MSR1) + avg(MSR2) + 0.01", twoDaysBack, now));
+    Map<String, String> props = new HashMap<String, String>();
+    col1.addProperties(props);
+    ExprColumn col1FromProps = new ExprColumn("multiExprCol", props);
+    assertEquals(col1, col1FromProps);
+    assertEquals(col1.hashCode(), col1FromProps.hashCode());
+    Iterator<ExprSpec> it = col1FromProps.getExpressionSpecs().iterator();
+    assertEquals(it.next().getStartTime(), twoDaysBackUptoHours);
+    assertEquals(it.next().getStartTime(), nowUptoHours);
+    ExprSpec endTimeSpecified = it.next();
+    assertNull(endTimeSpecified.getStartTime());
+    assertEquals(endTimeSpecified.getEndTime(), twoDaysBackUptoHours);
+    ExprSpec last = it.next();
+    assertEquals(last.getStartTime(), twoDaysBackUptoHours);
+    assertEquals(last.getEndTime(), nowUptoHours);
+    assertFalse(it.hasNext());
+  }
+
+  @Test
+  public void testExprColumnCreationErrors() {
+    FieldSchema colSchema = new FieldSchema("errorColumn", "double", "multi exprcol");
+
+    // no expression spec passed
+    try {
+      ExprColumn col1 = new ExprColumn(colSchema, "NoExprSpec", (ExprSpec[])null);
+      fail(col1 + " should not be created");
+    } catch (IllegalArgumentException e) {
+      assertTrue(e.getMessage().contains("No expressions specified for column errorColumn"));
+    }
+
+    // no expression passed in exprspec
+    try {
+      ExprColumn col1 = new ExprColumn(colSchema, "NoExprInExprSpec", new ExprSpec(null, null, null));
+      fail(col1 + " should not be created");
+    } catch (IllegalArgumentException e) {
+      assertTrue(e.getMessage().contains("No expression string specified for column errorColumn at index:0"));
+    }
+
+    // Parse error in expr passed in exprspec
+    try {
+      ExprColumn col1 = new ExprColumn(colSchema, "NoExprInExprSpec", new ExprSpec("(a+b", null, null));
+      fail(col1 + " should not be created");
+    } catch (IllegalArgumentException e) {
+      assertTrue(e.getMessage().contains("Expression can't be parsed: (a+b"), e.getMessage());
+    }
+
+    // Parse error in expr passed in exprspec
+    try {
+      ExprColumn col1 = new ExprColumn(colSchema, "NoExprInExprSpec", new ExprSpec("a + b", null, null),
+        new ExprSpec("(a+b", null, null));
+      fail(col1 + " should not be created");
+    } catch (IllegalArgumentException e) {
+      assertTrue(e.getMessage().contains("Expression can't be parsed: (a+b"));
+    }
+
+    // no expression passed in exprspec
+    try {
+      ExprColumn col1 = new ExprColumn(colSchema, "NoExprInExprSpecAt1", new ExprSpec("a + b", null, null),
+        new ExprSpec(null, null, null));
+      fail(col1 + " should not be created");
+    } catch (IllegalArgumentException e) {
+      assertTrue(e.getMessage().contains("No expression string specified for column errorColumn at index:1"));
+    }
+
+    // startTime after endTime
+    try {
+      ExprColumn col1 = new ExprColumn(colSchema, "startTimeAfterEndTime", new ExprSpec("a + b", now, twoDaysBack));
+      fail(col1 + " should not be created");
+    } catch (IllegalArgumentException e) {
+      assertTrue(e.getMessage().contains("Start time is after end time for column errorColumn "
+        + "for expression at index:0"), e.getMessage());
+    }
+
+    // startTime after endTime
+    try {
+      ExprColumn col1 = new ExprColumn(colSchema, "startTimeAfterEndTimeAt1", new ExprSpec("a + b", null, null),
+        new ExprSpec("a + b", now, twoDaysBack));
+      fail(col1 + " should not be created");
+    } catch (IllegalArgumentException e) {
+      assertTrue(e.getMessage().contains("Start time is after end time for column errorColumn "
+        + "for expression at index:1"), e.getMessage());
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/365f5f72/lens-examples/src/main/resources/customer.xml
----------------------------------------------------------------------
diff --git a/lens-examples/src/main/resources/customer.xml b/lens-examples/src/main/resources/customer.xml
index e78dab2..64024f1 100644
--- a/lens-examples/src/main/resources/customer.xml
+++ b/lens-examples/src/main/resources/customer.xml
@@ -33,7 +33,7 @@
         <chain_ref_column chain_name="customer_city" ref_col="name" />
       </ref_spec>
     </dim_attribute>
-    <dim_attribute name="customer_credit_status" type="STRING" start_time='2015-03-01-00:00'/>
+    <dim_attribute name="customer_credit_status" type="STRING" start_time='2015-03-01T00:00:00'/>
   </attributes>
   <join_chains>
     <join_chain name="customer_city">

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/365f5f72/lens-examples/src/main/resources/product.xml
----------------------------------------------------------------------
diff --git a/lens-examples/src/main/resources/product.xml b/lens-examples/src/main/resources/product.xml
index 8093561..bb91997 100644
--- a/lens-examples/src/main/resources/product.xml
+++ b/lens-examples/src/main/resources/product.xml
@@ -30,6 +30,12 @@
     <dim_attribute name="category" type="STRING" />
     <dim_attribute name="manufacturer" type="STRING" />
   </attributes>
+  <expressions>
+    <expression name="isHeavy" type="boolean">
+      <expr_spec expr = "weight > 10" end_time="2015-04-12T00:00:00"/>
+      <expr_spec expr = "weight > 50" start_time="2015-04-12T00:00:00"/>
+    </expression>
+  </expressions>
   <properties>
     <property name="dimension.product.timed.dimension" value="dt"/>
   </properties>

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/365f5f72/lens-examples/src/main/resources/sales-cube.xml
----------------------------------------------------------------------
diff --git a/lens-examples/src/main/resources/sales-cube.xml b/lens-examples/src/main/resources/sales-cube.xml
index 951ace9..f5e2f93 100644
--- a/lens-examples/src/main/resources/sales-cube.xml
+++ b/lens-examples/src/main/resources/sales-cube.xml
@@ -25,7 +25,7 @@
     <measure name="unit_sales" type="BIGINT" default_aggr="SUM" display_string="Unit Sales" format_string="#,###"/>
     <measure name="store_sales" type="DOUBLE" default_aggr="SUM" display_string="Store Sales" format_string="#,###.##"/>
     <measure name="store_cost" type="DOUBLE" default_aggr="SUM" display_string="Store Cost" format_string="#,###.00"
-     start_time='2015-03-01-00:00' />
+     start_time='2015-03-01T00:00:00' />
     <measure name="line_item_quantity" type="INT" default_aggr="SUM" display_string="Line item quantity"/>
     <measure name="line_item_product_price" type="FLOAT" default_aggr="SUM" display_string="Line item product price"/>
     <measure name="line_item_discount_amount" type="FLOAT" default_aggr="SUM" display_string="Line item discount"/>
@@ -35,9 +35,14 @@
      display_string="Maximum Line item discount"/>
   </measures>
   <expressions>
-    <expression name="profit" type="DOUBLE" expr="store_sales - store_cost" display_string="Profit"/>
-    <expression name="promotion_sales" type="DOUBLE" expr="sum(case when promotion_id = 0 then 0 else store_sales end)"
-     display_string="Promotion sales"/>
+    <expression name="profit" type="DOUBLE" display_string="Profit">
+      <expr_spec expr="store_sales - store_cost"/>
+    </expression>
+    <expression name="promotion_sales" type="DOUBLE" display_string="Promotion sales">
+      <expr_spec expr="sum(case when promotion_id = 0 then 0 else store_sales end)" start_time='2015-04-12T00:00:00'/>
+      <expr_spec expr="sum(case when promotion_id = -1 then 0 when promotion_id = -1 then 0 else store_sales end)"
+       end_time="2015-04-12T00:00:00"/>
+    </expression>
   </expressions>
   <dim_attributes>
     <dim_attribute name="customer_id" type="INT" />
@@ -47,7 +52,7 @@
     <dim_attribute name="order_line_number" type="INT" />
     <dim_attribute name="order_time" type="TIMESTAMP" />
     <dim_attribute name="delivery_time" type="TIMESTAMP" />
-    <dim_attribute name="customer_city_id" type="INT" start_time='2015-03-01-00:00' />
+    <dim_attribute name="customer_city_id" type="INT" start_time='2015-03-01T00:00:00' />
     <dim_attribute name="production_city_id" type="INT" />
     <dim_attribute name="delivery_city_id" type="INT" />
     <dim_attribute name="customer_city_name" type="string" description="City name to which the customer belongs"

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/365f5f72/lens-examples/src/main/resources/sample-cube.xml
----------------------------------------------------------------------
diff --git a/lens-examples/src/main/resources/sample-cube.xml b/lens-examples/src/main/resources/sample-cube.xml
index f24a8cc..0d97842 100644
--- a/lens-examples/src/main/resources/sample-cube.xml
+++ b/lens-examples/src/main/resources/sample-cube.xml
@@ -24,15 +24,18 @@
   <measures>
     <measure name="measure1" type="BIGINT"/>
     <measure name="measure2" type="INT" default_aggr="SUM"/>
-    <measure name="measure3" type="FLOAT" default_aggr="MAX" start_time='2013-12-12-00:00'/>
+    <measure name="measure3" type="FLOAT" default_aggr="MAX" start_time='2013-12-12T00:00:00'/>
     <measure name="measure4" type="DOUBLE" default_aggr="MIN"/>
   </measures>
   <expressions>
-    <expression name="expr_msr5" type="DOUBLE" expr="measure3 + measure4"/>
+    <expression name="expr_msr5" type="DOUBLE">
+      <expr_spec expr = "measure3 + measure4" start_time='2013-12-12T00:00:00' />
+      <expr_spec expr = "measure3 + measure4 + 0.01" end_time='2013-12-12T00:00:00'/>
+    </expression>
   </expressions>
   <dim_attributes>
     <dim_attribute name="dim1" type="INT"/>
-    <dim_attribute name="dim2" type="INT" start_time='2013-12-01-00:00'/>
+    <dim_attribute name="dim2" type="INT" start_time='2013-12-01T00:00:00'/>
     <dim_attribute name="dim3" type="INT" join_key="true">
       <ref_spec>
         <table_references>

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/365f5f72/lens-examples/src/main/resources/sample-db-only-dimension.xml
----------------------------------------------------------------------
diff --git a/lens-examples/src/main/resources/sample-db-only-dimension.xml b/lens-examples/src/main/resources/sample-db-only-dimension.xml
index 63a8bf8..4c6bec6 100644
--- a/lens-examples/src/main/resources/sample-db-only-dimension.xml
+++ b/lens-examples/src/main/resources/sample-db-only-dimension.xml
@@ -24,8 +24,8 @@
   <attributes>
     <dim_attribute name="id" type="INT"/>
     <dim_attribute name="name" type="STRING"/>
-    <dim_attribute name="detail" type="STRING" start_time='2013-12-01-00:00'/>
-    <dim_attribute name="d2id" type="INT" start_time='2013-12-01-00:00' join_key="true">
+    <dim_attribute name="detail" type="STRING" start_time='2013-12-01T00:00:00'/>
+    <dim_attribute name="d2id" type="INT" start_time='2013-12-01T00:00:00' join_key="true">
       <ref_spec>
         <table_references>
           <table_reference table="sample_dim2" column="id"/>

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/365f5f72/lens-examples/src/main/resources/sample-dimension.xml
----------------------------------------------------------------------
diff --git a/lens-examples/src/main/resources/sample-dimension.xml b/lens-examples/src/main/resources/sample-dimension.xml
index af4ab21..9b97da7 100644
--- a/lens-examples/src/main/resources/sample-dimension.xml
+++ b/lens-examples/src/main/resources/sample-dimension.xml
@@ -24,8 +24,8 @@
   <attributes>
     <dim_attribute name="id" type="INT"/>
     <dim_attribute name="name" type="STRING"/>
-    <dim_attribute name="detail" type="STRING" start_time='2013-12-01-00:00'/>
-    <dim_attribute name="d2id" type="INT" start_time='2013-12-01-00:00' join_key="true">
+    <dim_attribute name="detail" type="STRING" start_time='2013-12-01T00:00:00'/>
+    <dim_attribute name="d2id" type="INT" start_time='2013-12-01T00:00:00' join_key="true">
       <ref_spec>
         <table_references>
           <table_reference table="sample_dim2" column="id"/>

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/365f5f72/lens-examples/src/main/resources/sample-dimension2.xml
----------------------------------------------------------------------
diff --git a/lens-examples/src/main/resources/sample-dimension2.xml b/lens-examples/src/main/resources/sample-dimension2.xml
index d1ac80d..9740507 100644
--- a/lens-examples/src/main/resources/sample-dimension2.xml
+++ b/lens-examples/src/main/resources/sample-dimension2.xml
@@ -24,7 +24,7 @@
   <attributes>
     <dim_attribute name="id" type="INT"/>
     <dim_attribute name="name" type="STRING"/>
-    <dim_attribute name="detail2" type="STRING" start_time='2013-12-01-00:00'/>
+    <dim_attribute name="detail2" type="STRING" start_time='2013-12-01T00:00:00'/>
   </attributes>
   <properties>
     <property name="dimension.sample_dim2.timed.dimension" value="dt"/>

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/365f5f72/lens-server/src/main/java/org/apache/lens/server/metastore/JAXBUtils.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/metastore/JAXBUtils.java b/lens-server/src/main/java/org/apache/lens/server/metastore/JAXBUtils.java
index 3cf7011..a628c90 100644
--- a/lens-server/src/main/java/org/apache/lens/server/metastore/JAXBUtils.java
+++ b/lens-server/src/main/java/org/apache/lens/server/metastore/JAXBUtils.java
@@ -28,6 +28,7 @@ import javax.xml.datatype.XMLGregorianCalendar;
 
 import org.apache.lens.api.metastore.*;
 import org.apache.lens.cube.metadata.*;
+import org.apache.lens.cube.metadata.ExprColumn.ExprSpec;
 
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
@@ -204,6 +205,14 @@ public final class JAXBUtils {
     return hiveDim;
   }
 
+  /**
+   * Get XMLGregorianCalendar from Date.
+   *
+   * Useful for converting from java code to XML spec.
+   *
+   * @param d Date value
+   * @return XML value
+   */
   public static XMLGregorianCalendar getXMLGregorianCalendar(Date d) {
     if (d == null) {
       return null;
@@ -219,6 +228,14 @@ public final class JAXBUtils {
     }
   }
 
+  /**
+   * Get Date from XMLGregorianCalendar
+   *
+   * Useful for converting from XML spec to java code.
+   *
+   * @param cal XML value
+   * @return Date value
+   */
   public static Date getDateFromXML(XMLGregorianCalendar cal) {
     if (cal == null) {
       return null;
@@ -262,10 +279,30 @@ public final class JAXBUtils {
     xe.setType(ec.getType());
     xe.setDescription(ec.getDescription());
     xe.setDisplayString(ec.getDisplayString());
-    xe.setExpr(ec.getExpr());
+    xe.getExprSpec().addAll(xExprSpecFromExprColumn(ec.getExpressionSpecs()));
     return xe;
   }
 
+  private static Collection<XExprSpec> xExprSpecFromExprColumn(Collection<ExprSpec> esSet) {
+    List<XExprSpec> xes = new ArrayList<XExprSpec>();
+    for (ExprSpec es : esSet) {
+      XExprSpec e = new XExprSpec();
+      e.setExpr(es.getExpr());
+      e.setStartTime(getXMLGregorianCalendar(es.getStartTime()));
+      e.setEndTime(getXMLGregorianCalendar(es.getEndTime()));
+      xes.add(e);
+    }
+    return xes;
+  }
+
+  private static ExprSpec[] exprSpecFromXExprColumn(Collection<XExprSpec> xesList) {
+    List<ExprSpec> esArray = new ArrayList<ExprSpec>(xesList.size());
+    for (XExprSpec xes : xesList) {
+      esArray.add(new ExprSpec(xes.getExpr(), getDateFromXML(xes.getStartTime()), getDateFromXML(xes.getEndTime())));
+    }
+    return esArray.toArray(new ExprSpec[0]);
+  }
+
   /**
    * Create XDimAttribute from CubeDimAttribute
    */
@@ -274,6 +311,8 @@ public final class JAXBUtils {
     xd.setName(cd.getName());
     xd.setDescription(cd.getDescription());
     xd.setDisplayString(cd.getDisplayString());
+    xd.setStartTime(getXMLGregorianCalendar(cd.getStartTime()));
+    xd.setEndTime(getXMLGregorianCalendar(cd.getEndTime()));
     if (cd instanceof ReferencedDimAtrribute) {
       ReferencedDimAtrribute rd = (ReferencedDimAtrribute) cd;
       List<TableReference> dimRefs = rd.getReferences();
@@ -405,7 +444,7 @@ public final class JAXBUtils {
     ExprColumn ec = new ExprColumn(new FieldSchema(xe.getName(), xe.getType().toLowerCase(),
       xe.getDescription()),
       xe.getDisplayString(),
-      xe.getExpr());
+      exprSpecFromXExprColumn(xe.getExprSpec()));
     return ec;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/365f5f72/lens-server/src/main/java/org/apache/lens/server/ui/MetastoreUIResource.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/ui/MetastoreUIResource.java b/lens-server/src/main/java/org/apache/lens/server/ui/MetastoreUIResource.java
index afb404b..b7c60bb 100644
--- a/lens-server/src/main/java/org/apache/lens/server/ui/MetastoreUIResource.java
+++ b/lens-server/src/main/java/org/apache/lens/server/ui/MetastoreUIResource.java
@@ -194,7 +194,7 @@ public class MetastoreUIResource {
         for (XExprColumn expr : table.getExpressions().getExpression()) {
           try {
             attribList.put(new JSONObject().put("name", expr.getName()).put("type", "expression")
-              .put("expression", expr.getExpr()));
+              .put("expression", expr.getExprSpec()));
           } catch (JSONException j) {
             LOG.error(j);
           }

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/365f5f72/lens-server/src/test/java/org/apache/lens/server/metastore/TestMetastoreService.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/metastore/TestMetastoreService.java b/lens-server/src/test/java/org/apache/lens/server/metastore/TestMetastoreService.java
index a35a8ff..885a69a 100644
--- a/lens-server/src/test/java/org/apache/lens/server/metastore/TestMetastoreService.java
+++ b/lens-server/src/test/java/org/apache/lens/server/metastore/TestMetastoreService.java
@@ -42,6 +42,7 @@ import org.apache.lens.api.LensSessionHandle;
 import org.apache.lens.api.StringList;
 import org.apache.lens.api.metastore.*;
 import org.apache.lens.cube.metadata.*;
+import org.apache.lens.cube.metadata.ExprColumn.ExprSpec;
 import org.apache.lens.server.LensJerseyTest;
 import org.apache.lens.server.LensServices;
 import org.apache.lens.server.LensTestUtil;
@@ -383,9 +384,34 @@ public class TestMetastoreService extends LensJerseyTest {
     xe1.setType("DOUBLE");
     xe1.setDescription("first expression");
     xe1.setDisplayString("Expression1");
-    xe1.setExpr("msr1/1000");
+    XExprSpec es = new XExprSpec();
+    es.setExpr("msr1/1000");
+    xe1.getExprSpec().add(es);
+
+    XExprColumn xe2 = new XExprColumn();
+    xe2.setName("expr2");
+    xe2.setType("float");
+    xe2.setDescription("multi expression");
+    xe2.setDisplayString("Expression2");
+    XExprSpec es1 = new XExprSpec();
+    es1.setExpr("msr1/1000");
+    xe2.getExprSpec().add(es1);
+    XExprSpec es2 = new XExprSpec();
+    es2.setExpr("(msr1/1000) + 0.01");
+    es2.setStartTime(startDate);
+    xe2.getExprSpec().add(es2);
+    XExprSpec es3 = new XExprSpec();
+    es3.setExpr("(msr1/1000) + 0.03");
+    es3.setEndTime(endDate);
+    xe2.getExprSpec().add(es3);
+    XExprSpec es4 = new XExprSpec();
+    es4.setExpr("(msr1/1000) - 0.01");
+    es4.setStartTime(startDate);
+    es4.setEndTime(endDate);
+    xe2.getExprSpec().add(es4);
 
     cube.getExpressions().getExpression().add(xe1);
+    cube.getExpressions().getExpression().add(xe2);
 
     XProperty xp1 = cubeObjectFactory.createXProperty();
     xp1.setName("foo");
@@ -650,6 +676,27 @@ public class TestMetastoreService extends LensJerseyTest {
       assertNotNull(hcube.getExpressionByName("expr1"));
       assertEquals(hcube.getExpressionByName("expr1").getDescription(), "first expression");
       assertEquals(hcube.getExpressionByName("expr1").getDisplayString(), "Expression1");
+      assertNotNull(hcube.getExpressionByName("expr2"));
+      assertEquals(hcube.getExpressionByName("expr2").getExpressions().size(), 4);
+      ExprColumn expr2 = hcube.getExpressionByName("expr2");
+      Iterator<ExprSpec> esIter = expr2.getExpressionSpecs().iterator();
+      ExprSpec first = esIter.next();
+      assertEquals(first.getExpr(), "msr1/1000");
+      assertNull(first.getStartTime());
+      assertNull(first.getEndTime());
+      ExprSpec second = esIter.next();
+      assertEquals(second.getExpr(), "(msr1/1000) + 0.01");
+      assertNotNull(second.getStartTime());
+      assertNull(second.getEndTime());
+      ExprSpec third = esIter.next();
+      assertEquals(third.getExpr(), "(msr1/1000) + 0.03");
+      assertNull(third.getStartTime());
+      assertNotNull(third.getEndTime());
+      ExprSpec last = esIter.next();
+      assertEquals(last.getExpr(), "(msr1/1000) - 0.01");
+      assertNotNull(last.getStartTime());
+      assertNotNull(last.getEndTime());
+      assertFalse(esIter.hasNext());
       Assert.assertFalse(hcube.getJoinChains().isEmpty());
       Assert.assertEquals(hcube.getJoinChains().size(), 2);
       Assert.assertTrue(hcube.getJoinChainNames().contains("chain1"));
@@ -1030,7 +1077,9 @@ public class TestMetastoreService extends LensJerseyTest {
     xe1.setType("STRING");
     xe1.setDescription("dimension expression");
     xe1.setDisplayString("Dim Expression");
-    xe1.setExpr("substr(col1, 3)");
+    XExprSpec es = new XExprSpec();
+    es.setExpr("substr(col1, 3)");
+    xe1.getExprSpec().add(es);
     dimension.getExpressions().getExpression().add(xe1);
 
     XProperty xp1 = cubeObjectFactory.createXProperty();
@@ -2298,6 +2347,7 @@ public class TestMetastoreService extends LensJerseyTest {
         "flattestcube.testdim2col2",
         "flattestcube.dim4",
         "flattestcube.expr1",
+        "flattestcube.expr2",
         "chain1-testdim.col2",
         "chain1-testdim.col1",
         "chain1-testdim.dimexpr",

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/365f5f72/lens-server/src/test/java/org/apache/lens/server/query/TestQueryService.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/query/TestQueryService.java b/lens-server/src/test/java/org/apache/lens/server/query/TestQueryService.java
index 6f45e88..8ecf2a8 100644
--- a/lens-server/src/test/java/org/apache/lens/server/query/TestQueryService.java
+++ b/lens-server/src/test/java/org/apache/lens/server/query/TestQueryService.java
@@ -259,8 +259,7 @@ public class TestQueryService extends LensJerseyTest {
     assertEquals(ctx.getDriverFinishTime(), 0);
     assertTrue(ctx.getFinishTime() > 0);
     Assert.assertEquals(ctx.getStatus().getStatus(), QueryStatus.Status.FAILED);
-    System.out.println("%% " + metricsSvc.getTotalFailedQueries());
-    Assert.assertEquals(metricsSvc.getTotalFailedQueries(), failedQueries + 1);
+    Assert.assertTrue(metricsSvc.getTotalFailedQueries() >= failedQueries + 1);
   }
 
   // test with execute async post, get all queries, get query context,