feat(reasoner): add lube-api (#32)

* fix: scala style checker

* feat(reasoner): add lube-api.
This commit is contained in:
FishJoy 2023-12-12 19:03:02 +08:00 committed by GitHub
parent 09e7461113
commit 40a22007e2
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
22 changed files with 153 additions and 151 deletions

View File

@ -16,13 +16,12 @@ package com.antgroup.openspg.reasoner.lube.block
import scala.collection.mutable.ListBuffer
import com.antgroup.openspg.reasoner.lube.common.expr.Aggregator
import com.antgroup.openspg.reasoner.lube.common.graph.{IRField, IRGraph}
import com.antgroup.openspg.reasoner.lube.common.graph.IRField
final case class AggregationBlock(
dependencies: List[Block],
aggregations: Aggregations,
group: List[String],
graph: IRGraph)
group: List[String])
extends BasicBlock[Fields](BlockType("aggregation")) {
override def binds: Fields = {

View File

@ -13,6 +13,10 @@
package com.antgroup.openspg.reasoner.lube.block
import com.antgroup.openspg.reasoner.lube.common.graph.IRGraph
abstract class BasicBlock[B <: Binds](override val blockType: BlockType) extends Block {
override def binds: B
override def graph: IRGraph = dependencies.head.graph
}

View File

@ -13,7 +13,6 @@
package com.antgroup.openspg.reasoner.lube.block
import com.antgroup.openspg.reasoner.lube.common.graph.IRGraph
import com.antgroup.openspg.reasoner.lube.common.rule.Rule
/**
@ -23,7 +22,7 @@ import com.antgroup.openspg.reasoner.lube.common.rule.Rule
* @param rules
* @param graph
*/
final case class FilterBlock(dependencies: List[Block], rules: Rule, graph: IRGraph)
final case class FilterBlock(dependencies: List[Block], rules: Rule)
extends BasicBlock[Binds](BlockType("filter")) {
override def binds: Binds = {

View File

@ -13,14 +13,8 @@
package com.antgroup.openspg.reasoner.lube.block
import scala.collection.mutable
import com.antgroup.openspg.reasoner.lube.common.graph.{IRArray, IREdge, IRGraph, IRNode, IRPath}
import com.antgroup.openspg.reasoner.lube.common.pattern.{
Connection,
GraphPath,
VariablePatternConnection
}
import com.antgroup.openspg.reasoner.lube.common.graph.{IREdge, IRNode, IRPath, IRRepeatPath}
import com.antgroup.openspg.reasoner.lube.common.pattern.{Connection, GraphPath, VariablePatternConnection}
/**
* parse from "GraphStructure" block to match path
@ -31,34 +25,34 @@ import com.antgroup.openspg.reasoner.lube.common.pattern.{
*/
final case class MatchBlock(
dependencies: List[Block],
patterns: Map[String, GraphPath],
graph: IRGraph)
patterns: Map[String, GraphPath])
extends BasicBlock[Binds](BlockType("match")) {
override def binds: Binds = {
val props = patterns.values.head.graphPattern.properties
val nodes = patterns.values
.flatMap(path =>
path.graphPattern.nodes.map(node => IRNode(node._1, new mutable.HashSet[String]())))
path.graphPattern.nodes.map(node => IRNode(node._1, props(node._1))))
.toList
val edges = patterns.values
.flatMap(path =>
path.graphPattern.edges
.map(pair => pair._2.map(rel => edgeToIRField(rel)).toList)
.map(pair => pair._2.map(rel => edgeToIRField(rel, props)).toList)
.flatten)
.toList
Fields(nodes.++(edges))
}
private def edgeToIRField(edge: Connection) = {
private def edgeToIRField(edge: Connection, props: Map[String, Set[String]]) = {
edge match {
case connection: VariablePatternConnection =>
val start = IRNode(connection.source, new mutable.HashSet[String]())
val end = IRNode(connection.target, new mutable.HashSet[String]())
val irEdge = IREdge(connection.alias, new mutable.HashSet[String]())
val start = IRNode(connection.source, props(connection.source))
val end = IRNode(connection.target, props(connection.target))
val irEdge = IREdge(connection.alias, props(connection.alias))
val path = IRPath(connection.alias, List.apply(start, irEdge, end))
IRArray(path)
IRRepeatPath(path, connection.lower, connection.upper)
case _ =>
IREdge(edge.alias, new mutable.HashSet[String]())
IREdge(edge.alias, props(edge.alias))
}
}

View File

@ -20,8 +20,7 @@ final case class OrderAndSliceBlock(
dependencies: List[Block],
orderBy: Seq[SortItem],
limit: Option[Int],
group: List[String],
graph: IRGraph)
group: List[String])
extends BasicBlock[Binds](BlockType("order-and-slice")) {
override def binds: Binds = dependencies.head.binds
}

View File

@ -15,7 +15,7 @@ package com.antgroup.openspg.reasoner.lube.block
import scala.collection.mutable.ListBuffer
import com.antgroup.openspg.reasoner.lube.common.graph.{IRField, IRGraph, IRVariable}
import com.antgroup.openspg.reasoner.lube.common.graph.IRField
import com.antgroup.openspg.reasoner.lube.common.rule.Rule
/**
@ -26,8 +26,7 @@ import com.antgroup.openspg.reasoner.lube.common.rule.Rule
*/
final case class ProjectBlock(
dependencies: List[Block],
projects: ProjectFields = ProjectFields(),
graph: IRGraph)
projects: ProjectFields = ProjectFields())
extends BasicBlock[Fields](BlockType("project")) {
override def binds: Fields = {

View File

@ -13,9 +13,6 @@
package com.antgroup.openspg.reasoner.lube.block
import scala.collection.mutable.ListBuffer
import com.antgroup.openspg.reasoner.common.exception.UnsupportedOperationException
import com.antgroup.openspg.reasoner.common.types.KgType
import com.antgroup.openspg.reasoner.lube.common.expr.Expr
import com.antgroup.openspg.reasoner.lube.common.graph._
@ -24,7 +21,7 @@ import com.antgroup.openspg.reasoner.lube.common.pattern.{Element, PatternElemen
/**
* every operator block tree of root is result block
*/
sealed trait ResultBlock extends Block {}
abstract class ResultBlock[B <: Binds] extends BasicBlock[B](BlockType("result"))
/**
* output as table
@ -35,16 +32,15 @@ sealed trait ResultBlock extends Block {}
final case class TableResultBlock(
dependencies: List[Block],
selectList: OrderedFields,
asList: List[String],
graph: IRGraph)
extends ResultBlock {
asList: List[String])
extends ResultBlock[OrderedFields] {
/**
* The metadata output by the current block
*
* @return
*/
override def binds: Binds = selectList
override def binds: OrderedFields = selectList
}
/**
@ -53,11 +49,8 @@ final case class TableResultBlock(
* @param outputGraphPath the path name array for output
* @param graph
*/
final case class GraphResultBlock(
dependencies: List[Block],
outputGraphPath: List[String],
graph: IRGraph)
extends ResultBlock {
final case class GraphResultBlock(dependencies: List[Block], outputGraphPath: List[String])
extends ResultBlock[Binds] {
override val binds: Binds = dependencies.head.binds
}
@ -94,34 +87,14 @@ case class AddPredicate(predicate: PredicateElement) extends DDLOp
* @param dependencies
* @param graph
*/
case class DDLBlock(ddlOp: Set[DDLOp], dependencies: List[Block], graph: IRGraph)
extends ResultBlock {
case class DDLBlock(ddlOp: Set[DDLOp], dependencies: List[Block]) extends ResultBlock[Fields] {
/**
* The metadata output by the current block
*
* @return
*/
override def binds: Binds = {
val fields = dependencies.head.binds.fields
ddlOp.head match {
case AddProperty(s, propertyType, _) =>
val field = fields.find(f => f.name.equals(s.alias)).get
if (field.isInstanceOf[IRNode]) {
field.asInstanceOf[IRNode].fields.add(propertyType)
} else if (field.isInstanceOf[IREdge]) {
field.asInstanceOf[IREdge].fields.add(propertyType)
}
Fields(fields)
case AddPredicate(predicate) =>
val newFields = new ListBuffer[IRField]
newFields.++=(dependencies.head.binds.fields)
newFields.+=(IREdge(predicate.alias, null))
Fields(newFields.toList)
case other =>
throw UnsupportedOperationException(s"$other ddlop unsupported")
}
}
override def binds: Fields = Fields.empty
}

View File

@ -15,7 +15,8 @@ package com.antgroup.openspg.reasoner.lube.block
import com.antgroup.openspg.reasoner.lube.common.graph.IRGraph
case class SourceBlock(graph: IRGraph) extends BasicBlock[Binds](BlockType("source")) {
case class SourceBlock(override val graph: IRGraph)
extends BasicBlock[Binds](BlockType("source")) {
override val dependencies: List[Block] = List.empty
override val binds: Binds = Binds.empty
}

View File

@ -24,9 +24,8 @@ import org.json4s.ext.EnumNameSerializer
import org.json4s.jackson.Serialization
import org.json4s.jackson.Serialization.write
class PropertyGraphSchema(
val nodes: mutable.Map[String, Node],
val edges: mutable.Map[SPO, Edge]) extends Serializable {
class PropertyGraphSchema(val nodes: mutable.Map[String, Node], val edges: mutable.Map[SPO, Edge])
extends Serializable {
def getNodeField(nodeTypes: Set[String], fieldName: String): Field = {
for (nodeType <- nodeTypes) {
@ -52,6 +51,14 @@ class PropertyGraphSchema(
}
}
def addNode(
typeName: String,
nodeType: NodeType.Value,
properties: Set[Field],
resolved: Boolean): Unit = {
nodes.put(typeName, Node(typeName, nodeType, properties, resolved))
}
def addVertexField(nodeType: String, field: Field): Unit = {
val node = nodes.get(nodeType)
if (!node.isEmpty) {

View File

@ -17,8 +17,8 @@ import scala.collection.mutable
import com.antgroup.openspg.reasoner.common.exception.NotDefineException
import com.antgroup.openspg.reasoner.common.graph.edge.{Direction, SPO}
import com.antgroup.openspg.reasoner.common.types.KgType
import com.antgroup.openspg.reasoner.lube.catalog.struct.{Edge, Field, Node}
import com.antgroup.openspg.reasoner.common.types.{KgType, KTString}
import com.antgroup.openspg.reasoner.lube.catalog.struct.{Edge, Field, Node, NodeType}
/**
* A graph defined by Property Graph Model with enhanced semantics.
@ -49,12 +49,22 @@ class SemanticPropertyGraph(
graphSchema.addVertexField(nodeLabel, property)
}
def getNode(nodeLabel: String): Node = {
if (nodeLabel.contains("/")) {
graphSchema.nodes(nodeLabel.split("/")(0))
} else {
graphSchema.nodes(nodeLabel)
def addNode(nodeLabel: String, nodeType: NodeType.Value, properties: Set[Field]): Unit = {
val node = graphSchema.nodes.get(nodeLabel)
if (node.isDefined) {
return
}
val finalProperties =
properties ++ Set.apply(new Field("id", KTString, true), new Field("name", KTString, true))
if (nodeType == NodeType.CONCEPT) {
graphSchema.addNode(nodeLabel, nodeType, finalProperties, true)
} else {
graphSchema.addNode(nodeLabel, nodeType, finalProperties, false)
}
}
def getNode(nodeLabel: String): Node = {
graphSchema.nodes(nodeLabel)
}
def getEdge(spoStr: String): Edge = {
@ -66,11 +76,7 @@ class SemanticPropertyGraph(
}
def containsNode(nodeLabel: String): Boolean = {
if (nodeLabel.contains("/")) {
graphSchema.nodes.contains(nodeLabel.split("/")(0))
} else {
graphSchema.nodes.contains(nodeLabel)
}
graphSchema.nodes.contains(nodeLabel)
}
def containsEdge(spoStr: String): Boolean = {

View File

@ -61,6 +61,7 @@ class JSONGraphCatalog(val propertyGraph: String) extends Catalog {
: Set[Field] = {
Set.apply(
new Field(Constants.NODE_ID_KEY, KTString, true),
new Field(Constants.VERTEX_INTERNAL_ID_KEY, KTString, true),
new Field(Constants.CONTEXT_LABEL, KTString, true))
}
@ -70,8 +71,13 @@ class JSONGraphCatalog(val propertyGraph: String) extends Catalog {
override def getDefaultEdgeProperties()
: Set[Field] = {
Set.apply(
new Field(Constants.CONTEXT_LABEL, KTString, true),
new Field(Constants.EDGE_FROM_ID_KEY, KTString, true),
new Field(Constants.EDGE_TO_ID_KEY, KTString, true)
new Field(Constants.EDGE_TO_ID_KEY, KTString, true),
new Field(Constants.EDGE_FROM_INTERNAL_ID_KEY, KTString, true),
new Field(Constants.EDGE_TO_INTERNAL_ID_KEY, KTString, true),
new Field(Constants.EDGE_FROM_ID_TYPE_KEY, KTString, true),
new Field(Constants.EDGE_TO_ID_TYPE_KEY, KTString, true)
)
}

View File

@ -11,10 +11,6 @@
* or implied.
*/
/**
* Alipay.com Inc.
* Copyright (c) 2004-2023 All Rights Reserved.
*/
package com.antgroup.openspg.reasoner.lube.catalog.impl
import scala.collection.mutable
@ -77,6 +73,7 @@ class PropertyGraphCatalog(val propertyGraphSchema: Map[String, Set[String]]) ex
: Set[Field] = {
Set.apply(
new Field(Constants.NODE_ID_KEY, KTString, true),
new Field(Constants.VERTEX_INTERNAL_ID_KEY, KTString, true),
new Field(Constants.CONTEXT_LABEL, KTString, true))
}
@ -86,8 +83,13 @@ class PropertyGraphCatalog(val propertyGraphSchema: Map[String, Set[String]]) ex
override def getDefaultEdgeProperties()
: Set[Field] = {
Set.apply(
new Field(Constants.CONTEXT_LABEL, KTString, true),
new Field(Constants.EDGE_FROM_ID_KEY, KTString, true),
new Field(Constants.EDGE_TO_ID_KEY, KTString, true)
new Field(Constants.EDGE_TO_ID_KEY, KTString, true),
new Field(Constants.EDGE_FROM_INTERNAL_ID_KEY, KTString, true),
new Field(Constants.EDGE_TO_INTERNAL_ID_KEY, KTString, true),
new Field(Constants.EDGE_FROM_ID_TYPE_KEY, KTString, true),
new Field(Constants.EDGE_TO_ID_TYPE_KEY, KTString, true)
)
}
}

View File

@ -13,8 +13,6 @@
package com.antgroup.openspg.reasoner.lube.common.graph
import scala.collection.mutable
sealed trait IRField {
def name: String
}
@ -26,14 +24,14 @@ sealed trait RichIRField extends IRField
* @param name alias
* @param fields the field names of node
*/
case class IRNode(name: String, fields: mutable.Set[String]) extends IRField
case class IRNode(name: String, fields: Set[String]) extends IRField
/**
* Used to represent Edge in QueryGraph, the name of IREdge is represented by alias
* @param name alias
* @param fields the field names of edge
*/
case class IREdge(name: String, fields: mutable.Set[String]) extends IRField
case class IREdge(name: String, fields: Set[String]) extends IRField
/**
* Used to represent prop of [[IRNode]] and [[IREdge]]
@ -65,6 +63,6 @@ case class IRPath(name: String, elements: List[IRField]) extends RichIRField
* @param name
* @param element
*/
case class IRArray(element: IRField) extends RichIRField {
case class IRRepeatPath(element: IRPath, lower: Int, upper: Int) extends RichIRField {
override def name: String = element.name
}

View File

@ -13,31 +13,22 @@
package com.antgroup.openspg.reasoner.lube.common.graph
import java.util.concurrent.atomic.AtomicInteger
/**
* A graph defined in query, which has a graph name only. Usually the name is KG
*/
trait IRGraph {
def graphName: String
def nodes: Map[String, IRNode]
def edges: Map[String, IREdge]
}
final case class KG() extends IRGraph {
final case class KG(nodes: Map[String, IRNode], edges: Map[String, IREdge]) extends IRGraph {
override def graphName: String = IRGraph.defaultGraphName
}
final case class View(graphName: String) extends IRGraph
final case class View(graphName: String, nodes: Map[String, IRNode], edges: Map[String, IREdge])
extends IRGraph
object IRGraph {
val defaultGraphName = "KG"
private val graphId: AtomicInteger = new AtomicInteger(0)
def generate: IRGraph = {
if (graphId.get() == 0) {
graphId.incrementAndGet()
KG()
} else {
View(defaultGraphName + "_" + graphId.getAndAdd(1))
}
}
}

View File

@ -14,7 +14,6 @@
package com.antgroup.openspg.reasoner.lube.common.pattern
import scala.collection.mutable
import scala.reflect.runtime.universe.TypeTag
import com.antgroup.openspg.reasoner.common.graph.edge.{Direction, SPO}
import com.antgroup.openspg.reasoner.lube.catalog.SemanticPropertyGraph
@ -130,7 +129,8 @@ case class PartialGraphPattern(
case class GraphPattern(
rootAlias: String,
nodes: Map[String, Element],
edges: Map[String, Set[Connection]]) {
edges: Map[String, Set[Connection]],
properties: Map[String, Set[String]]) {
/**
* Get pattern element by node alias

View File

@ -92,7 +92,7 @@ final case class LogicRule(ruleName: String, ruleExplain: String, expr: Expr)
newRule.addDependency(r)
}
for (r <- rule.getDependencies) {
addDependency(r)
newRule.addDependency(r)
}
newRule
}

View File

@ -30,7 +30,7 @@ object BlockUtils {
def getDefine(block: Block): Set[String] = {
val defines = new mutable.HashSet[String]()
block match {
case DDLBlock(ddlOps, _, _) =>
case DDLBlock(ddlOps, _) =>
ddlOps.foreach(op => {
op match {
case AddPredicate(predicate) =>

View File

@ -54,10 +54,10 @@ object ExprUtils {
case GetField(fieldName) =>
val refName = arg.asInstanceOf[Ref].refName
if (edgeAlias != null && edgeAlias.contains(refName)) {
List.apply(IREdge(refName, mutable.Set.apply(fieldName)))
List.apply(IREdge(refName, Set.apply(fieldName)))
} else {
// other as ir node
List.apply(IRNode(refName, mutable.Set.apply(fieldName)))
List.apply(IRNode(refName, Set.apply(fieldName)))
}
case _ => c.filter(Option(_).isDefined).flatten
}
@ -67,7 +67,14 @@ object ExprUtils {
}
} else {
e match {
case Ref(refName) => List.apply(IRVariable(refName))
case Ref(refName) =>
if (nodesAlias != null && nodesAlias.contains(refName)) {
List.apply(IRNode(refName, Set.empty))
} else if (edgeAlias != null && edgeAlias.contains(refName)) {
List.apply(IREdge(refName, Set.empty))
} else {
List.apply(IRVariable(refName))
}
case _ => List.empty
}
}
@ -139,8 +146,8 @@ object ExprUtils {
* @return
*/
def mergeListIRField(c: List[IRField]): List[IRField] = {
var nodesMap = Map[String, mutable.Set[String]]()
var edgesMap = Map[String, mutable.Set[String]]()
var nodesMap = Map[String, Set[String]]()
var edgesMap = Map[String, Set[String]]()
var refSet = mutable.Set[String]()
var variable = c
.filter(Option(_).isDefined)

View File

@ -20,7 +20,7 @@ import com.antgroup.openspg.reasoner.lube.utils.transformer.BlockTransformer
class Block2GraphPathTransformer extends BlockTransformer[GraphPath] {
def transformBlock2GraphPath(block: Block): List[GraphPath] = {
block match {
case MatchBlock(_, patterns, _) => patterns.values.toList
case MatchBlock(_, patterns) => patterns.values.toList
case _ => null
}
}

View File

@ -53,7 +53,23 @@ class Expr2QlexpressTransformer extends ExprTransformer[String] {
def trans(e: Expr, params: List[String]): String = {
val opTrans: PartialFunction[Expr, String] = {
case BinaryOpExpr(name, l, r) => params.head + binaryOpSetTrans(name) + params(1)
case BinaryOpExpr(name, l, r) =>
val opStr = binaryOpSetTrans(name)
name match {
case BIn|BLike|BRLike|BAssign
|BEqual|BNotEqual|BGreaterThan
|BNotGreaterThan|BSmallerThan|BNotSmallerThan => params.head + opStr + params(1)
case _ =>
val leftStr = l match {
case c: BinaryOpExpr => "(" + params.head + ")"
case _ => params.head
}
val rightStr = r match {
case c: BinaryOpExpr => "(" + params(1) + ")"
case _ => params(1)
}
leftStr + opStr + rightStr
}
case UnaryOpExpr(name, arg) => unaryOpSetTrans(name).format(params.head)
case FunctionExpr(name, funcArgs) => "%s(%s)".format(name, params.mkString(","))
case Ref(refName) => refName
@ -74,7 +90,7 @@ class Expr2QlexpressTransformer extends ExprTransformer[String] {
val l = list
.map(x =>
if (listType.equals(KTString)) {
"'%s'".format(x)
"\"%s\"".format(x)
} else {
x
})

View File

@ -1,24 +0,0 @@
/*
* Copyright 2023 Ant Group CO., Ltd.
*
* Licensed 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.
*/
package com.antgroup.openspg.reasoner.lube.common.graph
import org.scalatest.funspec.AnyFunSpec
import org.scalatest.matchers.should.Matchers.{convertToAnyShouldWrapper, equal}
class IRGraphTests extends AnyFunSpec{
it ("test generate graph") {
IRGraph.generate should equal(KG())
IRGraph.generate.graphName should equal("KG_1")
}
}

View File

@ -13,8 +13,6 @@
package com.antgroup.openspg.reasoner.lube.parser
import scala.collection.mutable
import com.antgroup.openspg.reasoner.common.graph.edge.Direction
import com.antgroup.openspg.reasoner.common.types.{KTInteger, KTLong, KTObject}
import com.antgroup.openspg.reasoner.lube.block.{MatchBlock, ProjectBlock, ProjectFields, SourceBlock}
@ -32,7 +30,7 @@ class TransformerTest extends AnyFunSpec {
val block = ProjectBlock(
List.apply(
MatchBlock(
List.apply(SourceBlock(KG())),
List.apply(SourceBlock(KG(Map.empty, Map.empty))),
Map.apply("t" -> GraphPath(
"s",
GraphPattern(
@ -48,13 +46,11 @@ class TransformerTest extends AnyFunSpec {
Set.apply("belong"),
"d",
Direction.IN,
null)))),
false)),
KG())),
null))), Map.empty),
false)))),
ProjectFields(
Map.apply(IRVariable("total_domain_num") ->
ProjectRule(IRVariable("total_domain_num"), KTInteger, Ref("o")))),
KG())
ProjectRule(IRVariable("total_domain_num"), KTInteger, Ref("o")))))
val p = BlockUtils.transBlock2Graph(block)
p.size should equal(1)
p.head.graphPattern.nodes.size should equal(2)
@ -209,14 +205,14 @@ class TransformerTest extends AnyFunSpec {
res(1) should equal(IRVariable("DayliyAmount"))
res(2) should equal(IRVariable("MonthAmount"))
res(3) should equal(IRNode("user", mutable.Set.apply("sex")))
res(3) should equal(IRNode("user", Set.apply("sex")))
val qlTransformer = new Expr2QlexpressTransformer()
val qlExpress = qlTransformer.transform(r5)
qlExpress.size should equal(5)
qlExpress.head should equal("r0 = r0 = 123")
qlExpress.last should equal("R3 && R1 && !(R4 && R1)")
qlExpress.last should equal("(R3 && R1) && !(R4 && R1)")
}
it("variable_rule2_to_expr") {
@ -227,7 +223,36 @@ class TransformerTest extends AnyFunSpec {
val qlExpress = qlTransformer.transform(expr)
print(qlExpress)
qlExpress.head should
equal("DayliyAmount > 300 && R1 && !(MonthAmount < 500 && R1)")
equal("((DayliyAmount > 300) && R1) && !((MonthAmount < 500) && R1)")
}
it("agg rule flat") {
val r0 = LogicRule("tmp", "",
BinaryOpExpr(BGreaterThan, UnaryOpExpr(GetField("amount"), Ref("E1")), VLong("10")))
val r = ProjectRule(IRVariable("g"),
KTLong,
OpChainExpr(
GraphAggregatorExpr(
"unresolved_default_path",
List.apply(Ref("A"), Ref("B")),
null
),
OpChainExpr(
AggIfOpExpr(
AggOpExpr(
Count,
Ref("E1")
),
Ref("tmp")
),
null
)
)
)
r.addDependency(r0)
val transformer = new Rule2ExprTransformer()
val expr = transformer.transform(r)
println(expr.pretty)
}
it("null ql") {