You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@griffin.apache.org by gu...@apache.org on 2017/09/30 08:35:21 UTC

[06/11] incubator-griffin git commit: Dsl modify

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/4aa6f779/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/PersistType.scala
----------------------------------------------------------------------
diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/PersistType.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/PersistType.scala
new file mode 100644
index 0000000..10b83c8
--- /dev/null
+++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/PersistType.scala
@@ -0,0 +1,58 @@
+/*
+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.griffin.measure.rule.dsl
+
+import scala.util.matching.Regex
+
+sealed trait PersistType {
+  val regex: Regex
+  val desc: String
+//  def temp: Boolean = false
+//  def persist: Boolean = false
+//  def collect: Boolean = false
+}
+
+object PersistType {
+  private val persistTypes: List[PersistType] = List(RecordPersistType, MetricPersistType, NonePersistType)
+  def apply(ptn: String): PersistType = {
+    persistTypes.filter(tp => ptn match {
+      case tp.regex() => true
+      case _ => false
+    }).headOption.getOrElse(NonePersistType)
+  }
+  def unapply(pt: PersistType): Option[String] = Some(pt.desc)
+}
+
+final case object NonePersistType extends PersistType {
+  val regex: Regex = "".r
+  val desc: String = "none"
+}
+
+final case object RecordPersistType extends PersistType {
+  val regex: Regex = "^(?i)record$".r
+  val desc: String = "record"
+//  override def temp: Boolean = true
+}
+
+final case object MetricPersistType extends PersistType {
+  val regex: Regex = "^(?i)metric$".r
+  val desc: String = "metric"
+//  override def temp: Boolean = true
+//  override def collect: Boolean = true
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/4aa6f779/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/analyzer/AccuracyAnalyzer.scala
----------------------------------------------------------------------
diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/analyzer/AccuracyAnalyzer.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/analyzer/AccuracyAnalyzer.scala
new file mode 100644
index 0000000..7efb32e
--- /dev/null
+++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/analyzer/AccuracyAnalyzer.scala
@@ -0,0 +1,41 @@
+/*
+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.griffin.measure.rule.dsl.analyzer
+
+import org.apache.griffin.measure.rule.dsl.expr._
+
+
+case class AccuracyAnalyzer(expr: LogicalExpr, sourceName: String, targetName: String) extends BasicAnalyzer {
+
+  val dataSourceNames = expr.preOrderTraverseDepthFirst(Set[String]())(seqDataSourceNames, combDataSourceNames)
+
+  val sourceSelectionExprs = {
+    val seq = seqSelectionExprs(sourceName)
+    expr.preOrderTraverseDepthFirst(Seq[SelectionExpr]())(seq, combSelectionExprs)
+  }
+  val targetSelectionExprs = {
+    val seq = seqSelectionExprs(targetName)
+    expr.preOrderTraverseDepthFirst(Seq[SelectionExpr]())(seq, combSelectionExprs)
+  }
+
+  val selectionExprs = sourceSelectionExprs ++ {
+    expr.preOrderTraverseDepthFirst(Seq[AliasableExpr]())(seqWithAliasExprs, combWithAliasExprs)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/4aa6f779/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/analyzer/BasicAnalyzer.scala
----------------------------------------------------------------------
diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/analyzer/BasicAnalyzer.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/analyzer/BasicAnalyzer.scala
new file mode 100644
index 0000000..063eb7b
--- /dev/null
+++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/analyzer/BasicAnalyzer.scala
@@ -0,0 +1,53 @@
+/*
+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.griffin.measure.rule.dsl.analyzer
+
+import org.apache.griffin.measure.rule.dsl.expr._
+
+
+trait BasicAnalyzer extends Serializable {
+
+  val expr: Expr
+
+  val seqDataSourceNames = (expr: Expr, v: Set[String]) => {
+    expr match {
+      case DataSourceHeadExpr(name) => v + name
+      case _ => v
+    }
+  }
+  val combDataSourceNames = (a: Set[String], b: Set[String]) => a ++ b
+
+  val seqSelectionExprs = (dsName: String) => (expr: Expr, v: Seq[SelectionExpr]) => {
+    expr match {
+      case se @ SelectionExpr(head: DataSourceHeadExpr, _, _) if (head.desc == dsName) => v :+ se
+      case _ => v
+    }
+  }
+  val combSelectionExprs = (a: Seq[SelectionExpr], b: Seq[SelectionExpr]) => a ++ b
+
+  val seqWithAliasExprs = (expr: Expr, v: Seq[AliasableExpr]) => {
+    expr match {
+      case se: SelectExpr => v
+      case a: AliasableExpr if (a.alias.nonEmpty) => v :+ a
+      case _ => v
+    }
+  }
+  val combWithAliasExprs = (a: Seq[AliasableExpr], b: Seq[AliasableExpr]) => a ++ b
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/4aa6f779/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/analyzer/ProfilingAnalyzer.scala
----------------------------------------------------------------------
diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/analyzer/ProfilingAnalyzer.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/analyzer/ProfilingAnalyzer.scala
new file mode 100644
index 0000000..34bdbd3
--- /dev/null
+++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/analyzer/ProfilingAnalyzer.scala
@@ -0,0 +1,52 @@
+/*
+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.griffin.measure.rule.dsl.analyzer
+
+import org.apache.griffin.measure.rule.dsl.expr._
+
+
+case class ProfilingAnalyzer(expr: ProfilingClause, sourceName: String) extends BasicAnalyzer {
+
+  val dataSourceNames = expr.preOrderTraverseDepthFirst(Set[String]())(seqDataSourceNames, combDataSourceNames)
+
+  val sourceSelectionExprs = {
+    val seq = seqSelectionExprs(sourceName)
+    expr.selectClause.preOrderTraverseDepthFirst(Seq[SelectionExpr]())(seq, combSelectionExprs)
+  }
+
+  val selectionExprs = expr.selectClause.exprs.map(_.extractSelf)
+  def containsAllSelectionExpr = {
+    selectionExprs.filter { expr =>
+      expr match {
+        case SelectionExpr(head: ALLSelectHeadExpr, selectors: Seq[SelectExpr], _) => {
+          selectors.isEmpty
+        }
+        case SelectionExpr(head: DataSourceHeadExpr, selectors: Seq[SelectExpr], _) => {
+          (head == sourceName) && (selectors.size == 1) && (selectors.head.isInstanceOf[AllFieldsSelectExpr])
+        }
+        case _ => false
+      }
+    }.size > 0
+  }
+
+  val groupbyExprOpt = expr.groupbyClauseOpt
+  val preGroupbyExprs = expr.preGroupbyClauses.map(_.extractSelf)
+  val postGroupbyExprs = expr.postGroupbyClauses.map(_.extractSelf)
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/4aa6f779/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/AliasableExpr.scala
----------------------------------------------------------------------
diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/AliasableExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/AliasableExpr.scala
new file mode 100644
index 0000000..33a12e0
--- /dev/null
+++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/AliasableExpr.scala
@@ -0,0 +1,25 @@
+/*
+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.griffin.measure.rule.dsl.expr
+
+trait AliasableExpr extends Expr {
+
+  def alias: Option[String]
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/4aa6f779/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/ClauseExpression.scala
----------------------------------------------------------------------
diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/ClauseExpression.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/ClauseExpression.scala
new file mode 100644
index 0000000..26882b4
--- /dev/null
+++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/ClauseExpression.scala
@@ -0,0 +1,150 @@
+/*
+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.griffin.measure.rule.dsl.expr
+
+trait ClauseExpression extends Expr {
+}
+
+case class SelectClause(exprs: Seq[Expr]) extends ClauseExpression {
+
+  addChildren(exprs)
+
+  def desc: String = s"${exprs.map(_.desc).mkString(", ")}"
+  def coalesceDesc: String = s"${exprs.map(_.desc).mkString(", ")}"
+
+}
+
+case class WhereClause(expr: Expr) extends ClauseExpression {
+
+  addChild(expr)
+
+  def desc: String = s"WHERE ${expr.desc}"
+  def coalesceDesc: String = s"WHERE ${expr.coalesceDesc}"
+
+}
+
+case class GroupbyClause(exprs: Seq[Expr], havingClauseOpt: Option[Expr]) extends ClauseExpression {
+
+  addChildren(exprs ++ havingClauseOpt.toSeq)
+
+  def desc: String = {
+    val gbs = exprs.map(_.desc).mkString(", ")
+    havingClauseOpt match {
+      case Some(having) => s"GROUP BY ${gbs} HAVING ${having.desc}"
+      case _ => s"GROUP BY ${gbs}"
+    }
+  }
+  def coalesceDesc: String = {
+    val gbs = exprs.map(_.desc).mkString(", ")
+    havingClauseOpt match {
+      case Some(having) => s"GROUP BY ${gbs} HAVING ${having.coalesceDesc}"
+      case _ => s"GROUP BY ${gbs}"
+    }
+  }
+
+  def merge(other: GroupbyClause): GroupbyClause = {
+    val newHavingClauseOpt = (havingClauseOpt, other.havingClauseOpt) match {
+      case (Some(hc), Some(ohc)) => {
+        val logical1 = LogicalFactorExpr(hc, false, None)
+        val logical2 = LogicalFactorExpr(ohc, false, None)
+        Some(BinaryLogicalExpr(logical1, ("AND", logical2) :: Nil))
+      }
+      case (a @ Some(_), _) => a
+      case (_, b @ Some(_)) => b
+      case (_, _) => None
+    }
+    GroupbyClause(exprs ++ other.exprs, newHavingClauseOpt)
+  }
+
+}
+
+case class OrderbyItem(expr: Expr, orderOpt: Option[String]) extends Expr {
+  addChild(expr)
+  def desc: String = {
+    orderOpt match {
+      case Some(os) => s"${expr.desc} ${os.toUpperCase}"
+      case _ => s"${expr.desc}"
+    }
+  }
+  def coalesceDesc: String = desc
+}
+
+case class OrderbyClause(items: Seq[OrderbyItem]) extends ClauseExpression {
+
+  addChildren(items.map(_.expr))
+
+  def desc: String = {
+    val obs = items.map(_.desc).mkString(", ")
+    s"ORDER BY ${obs}"
+  }
+  def coalesceDesc: String = {
+    val obs = items.map(_.desc).mkString(", ")
+    s"ORDER BY ${obs}"
+  }
+}
+
+case class LimitClause(expr: Expr) extends ClauseExpression {
+
+  addChild(expr)
+
+  def desc: String = s"LIMIT ${expr.desc}"
+  def coalesceDesc: String = s"LIMIT ${expr.coalesceDesc}"
+}
+
+case class CombinedClause(selectClause: SelectClause, tails: Seq[ClauseExpression]
+                         ) extends ClauseExpression {
+
+  addChildren(selectClause +: tails)
+
+  def desc: String = {
+    tails.foldLeft(selectClause.desc) { (head, tail) =>
+      s"${head} ${tail.desc}"
+    }
+  }
+  def coalesceDesc: String = {
+    tails.foldLeft(selectClause.coalesceDesc) { (head, tail) =>
+      s"${head} ${tail.coalesceDesc}"
+    }
+  }
+}
+
+case class ProfilingClause(selectClause: SelectClause, groupbyClauseOpt: Option[GroupbyClause],
+                           preGroupbyClauses: Seq[ClauseExpression],
+                           postGroupbyClauses: Seq[ClauseExpression]
+                          ) extends ClauseExpression {
+  addChildren(groupbyClauseOpt match {
+    case Some(gc) => (selectClause +: preGroupbyClauses) ++ (gc +: postGroupbyClauses)
+    case _ => (selectClause +: preGroupbyClauses) ++ postGroupbyClauses
+  })
+
+  def desc: String = {
+    val selectDesc = selectClause.desc
+    val groupbyDesc = groupbyClauseOpt.map(_.desc).mkString(" ")
+    val preDesc = preGroupbyClauses.map(_.desc).mkString(" ")
+    val postDesc = postGroupbyClauses.map(_.desc).mkString(" ")
+    s"${selectDesc} ${preDesc} ${groupbyDesc} ${postDesc}"
+  }
+  def coalesceDesc: String = {
+    val selectDesc = selectClause.coalesceDesc
+    val groupbyDesc = groupbyClauseOpt.map(_.coalesceDesc).mkString(" ")
+    val preDesc = preGroupbyClauses.map(_.coalesceDesc).mkString(" ")
+    val postDesc = postGroupbyClauses.map(_.coalesceDesc).mkString(" ")
+    s"${selectDesc} ${preDesc} ${groupbyDesc} ${postDesc}"
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/4aa6f779/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/Expr.scala
----------------------------------------------------------------------
diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/Expr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/Expr.scala
new file mode 100644
index 0000000..850579c
--- /dev/null
+++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/Expr.scala
@@ -0,0 +1,29 @@
+/*
+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.griffin.measure.rule.dsl.expr
+
+trait Expr extends TreeNode with Serializable {
+
+  def desc: String
+
+  def coalesceDesc: String
+
+  def extractSelf: Expr = this
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/4aa6f779/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/FunctionExpr.scala
----------------------------------------------------------------------
diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/FunctionExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/FunctionExpr.scala
new file mode 100644
index 0000000..b82fd96
--- /dev/null
+++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/FunctionExpr.scala
@@ -0,0 +1,29 @@
+/*
+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.griffin.measure.rule.dsl.expr
+
+case class FunctionExpr(functionName: String, args: Seq[Expr], aliasOpt: Option[String]
+                       ) extends Expr with AliasableExpr {
+
+  addChildren(args)
+
+  def desc: String = s"${functionName}(${args.map(_.desc).mkString(", ")})"
+  def coalesceDesc: String = desc
+  def alias: Option[String] = if (aliasOpt.isEmpty) Some(functionName) else aliasOpt
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/4aa6f779/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/LiteralExpr.scala
----------------------------------------------------------------------
diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/LiteralExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/LiteralExpr.scala
new file mode 100644
index 0000000..60290bc
--- /dev/null
+++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/LiteralExpr.scala
@@ -0,0 +1,72 @@
+/*
+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.griffin.measure.rule.dsl.expr
+
+import org.apache.griffin.measure.utils.TimeUtil
+
+trait LiteralExpr extends Expr {
+  def coalesceDesc: String = desc
+}
+
+case class LiteralNullExpr(str: String) extends LiteralExpr {
+  def desc: String = "NULL"
+}
+
+case class LiteralNanExpr(str: String) extends LiteralExpr {
+  def desc: String = "NaN"
+}
+
+case class LiteralStringExpr(str: String) extends LiteralExpr {
+  def desc: String = str
+}
+
+case class LiteralNumberExpr(str: String) extends LiteralExpr {
+  def desc: String = {
+    try {
+      if (str.contains(".")) {
+        str.toDouble.toString
+      } else {
+        str.toLong.toString
+      }
+    } catch {
+      case e: Throwable => throw new Exception(s"${str} is invalid number")
+    }
+  }
+}
+
+case class LiteralTimeExpr(str: String) extends LiteralExpr {
+  def desc: String = {
+    TimeUtil.milliseconds(str) match {
+      case Some(t) => t.toString
+      case _ => throw new Exception(s"${str} is invalid time")
+    }
+  }
+}
+
+case class LiteralBooleanExpr(str: String) extends LiteralExpr {
+  final val TrueRegex = """(?i)true""".r
+  final val FalseRegex = """(?i)false""".r
+  def desc: String = {
+    str match {
+      case TrueRegex() => true.toString
+      case FalseRegex() => false.toString
+      case _ => throw new Exception(s"${str} is invalid boolean")
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/4aa6f779/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/LogicalExpr.scala
----------------------------------------------------------------------
diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/LogicalExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/LogicalExpr.scala
new file mode 100644
index 0000000..4b16219
--- /dev/null
+++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/LogicalExpr.scala
@@ -0,0 +1,170 @@
+/*
+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.griffin.measure.rule.dsl.expr
+
+trait LogicalExpr extends Expr {
+}
+
+case class InExpr(head: Expr, is: Boolean, range: Seq[Expr]) extends LogicalExpr {
+
+  addChildren(head +: range)
+
+  def desc: String = {
+    val notStr = if (is) "" else " NOT"
+    s"${head.desc}${notStr} IN (${range.map(_.desc).mkString(", ")})"
+  }
+  def coalesceDesc: String = {
+    val notStr = if (is) "" else " NOT"
+    s"${head.coalesceDesc}${notStr} IN (${range.map(_.coalesceDesc).mkString(", ")})"
+  }
+}
+
+case class BetweenExpr(head: Expr, is: Boolean, range: Seq[Expr]) extends LogicalExpr {
+
+  range match {
+    case first :: second :: _ => addChildren(head :: first :: second :: Nil)
+    case _ => throw new Exception("between expression exception: range less than 2")
+  }
+
+  def desc: String = {
+    val notStr = if (is) "" else " NOT"
+    val rangeStr = range match {
+      case first :: second :: _ => s"${first.desc} AND ${second.desc}"
+      case _ => throw new Exception("between expression exception: range less than 2")
+    }
+    s"${head.desc}${notStr} BETWEEN ${rangeStr}"
+  }
+  def coalesceDesc: String = {
+    val notStr = if (is) "" else " NOT"
+    val rangeStr = range match {
+      case first :: second :: _ => s"${first.coalesceDesc} AND ${second.coalesceDesc}"
+      case _ => throw new Exception("between expression exception: range less than 2")
+    }
+    s"${head.coalesceDesc}${notStr} BETWEEN ${rangeStr}"
+  }
+}
+
+case class LikeExpr(head: Expr, is: Boolean, value: Expr) extends LogicalExpr {
+
+  addChildren(head :: value :: Nil)
+
+  def desc: String = {
+    val notStr = if (is) "" else " NOT"
+    s"${head.desc}${notStr} LIKE ${value.desc}"
+  }
+  def coalesceDesc: String = {
+    val notStr = if (is) "" else " NOT"
+    s"${head.coalesceDesc}${notStr} LIKE ${value.coalesceDesc}"
+  }
+}
+
+case class IsNullExpr(head: Expr, is: Boolean) extends LogicalExpr {
+
+  addChild(head)
+
+  def desc: String = {
+    val notStr = if (is) "" else " NOT"
+    s"${head.desc} IS${notStr} NULL"
+  }
+  def coalesceDesc: String = desc
+}
+
+case class IsNanExpr(head: Expr, is: Boolean) extends LogicalExpr {
+
+  addChild(head)
+
+  def desc: String = {
+    val notStr = if (is) "" else "NOT "
+    s"${notStr}isnan(${head.desc})"
+  }
+  def coalesceDesc: String = desc
+}
+
+// -----------
+
+case class LogicalFactorExpr(factor: Expr, withBracket: Boolean, aliasOpt: Option[String]
+                            ) extends LogicalExpr with AliasableExpr {
+
+  addChild(factor)
+
+  def desc: String = if (withBracket) s"(${factor.desc})" else factor.desc
+  def coalesceDesc: String = factor.coalesceDesc
+  def alias: Option[String] = aliasOpt
+  override def extractSelf: Expr = {
+    if (aliasOpt.nonEmpty) this
+    else factor.extractSelf
+  }
+}
+
+case class UnaryLogicalExpr(oprs: Seq[String], factor: LogicalExpr) extends LogicalExpr {
+
+  addChild(factor)
+
+  def desc: String = {
+    oprs.foldRight(factor.desc) { (opr, fac) =>
+      s"(${trans(opr)} ${fac})"
+    }
+  }
+  def coalesceDesc: String = {
+    oprs.foldRight(factor.coalesceDesc) { (opr, fac) =>
+      s"(${trans(opr)} ${fac})"
+    }
+  }
+  private def trans(s: String): String = {
+    s match {
+      case "!" => "NOT"
+      case _ => s.toUpperCase
+    }
+  }
+  override def extractSelf: Expr = {
+    if (oprs.nonEmpty) this
+    else factor.extractSelf
+  }
+}
+
+case class BinaryLogicalExpr(factor: LogicalExpr, tails: Seq[(String, LogicalExpr)]) extends LogicalExpr {
+
+  addChildren(factor +: tails.map(_._2))
+
+  def desc: String = {
+    val res = tails.foldLeft(factor.desc) { (fac, tail) =>
+      val (opr, expr) = tail
+      s"${fac} ${trans(opr)} ${expr.desc}"
+    }
+    if (tails.size <= 0) res else s"${res}"
+  }
+  def coalesceDesc: String = {
+    val res = tails.foldLeft(factor.coalesceDesc) { (fac, tail) =>
+      val (opr, expr) = tail
+      s"${fac} ${trans(opr)} ${expr.coalesceDesc}"
+    }
+    if (tails.size <= 0) res else s"${res}"
+  }
+  private def trans(s: String): String = {
+    s match {
+      case "&&" => "AND"
+      case "||" => "OR"
+      case _ => s.trim.toUpperCase
+    }
+  }
+  override def extractSelf: Expr = {
+    if (tails.nonEmpty) this
+    else factor.extractSelf
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/4aa6f779/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/MathExpr.scala
----------------------------------------------------------------------
diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/MathExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/MathExpr.scala
new file mode 100644
index 0000000..b3d3db4
--- /dev/null
+++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/MathExpr.scala
@@ -0,0 +1,80 @@
+/*
+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.griffin.measure.rule.dsl.expr
+
+trait MathExpr extends Expr {
+}
+
+case class MathFactorExpr(factor: Expr, withBracket: Boolean, aliasOpt: Option[String]
+                         ) extends MathExpr with AliasableExpr {
+
+  addChild(factor)
+
+  def desc: String = if (withBracket) s"(${factor.desc})" else factor.desc
+  def coalesceDesc: String = factor.coalesceDesc
+  def alias: Option[String] = aliasOpt
+  override def extractSelf: Expr = {
+    if (aliasOpt.nonEmpty) this
+    else factor.extractSelf
+  }
+}
+
+case class UnaryMathExpr(oprs: Seq[String], factor: MathExpr) extends MathExpr {
+
+  addChild(factor)
+
+  def desc: String = {
+    oprs.foldRight(factor.desc) { (opr, fac) =>
+      s"(${opr}${fac})"
+    }
+  }
+  def coalesceDesc: String = {
+    oprs.foldRight(factor.coalesceDesc) { (opr, fac) =>
+      s"(${opr}${fac})"
+    }
+  }
+  override def extractSelf: Expr = {
+    if (oprs.nonEmpty) this
+    else factor.extractSelf
+  }
+}
+
+case class BinaryMathExpr(factor: MathExpr, tails: Seq[(String, MathExpr)]) extends MathExpr {
+
+  addChildren(factor +: tails.map(_._2))
+
+  def desc: String = {
+    val res = tails.foldLeft(factor.desc) { (fac, tail) =>
+      val (opr, expr) = tail
+      s"${fac} ${opr} ${expr.desc}"
+    }
+    if (tails.size <= 0) res else s"${res}"
+  }
+  def coalesceDesc: String = {
+    val res = tails.foldLeft(factor.coalesceDesc) { (fac, tail) =>
+      val (opr, expr) = tail
+      s"${fac} ${opr} ${expr.coalesceDesc}"
+    }
+    if (tails.size <= 0) res else s"${res}"
+  }
+  override def extractSelf: Expr = {
+    if (tails.nonEmpty) this
+    else factor.extractSelf
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/4aa6f779/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/SelectExpr.scala
----------------------------------------------------------------------
diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/SelectExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/SelectExpr.scala
new file mode 100644
index 0000000..fd803a8
--- /dev/null
+++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/SelectExpr.scala
@@ -0,0 +1,115 @@
+/*
+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.griffin.measure.rule.dsl.expr
+
+trait HeadExpr extends Expr {
+
+}
+
+case class DataSourceHeadExpr(name: String) extends HeadExpr {
+  def desc: String = name
+  def coalesceDesc: String = desc
+}
+
+case class FieldNameHeadExpr(field: String) extends HeadExpr {
+  def desc: String = field
+  def coalesceDesc: String = desc
+}
+
+case class ALLSelectHeadExpr() extends HeadExpr {
+  def desc: String = "*"
+  def coalesceDesc: String = desc
+}
+
+case class OtherHeadExpr(expr: Expr) extends HeadExpr {
+
+  addChild(expr)
+
+  def desc: String = expr.desc
+  def coalesceDesc: String = expr.coalesceDesc
+}
+
+// -------------
+
+trait SelectExpr extends Expr with AliasableExpr {
+}
+
+case class AllFieldsSelectExpr() extends SelectExpr {
+  def desc: String = s".*"
+  def coalesceDesc: String = desc
+  def alias: Option[String] = None
+}
+
+case class FieldSelectExpr(field: String) extends SelectExpr {
+  def desc: String = s".${field}"
+  def coalesceDesc: String = desc
+  def alias: Option[String] = Some(field)
+}
+
+case class IndexSelectExpr(index: Expr) extends SelectExpr {
+
+  addChild(index)
+
+  def desc: String = s"[${index.desc}]"
+  def coalesceDesc: String = desc
+  def alias: Option[String] = Some(desc)
+}
+
+case class FunctionSelectExpr(functionName: String, args: Seq[Expr]) extends SelectExpr {
+
+  addChildren(args)
+
+  def desc: String = ""
+  def coalesceDesc: String = desc
+  def alias: Option[String] = Some(functionName)
+}
+
+// -------------
+
+case class SelectionExpr(head: HeadExpr, selectors: Seq[SelectExpr], aliasOpt: Option[String]) extends SelectExpr {
+
+  addChildren(head +: selectors)
+
+  def desc: String = {
+    selectors.foldLeft(head.desc) { (hd, sel) =>
+      sel match {
+        case FunctionSelectExpr(funcName, args) => {
+          val nargs = hd +: args.map(_.desc)
+          s"${funcName}(${nargs.mkString(", ")})"
+        }
+        case _ => s"${hd}${sel.desc}"
+      }
+    }
+  }
+  def coalesceDesc: String = {
+    selectors.lastOption match {
+      case None => desc
+      case Some(sel: FunctionSelectExpr) => desc
+      case _ => s"coalesce(${desc}, 'null')"
+    }
+  }
+  def alias: Option[String] = {
+    if (aliasOpt.isEmpty) {
+      selectors.lastOption match {
+        case Some(last) => last.alias
+        case _ => None
+      }
+    } else aliasOpt
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/4aa6f779/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/TreeNode.scala
----------------------------------------------------------------------
diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/TreeNode.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/TreeNode.scala
new file mode 100644
index 0000000..aab16b4
--- /dev/null
+++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/TreeNode.scala
@@ -0,0 +1,45 @@
+/*
+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.griffin.measure.rule.dsl.expr
+
+trait TreeNode extends Serializable {
+
+  var children = Seq[TreeNode]()
+
+  def addChild(expr: TreeNode) = { children :+= expr }
+  def addChildren(exprs: Seq[TreeNode]) = { children ++= exprs }
+
+  def preOrderTraverseDepthFirst[T, A <: TreeNode](z: T)(seqOp: (A, T) => T, combOp: (T, T) => T): T = {
+    if (this.isInstanceOf[A]) {
+      val tv = seqOp(this.asInstanceOf[A], z)
+      children.foldLeft(combOp(z, tv)) { (ov, tn) =>
+        combOp(ov, tn.preOrderTraverseDepthFirst(z)(seqOp, combOp))
+      }
+    } else z
+  }
+  def postOrderTraverseDepthFirst[T, A <: TreeNode](z: T)(seqOp: (A, T) => T, combOp: (T, T) => T): T = {
+    if (this.isInstanceOf[A]) {
+      val cv = children.foldLeft(z) { (ov, tn) =>
+        combOp(ov, tn.postOrderTraverseDepthFirst(z)(seqOp, combOp))
+      }
+      combOp(z, seqOp(this.asInstanceOf[A], cv))
+    } else z
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/4aa6f779/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParser.scala
----------------------------------------------------------------------
diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParser.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParser.scala
new file mode 100644
index 0000000..0431354
--- /dev/null
+++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParser.scala
@@ -0,0 +1,337 @@
+/*
+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.griffin.measure.rule.dsl.parser
+
+import org.apache.griffin.measure.rule.dsl.expr._
+
+import scala.util.parsing.combinator.JavaTokenParsers
+
+trait BasicParser extends JavaTokenParsers with Serializable {
+
+  val dataSourceNames: Seq[String]
+  val functionNames: Seq[String]
+
+  /**
+    * BNF for basic parser
+    *
+    * -- literal --
+    * <literal> ::= <literal-string> | <literal-number> | <literal-time> | <literal-boolean> | <literal-null> | <literal-nan>
+    * <literal-string> ::= <any-string>
+    * <literal-number> ::= <integer> | <double>
+    * <literal-time> ::= <integer> ("d"|"h"|"m"|"s"|"ms")
+    * <literal-boolean> ::= true | false
+    * <literal-null> ::= null
+    * <literal-nan> ::= nan
+    *
+    * -- selection --
+    * <selection> ::= <selection-head> [ <field-sel> | <index-sel> | <function-sel> ]*
+    * <selection-head> ::= ("data source name registered") | <function>
+    * <field-sel> ::= "." <field-name> | "[" <quote-field-name> "]"
+    * <index-sel> ::= "[" <arg> "]"
+    * <function-sel> ::= "." <function-name> "(" [<arg>]? [, <arg>]* ")"
+    * <arg> ::= <math-expr>
+    *
+    * -- math expr --
+    * <math-factor> ::= <literal> | <alias-expr> | <function> | <selection> | "(" <math-expr> ")"
+    * <unary-math-expr> ::= [<unary-opr>]* <math-factor>
+    * <binary-math-expr> ::= <unary-math-expr> [<binary-opr> <unary-math-expr>]+
+    * <math-expr> ::= <binary-math-expr>
+    *
+    * -- logical expr --
+    * <in-expr> ::= <math-expr> [<not>]? <in> <range-expr>
+    * <between-expr> ::= <math-expr> [<not>]? <between> (<math-expr> <and> <math-expr> | <range-expr>)
+    * <range-expr> ::= "(" [<math-expr>]? [, <math-expr>]+ ")"
+    * <like-expr> ::= <math-expr> [<not>]? <like> <math-expr>
+    * <is-null-expr> ::= <math-expr> <is> [<not>]? <null>
+    * <is-nan-expr> ::= <math-expr> <is> [<not>]? <nan>
+    *
+    * <logical-factor> ::= <math-expr> | <in-expr> | <between-expr> | <like-expr> | <is-null-expr> | <is-nan-expr> | "(" <logical-expr> ")"
+    * <unary-logical-expr> ::= [<unary-logical-opr>]* <logical-factor>
+    * <binary-logical-expr> ::= <unary-logical-expr> [<binary-logical-opr> <unary-logical-expr>]+
+    * <logical-expr> ::= <binary-logical-expr>
+    *
+    * -- expression --
+    * <expr> = <math-expr> | <logical-expr>
+    *
+    * -- function expr --
+    * <function> ::= <function-name> "(" [<arg>] [, <arg>]+ ")"
+    * <function-name> ::= ("function name registered")
+    * <arg> ::= <expr>
+    *
+    * -- alias expr --
+    * <alias-expr> = <expr> <as> <name>
+    */
+
+  protected def genNamesParser(names: Seq[String]): Parser[String] = {
+    names.reverse.map {
+      fn => s"""(?i)${fn}""".r: Parser[String]
+    }.reduce(_ | _)
+  }
+
+  object Literal {
+    val NULL: Parser[String] = """(?i)null""".r
+    val NAN: Parser[String] = """(?i)nan""".r
+  }
+  import Literal._
+
+  object Operator {
+    val MATH_UNARY: Parser[String] = "+" | "-"
+    val MATH_BINARIES: Seq[Parser[String]] = Seq(("*" | "/" | "%"), ("+" | "-"))
+
+    val NOT: Parser[String] = """(?i)not\s""".r | "!"
+    val AND: Parser[String] = """(?i)and\s""".r | "&&"
+    val OR: Parser[String] = """(?i)or\s""".r | "||"
+    val IN: Parser[String] = """(?i)in\s""".r
+    val BETWEEN: Parser[String] = """(?i)between\s""".r
+    val AND_ONLY: Parser[String] = """(?i)and\s""".r
+    val IS: Parser[String] = """(?i)is\s""".r
+    val LIKE: Parser[String] = """(?i)like\s""".r
+    val COMPARE: Parser[String] = "=" | "!=" | "<>" | "<=" | ">=" | "<" | ">"
+    val LOGICAL_UNARY: Parser[String] = NOT
+    val LOGICAL_BINARIES: Seq[Parser[String]] = Seq((COMPARE), (AND), (OR))
+
+    val LSQBR: Parser[String] = "["
+    val RSQBR: Parser[String] = "]"
+    val LBR: Parser[String] = "("
+    val RBR: Parser[String] = ")"
+
+    val DOT: Parser[String] = "."
+    val ALLSL: Parser[String] = "*"
+    val SQUOTE: Parser[String] = "'"
+    val DQUOTE: Parser[String] = "\""
+    val UQUOTE: Parser[String] = "`"
+    val COMMA: Parser[String] = ","
+
+    val AS: Parser[String] = """(?i)as\s""".r
+    val WHERE: Parser[String] = """(?i)where\s""".r
+    val GROUP: Parser[String] = """(?i)group\s""".r
+    val ORDER: Parser[String] = """(?i)order\s""".r
+    val BY: Parser[String] = """(?i)by\s""".r
+    val DESC: Parser[String] = """(?i)desc""".r
+    val ASC: Parser[String] = """(?i)asc""".r
+    val HAVING: Parser[String] = """(?i)having\s""".r
+    val LIMIT: Parser[String] = """(?i)limit\s""".r
+  }
+  import Operator._
+
+  object Strings {
+    def AnyString: Parser[String] = """"(?:[^\"]|\")*"""".r | """'(?:[^']|\')*'""".r
+    def UQuoteTableFieldName: Parser[String] = """`(?:[^`]|[\\][`])*`""".r
+    def TableFieldName: Parser[String] = UQuoteTableFieldName | """[a-zA-Z_]\w*""".r
+    def DataSourceName: Parser[String] = genNamesParser(dataSourceNames)
+    def FunctionName: Parser[String] = genNamesParser(functionNames)
+
+    def IntegerNumber: Parser[String] = """[+\-]?\d+""".r
+    def DoubleNumber: Parser[String] = """[+\-]?(\.\d+|\d+\.\d*)""".r
+    def IndexNumber: Parser[String] = IntegerNumber
+
+    def TimeString: Parser[String] = """([+\-]?\d+)(d|h|m|s|ms)""".r
+    def BooleanString: Parser[String] = """(?i)true|false""".r
+  }
+  import Strings._
+
+  /**
+    * -- literal --
+    * <literal> ::= <literal-string> | <literal-number> | <literal-time> | <literal-boolean> | <literal-null> | <literal-nan>
+    * <literal-string> ::= <any-string>
+    * <literal-number> ::= <integer> | <double>
+    * <literal-time> ::= <integer> ("d"|"h"|"m"|"s"|"ms")
+    * <literal-boolean> ::= true | false
+    * <literal-null> ::= null
+    * <literal-nan> ::= nan
+    */
+  def literal: Parser[LiteralExpr] = literalNull | literalNan | literalBoolean | literalString | literalTime | literalNumber
+  def literalNull: Parser[LiteralNullExpr] = NULL ^^ { LiteralNullExpr(_) }
+  def literalNan: Parser[LiteralNanExpr] = NAN ^^ { LiteralNanExpr(_) }
+  def literalString: Parser[LiteralStringExpr] = AnyString ^^ { LiteralStringExpr(_) }
+  def literalNumber: Parser[LiteralNumberExpr] = (DoubleNumber | IntegerNumber) ^^ { LiteralNumberExpr(_) }
+  def literalTime: Parser[LiteralTimeExpr] = TimeString ^^ { LiteralTimeExpr(_) }
+  def literalBoolean: Parser[LiteralBooleanExpr] = BooleanString ^^ { LiteralBooleanExpr(_) }
+
+  /**
+    * -- selection --
+    * <selection> ::= <selection-head> [ <field-sel> | <index-sel> | <function-sel> ]*
+    * <selection-head> ::= ("data source name registered") | <function>
+    * <field-sel> ::= "." <field-name> | "[" <quote-field-name> "]"
+    * <index-sel> ::= "[" <arg> "]"
+    * <function-sel> ::= "." <function-name> "(" [<arg>]? [, <arg>]* ")"
+    * <arg> ::= <math-expr>
+    */
+
+  def selection: Parser[SelectionExpr] = selectionHead ~ rep(selector) ~ opt(asAlias) ^^ {
+    case head ~ sels ~ aliasOpt => SelectionExpr(head, sels, aliasOpt)
+  }
+  def selectionHead: Parser[HeadExpr] = DataSourceName ^^ {
+    DataSourceHeadExpr(_)
+  } | function ^^ {
+    OtherHeadExpr(_)
+  } | TableFieldName ^^ {
+    FieldNameHeadExpr(_)
+  } | ALLSL ^^ { _ =>
+    ALLSelectHeadExpr()
+  }
+  def selector: Parser[SelectExpr] = functionSelect | allFieldsSelect | fieldSelect | indexSelect
+  def allFieldsSelect: Parser[AllFieldsSelectExpr] = DOT ~> ALLSL ^^ { _ => AllFieldsSelectExpr() }
+  def fieldSelect: Parser[FieldSelectExpr] = DOT ~> TableFieldName ^^ { FieldSelectExpr(_) }
+  def indexSelect: Parser[IndexSelectExpr] = LSQBR ~> argument <~ RSQBR ^^ { IndexSelectExpr(_) }
+  def functionSelect: Parser[FunctionSelectExpr] = DOT ~ FunctionName ~ LBR ~ repsep(argument, COMMA) ~ RBR ^^ {
+    case _ ~ name ~ _ ~ args ~ _ => FunctionSelectExpr(name, args)
+  }
+
+  def asAlias: Parser[String] = AS ~> TableFieldName
+
+  /**
+    * -- math expr --
+    * <math-factor> ::= <literal> | <alias-expr> | <function> | <selection> | "(" <math-expr> ")"
+    * <unary-math-expr> ::= [<unary-opr>]* <math-factor>
+    * <binary-math-expr> ::= <unary-math-expr> [<binary-opr> <unary-math-expr>]+
+    * <math-expr> ::= <binary-math-expr>
+    */
+
+  def mathFactor: Parser[MathExpr] = (literal | function | selection) ^^ {
+    MathFactorExpr(_, false, None)
+  } | LBR ~ mathExpression ~ RBR ~ opt(asAlias) ^^ {
+    case _ ~ expr ~ _ ~ aliasOpt => MathFactorExpr(expr, true, aliasOpt)
+  }
+  def unaryMathExpression: Parser[MathExpr] = rep(MATH_UNARY) ~ mathFactor ^^ {
+    case Nil ~ a => a
+    case list ~ a => UnaryMathExpr(list, a)
+  }
+  def binaryMathExpressions: Seq[Parser[MathExpr]] =
+    MATH_BINARIES.foldLeft(List[Parser[MathExpr]](unaryMathExpression)) { (parsers, binaryParser) =>
+      val pre = parsers.head
+      val cur = pre ~ rep(binaryParser ~ pre) ^^ {
+        case a ~ Nil => a
+        case a ~ list => BinaryMathExpr(a, list.map(c => (c._1, c._2)))
+      }
+      cur :: parsers
+    }
+  def mathExpression: Parser[MathExpr] = binaryMathExpressions.head
+
+  /**
+    * -- logical expr --
+    * <in-expr> ::= <math-expr> [<not>]? <in> <range-expr>
+    * <between-expr> ::= <math-expr> [<not>]? <between> (<math-expr> <and> <math-expr> | <range-expr>)
+    * <range-expr> ::= "(" [<math-expr>]? [, <math-expr>]+ ")"
+    * <like-expr> ::= <math-expr> [<not>]? <like> <math-expr>
+    * <is-null-expr> ::= <math-expr> <is> [<not>]? <null>
+    * <is-nan-expr> ::= <math-expr> <is> [<not>]? <nan>
+    *
+    * <logical-factor> ::= <math-expr> | <in-expr> | <between-expr> | <like-expr> | <is-null-expr> | <is-nan-expr> | "(" <logical-expr> ")"
+    * <unary-logical-expr> ::= [<unary-logical-opr>]* <logical-factor>
+    * <binary-logical-expr> ::= <unary-logical-expr> [<binary-logical-opr> <unary-logical-expr>]+
+    * <logical-expr> ::= <binary-logical-expr>
+    */
+
+  def inExpr: Parser[LogicalExpr] = mathExpression ~ opt(NOT) ~ IN ~ LBR ~ repsep(mathExpression, COMMA) ~ RBR ^^ {
+    case head ~ notOpt ~ _ ~ _ ~ list ~ _ => InExpr(head, notOpt.isEmpty, list)
+  }
+  def betweenExpr: Parser[LogicalExpr] = mathExpression ~ opt(NOT) ~ BETWEEN ~ LBR ~ repsep(mathExpression, COMMA) ~ RBR ^^ {
+    case head ~ notOpt ~ _ ~ _ ~ list ~ _ => BetweenExpr(head, notOpt.isEmpty, list)
+  } | mathExpression ~ opt(NOT) ~ BETWEEN ~ mathExpression ~ AND_ONLY ~ mathExpression ^^ {
+    case head ~ notOpt ~ _ ~ first ~ _ ~ second => BetweenExpr(head, notOpt.isEmpty, Seq(first, second))
+  }
+  def likeExpr: Parser[LogicalExpr] = mathExpression ~ opt(NOT) ~ LIKE ~ mathExpression ^^ {
+    case head ~ notOpt ~ _ ~ value => LikeExpr(head, notOpt.isEmpty, value)
+  }
+  def isNullExpr: Parser[LogicalExpr] = mathExpression ~ IS ~ opt(NOT) ~ NULL ^^ {
+    case head ~ _ ~ notOpt ~ _ => IsNullExpr(head, notOpt.isEmpty)
+  }
+  def isNanExpr: Parser[LogicalExpr] = mathExpression ~ IS ~ opt(NOT) ~ NAN ^^ {
+    case head ~ _ ~ notOpt ~ _ => IsNanExpr(head, notOpt.isEmpty)
+  }
+
+  def logicalFactor: Parser[LogicalExpr] = (inExpr | betweenExpr | likeExpr | isNullExpr | isNanExpr | mathExpression) ^^ {
+    LogicalFactorExpr(_, false, None)
+  } | LBR ~ logicalExpression ~ RBR ~ opt(asAlias) ^^ {
+    case _ ~ expr ~ _ ~ aliasOpt => LogicalFactorExpr(expr, true, aliasOpt)
+  }
+  def unaryLogicalExpression: Parser[LogicalExpr] = rep(LOGICAL_UNARY) ~ logicalFactor ^^ {
+    case Nil ~ a => a
+    case list ~ a => UnaryLogicalExpr(list, a)
+  }
+  def binaryLogicalExpressions: Seq[Parser[LogicalExpr]] =
+    LOGICAL_BINARIES.foldLeft(List[Parser[LogicalExpr]](unaryLogicalExpression)) { (parsers, binaryParser) =>
+      val pre = parsers.head
+      val cur = pre ~ rep(binaryParser ~ pre) ^^ {
+        case a ~ Nil => a
+        case a ~ list => BinaryLogicalExpr(a, list.map(c => (c._1, c._2)))
+      }
+      cur :: parsers
+    }
+  def logicalExpression: Parser[LogicalExpr] = binaryLogicalExpressions.head
+
+  /**
+    * -- expression --
+    * <expr> = <math-expr> | <logical-expr>
+    */
+
+  def expression: Parser[Expr] = logicalExpression | mathExpression
+
+  /**
+    * -- function expr --
+    * <function> ::= <function-name> "(" [<arg>] [, <arg>]+ ")"
+    * <function-name> ::= ("function name registered")
+    * <arg> ::= <expr>
+    */
+
+  def function: Parser[FunctionExpr] = FunctionName ~ LBR ~ repsep(argument, COMMA) ~ RBR ~ opt(asAlias) ^^ {
+    case name ~ _ ~ args ~ _ ~ aliasOpt => FunctionExpr(name, args, aliasOpt)
+  }
+  def argument: Parser[Expr] = expression
+
+  /**
+    * -- clauses --
+    * <select-clause> = <expr> [, <expr>]*
+    * <where-clause> = <where> <expr>
+    * <having-clause> = <having> <expr>
+    * <groupby-clause> = <group> <by> <expr> [ <having-clause> ]?
+    * <orderby-item> = <expr> [ <DESC> ]?
+    * <orderby-clause> = <order> <by> <orderby-item> [ , <orderby-item> ]*
+    * <limit-clause> = <limit> <expr>
+    */
+
+  def selectClause: Parser[SelectClause] = rep1sep(expression, COMMA) ^^ { SelectClause(_) }
+  def whereClause: Parser[WhereClause] = WHERE ~> expression ^^ { WhereClause(_) }
+  def havingClause: Parser[Expr] = HAVING ~> expression
+  def groupbyClause: Parser[GroupbyClause] = GROUP ~ BY ~ rep1sep(expression, COMMA) ~ opt(havingClause) ^^ {
+    case _ ~ _ ~ cols ~ havingOpt => GroupbyClause(cols, havingOpt)
+  }
+  def orderbyItem: Parser[OrderbyItem] = expression ~ opt(DESC | ASC) ^^ {
+    case expr ~ orderOpt => OrderbyItem(expr, orderOpt)
+  }
+  def orderbyClause: Parser[OrderbyClause] = ORDER ~ BY ~ rep1sep(orderbyItem, COMMA) ^^ {
+    case _ ~ _ ~ cols => OrderbyClause(cols)
+  }
+  def limitClause: Parser[LimitClause] = LIMIT ~> expression ^^ { LimitClause(_) }
+
+  /**
+    * -- combined clauses --
+    * <combined-clauses> = <select-clause> [ <where-clause> ]+ [ <groupby-clause> ]+ [ <orderby-clause> ]+ [ <limit-clause> ]+
+    */
+
+  def combinedClause: Parser[CombinedClause] = selectClause ~ opt(whereClause) ~
+    opt(groupbyClause) ~ opt(orderbyClause) ~ opt(limitClause) ^^ {
+    case sel ~ whereOpt ~ groupbyOpt ~ orderbyOpt ~ limitOpt => {
+      val tails = Seq(whereOpt, groupbyOpt, orderbyOpt, limitOpt).flatMap(opt => opt)
+      CombinedClause(sel, tails)
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/4aa6f779/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/GriffinDslParser.scala
----------------------------------------------------------------------
diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/GriffinDslParser.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/GriffinDslParser.scala
new file mode 100644
index 0000000..637decb
--- /dev/null
+++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/GriffinDslParser.scala
@@ -0,0 +1,50 @@
+/*
+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.griffin.measure.rule.dsl.parser
+
+import org.apache.griffin.measure.rule.dsl._
+import org.apache.griffin.measure.rule.dsl.expr._
+
+case class GriffinDslParser(dataSourceNames: Seq[String], functionNames: Seq[String]
+                           ) extends BasicParser {
+
+  /**
+    * -- profiling clauses --
+    * <profiling-clauses> = <select-clause> [ <where-clause> ]+ [ <groupby-clause> ]+ [ <orderby-clause> ]+ [ <limit-clause> ]+
+    */
+
+  def profilingClause: Parser[ProfilingClause] = selectClause ~ opt(whereClause) ~
+    opt(groupbyClause) ~ opt(orderbyClause) ~ opt(limitClause) ^^ {
+    case sel ~ whereOpt ~ groupbyOpt ~ orderbyOpt ~ limitOpt => {
+      val preClauses = Seq(whereOpt).flatMap(opt => opt)
+      val postClauses = Seq(orderbyOpt, limitOpt).flatMap(opt => opt)
+      ProfilingClause(sel, groupbyOpt, preClauses, postClauses)
+    }
+  }
+
+  def parseRule(rule: String, dqType: DqType): ParseResult[Expr] = {
+    val rootExpr = dqType match {
+      case AccuracyType => logicalExpression
+      case ProfilingType => profilingClause
+      case _ => expression
+    }
+    parseAll(rootExpr, rule)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/4aa6f779/measure/src/main/scala/org/apache/griffin/measure/rule/expr/AnalyzableExpr.scala
----------------------------------------------------------------------
diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/AnalyzableExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/AnalyzableExpr.scala
deleted file mode 100644
index aefcaad..0000000
--- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/AnalyzableExpr.scala
+++ /dev/null
@@ -1,24 +0,0 @@
-/*
-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.griffin.measure.rule.expr
-
-
-trait AnalyzableExpr extends Serializable {
-  def getGroupbyExprPairs(dsPair: (String, String)): Seq[(Expr, Expr)] = Nil
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/4aa6f779/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Cacheable.scala
----------------------------------------------------------------------
diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Cacheable.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Cacheable.scala
deleted file mode 100644
index feb8156..0000000
--- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Cacheable.scala
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
-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.griffin.measure.rule.expr
-
-trait Cacheable extends DataSourceable {
-  protected def cacheUnit: Boolean = false
-  def cacheable(ds: String): Boolean = {
-    cacheUnit && !conflict() && ((ds.isEmpty && dataSources.isEmpty) || (ds.nonEmpty && contains(ds)))
-  }
-  protected def getCacheExprs(ds: String): Iterable[Cacheable]
-
-  protected def persistUnit: Boolean = false
-  def persistable(ds: String): Boolean = {
-    persistUnit && ((ds.isEmpty && dataSources.isEmpty) || (ds.nonEmpty && contains(ds)))
-  }
-  protected def getPersistExprs(ds: String): Iterable[Cacheable]
-}

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/4aa6f779/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Calculatable.scala
----------------------------------------------------------------------
diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Calculatable.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Calculatable.scala
deleted file mode 100644
index 904e823..0000000
--- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Calculatable.scala
+++ /dev/null
@@ -1,25 +0,0 @@
-/*
-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.griffin.measure.rule.expr
-
-trait Calculatable extends Serializable {
-
-  def calculate(values: Map[String, Any]): Option[Any]
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/4aa6f779/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ClauseExpr.scala
----------------------------------------------------------------------
diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ClauseExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ClauseExpr.scala
deleted file mode 100644
index a56e0db..0000000
--- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ClauseExpr.scala
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
-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.griffin.measure.rule.expr
-
-
-trait ClauseExpr extends Expr with AnalyzableExpr {
-  def valid(values: Map[String, Any]): Boolean = true
-  override def cacheUnit: Boolean = true
-}
-
-case class WhereClauseExpr(expr: LogicalExpr) extends ClauseExpr {
-  def calculateOnly(values: Map[String, Any]): Option[Any] = expr.calculate(values)
-  val desc: String = expr.desc
-  val dataSources: Set[String] = expr.dataSources
-  override def getSubCacheExprs(ds: String): Iterable[Expr] = {
-    expr.getCacheExprs(ds)
-  }
-  override def getSubFinalCacheExprs(ds: String): Iterable[Expr] = {
-    expr.getFinalCacheExprs(ds)
-  }
-  override def getSubPersistExprs(ds: String): Iterable[Expr] = {
-    expr.getPersistExprs(ds)
-  }
-
-  override def getGroupbyExprPairs(dsPair: (String, String)): Seq[(Expr, Expr)] = expr.getGroupbyExprPairs(dsPair)
-}
-
-case class WhenClauseExpr(expr: LogicalExpr) extends ClauseExpr {
-  def calculateOnly(values: Map[String, Any]): Option[Any] = expr.calculate(values)
-  val desc: String = s"WHEN ${expr.desc}"
-  val dataSources: Set[String] = expr.dataSources
-  override def getSubCacheExprs(ds: String): Iterable[Expr] = {
-    expr.getCacheExprs(ds)
-  }
-  override def getSubFinalCacheExprs(ds: String): Iterable[Expr] = {
-    expr.getFinalCacheExprs(ds)
-  }
-  override def getSubPersistExprs(ds: String): Iterable[Expr] = {
-    expr.getPersistExprs(ds)
-  }
-
-  override def getGroupbyExprPairs(dsPair: (String, String)): Seq[(Expr, Expr)] = expr.getGroupbyExprPairs(dsPair)
-}
-
-case class StatementExpr(whereClause: WhereClauseExpr, whenClauseOpt: Option[WhenClauseExpr]) extends ClauseExpr {
-  def calculateOnly(values: Map[String, Any]): Option[Any] = whereClause.calculate(values)
-  val desc: String = {
-    whenClauseOpt match {
-      case Some(expr) => s"${whereClause.desc} ${expr.desc}"
-      case _ => whereClause.desc
-    }
-  }
-  val dataSources: Set[String] = whereClause.dataSources
-  override def getSubCacheExprs(ds: String): Iterable[Expr] = {
-    whereClause.getCacheExprs(ds)
-  }
-  override def getSubFinalCacheExprs(ds: String): Iterable[Expr] = {
-    whereClause.getFinalCacheExprs(ds)
-  }
-  override def getSubPersistExprs(ds: String): Iterable[Expr] = {
-    whereClause.getPersistExprs(ds)
-  }
-
-  override def getGroupbyExprPairs(dsPair: (String, String)): Seq[(Expr, Expr)] = whereClause.getGroupbyExprPairs(dsPair)
-}
-
-//case class WhenClauseStatementExpr(expr: LogicalExpr, whenExpr: LogicalExpr) extends ClauseExpr {
-//  def calculateOnly(values: Map[String, Any]): Option[Any] = expr.calculate(values)
-//  val desc: String = s"${expr.desc} when ${whenExpr.desc}"
-//
-//  override def valid(values: Map[String, Any]): Boolean = {
-//    whenExpr.calculate(values) match {
-//      case Some(r: Boolean) => r
-//      case _ => false
-//    }
-//  }
-//
-//  val dataSources: Set[String] = expr.dataSources ++ whenExpr.dataSources
-//  override def getSubCacheExprs(ds: String): Iterable[Expr] = {
-//    expr.getCacheExprs(ds) ++ whenExpr.getCacheExprs(ds)
-//  }
-//  override def getSubFinalCacheExprs(ds: String): Iterable[Expr] = {
-//    expr.getFinalCacheExprs(ds) ++ whenExpr.getFinalCacheExprs(ds)
-//  }
-//  override def getSubPersistExprs(ds: String): Iterable[Expr] = {
-//    expr.getPersistExprs(ds) ++ whenExpr.getPersistExprs(ds)
-//  }
-//
-//  override def getGroupbyExprPairs(dsPair: (String, String)): Seq[(Expr, Expr)] = {
-//    expr.getGroupbyExprPairs(dsPair) ++ whenExpr.getGroupbyExprPairs(dsPair)
-//  }
-//  override def getWhenClauseExpr(): Option[LogicalExpr] = Some(whenExpr)
-//}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/4aa6f779/measure/src/main/scala/org/apache/griffin/measure/rule/expr/DataSourceable.scala
----------------------------------------------------------------------
diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/DataSourceable.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/DataSourceable.scala
deleted file mode 100644
index e2cf172..0000000
--- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/DataSourceable.scala
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
-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.griffin.measure.rule.expr
-
-trait DataSourceable extends Serializable {
-  val dataSources: Set[String]
-  protected def conflict(): Boolean = dataSources.size > 1
-  def contains(ds: String): Boolean = dataSources.contains(ds)
-  def dataSourceOpt: Option[String] = {
-    if (dataSources.size == 1) Some(dataSources.head) else None
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/4aa6f779/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Describable.scala
----------------------------------------------------------------------
diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Describable.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Describable.scala
deleted file mode 100644
index 393d7a6..0000000
--- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Describable.scala
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
-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.griffin.measure.rule.expr
-
-trait Describable extends Serializable {
-
-  val desc: String
-
-  protected def describe(v: Any): String = {
-    v match {
-      case s: Describable => s"${s.desc}"
-      case s: String => s"'${s}'"
-      case a => s"${a}"
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/4aa6f779/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Expr.scala
----------------------------------------------------------------------
diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Expr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Expr.scala
deleted file mode 100644
index 726b5b6..0000000
--- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Expr.scala
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
-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.griffin.measure.rule.expr
-
-import org.apache.spark.sql.types.DataType
-
-trait Expr extends Serializable with Describable with Cacheable with Calculatable {
-
-  protected val _defaultId: String = ExprIdCounter.emptyId
-
-  val _id = ExprIdCounter.genId(_defaultId)
-
-  protected def getSubCacheExprs(ds: String): Iterable[Expr] = Nil
-  final def getCacheExprs(ds: String): Iterable[Expr] = {
-    if (cacheable(ds)) getSubCacheExprs(ds).toList :+ this else getSubCacheExprs(ds)
-  }
-
-  protected def getSubFinalCacheExprs(ds: String): Iterable[Expr] = Nil
-  final def getFinalCacheExprs(ds: String): Iterable[Expr] = {
-    if (cacheable(ds)) Nil :+ this else getSubFinalCacheExprs(ds)
-  }
-
-  protected def getSubPersistExprs(ds: String): Iterable[Expr] = Nil
-  final def getPersistExprs(ds: String): Iterable[Expr] = {
-    if (persistable(ds)) getSubPersistExprs(ds).toList :+ this else getSubPersistExprs(ds)
-  }
-
-  final def calculate(values: Map[String, Any]): Option[Any] = {
-    values.get(_id) match {
-      case Some(v) => Some(v)
-      case _ => calculateOnly(values)
-    }
-  }
-  protected def calculateOnly(values: Map[String, Any]): Option[Any]
-
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/4aa6f779/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ExprDescOnly.scala
----------------------------------------------------------------------
diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ExprDescOnly.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ExprDescOnly.scala
deleted file mode 100644
index 01b7e3c..0000000
--- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ExprDescOnly.scala
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
-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.griffin.measure.rule.expr
-
-trait ExprDescOnly extends Describable {
-
-}
-
-
-case class SelectionHead(expr: String) extends ExprDescOnly {
-  private val headRegex = """\$(\w+)""".r
-  val head: String = expr match {
-    case headRegex(v) => v.toLowerCase
-    case _ => expr
-  }
-  val desc: String = "$" + head
-}
-
-case class RangeDesc(elements: Iterable[MathExpr]) extends ExprDescOnly {
-  val desc: String = {
-    val rangeDesc = elements.map(_.desc).mkString(", ")
-    s"(${rangeDesc})"
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/4aa6f779/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ExprIdCounter.scala
----------------------------------------------------------------------
diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ExprIdCounter.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ExprIdCounter.scala
deleted file mode 100644
index ae76aef..0000000
--- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ExprIdCounter.scala
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
-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.griffin.measure.rule.expr
-
-import java.util.concurrent.atomic.AtomicLong
-
-import scala.collection.mutable.{Set => MutableSet}
-
-object ExprIdCounter {
-
-  private val idCounter: AtomicLong = new AtomicLong(0L)
-
-  private val existIdSet: MutableSet[String] = MutableSet.empty[String]
-
-  private val invalidIdRegex = """^\d+$""".r
-
-  val emptyId: String = ""
-
-  def genId(defaultId: String): String = {
-    defaultId match {
-      case emptyId => increment.toString
-      case invalidIdRegex() => increment.toString
-//      case defId if (exist(defId)) => s"${increment}#${defId}"
-      case defId if (exist(defId)) => s"${defId}"
-      case _ => {
-        insertUserId(defaultId)
-        defaultId
-      }
-    }
-  }
-
-  private def exist(id: String): Boolean = {
-    existIdSet.contains(id)
-  }
-
-  private def insertUserId(id: String): Unit = {
-    existIdSet += id
-  }
-
-  private def increment(): Long = {
-    idCounter.incrementAndGet()
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/4aa6f779/measure/src/main/scala/org/apache/griffin/measure/rule/expr/FieldDescOnly.scala
----------------------------------------------------------------------
diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/FieldDescOnly.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/FieldDescOnly.scala
deleted file mode 100644
index dca037b..0000000
--- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/FieldDescOnly.scala
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
-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.griffin.measure.rule.expr
-
-import scala.util.{Success, Try}
-
-trait FieldDescOnly extends Describable with DataSourceable {
-
-}
-
-case class IndexDesc(expr: String) extends FieldDescOnly {
-  val index: Int = {
-    Try(expr.toInt) match {
-      case Success(v) => v
-      case _ => throw new Exception(s"${expr} is invalid index")
-    }
-  }
-  val desc: String = describe(index)
-  val dataSources: Set[String] = Set.empty[String]
-}
-
-case class FieldDesc(expr: String) extends FieldDescOnly {
-  val field: String = expr
-  val desc: String = describe(field)
-  val dataSources: Set[String] = Set.empty[String]
-}
-
-case class AllFieldsDesc(expr: String) extends FieldDescOnly {
-  val allFields: String = expr
-  val desc: String = allFields
-  val dataSources: Set[String] = Set.empty[String]
-}
-
-case class FieldRangeDesc(startField: FieldDescOnly, endField: FieldDescOnly) extends FieldDescOnly {
-  val desc: String = {
-    (startField, endField) match {
-      case (f1: IndexDesc, f2: IndexDesc) => s"(${f1.desc}, ${f2.desc})"
-      case _ => throw new Exception("invalid field range description")
-    }
-  }
-  val dataSources: Set[String] = Set.empty[String]
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/4aa6f779/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LiteralExpr.scala
----------------------------------------------------------------------
diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LiteralExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LiteralExpr.scala
deleted file mode 100644
index acf1589..0000000
--- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LiteralExpr.scala
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
-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.griffin.measure.rule.expr
-
-import org.apache.griffin.measure.utils.TimeUtil
-import org.apache.spark.sql.types._
-
-import scala.util.{Failure, Success, Try}
-
-trait LiteralExpr extends Expr {
-  val value: Option[Any]
-  def calculateOnly(values: Map[String, Any]): Option[Any] = value
-  val dataSources: Set[String] = Set.empty[String]
-}
-
-case class LiteralValueExpr(value: Option[Any]) extends LiteralExpr {
-  val desc: String = value.getOrElse("").toString
-}
-
-case class LiteralStringExpr(expr: String) extends LiteralExpr {
-  val value: Option[String] = Some(expr)
-  val desc: String = s"'${value.getOrElse("")}'"
-}
-
-case class LiteralNumberExpr(expr: String) extends LiteralExpr {
-  val value: Option[Any] = {
-    if (expr.contains(".")) {
-      Try (expr.toDouble) match {
-        case Success(v) => Some(v)
-        case _ => throw new Exception(s"${expr} is invalid number")
-      }
-    } else {
-      Try (expr.toLong) match {
-        case Success(v) => Some(v)
-        case _ => throw new Exception(s"${expr} is invalid number")
-      }
-    }
-  }
-  val desc: String = value.getOrElse("").toString
-}
-
-case class LiteralTimeExpr(expr: String) extends LiteralExpr {
-  final val TimeRegex = """(\d+)(d|h|m|s|ms)""".r
-  val value: Option[Long] = TimeUtil.milliseconds(expr)
-  val desc: String = expr
-}
-
-case class LiteralBooleanExpr(expr: String) extends LiteralExpr {
-  final val TrueRegex = """(?i)true""".r
-  final val FalseRegex = """(?i)false""".r
-  val value: Option[Boolean] = expr match {
-    case TrueRegex() => Some(true)
-    case FalseRegex() => Some(false)
-    case _ => throw new Exception(s"${expr} is invalid boolean")
-  }
-  val desc: String = value.getOrElse("").toString
-}
-
-case class LiteralNullExpr(expr: String) extends LiteralExpr {
-  val value: Option[Any] = Some(null)
-  val desc: String = "null"
-}
-
-case class LiteralNoneExpr(expr: String) extends LiteralExpr {
-  val value: Option[Any] = None
-  val desc: String = "none"
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/4aa6f779/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LogicalExpr.scala
----------------------------------------------------------------------
diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LogicalExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LogicalExpr.scala
deleted file mode 100644
index dd061d7..0000000
--- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LogicalExpr.scala
+++ /dev/null
@@ -1,178 +0,0 @@
-/*
-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.griffin.measure.rule.expr
-
-import org.apache.griffin.measure.rule.CalculationUtil._
-import org.apache.spark.sql.types.{BooleanType, DataType}
-
-trait LogicalExpr extends Expr with AnalyzableExpr {
-  override def cacheUnit: Boolean = true
-}
-
-case class LogicalSimpleExpr(expr: MathExpr) extends LogicalExpr {
-  def calculateOnly(values: Map[String, Any]): Option[Any] = expr.calculate(values)
-  val desc: String = expr.desc
-  val dataSources: Set[String] = expr.dataSources
-  override def cacheUnit: Boolean = false
-  override def getSubCacheExprs(ds: String): Iterable[Expr] = expr.getCacheExprs(ds)
-  override def getSubFinalCacheExprs(ds: String): Iterable[Expr] = expr.getFinalCacheExprs(ds)
-  override def getSubPersistExprs(ds: String): Iterable[Expr] = expr.getPersistExprs(ds)
-}
-
-case class LogicalCompareExpr(left: MathExpr, compare: String, right: MathExpr) extends LogicalExpr {
-  private val (eqOpr, neqOpr, btOpr, bteOpr, ltOpr, lteOpr) = ("""==?""".r, """!==?""".r, ">", ">=", "<", "<=")
-  def calculateOnly(values: Map[String, Any]): Option[Boolean] = {
-    val (lv, rv) = (left.calculate(values), right.calculate(values))
-    compare match {
-      case this.eqOpr() => lv === rv
-      case this.neqOpr() => lv =!= rv
-      case this.btOpr => lv > rv
-      case this.bteOpr => lv >= rv
-      case this.ltOpr => lv < rv
-      case this.lteOpr => lv <= rv
-      case _ => None
-    }
-  }
-  val desc: String = s"${left.desc} ${compare} ${right.desc}"
-  val dataSources: Set[String] = left.dataSources ++ right.dataSources
-  override def getSubCacheExprs(ds: String): Iterable[Expr] = {
-    left.getCacheExprs(ds) ++ right.getCacheExprs(ds)
-  }
-  override def getSubFinalCacheExprs(ds: String): Iterable[Expr] = {
-    left.getFinalCacheExprs(ds) ++ right.getFinalCacheExprs(ds)
-  }
-  override def getSubPersistExprs(ds: String): Iterable[Expr] = {
-    left.getPersistExprs(ds) ++ right.getPersistExprs(ds)
-  }
-
-  override def getGroupbyExprPairs(dsPair: (String, String)): Seq[(Expr, Expr)] = {
-    if (compare == "=" || compare == "==") {
-      (left.dataSourceOpt, right.dataSourceOpt) match {
-        case (Some(dsPair._1), Some(dsPair._2)) => (left, right) :: Nil
-        case (Some(dsPair._2), Some(dsPair._1)) => (right, left) :: Nil
-        case _ => Nil
-      }
-    } else Nil
-  }
-}
-
-case class LogicalRangeExpr(left: MathExpr, rangeOpr: String, range: RangeDesc) extends LogicalExpr {
-  private val (inOpr, ninOpr, btwnOpr, nbtwnOpr) = ("""(?i)in""".r, """(?i)not\s+in""".r, """(?i)between""".r, """(?i)not\s+between""".r)
-  def calculateOnly(values: Map[String, Any]): Option[Any] = {
-    val (lv, rvs) = (left.calculate(values), range.elements.map(_.calculate(values)))
-    rangeOpr match {
-      case this.inOpr() => lv in rvs
-      case this.ninOpr() => lv not_in rvs
-      case this.btwnOpr() => lv between rvs
-      case this.nbtwnOpr() => lv not_between rvs
-      case _ => None
-    }
-  }
-  val desc: String = s"${left.desc} ${rangeOpr} ${range.desc}"
-  val dataSources: Set[String] = left.dataSources ++ range.elements.flatMap(_.dataSources).toSet
-  override def getSubCacheExprs(ds: String): Iterable[Expr] = {
-    left.getCacheExprs(ds) ++ range.elements.flatMap(_.getCacheExprs(ds))
-  }
-  override def getSubFinalCacheExprs(ds: String): Iterable[Expr] = {
-    left.getFinalCacheExprs(ds) ++ range.elements.flatMap(_.getFinalCacheExprs(ds))
-  }
-  override def getSubPersistExprs(ds: String): Iterable[Expr] = {
-    left.getPersistExprs(ds) ++ range.elements.flatMap(_.getPersistExprs(ds))
-  }
-}
-
-// -- logical statement --
-//case class LogicalFactorExpr(self: LogicalExpr) extends LogicalExpr {
-//  def calculate(values: Map[String, Any]): Option[Any] = self.calculate(values)
-//  val desc: String = self.desc
-//}
-
-case class UnaryLogicalExpr(oprList: Iterable[String], factor: LogicalExpr) extends LogicalExpr {
-  private val notOpr = """(?i)not|!""".r
-  def calculateOnly(values: Map[String, Any]): Option[Any] = {
-    val fv = factor.calculate(values)
-    oprList.foldRight(fv) { (opr, v) =>
-      opr match {
-        case this.notOpr() => !v
-        case _ => None
-      }
-    }
-  }
-  val desc: String = oprList.foldRight(factor.desc) { (prev, ex) => s"${prev} ${ex}" }
-  val dataSources: Set[String] = factor.dataSources
-  override def getSubCacheExprs(ds: String): Iterable[Expr] = {
-    factor.getCacheExprs(ds)
-  }
-  override def getSubFinalCacheExprs(ds: String): Iterable[Expr] = {
-    factor.getFinalCacheExprs(ds)
-  }
-  override def getSubPersistExprs(ds: String): Iterable[Expr] = {
-    factor.getPersistExprs(ds)
-  }
-
-  override def getGroupbyExprPairs(dsPair: (String, String)): Seq[(Expr, Expr)] = {
-    val notOprList = oprList.filter { opr =>
-      opr match {
-        case this.notOpr() => true
-        case _ => false
-      }
-    }
-    if (notOprList.size % 2 == 0) factor.getGroupbyExprPairs(dsPair) else Nil
-  }
-}
-
-case class BinaryLogicalExpr(first: LogicalExpr, others: Iterable[(String, LogicalExpr)]) extends LogicalExpr {
-  private val (andOpr, orOpr) = ("""(?i)and|&&""".r, """(?i)or|\|\|""".r)
-  def calculateOnly(values: Map[String, Any]): Option[Any] = {
-    val fv = first.calculate(values)
-    others.foldLeft(fv) { (v, pair) =>
-      val (opr, next) = pair
-      val nv = next.calculate(values)
-      opr match {
-        case this.andOpr() => v && nv
-        case this.orOpr() => v || nv
-        case _ => None
-      }
-    }
-  }
-  val desc: String = others.foldLeft(first.desc) { (ex, next) => s"${ex} ${next._1} ${next._2.desc}" }
-  val dataSources: Set[String] = first.dataSources ++ others.flatMap(_._2.dataSources).toSet
-  override def getSubCacheExprs(ds: String): Iterable[Expr] = {
-    first.getCacheExprs(ds) ++ others.flatMap(_._2.getCacheExprs(ds))
-  }
-  override def getSubFinalCacheExprs(ds: String): Iterable[Expr] = {
-    first.getFinalCacheExprs(ds) ++ others.flatMap(_._2.getFinalCacheExprs(ds))
-  }
-  override def getSubPersistExprs(ds: String): Iterable[Expr] = {
-    first.getPersistExprs(ds) ++ others.flatMap(_._2.getPersistExprs(ds))
-  }
-
-  override def getGroupbyExprPairs(dsPair: (String, String)): Seq[(Expr, Expr)] = {
-    if (others.isEmpty) first.getGroupbyExprPairs(dsPair)
-    else {
-      val isAnd = others.exists(_._1 match {
-        case this.andOpr() => true
-        case _ => false
-      })
-      if (isAnd) {
-        first.getGroupbyExprPairs(dsPair) ++ others.flatMap(_._2.getGroupbyExprPairs(dsPair))
-      } else Nil
-    }
-  }
-}
\ No newline at end of file