Commit 35670609 by Neeru Gupta Committed by Dave Kantor

ATLAS-737 Add DSL support for Sum, Max, Min and count operations with and…

ATLAS-737 Add DSL support for Sum, Max, Min and count operations with and without group by (guptaneeru via dkantor)
parent 6fb7b82a
......@@ -11,7 +11,7 @@ The grammar for the DSL is below.
<verbatim>
queryWithPath: query ~ opt(WITHPATH)
query: querySrc ~ opt(loopExpression) ~ opt(selectClause) ~ opt(orderby) ~ opt(limitOffset)
query: querySrc ~ opt(loopExpression) ~ opt(groupByExpr) ~ opt(selectClause) ~ opt(orderby) ~ opt(limitOffset)
querySrc: rep1sep(singleQrySrc, opt(COMMA))
......@@ -22,6 +22,8 @@ singleQrySrc = FROM ~ fromSrc ~ opt(WHERE) ~ opt(expr ^? notIdExpression) |
fromSrc: identifier ~ AS ~ alias | identifier
groupByExpr = GROUPBY ~ (LPAREN ~> rep1sep(selectExpression, COMMA) <~ RPAREN)
orderby: ORDERBY ~ expr ~ opt (sortOrder)
limitOffset: LIMIT ~ lmt ~ opt (offset)
......@@ -34,6 +36,14 @@ loopExpression: LOOP ~ (LPAREN ~> query <~ RPAREN) ~ opt(intConstant <~ TIMES) ~
selectClause: SELECT ~ rep1sep(selectExpression, COMMA)
countClause = COUNT ~ LPAREN ~ RPAREN
maxClause = MAX ~ (LPAREN ~> expr <~ RPAREN)
minClause = MIN ~ (LPAREN ~> expr <~ RPAREN)
sumClause = SUM ~ (LPAREN ~> expr <~ RPAREN)
selectExpression: expr ~ opt(AS ~> alias)
expr: compE ~ opt(rep(exprRight))
......@@ -44,7 +54,7 @@ compE:
arithE ~ (LT | LTE | EQ | NEQ | GT | GTE) ~ arithE |
arithE ~ (ISA | IS) ~ ident |
arithE ~ HAS ~ ident |
arithE
arithE | countClause | maxClause | minClause | sumClause
arithE: multiE ~ opt(rep(arithERight))
......@@ -87,6 +97,7 @@ Language Notes:
* The _!WithPath_ clause can be used with transitive closure queries to retrieve the Path that
connects the two related Entities. (We also provide a higher level interface for Closure Queries
see scaladoc for 'org.apache.atlas.query.ClosureQuery')
* GROUPBY is optional. Group by can be specified with aggregate methods like max, min, sum and count. When group by is specified aggregated results are returned based on the method specified in select clause. Select expression is mandatory with group by expression.
* ORDERBY is optional. When order by clause is specified, case insensitive sorting is done based on the column specified.
For sorting in descending order specify 'DESC' after order by clause. If no order by is specified, then no default sorting is applied.
* LIMIT is optional. It limits the maximum number of objects to be fetched starting from specified optional offset. If no offset is specified count starts from beginning.
......@@ -116,6 +127,10 @@ DSL queries:
* Column where Column isa PII
* Table where name="sales_fact", columns
* Table where name="sales_fact", columns as column select column.name, column.dataType, column.comment
* DB groupby(owner) select owner, count()
* DB groupby(owner) select owner, max(name)
* DB groupby(owner) select owner, min(name)
* from Person select count() as 'count', max(Person.age) as 'max', min(Person.age)
* `Log Data`
---++ Full-text Search
......
......@@ -9,6 +9,7 @@ ATLAS-1060 Add composite indexes for exact match performance improvements for al
ATLAS-1127 Modify creation and modification timestamps to Date instead of Long(sumasai)
ALL CHANGES:
ATLAS-737 Add DSL support for Sum, Max, Min and count operations with and without group by (guptaneeru via dkantor)
ATLAS-1305 Fix potential NPEs in instance conversion code (sumasai)
ATLAS-1349 Reduce excessive exception logging (apoorvnaik via svimal2106)
ATLAS-1343 CTAS query is not captured by Atlas with Hive2 (svimal2106)
......
......@@ -22,6 +22,7 @@ import java.util.ArrayList;
import java.util.List;
import org.apache.atlas.AtlasException;
import org.apache.atlas.groovy.CastExpression;
import org.apache.atlas.groovy.ClosureExpression;
import org.apache.atlas.groovy.ComparisonExpression;
import org.apache.atlas.groovy.ComparisonOperatorExpression;
......@@ -34,6 +35,7 @@ import org.apache.atlas.groovy.LiteralExpression;
import org.apache.atlas.groovy.LogicalExpression;
import org.apache.atlas.groovy.RangeExpression;
import org.apache.atlas.groovy.TernaryOperatorExpression;
import org.apache.atlas.groovy.TypeCoersionExpression;
import org.apache.atlas.groovy.ComparisonExpression.ComparisonOperator;
import org.apache.atlas.groovy.LogicalExpression.LogicalOperator;
import org.apache.atlas.query.GraphPersistenceStrategies;
......@@ -53,8 +55,9 @@ public class Gremlin2ExpressionFactory extends GremlinExpressionFactory {
private static final String PATH_FIELD = "path";
private static final String ENABLE_PATH_METHOD = "enablePath";
private static final String BACK_METHOD = "back";
private static final String VERTEX_LIST_CLASS = "List<Vertex>";
private static final String VERTEX_ARRAY_CLASS = "Vertex[]";
private static final String LAST_METHOD = "last";
@Override
public GroovyExpression generateLogicalExpression(GroovyExpression parent, String operator, List<GroovyExpression> operands) {
return new FunctionCallExpression(parent, operator, operands);
......@@ -63,9 +66,12 @@ public class Gremlin2ExpressionFactory extends GremlinExpressionFactory {
@Override
public GroovyExpression generateBackReferenceExpression(GroovyExpression parent, boolean inSelect, String alias) {
if (inSelect) {
if (inSelect && parent == null) {
return getFieldInSelect();
}
else if (inSelect && parent != null) {
return parent;
}
else {
return new FunctionCallExpression(parent, BACK_METHOD, new LiteralExpression(alias));
}
......@@ -208,16 +214,44 @@ public class Gremlin2ExpressionFactory extends GremlinExpressionFactory {
return new FunctionCallExpression(parent, ORDER_METHOD, new ClosureExpression(comparisonFunction));
}
@Override
public GroovyExpression getAnonymousTraversalExpression() {
return new FunctionCallExpression("_");
}
@Override
public GroovyExpression generateGroupByExpression(GroovyExpression parent, GroovyExpression groupByExpression,
GroovyExpression aggregationFunction) {
GroovyExpression groupByClosureExpr = new ClosureExpression(groupByExpression);
GroovyExpression itClosure = new ClosureExpression(getItVariable());
GroovyExpression result = new FunctionCallExpression(parent, "groupBy", groupByClosureExpr, itClosure);
result = new FunctionCallExpression(result, "cap");
result = new FunctionCallExpression(result, "next");
result = new FunctionCallExpression(result, "values");
result = new FunctionCallExpression(result, "toList");
GroovyExpression mapValuesClosure = new ClosureExpression(getItVariable());
GroovyExpression aggregrationFunctionClosure = new ClosureExpression(aggregationFunction);
result = new FunctionCallExpression(result, "collect", aggregrationFunctionClosure);
return result;
}
@Override
public GroovyExpression getFieldInSelect() {
return getItVariable();
}
@Override
public GroovyExpression getGroupBySelectFieldParent() {
GroovyExpression itExpr = getItVariable();
return new FunctionCallExpression(itExpr, LAST_METHOD);
}
//assumes cast already performed
@Override
public GroovyExpression generateCountExpression(GroovyExpression itExpr) {
GroovyExpression collectionExpr = new CastExpression(itExpr,"Collection");
return new FunctionCallExpression(itExpr, "size");
}
}
......@@ -279,7 +279,9 @@ public class Gremlin3ExpressionFactory extends GremlinExpressionFactory {
public GroovyExpression generateOrderByExpression(GroovyExpression parent, List<GroovyExpression> translatedOrderBy,
boolean isAscending) {
GroovyExpression orderByClause = translatedOrderBy.get(0);
GroovyExpression orderByExpr = translatedOrderBy.get(0);
GroovyExpression orderByClosure = new ClosureExpression(orderByExpr);
GroovyExpression orderByClause = new TypeCoersionExpression(orderByClosure, FUNCTION_CLASS);
GroovyExpression aExpr = new IdentifierExpression("a");
GroovyExpression bExpr = new IdentifierExpression("b");
......@@ -322,4 +324,32 @@ public class Gremlin3ExpressionFactory extends GremlinExpressionFactory {
return new FunctionCallExpression(expr2, LAST_METHOD);
}
@Override
public GroovyExpression generateGroupByExpression(GroovyExpression parent, GroovyExpression groupByExpression,
GroovyExpression aggregationFunction) {
GroovyExpression result = new FunctionCallExpression(parent, "group");
GroovyExpression groupByClosureExpr = new TypeCoersionExpression(new ClosureExpression(groupByExpression), "Function");
result = new FunctionCallExpression(result, "by", groupByClosureExpr);
result = new FunctionCallExpression(result, "toList");
GroovyExpression mapValuesClosure = new ClosureExpression(new FunctionCallExpression(new CastExpression(getItVariable(), "Map"), "values"));
result = new FunctionCallExpression(result, "collect", mapValuesClosure);
//when we call Map.values(), we end up with an extra list around the result. We remove this by calling toList().get(0). This
//leaves us with a list of lists containing the vertices that match each group. We then apply the aggregation functions
//specified in the select list to each of these inner lists.
result = new FunctionCallExpression(result ,"toList");
result = new FunctionCallExpression(result, "get", new LiteralExpression(0));
GroovyExpression aggregrationFunctionClosure = new ClosureExpression(aggregationFunction);
result = new FunctionCallExpression(result, "collect", aggregrationFunctionClosure);
return result;
}
public GroovyExpression getGroupBySelectFieldParent() {
return null;
}
}
......@@ -23,6 +23,7 @@ import java.util.List;
import org.apache.atlas.AtlasException;
import org.apache.atlas.groovy.ArithmeticExpression;
import org.apache.atlas.groovy.CastExpression;
import org.apache.atlas.groovy.ClosureExpression;
import org.apache.atlas.groovy.FieldExpression;
import org.apache.atlas.groovy.FunctionCallExpression;
......@@ -214,7 +215,6 @@ public abstract class GremlinExpressionFactory {
*/
protected abstract GroovyExpression initialExpression(GraphPersistenceStrategies s, GroovyExpression varExpr);
/**
* Generates an expression that tests whether the vertex represented by the 'toTest'
* expression represents an instance of the specified type, checking both the type
......@@ -385,13 +385,12 @@ public abstract class GremlinExpressionFactory {
return new ArithmeticExpression(left, op, right);
}
public abstract GroovyExpression generateGroupByExpression(GroovyExpression parent, GroovyExpression groupByExpression, GroovyExpression aggregationFunction);
protected GroovyExpression getItVariable() {
return new IdentifierExpression(IT_VARIABLE);
}
protected GroovyExpression getAllVerticesExpr() {
GroovyExpression gExpr = getGraph();
return new FunctionCallExpression(gExpr, V_METHOD);
......@@ -401,9 +400,40 @@ public abstract class GremlinExpressionFactory {
return new IdentifierExpression(G_VARIABLE);
}
protected GroovyExpression getCurrentObjectExpression() {
return new FieldExpression(getItVariable(), OBJECT_FIELD);
}
//assumes cast already performed
public GroovyExpression generateCountExpression(GroovyExpression itExpr) {
GroovyExpression collectionExpr = new CastExpression(itExpr,"Collection");
return new FunctionCallExpression(collectionExpr, "size");
}
public GroovyExpression generateMinExpression(GroovyExpression itExpr, GroovyExpression mapFunction) {
return getAggregrationExpression(itExpr, mapFunction, "min");
}
public GroovyExpression generateMaxExpression(GroovyExpression itExpr, GroovyExpression mapFunction) {
return getAggregrationExpression(itExpr, mapFunction, "max");
}
public GroovyExpression generateSumExpression(GroovyExpression itExpr, GroovyExpression mapFunction) {
return getAggregrationExpression(itExpr, mapFunction, "sum");
}
private GroovyExpression getAggregrationExpression(GroovyExpression itExpr,
GroovyExpression mapFunction, String functionName) {
GroovyExpression collectionExpr = new CastExpression(itExpr,
"Collection");
ClosureExpression collectFunction = new ClosureExpression(mapFunction);
GroovyExpression transformedList = new FunctionCallExpression(
collectionExpr, "collect", collectFunction);
return new FunctionCallExpression(transformedList, functionName);
}
public GroovyExpression getClosureArgumentValue() {
return getItVariable();
}
public abstract GroovyExpression getGroupBySelectFieldParent();
}
\ No newline at end of file
......@@ -62,7 +62,7 @@ object Expressions {
trait Expression {
self: Product =>
def isAggregator = false
def children: Seq[Expression]
/**
......@@ -331,6 +331,12 @@ object Expressions {
def limit(lmt: Literal[Integer], offset : Literal[Integer]) = new LimitExpression(this, lmt, offset)
def order(odr: Expression, asc: Boolean) = new OrderExpression(this, odr, asc)
def max(maxClause: Expression) = new MaxExpression(maxClause)
def min(minClause: Expression) = new MinExpression(minClause)
def groupBy(groupBy: SelectExpression, selectExpr: SelectExpression) = new GroupByExpression(this, groupBy, selectExpr)
}
trait BinaryNode {
......@@ -393,7 +399,7 @@ object Expressions {
case class UnresolvedFieldExpression(child: Expression, fieldName: String) extends Expression
with UnaryNode {
override def toString = s"${child}.$fieldName"
override def isAggregator = child.isAggregator
override lazy val resolved = false
override def dataType = throw new UnresolvedException(this, "field")
......@@ -445,7 +451,7 @@ object Expressions {
override def namedExpressions = child.namedExpressions + (alias -> child)
override def toString = s"$child as $alias"
override def isAggregator = child.isAggregator
lazy val dataType = {
if (!resolved) {
throw new UnresolvedException(this,
......@@ -519,6 +525,14 @@ object Expressions {
def listLiteral[_ <: PrimitiveType[_]](typ: ArrayType, rawValue: List[Expressions.Literal[_]]) = new ListLiteral(typ, rawValue)
def count() = new CountExpression()
def maxExpr(maxClause: Expression) = new MaxExpression(maxClause)
def minExpr(minClause: Expression) = new MinExpression(minClause)
def sumExpr(sumClause: Expression) = new SumExpression(sumClause)
case class ArithmeticExpression(symbol: String,
left: Expression,
right: Expression)
......@@ -681,13 +695,25 @@ object Expressions {
override def toString = s"$child where $condExpr"
}
case class SelectExpression(child: Expression, selectList: List[Expression]) extends Expression {
case class SelectExpression(child: Expression, selectList: List[Expression], forGroupBy: Boolean = false) extends Expression {
val children = List(child) ::: selectList
def hasAggregation = {
var result = false;
selectList.foreach { expr =>
{
result = result || expr.isAggregator
}
}
result
}
lazy val selectListWithAlias = selectList.zipWithIndex map {
case (s: AliasExpression, _) => s
case (x, i) => new AliasExpression(x, s"${x}")
}
lazy val dataType = {
if (!resolved) {
throw new UnresolvedException(this,
......@@ -698,7 +724,14 @@ object Expressions {
override def namedExpressions = child.namedExpressions ++ (selectList.flatMap(_.namedExpressions))
override def toString = s"""$child select ${selectListWithAlias.mkString("", ", ", "")}"""
override def toString = {
//When this is part of a group by, the child is only present so that the select
//list gets translated correctly. It is not really part of the query. The child
//ends up both in the GroupByExpression as well as here. We only want to show it
//in the GroupByExpression. Hide it here.
var prefix = if(forGroupBy) { "" } else { s"""${child} select """ }
s"""${prefix}${selectListWithAlias.mkString("", ", ", "")}"""
}
}
case class LoopExpression(val input: Expression, val loopingExpression: Expression,
......@@ -797,4 +830,68 @@ object Expressions {
child.dataType
}
}
case class CountExpression() extends Expression {
override def isAggregator = true
override def toString = s"count()"
val children = Nil
lazy val dataType = {
DataTypes.LONG_TYPE
}
}
case class MaxExpression(maxClause: Expression) extends Expression {
override def toString = s"max($maxClause)"
override def isAggregator = true
val children = List(maxClause)
lazy val dataType = {
if (!resolved) {
throw new UnresolvedException(this,
s"datatype. Can not resolve due to unresolved children")
}
maxClause.dataType
}
}
case class MinExpression(minClause: Expression) extends Expression {
override def toString = s"min($minClause)"
override def isAggregator = true
val children = List(minClause)
lazy val dataType = {
if (!resolved) {
throw new UnresolvedException(this,
s"datatype. Can not resolve due to unresolved children")
}
minClause.dataType
}
}
case class SumExpression(sumClause: Expression) extends Expression {
override def toString = s"sum($sumClause)"
override def isAggregator = true
val children = List(sumClause)
lazy val dataType = {
if (!resolved) {
throw new UnresolvedException(this,
s"datatype. Can not resolve due to unresolved children")
}
sumClause.dataType
}
}
case class GroupByExpression(child: Expression, groupBy: SelectExpression, selExpr: SelectExpression) extends Expression{
override def toString = s"from ${child} groupby(${groupBy}) select ${selExpr}"
val children = List(child, groupBy, selExpr)
lazy val dataType = {
if (!resolved) {
throw new UnresolvedException(this,
s"datatype. Can not resolve due to unresolved children")
}
selExpr.dataType
}
}
}
......@@ -90,7 +90,7 @@ class GremlinEvaluator(qry: GremlinQuery, persistenceStrategy: GraphPersistenceS
if(debug) {
println(" rawRes " +rawRes)
}
if (!qry.hasSelectList) {
if (!qry.hasSelectList && ! qry.isGroupBy) {
val rows = rawRes.asInstanceOf[java.util.List[AnyRef]].map { v =>
val instObj = instanceObject(v)
val o = persistenceStrategy.constructInstance(oType, instObj)
......@@ -112,6 +112,19 @@ class GremlinEvaluator(qry: GremlinQuery, persistenceStrategy: GraphPersistenceS
sInstance.set(cName, persistenceStrategy.constructInstance(aE.dataType, v))
}
}
else if(qry.isGroupBy) {
//the order in the result will always match the order in the select list
val selExpr = qry.expr.asInstanceOf[GroupByExpression].selExpr
var idx = 0;
val row : java.util.List[Object] = rV.asInstanceOf[java.util.List[Object]]
selExpr.selectListWithAlias.foreach { aE =>
val cName = aE.alias
val cValue = row.get(idx);
sInstance.set(cName, persistenceStrategy.constructInstance(aE.dataType, cValue))
idx += 1;
}
}
addPathStruct(r, sInstance)
}
GremlinQueryResult(qry.expr.toString, rType, rows.toList)
......
......@@ -73,7 +73,12 @@ import org.apache.atlas.typesystem.types.DataTypes.TypeCategory
import org.apache.atlas.typesystem.types.IDataType
import org.apache.atlas.typesystem.types.TypeSystem
import org.joda.time.format.ISODateTimeFormat
import org.apache.atlas.query.Expressions.GroupByExpression
import org.apache.atlas.query.Expressions.MaxExpression
import org.apache.atlas.query.Expressions.MinExpression
import org.apache.atlas.query.Expressions.SumExpression
import org.apache.atlas.query.Expressions.CountExpression
import java.util.HashSet
trait IntSequence {
def next: Int
}
......@@ -84,11 +89,67 @@ case class GremlinQuery(expr: Expression, queryStr: String, resultMaping: Map[St
def isPathExpression = expr.isInstanceOf[PathExpression]
def isGroupBy = expr.isInstanceOf[GroupByExpression]
}
trait SelectExpressionHandling {
class AliasFinder extends PartialFunction[Expression,Unit] {
val aliases = new HashSet[String]()
def isDefinedAt(e: Expression) = true
def apply(e: Expression) = e match {
case e@AliasExpression(_, alias) => {
aliases.add(alias)
}
case x => Unit
}
}
class ReplaceAliasWithBackReference(aliases: HashSet[String]) extends PartialFunction[Expression, Expression] {
def isDefinedAt(e: Expression) = true
def apply(e: Expression) = e match {
case e@AliasExpression(child,alias) if aliases.contains(alias) => {
new BackReference(alias, child, None)
}
case x => x
}
}
//in groupby, convert alias expressions defined in the group by child to BackReferences
//in the groupby list and selectList.
val AddBackReferencesToGroupBy : PartialFunction[Expression, Expression] = {
case GroupByExpression(child, groupBy, selExpr) => {
val aliases = ArrayBuffer[AliasExpression]()
val finder = new AliasFinder();
child.traverseUp(finder);
val replacer = new ReplaceAliasWithBackReference(finder.aliases)
val newGroupBy = new SelectExpression(
groupBy.child.transformUp(replacer),
groupBy.selectList.map {
expr => expr.transformUp(replacer)
},
groupBy.forGroupBy);
val newSelExpr = new SelectExpression(
selExpr.child.transformUp(replacer),
selExpr.selectList.map {
expr => expr.transformUp(replacer)
},
selExpr.forGroupBy);
new GroupByExpression(child, newGroupBy, newSelExpr)
}
case x => x
}
/**
* To aide in gremlinQuery generation add an alias to the input of SelectExpressions
*/
......@@ -110,14 +171,22 @@ trait SelectExpressionHandling {
}
def apply(e: Expression) = e match {
case SelectExpression(aliasE@AliasExpression(_, _), selList) => {
case SelectExpression(aliasE@AliasExpression(_, _), selList, forGroupBy) => {
idx = idx + 1
SelectExpression(aliasE, selList.map(_.transformUp(new DecorateFieldWithAlias(aliasE))))
SelectExpression(aliasE, selList.map(_.transformUp(new DecorateFieldWithAlias(aliasE))), forGroupBy)
}
case SelectExpression(child, selList) => {
case SelectExpression(child, selList, forGroupBy) => {
idx = idx + 1
val aliasE = AliasExpression(child, s"_src$idx")
SelectExpression(aliasE, selList.map(_.transformUp(new DecorateFieldWithAlias(aliasE))))
SelectExpression(aliasE, selList.map(_.transformUp(new DecorateFieldWithAlias(aliasE))), forGroupBy)
}
case OrderExpression(aliasE@AliasExpression(_, _), order, asc) => {
OrderExpression(aliasE, order.transformUp(new DecorateFieldWithAlias(aliasE)),asc)
}
case OrderExpression(child, order, asc) => {
idx = idx + 1
val aliasE = AliasExpression(child, s"_src$idx")
OrderExpression(aliasE, order.transformUp(new DecorateFieldWithAlias(aliasE)),asc)
}
case _ => e
}
......@@ -133,7 +202,7 @@ trait SelectExpressionHandling {
}
def validateSelectExprHaveOneSrc: PartialFunction[Expression, Unit] = {
case SelectExpression(_, selList) => {
case SelectExpression(_, selList, forGroupBy) => {
selList.foreach { se =>
val srcs = getSelectExpressionSrc(se)
if (srcs.size > 1) {
......@@ -262,25 +331,25 @@ class GremlinTranslator(expr: Expression,
}
private def genQuery(parent: GroovyExpression, expr: Expression, inSelect: Boolean): GroovyExpression = expr match {
private def genQuery(parent: GroovyExpression, expr: Expression, inClosure: Boolean): GroovyExpression = expr match {
case ClassExpression(clsName) => typeTestExpression(parent, clsName)
case TraitExpression(clsName) => typeTestExpression(parent, clsName)
case fe@FieldExpression(fieldName, fInfo, child)
if fe.dataType.getTypeCategory == TypeCategory.PRIMITIVE || fe.dataType.getTypeCategory == TypeCategory.ARRAY => {
val fN = gPersistenceBehavior.fieldNameInVertex(fInfo.dataType, fInfo.attrInfo)
val childExpr = translateOptChild(parent, child, inSelect);
return GremlinExpressionFactory.INSTANCE.generateFieldExpression(childExpr, fInfo, fN, inSelect);
val childExpr = translateOptChild(parent, child, inClosure);
return GremlinExpressionFactory.INSTANCE.generateFieldExpression(childExpr, fInfo, fN, inClosure);
}
case fe@FieldExpression(fieldName, fInfo, child)
if fe.dataType.getTypeCategory == TypeCategory.CLASS || fe.dataType.getTypeCategory == TypeCategory.STRUCT => {
val childExpr = translateOptChild(parent, child, inSelect);
val childExpr = translateOptChild(parent, child, inClosure);
val direction = if (fInfo.isReverse) AtlasEdgeDirection.IN else AtlasEdgeDirection.OUT
val edgeLbl = gPersistenceBehavior.edgeLabel(fInfo)
return GremlinExpressionFactory.INSTANCE.generateAdjacentVerticesExpression(childExpr, direction, edgeLbl)
}
case fe@FieldExpression(fieldName, fInfo, child) if fInfo.traitName != null => {
val childExpr = translateOptChild(parent, child, inSelect);
val childExpr = translateOptChild(parent, child, inClosure);
val direction = gPersistenceBehavior.instanceToTraitEdgeDirection
val edgeLbl = gPersistenceBehavior.edgeLabel(fInfo)
return GremlinExpressionFactory.INSTANCE.generateAdjacentVerticesExpression(childExpr, direction, edgeLbl)
......@@ -289,25 +358,25 @@ class GremlinTranslator(expr: Expression,
case c@ComparisonExpression(symb, f@FieldExpression(fieldName, fInfo, ch), l) => {
val qualifiedPropertyName = s"${gPersistenceBehavior.fieldNameInVertex(fInfo.dataType, fInfo.attrInfo)}"
val childExpr = translateOptChild(parent, ch, inSelect)
val childExpr = translateOptChild(parent, ch, inClosure)
val persistentExprValue : GroovyExpression = if(l.isInstanceOf[Literal[_]]) {
translateLiteralValue(fInfo.attrInfo.dataType, l.asInstanceOf[Literal[_]]);
}
else {
genQuery(null, l, inSelect);
genQuery(null, l, inClosure);
}
return GremlinExpressionFactory.INSTANCE.generateHasExpression(gPersistenceBehavior, childExpr, qualifiedPropertyName, c.symbol, persistentExprValue, fInfo);
}
case fil@FilterExpression(child, condExpr) => {
val newParent = genQuery(parent, child, inSelect);
return genQuery(newParent, condExpr, inSelect);
val newParent = genQuery(parent, child, inClosure);
return genQuery(newParent, condExpr, inClosure);
}
case l@LogicalExpression(symb, children) => {
val translatedChildren : java.util.List[GroovyExpression] = translateList(children, true, inSelect);
val translatedChildren : java.util.List[GroovyExpression] = translateList(children, false);
return GremlinExpressionFactory.INSTANCE.generateLogicalExpression(parent, symb, translatedChildren);
}
case sel@SelectExpression(child, selList) => {
case sel@SelectExpression(child, selList, forGroupBy) => {
val m = groupSelectExpressionsBySrc(sel)
var srcNamesList: java.util.List[LiteralExpression] = new ArrayList()
var srcExprsList: List[java.util.List[GroovyExpression]] = List()
......@@ -316,7 +385,7 @@ class GremlinTranslator(expr: Expression,
while (it.hasNext) {
val (src, selExprs) = it.next
srcNamesList.add(new LiteralExpression(src));
val translatedSelExprs : java.util.List[GroovyExpression] = translateList(selExprs, false, true);
val translatedSelExprs : java.util.List[GroovyExpression] = translateList(selExprs, true);
srcExprsList = srcExprsList :+ translatedSelExprs
}
val srcExprsStringList : java.util.List[GroovyExpression] = new ArrayList();
......@@ -324,7 +393,7 @@ class GremlinTranslator(expr: Expression,
srcExprsStringList.add(new ListExpression(it));
}
val childExpr = genQuery(parent, child, inSelect)
val childExpr = genQuery(parent, child, inClosure)
return GremlinExpressionFactory.INSTANCE.generateSelectExpression(childExpr, srcNamesList, srcExprsStringList);
}
......@@ -337,16 +406,16 @@ class GremlinTranslator(expr: Expression,
null.asInstanceOf[Integer]
}
val alias = input.asInstanceOf[AliasExpression].alias;
val inputQry = genQuery(parent, input, inSelect)
val translatedLoopExpr = genQuery(GremlinExpressionFactory.INSTANCE.getLoopExpressionParent(inputQry), loopExpr, inSelect);
val inputQry = genQuery(parent, input, inClosure)
val translatedLoopExpr = genQuery(GremlinExpressionFactory.INSTANCE.getLoopExpressionParent(inputQry), loopExpr, inClosure);
return GremlinExpressionFactory.INSTANCE.generateLoopExpression(inputQry, gPersistenceBehavior, input.dataType, translatedLoopExpr, alias, times);
}
case BackReference(alias, _, _) => {
return GremlinExpressionFactory.INSTANCE.generateBackReferenceExpression(parent, inSelect, alias);
return GremlinExpressionFactory.INSTANCE.generateBackReferenceExpression(parent, inClosure, alias);
}
case AliasExpression(child, alias) => {
var childExpr = genQuery(parent, child, inSelect);
var childExpr = genQuery(parent, child, inClosure);
return GremlinExpressionFactory.INSTANCE.generateAliasExpression(childExpr, alias);
}
case isTraitLeafExpression(traitName, Some(clsExp)) => {
......@@ -371,11 +440,11 @@ class GremlinTranslator(expr: Expression,
}
}
case hasFieldUnaryExpression(fieldName, child) =>
val childExpr = genQuery(parent, child, inSelect);
val childExpr = genQuery(parent, child, inClosure);
return GremlinExpressionFactory.INSTANCE.generateUnaryHasExpression(childExpr, fieldName);
case ArithmeticExpression(symb, left, right) => {
val leftExpr = genQuery(parent, left, inSelect);
val rightExpr = genQuery(parent, right, inSelect);
val leftExpr = genQuery(parent, left, inClosure);
val rightExpr = genQuery(parent, right, inClosure);
return GremlinExpressionFactory.INSTANCE.generateArithmeticExpression(leftExpr, symb, rightExpr);
}
case l: Literal[_] => {
......@@ -386,23 +455,22 @@ class GremlinTranslator(expr: Expression,
return translateLiteralValue(l.dataType, l);
}
case list: ListLiteral[_] => {
val values : java.util.List[GroovyExpression] = translateList(list.rawValue, false, inSelect);
val values : java.util.List[GroovyExpression] = translateList(list.rawValue, true); //why hard coded
return new ListExpression(values);
}
case in@TraitInstanceExpression(child) => {
val childExpr = genQuery(parent, child, inSelect);
val childExpr = genQuery(parent, child, inClosure);
val direction = gPersistenceBehavior.traitToInstanceEdgeDirection;
return GremlinExpressionFactory.INSTANCE.generateAdjacentVerticesExpression(childExpr, direction);
}
case in@InstanceExpression(child) => {
return genQuery(parent, child, inSelect);
return genQuery(parent, child, inClosure);
}
case pe@PathExpression(child) => {
val childExpr = genQuery(parent, child, inSelect)
val childExpr = genQuery(parent, child, inClosure)
return GremlinExpressionFactory.INSTANCE.generatePathExpression(childExpr);
}
case order@OrderExpression(child, odr, asc) => {
var orderby = ""
var orderExpression = odr
if(odr.isInstanceOf[BackReference]) {
orderExpression = odr.asInstanceOf[BackReference].reference
......@@ -411,36 +479,95 @@ class GremlinTranslator(expr: Expression,
orderExpression = odr.asInstanceOf[AliasExpression].child
}
val childExpr = genQuery(parent, child, inSelect);
val childExpr = genQuery(parent, child, inClosure);
var orderByParents : java.util.List[GroovyExpression] = GremlinExpressionFactory.INSTANCE.getOrderFieldParents();
val translatedParents : java.util.List[GroovyExpression] = new ArrayList[GroovyExpression]();
var translatedOrderParents = orderByParents.foreach { it =>
translatedParents.add(genQuery(it, orderExpression, false));
translatedParents.add(genQuery(it, orderExpression, true));
}
return GremlinExpressionFactory.INSTANCE.generateOrderByExpression(childExpr, translatedParents,asc);
}
case limitOffset@LimitExpression(child, limit, offset) => {
val childExpr = genQuery(parent, child, inSelect);
val childExpr = genQuery(parent, child, inClosure);
val totalResultRows = limit.value + offset.value;
return GremlinExpressionFactory.INSTANCE.generateLimitExpression(childExpr, offset.value, totalResultRows);
}
case count@CountExpression() => {
val listExpr = GremlinExpressionFactory.INSTANCE.getClosureArgumentValue();
GremlinExpressionFactory.INSTANCE.generateCountExpression(listExpr);
}
case max@MaxExpression(child) => {
//use "it" as the parent since the child will become
//part of a closure. Its value will be whatever vertex
//we are looking at in the collection.
val childExprParent = null;
val childExpr = genQuery(childExprParent, child, true);
val listExpr = GremlinExpressionFactory.INSTANCE.getClosureArgumentValue();
GremlinExpressionFactory.INSTANCE.generateMaxExpression(listExpr, childExpr);
}
case min@MinExpression(child) => {
//use "it" as the parent since the child will become
//part of a closure. Its value will be whatever vertex
//we are looking at in the collection.
val childExprParent = null;
val childExpr = genQuery(childExprParent, child, true);
val listExpr = GremlinExpressionFactory.INSTANCE.getClosureArgumentValue();
GremlinExpressionFactory.INSTANCE.generateMinExpression(listExpr, childExpr);
}
case sum@SumExpression(child) => {
//use "it" as the parent since the child will become
//part of a closure. Its value will be whatever vertex
//we are looking at in the collection.
val childExprParent = null;
val childExpr = genQuery(childExprParent, child, true);
val listExpr = GremlinExpressionFactory.INSTANCE.getClosureArgumentValue();
GremlinExpressionFactory.INSTANCE.generateSumExpression(listExpr, childExpr);
}
case groupByExpr@GroupByExpression(child, groupBy, selExpr) => {
//remove aliases
val groupByExprListToTranslate = (groupBy.asInstanceOf[SelectExpression]).selectListWithAlias.map {
x => x.child;
}
val grpByExprsList = translateList(groupByExprListToTranslate, true);
val groupByValue = new ListExpression(grpByExprsList);
//reduction only aggregate methods are supported here as of now.(Max, Min, Count)
//remove aliases
val srcExprListToTranslate = selExpr.selectListWithAlias.map {
x => x.child;
}
val srcExprsList = translateList(srcExprListToTranslate, true, true);
val srcExprsStringList = new ListExpression(srcExprsList)
val childExpr = genQuery(parent, child, inClosure);
return GremlinExpressionFactory.INSTANCE.generateGroupByExpression(childExpr, groupByValue, srcExprsStringList);
}
case x => throw new GremlinTranslationException(x, "expression not yet supported")
}
def translateList(exprs : List[Expressions.Expression], isAnonymousTraveral: Boolean, inSelect : Boolean) : java.util.List[GroovyExpression] = {
var parent = if(isAnonymousTraveral) {GremlinExpressionFactory.INSTANCE.getAnonymousTraversalExpression() } else { null }
def translateList(exprs : List[Expressions.Expression], inClosure : Boolean, inGroupBy: Boolean = false) : java.util.List[GroovyExpression] = {
var parent = if (inGroupBy) {
GremlinExpressionFactory.INSTANCE.getGroupBySelectFieldParent();
}
else if(inClosure) {
null;
}
else {
GremlinExpressionFactory.INSTANCE.getAnonymousTraversalExpression()
}
var result : java.util.List[GroovyExpression] = new java.util.ArrayList(exprs.size);
exprs.foreach { it =>
result.add(genQuery(parent, it, inSelect));
result.add(genQuery(parent, it, inClosure));
}
return result;
}
def translateOptChild(parent : GroovyExpression, child : Option[Expressions.Expression] , inSelect: Boolean) : GroovyExpression = child match {
def translateOptChild(parent : GroovyExpression, child : Option[Expressions.Expression] , inClosure: Boolean) : GroovyExpression = child match {
case Some(x) => genQuery(parent, x, inSelect)
case Some(x) => genQuery(parent, x, inClosure)
case None => parent
}
......@@ -532,6 +659,7 @@ class GremlinTranslator(expr: Expression,
var e1 = expr.transformUp(wrapAndRule)
e1.traverseUp(validateComparisonForm)
e1 = e1.transformUp(AddBackReferencesToGroupBy)
e1 = e1.transformUp(new AddAliasToSelectInput)
e1.traverseUp(validateSelectExprHaveOneSrc)
e1 = e1.transformUp(addAliasToLoopInput())
......@@ -556,7 +684,7 @@ class GremlinTranslator(expr: Expression,
*/
def extractSelectExpression(child: Expression): Option[SelectExpression] = {
child match {
case se@SelectExpression(child, selectList) =>{
case se@SelectExpression(child, selectList, false) =>{
Some(se)
}
case limit@LimitExpression(child, lmt, offset) => {
......
......@@ -24,6 +24,8 @@ import scala.util.parsing.combinator.lexical.StdLexical
import scala.util.parsing.combinator.syntactical.StandardTokenParsers
import scala.util.parsing.combinator.{ImplicitConversions, PackratParsers}
import scala.util.parsing.input.CharArrayReader._
import org.apache.atlas.AtlasException
import org.apache.atlas.typesystem.types.DataTypes
trait QueryKeywords {
this: StandardTokenParsers =>
......@@ -67,6 +69,10 @@ trait QueryKeywords {
protected val LIMIT = Keyword("limit")
protected val OFFSET = Keyword("offset")
protected val ORDERBY = Keyword("orderby")
protected val COUNT = Keyword("count")
protected val MAX = Keyword("max")
protected val MIN = Keyword("min")
protected val SUM = Keyword("sum")
}
trait ExpressionUtils {
......@@ -79,14 +85,14 @@ trait ExpressionUtils {
case (c, t, Some(a)) => input.loop(c, t.get).as(a)
}
def select(input: Expression, s: List[(Expression, Option[String])]) = {
def select(input: Expression, s: List[(Expression, Option[String])], forGroupBy: Boolean = false) = {
val selList = s.map { t =>
t._2 match {
case None => t._1.as(s"${t._1}")
case _ => t._1.as(t._2.get)
}
}
input.select(selList: _*)
new SelectExpression(input, selList, forGroupBy)
}
def limit(input: Expression, lmt: Literal[Integer], offset: Literal[Integer]) = {
......@@ -117,6 +123,10 @@ trait ExpressionUtils {
val leftSrcId = leftmostId(sngQuery2)
sngQuery2.transformUp(replaceIdWithField(leftSrcId, snglQuery1.field(leftSrcId.name)))
}
def groupBy(input: Expression, groupByExpr: SelectExpression, selectExpr: SelectExpression) = {
input.groupBy(groupByExpr, selectExpr)
}
}
case class QueryParams(limit: Int, offset: Int)
......@@ -164,8 +174,8 @@ object QueryParser extends StandardTokenParsers with QueryKeywords with Expressi
*
* @return
*/
def query(implicit queryParams: QueryParams) = querySrc ~ opt(loopExpression) ~ opt(selectClause) ~ opt(orderby) ~ opt(limitOffset) ^^ {
case s ~ l ~ sel ~ odr ~ lmtoff => {
def query(implicit queryParams: QueryParams) = querySrc ~ opt(loopExpression) ~ opt(groupByExpr) ~ opt(selectClause) ~ opt(orderby) ~ opt(limitOffset) ^^ {
case s ~ l ~ grp ~ sel ~ odr ~ lmtoff => {
var expressiontree = s
if (l.isDefined) //Note: The order of if statements is important.
{
......@@ -175,10 +185,6 @@ object QueryParser extends StandardTokenParsers with QueryKeywords with Expressi
{
expressiontree = order(expressiontree, odr.get._1, odr.get._2)
}
if (sel.isDefined)
{
expressiontree = select(expressiontree, sel.get)
}
if (queryParams != null && lmtoff.isDefined)
{
val mylimit = int(min(queryParams.limit, max(lmtoff.get._1 - queryParams.offset, 0)))
......@@ -189,6 +195,34 @@ object QueryParser extends StandardTokenParsers with QueryKeywords with Expressi
} else if(queryParams != null) {
expressiontree = limit(expressiontree, int(queryParams.limit), int(queryParams.offset))
}
if (grp.isDefined && sel.isDefined)
{
var child = expressiontree
var selectExpr: SelectExpression = select(child, sel.get, true)
var grpBySelectExpr: SelectExpression = select(child, grp.get, true)
expressiontree = groupBy(child, grpBySelectExpr, selectExpr)
}
else if (grp.isDefined)
{
throw new AtlasException("groupby without select is not allowed");
}
else if (sel.isDefined)
{
var selectChild = expressiontree
val selExpr : SelectExpression = select(selectChild, sel.get);
if(selExpr.hasAggregation) {
//In order to do the aggregation, we need to add an implicit group by. Having the
//group by expression be a constant values forces all of the vertices into one group.
val groupByConstant : Expression = Expressions.literal(DataTypes.STRING_TYPE, "dummy");
val groupBySelExpr : SelectExpression = select(selectChild, sel.get, true);
val groupByListExpr : SelectExpression = select(selectChild, List((groupByConstant,None)), true)
expressiontree = groupBy(selectChild, groupByListExpr, groupBySelExpr)
}
else {
expressiontree = selExpr
}
}
expressiontree
}
}
......@@ -279,7 +313,7 @@ object QueryParser extends StandardTokenParsers with QueryKeywords with Expressi
arithE ~ (LT | LTE | EQ | NEQ | GT | GTE) ~ arithE ^^ { case l ~ op ~ r => l.compareOp(op)(r)} |
arithE ~ (ISA | IS) ~ ident ^^ { case l ~ i ~ t => l.isTrait(t)} |
arithE ~ HAS ~ ident ^^ { case l ~ i ~ f => l.hasField(f)} |
arithE
arithE | countClause | maxClause | minClause | sumClause
def arithE = multiE ~ opt(rep(arithERight)) ^^ {
case l ~ None => l
......@@ -351,6 +385,22 @@ object QueryParser extends StandardTokenParsers with QueryKeywords with Expressi
def doubleConstant: Parser[String] =
elem("int", _.isInstanceOf[lexical.DoubleLiteral]) ^^ (_.chars)
def countClause = COUNT ~ LPAREN ~ RPAREN ^^ {
case c => count()
}
def maxClause = MAX ~ (LPAREN ~> expr <~ RPAREN) ^^ {
case m ~ e => maxExpr(e)
}
def minClause = MIN ~ (LPAREN ~> expr <~ RPAREN) ^^ {
case m ~ e => minExpr(e)
}
def sumClause = SUM ~ (LPAREN ~> expr <~ RPAREN) ^^ {
case m ~ e => sumExpr(e)
}
def groupByExpr = GROUPBY ~ (LPAREN ~> rep1sep(selectExpression, COMMA) <~ RPAREN) ^^ {
case g ~ ce => ce
}
}
class QueryLexer(val keywords: Seq[String], val delims: Seq[String]) extends StdLexical with ImplicitConversions {
......
......@@ -81,11 +81,11 @@ class Resolver(srcExpr: Option[Expression] = None, aliases: Map[String, Expressi
val r = new Resolver(Some(inputExpr), inputExpr.namedExpressions)
return new FilterExpression(inputExpr, condExpr.transformUp(r))
}
case SelectExpression(child, selectList) if child.resolved => {
case SelectExpression(child, selectList, forGroupBy) if child.resolved => {
val r = new Resolver(Some(child), child.namedExpressions)
return new SelectExpression(child, selectList.map {
_.transformUp(r)
})
}, forGroupBy)
}
case l@LoopExpression(inputExpr, loopExpr, t) if inputExpr.resolved => {
val r = new Resolver(Some(inputExpr), inputExpr.namedExpressions, true)
......@@ -145,11 +145,15 @@ object FieldValidator extends PartialFunction[Expression, Expression] {
new FilterExpression(inputExpr, validatedCE)
}
}
case SelectExpression(child, selectList) if child.resolved => {
case SelectExpression(child, selectList, forGroupBy) if child.resolved => {
val v = validateQualifiedField(child.dataType)
return new SelectExpression(child, selectList.map {
_.transformUp(v)
})
}, forGroupBy)
}
case OrderExpression(child, order, asc) => {
val v = validateQualifiedField(child.dataType)
OrderExpression(child, order.transformUp(v), asc)
}
case l@LoopExpression(inputExpr, loopExpr, t) => {
val validatedLE = loopExpr.transformUp(validateQualifiedField(inputExpr.dataType))
......
......@@ -66,15 +66,16 @@ public class BaseRepositoryTest {
protected MetadataRepository repository;
protected void setUp() throws Exception {
//force graph initialization / built in type registration
TestUtils.getGraph();
setUpDefaultTypes();
setUpTypes();
TestUtils.getGraph().commit();
new GraphBackedSearchIndexer(new AtlasTypeRegistry());
TestUtils.resetRequestContext();
setupInstances();
TestUtils.getGraph().commit();
TestUtils.dumpGraph(TestUtils.getGraph());
}
......
......@@ -156,7 +156,8 @@ public final class TestUtils {
createOptionalAttrDef("salary", DataTypes.DOUBLE_TYPE),
createOptionalAttrDef("age", DataTypes.FLOAT_TYPE),
createOptionalAttrDef("numberOfStarsEstimate", DataTypes.BIGINTEGER_TYPE),
createOptionalAttrDef("approximationOfPi", DataTypes.BIGDECIMAL_TYPE)
createOptionalAttrDef("approximationOfPi", DataTypes.BIGDECIMAL_TYPE),
createOptionalAttrDef("isOrganDonor", DataTypes.BOOLEAN_TYPE)
);
HierarchicalTypeDefinition<ClassType> managerTypeDef = createClassTypeDef("Manager", "Manager"+_description, ImmutableSet.of("Person"),
......@@ -195,6 +196,7 @@ public final class TestUtils {
john.set("address", johnAddr);
john.set("birthday",new Date(1950, 5, 15));
john.set("isOrganDonor", true);
john.set("hasPets", true);
john.set("numberOfCars", 1);
john.set("houseNumber", 153);
......@@ -227,6 +229,7 @@ public final class TestUtils {
max.set("manager", jane);
max.set("mentor", julius);
max.set("birthday",new Date(1979, 3, 15));
max.set("isOrganDonor", true);
max.set("hasPets", true);
max.set("age", 36);
max.set("numberOfCars", 2);
......
......@@ -18,7 +18,25 @@
package org.apache.atlas.discovery;
import com.google.common.collect.ImmutableSet;
import static org.apache.atlas.typesystem.types.utils.TypesUtil.createClassTypeDef;
import static org.apache.atlas.typesystem.types.utils.TypesUtil.createOptionalAttrDef;
import static org.apache.atlas.typesystem.types.utils.TypesUtil.createRequiredAttrDef;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertNotNull;
import java.text.SimpleDateFormat;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Date;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import javax.inject.Inject;
import org.apache.atlas.AtlasException;
import org.apache.atlas.BaseRepositoryTest;
......@@ -42,6 +60,7 @@ import org.apache.atlas.typesystem.types.IDataType;
import org.apache.atlas.typesystem.types.Multiplicity;
import org.apache.atlas.typesystem.types.TypeSystem;
import org.codehaus.jettison.json.JSONArray;
import org.codehaus.jettison.json.JSONException;
import org.codehaus.jettison.json.JSONObject;
import org.testng.Assert;
import org.testng.annotations.AfterClass;
......@@ -51,25 +70,7 @@ import org.testng.annotations.DataProvider;
import org.testng.annotations.Guice;
import org.testng.annotations.Test;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.Date;
import java.text.SimpleDateFormat;
import javax.inject.Inject;
import static org.apache.atlas.typesystem.types.utils.TypesUtil.createClassTypeDef;
import static org.apache.atlas.typesystem.types.utils.TypesUtil.createOptionalAttrDef;
import static org.apache.atlas.typesystem.types.utils.TypesUtil.createRequiredAttrDef;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertNotNull;
import static org.testng.Assert.assertTrue;
import com.google.common.collect.ImmutableSet;
@Guice(modules = RepositoryMetadataModule.class)
public class GraphBackedDiscoveryServiceTest extends BaseRepositoryTest {
......@@ -647,88 +648,221 @@ public class GraphBackedDiscoveryServiceTest extends BaseRepositoryTest {
private Object[][] createDSLQueriesWithOrderBy() {
Boolean isAscending = Boolean.TRUE;
return new Object[][]{
//test with alias
// {"from hive_db select hive_db.name as 'o' orderby o limit 3", 3, "name", isAscending},
{"from hive_db as h orderby h.owner limit 3", 3, "owner", isAscending},
{"hive_column as c select c.name orderby hive_column.name ", 37, "c.name", isAscending},
{"hive_column as c select c.name orderby hive_column.name limit 5", 5, "c.name", isAscending},
{"hive_column as c select c.name orderby hive_column.name desc limit 5", 5, "c.name", !isAscending},
{"from hive_db orderby hive_db.owner limit 3", 3, "owner", isAscending},
{"hive_column select hive_column.name orderby hive_column.name ", 37, "hive_column.name", isAscending},
{"hive_column select hive_column.name orderby hive_column.name limit 5", 5, "hive_column.name", isAscending},
{"hive_column select hive_column.name orderby hive_column.name desc limit 5", 5, "hive_column.name", !isAscending},
{"from hive_db orderby owner limit 3", 3, "owner", isAscending},
{"hive_column select hive_column.name orderby name ", 37, "hive_column.name", isAscending},
{"hive_column select hive_column.name orderby name limit 5", 5, "hive_column.name", isAscending},
{"hive_column select hive_column.name orderby name desc limit 5", 5, "hive_column.name", !isAscending},
//Not working, the problem is in server code not figuring out how to sort. not sure if it is valid use case.
// {"hive_db hive_table orderby 'hive_db.owner'", 8, "owner", isAscending},
// {"hive_db hive_table orderby 'hive_db.owner'", 10, "owner", isAscending},
// {"hive_db hive_table orderby 'hive_db.owner' limit 5", 5, "owner", isAscending},
// {"hive_db hive_table orderby 'hive_db.owner' limit 5 offset 5", 3, "owner", isAscending},
{"hive_column select hive_column.name orderby name limit 10 withPath", 10, "name", isAscending},
{"hive_column select hive_column.name orderby name asc limit 10 withPath", 10, "name", isAscending},
{"hive_column select hive_column.name orderby 'hive_column.name' desc limit 10 withPath", 10, "name", !isAscending},
{"from hive_db orderby owner limit 3", 3, "owner", isAscending},
{"hive_db where hive_db.name=\"Reporting\" orderby owner", 1, "owner", isAscending},
{"hive_db where hive_db.name=\"Reporting\" orderby owner limit 10", 1, "owner", isAscending},
{"hive_db where hive_db.name=\"Reporting\" select name, owner orderby name", 1, "name", isAscending},
{"hive_db has name orderby 'hive_db.owner' limit 10 offset 0", 3, "owner", isAscending},
{"hive_db select hive_db.description orderby hive_db.description limit 10 withPath", 3, "hive_db.description", isAscending},
{"hive_db select hive_db.description orderby hive_db.description desc limit 10 withPath", 3, "hive_db.description", !isAscending},
{"from hive_table orderby owner", 10, "owner", isAscending},
{"from hive_table orderby 'hive_table.owner' limit 8", 8, "owner", isAscending},
{"hive_column select hive_column.name orderby hive_column.name limit 10 withPath", 10, "hive_column.name", isAscending},
{"hive_column select hive_column.name orderby hive_column.name asc limit 10 withPath", 10, "hive_column.name", isAscending},
{"hive_column select hive_column.name orderby hive_column.name desc limit 10 withPath", 10, "hive_column.name", !isAscending},
{"from hive_db orderby hive_db.owner limit 3", 3, "owner", isAscending},
{"hive_db where hive_db.name=\"Reporting\" orderby 'owner'", 1, "owner", isAscending},
{"hive_table orderby owner", 10, "owner", isAscending},
{"hive_table orderby owner limit 8", 8, "owner", isAscending},
{"hive_table orderby owner limit 8 offset 0", 8, "owner", isAscending},
{"hive_table orderby 'hive_table.owner' desc limit 8 offset 0", 8, "owner", !isAscending},
{"hive_db where hive_db.name=\"Reporting\" orderby hive_db.owner limit 10 ", 1, "owner", isAscending},
{"hive_db where hive_db.name=\"Reporting\" select name, owner orderby hive_db.name ", 1, "name", isAscending},
{"hive_db has name orderby hive_db.owner limit 10 offset 0", 3, "owner", isAscending},
{"hive_table isa Dimension orderby owner", 3, "owner", isAscending},//order not working
{"hive_table isa Dimension orderby owner limit 3", 3, "owner", isAscending},
{"hive_table isa Dimension orderby owner limit 3 offset 0", 3, "owner", isAscending},
{"hive_table isa Dimension orderby 'hive_table.owner' desc limit 3 offset 0", 3, "owner", !isAscending},
{"from hive_table select hive_table.owner orderby hive_table.owner", 10, "hive_table.owner", isAscending},
{"from hive_table select hive_table.owner orderby hive_table.owner limit 8", 8, "hive_table.owner", isAscending},
{"hive_column where hive_column isa PII orderby name", 8, "name", isAscending},
{"hive_column where hive_column isa PII orderby name limit 5", 5, "name", isAscending},
{"hive_column where hive_column isa PII orderby name limit 5 offset 1", 5, "name", isAscending},
{"hive_column where hive_column isa PII orderby 'hive_column.name' desc limit 5 offset 1", 5, "name", !isAscending},
{"hive_table orderby hive_table.name", 10, "name", isAscending},
{"hive_table orderby hive_table.owner", 10, "owner", isAscending},
{"hive_table orderby hive_table.owner limit 8", 8, "owner", isAscending},
{"hive_table orderby hive_table.owner limit 8 offset 0", 8, "owner", isAscending},
{"hive_table orderby hive_table.owner desc limit 8 offset 0", 8, "owner", !isAscending},
//Not working because of existing bug Atlas-175
// {"hive_table isa Dimension orderby hive_table.owner", 3, "hive_table.owner", isAscending},//order not working
// {"hive_table isa Dimension orderby hive_table.owner limit 3", 3, "hive_table.owner", isAscending},
// {"hive_table isa Dimension orderby hive_table.owner limit 3 offset 0", 3, "hive_table.owner", isAscending},
// {"hive_table isa Dimension orderby hive_table.owner desc limit 3 offset 0", 3, "hive_table.owner", !isAscending},
//
// {"hive_column where hive_column isa PII orderby hive_column.name", 6, "hive_column.name", isAscending},
// {"hive_column where hive_column isa PII orderby hive_column.name limit 5", 5, "hive_column.name", isAscending},
// {"hive_column where hive_column isa PII orderby hive_column.name limit 5 offset 1", 5, "hive_column.name", isAscending},
// {"hive_column where hive_column isa PII orderby hive_column.name desc limit 5 offset 1", 5, "hive_column.name", !isAscending},
{"hive_column select hive_column.name orderby hive_column.name ", 37, "hive_column.name", isAscending},
{"hive_column select hive_column.name orderby hive_column.name limit 5", 5, "hive_column.name", isAscending},
{"hive_column select hive_column.name orderby hive_column.name desc limit 5", 5, "hive_column.name", !isAscending},
{"hive_column select hive_column.name orderby hive_column.name limit 5 offset 28", 5, "hive_column.name", isAscending},
{"hive_column select name orderby hive_column.name", 37, "name", isAscending},
{"hive_column select name orderby hive_column.name limit 5", 5, "name", isAscending},
{"hive_column select name orderby hive_column.name desc", 37, "name", !isAscending},
{"hive_column where hive_column.name=\"customer_id\" orderby hive_column.name", 6, "name", isAscending},
{"hive_column where hive_column.name=\"customer_id\" orderby hive_column.name limit 2", 2, "name", isAscending},
{"hive_column where hive_column.name=\"customer_id\" orderby hive_column.name limit 2 offset 1", 2, "name", isAscending},
{"from hive_table select owner orderby hive_table.owner",10, "owner", isAscending},
{"from hive_table select owner orderby hive_table.owner limit 5", 5, "owner", isAscending},
{"from hive_table select owner orderby hive_table.owner desc limit 5", 5, "owner", !isAscending},
{"from hive_table select owner orderby hive_table.owner limit 5 offset 5", 5, "owner", isAscending},
{"hive_db where (name = \"Reporting\") orderby hive_db.name", 1, "name", isAscending},
{"hive_db where (name = \"Reporting\") orderby hive_db.name limit 10", 1, "name", isAscending},
{"hive_db where hive_db has name orderby hive_db.owner", 3, "owner", isAscending},
{"hive_db where hive_db has name orderby hive_db.owner limit 5", 3, "owner", isAscending},
{"hive_db where hive_db has name orderby hive_db.owner limit 2 offset 0", 2, "owner", isAscending},
{"hive_db where hive_db has name orderby hive_db.owner limit 2 offset 1", 2, "owner", isAscending},
{"hive_column select hive_column.name orderby name", 37, "hive_column.name", isAscending},
{"hive_column select hive_column.name orderby name limit 5", 5, "hive_column.name", isAscending},
{"hive_column select hive_column.name orderby name desc limit 5", 5, "hive_column.name", !isAscending},
{"hive_column select hive_column.name orderby 'hive_column.name' limit 5 offset 36", 1, "hive_column.name", isAscending},
{"hive_column select name orderby name", 37, "name", isAscending},
{"hive_column select name orderby name limit 5", 5, "name", isAscending},
{"hive_column select name orderby 'hive_column.name' desc", 37, "name", !isAscending},
{"hive_column where hive_column.name=\"customer_id\" orderby name", 6, "name", isAscending},
{"hive_column where hive_column.name=\"customer_id\" orderby name limit 2", 2, "name", isAscending},
{"hive_column where hive_column.name=\"customer_id\" orderby 'hive_column.name' limit 2 offset 1", 2, "name", isAscending},
{"from hive_table select owner orderby owner", 10, "owner", isAscending},
{"from hive_table select owner orderby owner limit 5", 5, "owner", isAscending},
{"from hive_table select owner orderby owner desc limit 5", 5, "owner", !isAscending},
{"from hive_table select owner orderby 'hive_table.owner' limit 5 offset 5", 5, "owner", isAscending},
{"hive_db where (name = \"Reporting\") orderby name", 1, "name", isAscending},
{"hive_db where (name = \"Reporting\") orderby 'hive_db.name' limit 10", 1, "name", isAscending},
{"hive_db where (name = \"Reporting\") select name as _col_0, owner as _col_1 orderby '_col_1'", 1, "_col_1", isAscending}, //will it work
{"hive_db where (name = \"Reporting\") select name as _col_0, owner as _col_1 orderby owner limit 10", 1, "_col_1", isAscending},
{"hive_db where hive_db has name orderby owner", 3, "owner", isAscending},
{"hive_db where hive_db has name orderby owner limit 5", 3, "owner", isAscending},
{"hive_db where hive_db has name orderby owner limit 2 offset 0", 2, "owner", isAscending},
{"hive_db where hive_db has name orderby 'hive_db.owner' limit 2 offset 1", 2, "owner", isAscending},
{"hive_db where (name = \"Reporting\") select name as _col_0, (createTime + 1) as _col_1 orderby '_col_1'", 1, "_col_1", isAscending},
{"hive_db where (name = \"Reporting\") select name as _col_0, (createTime + 1) as _col_1 orderby '_col_1' limit 10", 1, "_col_1", isAscending},
{"hive_db where (name = \"Reporting\") select name as _col_0, (createTime + 1) as _col_1 orderby '_col_1' limit 10 offset 1", 0, "_col_1", isAscending},
{"hive_db where (name = \"Reporting\") select name as _col_0, (createTime + 1) as _col_1 orderby '_col_1' limit 10 offset 0", 1, "_col_1", isAscending},
{"hive_table where (name = \"sales_fact\" and createTime > \"2014-01-01\" ) select name as _col_0, createTime as _col_1 orderby createTime", 1, "_col_1", isAscending},
{"hive_table where (name = \"sales_fact\" and createTime > \"2014-01-01\" ) select name as _col_0, createTime as _col_1 orderby createTime limit 10 ", 1, "_col_1", isAscending},
{"hive_table where (name = \"sales_fact\" and createTime > \"2014-01-01\" ) select name as _col_0, createTime as _col_1 orderby createTime limit 10 offset 0", 1, "_col_1", isAscending},
{"hive_table where (name = \"sales_fact\" and createTime > \"2014-01-01\" ) select name as _col_0, createTime as _col_1 orderby '_col_1' ", 1, "_col_1", isAscending},
{"hive_table where (name = \"sales_fact\" and createTime > \"2014-01-01\" ) select name as _col_0, createTime as _col_1 orderby '_col_1' limit 10 ", 1, "_col_1", isAscending},
{"hive_table where (name = \"sales_fact\" and createTime > \"2014-01-01\" ) select name as _col_0, createTime as _col_1 orderby '_col_1' limit 10 offset 0", 1, "_col_1", isAscending},
{"hive_table where (name = \"sales_fact\" and createTime > \"2014-01-01\" ) select name as _col_0, createTime as _col_1 orderby '_col_1' limit 10 offset 5", 0, "_col_1", isAscending},
{"hive_table where (name = \"sales_fact\" and createTime >= \"2014-12-11T02:35:58.440Z\" ) select name as _col_0, createTime as _col_1 orderby name", 1, "_col_0", isAscending},
{"hive_table where (name = \"sales_fact\" and createTime >= \"2014-12-11T02:35:58.440Z\" ) select name as _col_0, createTime as _col_1 orderby name limit 10 offset 0", 1, "_col_0", isAscending},
{"hive_table where (name = \"sales_fact\" and createTime >= \"2014-12-11T02:35:58.440Z\" ) select name as _col_0, createTime as _col_1 orderby name limit 10 offset 1", 0, "_col_0", isAscending},
{"hive_table where (name = \"sales_fact\" and createTime >= \"2014-12-11T02:35:58.440Z\" ) select name as _col_0, createTime as _col_1 orderby name limit 10", 1, "_col_0", isAscending},
{"hive_table where (name = \"sales_fact\" and createTime >= \"2014-12-11T02:35:58.440Z\" ) select name as _col_0, createTime as _col_1 orderby '_col_0' ", 1, "_col_0", isAscending},
{"hive_table where (name = \"sales_fact\" and createTime >= \"2014-12-11T02:35:58.440Z\" ) select name as _col_0, createTime as _col_1 orderby '_col_0' limit 10 offset 0", 1, "_col_0", isAscending},
{"hive_table where (name = \"sales_fact\" and createTime >= \"2014-12-11T02:35:58.440Z\" ) select name as _col_0, createTime as _col_1 orderby '_col_0' limit 10 offset 1", 0, "_col_0", isAscending},
{"hive_table where (name = \"sales_fact\" and createTime >= \"2014-12-11T02:35:58.440Z\" ) select name as _col_0, createTime as _col_1 orderby '_col_0' limit 10", 1, "_col_0", isAscending},
{"hive_table where (name = \"sales_fact\" and createTime >= \"2014-12-11T02:35:58.440Z\" ) select name as _col_0, createTime as _col_1 orderby '_col_0' limit 0 offset 1", 0, "_col_0", isAscending},
//Test if proeprty is not defined. it should not fail the query
{"hive_column select hive_column.name orderby hive_column.name limit 10 withPath", 10, "hive_column.name", isAscending},
{"hive_column select hive_column.name orderby hive_column.name limit 10 withPath", 10, "hive_column.name", isAscending},
{"hive_table orderby 'hive_table.owner_notdefined'", 10, null, isAscending},
};
}
@DataProvider(name = "dslGroupByQueriesProvider")
private Object[][] createDSLGroupByQueries() {
return new Object[][]{
{ "from Person as p, mentor as m groupby(m.name) select m.name, count()",
new FieldValueValidator().withFieldNames("m.name", "count()").withExpectedValues("Max", 1)
.withExpectedValues("Julius", 1) },
// This variant of this query is currently failing. See OMS-335 for details.
{ "from Person as p, mentor groupby(mentor.name) select mentor.name, count()",
new FieldValueValidator().withFieldNames("mentor.name", "count()").withExpectedValues("Max", 1)
.withExpectedValues("Julius", 1) },
{ "from Person, mentor groupby(mentor.name) select mentor.name, count()",
new FieldValueValidator().withFieldNames("mentor.name", "count()").withExpectedValues("Max", 1)
.withExpectedValues("Julius", 1) },
{ "from Person, mentor as m groupby(m.name) select m.name, count()",
new FieldValueValidator().withFieldNames("m.name", "count()").withExpectedValues("Max", 1)
.withExpectedValues("Julius", 1) },
{ "from Person groupby (isOrganDonor) select count()",
new FieldValueValidator().withFieldNames("count()").withExpectedValues(2)
.withExpectedValues(2) },
{ "from Person groupby (isOrganDonor) select Person.isOrganDonor, count()",
new FieldValueValidator().withFieldNames("Person.isOrganDonor", "count()")
.withExpectedValues(true, 2).withExpectedValues(false, 2) },
{ "from Person groupby (isOrganDonor) select Person.isOrganDonor as 'organDonor', count() as 'count', max(Person.age) as 'max', min(Person.age) as 'min'",
new FieldValueValidator().withFieldNames("organDonor", "max", "min", "count")
.withExpectedValues(true, 50, 36, 2).withExpectedValues(false, 0, 0, 2) },
{ "from hive_db groupby (owner, name) select count() ", new FieldValueValidator()
.withFieldNames("count()").withExpectedValues(1).withExpectedValues(1).withExpectedValues(1) },
{ "from hive_db groupby (owner, name) select hive_db.owner, hive_db.name, count() ",
new FieldValueValidator().withFieldNames("hive_db.owner", "hive_db.name", "count()")
.withExpectedValues("Jane BI", "Reporting", 1)
.withExpectedValues("Tim ETL", "Logging", 1)
.withExpectedValues("John ETL", "Sales", 1) },
{ "from hive_db groupby (owner) select count() ",
new FieldValueValidator().withFieldNames("count()").withExpectedValues(1).withExpectedValues(1)
.withExpectedValues(1) },
{ "from hive_db groupby (owner) select hive_db.owner, count() ",
new FieldValueValidator().withFieldNames("hive_db.owner", "count()")
.withExpectedValues("Jane BI", 1).withExpectedValues("Tim ETL", 1)
.withExpectedValues("John ETL", 1) },
{ "from hive_db groupby (owner) select hive_db.owner, max(hive_db.name) ",
new FieldValueValidator().withFieldNames("hive_db.owner", "max(hive_db.name)")
.withExpectedValues("Tim ETL", "Logging").withExpectedValues("Jane BI", "Reporting")
.withExpectedValues("John ETL", "Sales") },
{ "from hive_db groupby (owner) select max(hive_db.name) ",
new FieldValueValidator().withFieldNames("max(hive_db.name)").withExpectedValues("Logging")
.withExpectedValues("Reporting").withExpectedValues("Sales") },
{ "from hive_db groupby (owner) select owner, hive_db.name, min(hive_db.name) ",
new FieldValueValidator().withFieldNames("owner", "hive_db.name", "min(hive_db.name)")
.withExpectedValues("Tim ETL", "Logging", "Logging")
.withExpectedValues("Jane BI", "Reporting", "Reporting")
.withExpectedValues("John ETL", "Sales", "Sales") },
{ "from hive_db groupby (owner) select owner, min(hive_db.name) ",
new FieldValueValidator().withFieldNames("owner", "min(hive_db.name)")
.withExpectedValues("Tim ETL", "Logging").withExpectedValues("Jane BI", "Reporting")
.withExpectedValues("John ETL", "Sales") },
{ "from hive_db groupby (owner) select min(name) ",
new FieldValueValidator().withFieldNames("min(name)")
.withExpectedValues("Reporting").withExpectedValues("Logging")
.withExpectedValues("Sales") },
{ "from hive_db groupby (owner) select min('name') ",
new FieldValueValidator().withFieldNames("min(\"name\")").withExpectedValues("name")
.withExpectedValues("name").withExpectedValues("name") }, //finding the minimum of a constant literal expression...
{ "from hive_db groupby (owner) select name ",
new FieldValueValidator().withFieldNames("name").withExpectedValues("Reporting")
.withExpectedValues("Sales").withExpectedValues("Logging") },
//implied group by
{ "from hive_db select count() ",
new FieldValueValidator().withFieldNames("count()").withExpectedValues(3) },
//implied group by
{ "from Person select count() as 'count', max(Person.age) as 'max', min(Person.age) as 'min'",
new FieldValueValidator().withFieldNames("max", "min", "count").withExpectedValues(50, 0, 4) },
//Sum
{ "from Person groupby (isOrganDonor) select count() as 'count', sum(Person.age) as 'sum'",
new FieldValueValidator().withFieldNames("count", "sum").withExpectedValues(2, 0)
.withExpectedValues(2, 86) },
{ "from Person groupby (isOrganDonor) select Person.isOrganDonor as 'organDonor', count() as 'count', sum(Person.age) as 'sum'",
new FieldValueValidator().withFieldNames("organDonor", "count", "sum").withExpectedValues(false, 2, 0)
.withExpectedValues(true, 2, 86) },
{ "from Person select count() as 'count', sum(Person.age) as 'sum'",
new FieldValueValidator().withFieldNames("count", "sum").withExpectedValues(4, 86) },
// tests to ensure that group by works with order by and limit
{ "from hive_db groupby (owner) select min(name) orderby name limit 2 ",
new FieldValueValidator().withFieldNames("min(name)")
.withExpectedValues("Logging").withExpectedValues("Reporting")
},
{ "from hive_db groupby (owner) select min(name) orderby name desc limit 2 ",
new FieldValueValidator().withFieldNames("min(name)")
.withExpectedValues("Reporting").withExpectedValues("Sales")
},
};
}
@Test(dataProvider = "dslOrderByQueriesProvider")
public void testSearchByDSLQueriesWithOrderBy(String dslQuery, Integer expectedNumRows, String orderBy, boolean ascending) throws Exception {
System.out.println("Executing dslQuery = " + dslQuery);
......@@ -759,7 +893,7 @@ public class GraphBackedDiscoveryServiceTest extends BaseRepositoryTest {
}
catch(Exception ex)
{
System.out.println( " Exception occured " + ex.getMessage());
System.out.println( " Exception occured " + ex.getMessage() + " found row: "+row);
}
}
Iterator<String> iter = returnedList.iterator();
......@@ -771,10 +905,11 @@ public class GraphBackedDiscoveryServiceTest extends BaseRepositoryTest {
_prev = _current;
_current = iter.next().toLowerCase();
if (_prev != null && _prev.compareTo(_current) != 0) {
if (ascending) {
Assert.assertTrue(_prev.compareTo(_current) < 0);
} else {
Assert.assertTrue(_prev.compareTo(_current) > 0);
if(ascending) {
Assert.assertTrue(_prev.compareTo(_current) < 0, _prev + " is greater than " + _current);
}
else {
Assert.assertTrue(_prev.compareTo(_current) > 0, _prev + " is less than " + _current);
}
}
}
......@@ -886,4 +1021,155 @@ public class GraphBackedDiscoveryServiceTest extends BaseRepositoryTest {
repositoryService.createEntities(typedInstance);
}
private void runCountGroupByQuery(String dslQuery, ResultChecker checker) throws Exception {
System.out.println("Executing dslQuery = " + dslQuery);
String jsonResults = searchByDSL(dslQuery);
assertNotNull(jsonResults);
JSONObject results = new JSONObject(jsonResults);
assertEquals(results.length(), 3);
Object query = results.get("query");
assertNotNull(query);
JSONArray rows = results.getJSONArray("rows");
assertNotNull(rows);
if (checker != null) {
checker.validateResult(dslQuery, rows);
}
System.out.println("query [" + dslQuery + "] returned [" + rows.length() + "] rows");
}
@Test(dataProvider = "dslGroupByQueriesProvider")
public void testSearchGroupByDSLQueries(String dslQuery, ResultChecker checker) throws Exception {
runCountGroupByQuery(dslQuery, checker);
}
private interface ResultChecker {
void validateResult(String dslQuery, JSONArray foundRows) throws JSONException;
}
static class FieldValueValidator implements ResultChecker {
static class ResultObject {
@Override
public String toString() {
return "ResultObject [fieldValues_=" + fieldValues_ + "]";
}
Map<String, Object> fieldValues_ = new HashMap<>();
public void setFieldValue(String string, Object object) {
fieldValues_.put(string, object);
}
public boolean matches(JSONObject object) throws JSONException {
for (Map.Entry<String, Object> requiredFieldsEntry : fieldValues_.entrySet()) {
String fieldName = requiredFieldsEntry.getKey();
Object expectedValue = requiredFieldsEntry.getValue();
Object foundValue = null;
if (expectedValue.getClass() == Integer.class) {
foundValue = object.getInt(fieldName);
} else {
foundValue = object.get(fieldName);
}
if (foundValue == null || !expectedValue.equals(foundValue)) {
return false;
}
}
return true;
}
}
private String[] fieldNames_;
private List<ResultObject> expectedObjects_ = new ArrayList<>();
public FieldValueValidator() {
}
public FieldValueValidator withFieldNames(String... fields) {
fieldNames_ = fields;
return this;
}
public FieldValueValidator withExpectedValues(Object... values) {
ResultObject obj = new ResultObject();
for (int i = 0; i < fieldNames_.length; i++) {
obj.setFieldValue(fieldNames_[i], values[i]);
}
expectedObjects_.add(obj);
return this;
}
@Override
public void validateResult(String dslQuery, JSONArray foundRows) throws JSONException {
//make sure that all required rows are found
Assert.assertEquals(foundRows.length(), expectedObjects_.size(),
"The wrong number of objects was returned for query " + dslQuery + ". Expected "
+ expectedObjects_.size() + ", found " + foundRows.length());
for (ResultObject required : expectedObjects_) {
//not exactly efficient, but this is test code
boolean found = false;
for (int i = 0; i < foundRows.length(); i++) {
JSONObject row = foundRows.getJSONObject(i);
System.out.println(" found row "+ row);
if (required.matches(row)) {
found = true;
break;
}
}
if (!found) {
Assert.fail("The result for " + dslQuery + " is wrong. The required row " + required
+ " was not found in " + foundRows);
}
}
}
}
static class CountOnlyValidator implements ResultChecker {
private List<Integer> expectedCounts = new ArrayList<Integer>();
private int countColumn = 0;
public CountOnlyValidator() {
}
public CountOnlyValidator withCountColumn(int col) {
countColumn = col;
return this;
}
public CountOnlyValidator withExpectedCounts(Integer... counts) {
expectedCounts.addAll(Arrays.asList(counts));
return this;
}
@Override
public void validateResult(String dslQuery, JSONArray foundRows) throws JSONException {
assertEquals(foundRows.length(), expectedCounts.size());
for (int i = 0; i < foundRows.length(); i++) {
JSONArray row = foundRows.getJSONArray(i);
assertEquals(row.length(), 1);
int foundCount = row.getInt(countColumn);
// assertTrue(expectedCounts.contains(foundCount));
}
}
}
private FieldValueValidator makeCountValidator(int count) {
return new FieldValueValidator().withFieldNames("count()").withExpectedValues(count);
}
private FieldValueValidator makeNoResultsValidator() {
return new FieldValueValidator();
}
}
\ No newline at end of file
Markdown is supported
0% or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment