diff --git a/src/main/scala/org/deepdive/ddlog/DeepDiveLog.scala b/src/main/scala/org/deepdive/ddlog/DeepDiveLog.scala index a6d5fcc..2ad8f75 100644 --- a/src/main/scala/org/deepdive/ddlog/DeepDiveLog.scala +++ b/src/main/scala/org/deepdive/ddlog/DeepDiveLog.scala @@ -23,6 +23,7 @@ object DeepDiveLog { , inputFiles: List[String] = List() , query: String = null , mode: Mode = ORIGINAL + , skipDesugar: Boolean = false ) val commandLine = new scopt.OptionParser[Config]("ddlog") { val commonProgramOpts = List( @@ -38,6 +39,7 @@ object DeepDiveLog { opt[Unit]('i', "incremental") optional() action { (_, c) => c.copy(mode = INCREMENTAL) } text("Whether to derive delta rules") opt[Unit]("materialization") optional() action { (_, c) => c.copy(mode = MATERIALIZATION) } text("Whether to materialize origin data") opt[Unit]("merge") optional() action { (_, c) => c.copy(mode = MERGE) } text("Whether to merge delta data") + opt[Unit]("skip-desugar") optional() action { (_, c) => c.copy(skipDesugar = true) } text("Whether to skip desugaring and assume no sugar") arg[String]("FILE...") minOccurs(0) unbounded() action { (f, c) => c.copy(inputFiles = c.inputFiles ++ List(f)) } text("Path to DDLog program files") checkConfig { c => if (c.handler == null) failure("No command specified") @@ -64,8 +66,14 @@ trait DeepDiveLogHandler { def run(config: DeepDiveLog.Config): Unit = try { // parse each file into a single program val parsedProgram = parseFiles(config.inputFiles) + + // desugar unless explicitly said to skip so + val programToRun = + if (config.skipDesugar) parsedProgram + else DeepDiveLogDesugarRewriter.derive(parsedProgram) + // run handler with the parsed program - run(parsedProgram, config) + run(programToRun, config) } catch { case e: RuntimeException => if (sys.env contains "DDLOG_STACK_TRACE") throw e diff --git a/src/main/scala/org/deepdive/ddlog/DeepDiveLogCompiler.scala b/src/main/scala/org/deepdive/ddlog/DeepDiveLogCompiler.scala index 411dba3..9f8d7a8 100644 --- a/src/main/scala/org/deepdive/ddlog/DeepDiveLogCompiler.scala +++ b/src/main/scala/org/deepdive/ddlog/DeepDiveLogCompiler.scala @@ -350,7 +350,8 @@ class QueryCompiler(cq : ConjunctiveQuery, ss: CompilationState) { } // resolve an expression - def compileExpr(e: Expr) : String = { + def compileExpr(e: Expr) : String = compileExpr(e, 0) + def compileExpr(e: Expr, level: Int) : String = { e match { case VarExpr(name) => compileVariable(name) case NullConst() => "NULL" @@ -364,30 +365,37 @@ class QueryCompiler(cq : ConjunctiveQuery, ss: CompilationState) { resolved } case BinaryOpExpr(lhs, op, rhs) => { - val resovledLhs = compileExpr(lhs) - val resovledRhs = compileExpr(rhs) - s"(${resovledLhs} ${op} ${resovledRhs})" + val resovledLhs = compileExpr(lhs, level + 1) + val resovledRhs = compileExpr(rhs, level + 1) + val sql = s"${resovledLhs} ${op} ${resovledRhs}" + if (level == 0) sql else s"(${sql})" } case TypecastExpr(lhs, rhs) => { val resovledLhs = compileExpr(lhs) s"(${resovledLhs} :: ${rhs})" } + case IfThenElseExpr(ifCondThenExprPairs, optElseExpr) => { + (ifCondThenExprPairs map { + case (ifCond, thenExpr) => s"WHEN ${compileCond(ifCond)} THEN ${compileExpr(thenExpr)}" + }) ++ List(optElseExpr map compileExpr mkString("ELSE ", "", "")) + } mkString("\nCASE ", "\n ", "\nEND") } } // resolve a condition - def compileCond(cond: Cond) : String = { + def compileCond(cond: Cond) : String = compileCond(cond, 0) + def compileCond(cond: Cond, level: Int) : String = { cond match { - case ComparisonCond(lhs, op, rhs) => - s"${compileExpr(lhs)} ${op} ${compileExpr(rhs)}" - case NegationCond(c) => s"(NOT ${compileCond(c)})" + case ExprCond(e) => compileExpr(e) + case NegationCond(c) => s"NOT ${compileCond(c, level + 1)}" case CompoundCond(lhs, op, rhs) => { - val resolvedLhs = s"${compileCond(lhs)}" - val resolvedRhs = s"${compileCond(rhs)}" - op match { - case LogicOperator.AND => s"(${resolvedLhs} AND ${resolvedRhs})" - case LogicOperator.OR => s"(${resolvedLhs} OR ${resolvedRhs})" + val resolvedLhs = s"${compileCond(lhs, level + 1)}" + val resolvedRhs = s"${compileCond(rhs, level + 1)}" + val sql = op match { + case LogicOperator.AND => s"${resolvedLhs} AND ${resolvedRhs}" + case LogicOperator.OR => s"${resolvedLhs} OR ${resolvedRhs}" } + if (level == 0) sql else s"(${sql})" } case _ => "" } @@ -611,7 +619,7 @@ object DeepDiveLogCompiler extends DeepDiveLogHandler { if (stmt.supervision != None) { if (stmt.q.bodies.length > 1) ss.error(s"Scoping rule does not allow disjunction.\n") val headStr = qc.generateSQLHead(NoAlias) - val labelCol = qc.compileVariable(stmt.supervision.get) + val labelCol = qc.compileExpr(stmt.supervision.get) inputQueries += s"""SELECT DISTINCT ${ headStr }, 0 AS id, ${labelCol} AS label ${ qc.generateSQLBody(cqBody) } """ diff --git a/src/main/scala/org/deepdive/ddlog/DeepDiveLogDesugarRewriter.scala b/src/main/scala/org/deepdive/ddlog/DeepDiveLogDesugarRewriter.scala new file mode 100644 index 0000000..b06f5b1 --- /dev/null +++ b/src/main/scala/org/deepdive/ddlog/DeepDiveLogDesugarRewriter.scala @@ -0,0 +1,86 @@ +package org.deepdive.ddlog + +object DeepDiveLogDesugarRewriter { + + // Rewrite function call rules whose output coincides with normal rules. + def desugarUnionsImpliedByFunctionCallRules(program: DeepDiveLog.Program) = { + def indexByFirst[a,b](pairs: Seq[(a,b)]): Map[a,List[b]] = + pairs groupBy { _._1 } mapValues { _ map (_._2) toList } + + val schemaByName = indexByFirst(program collect { + case decl: SchemaDeclaration => decl.a.name -> decl + }) mapValues (_ head) + val rulesWithIndexByName = indexByFirst(program.zipWithIndex collect { + case (fncall: FunctionCallRule, i) => fncall.output -> (fncall, i) + case (rule : ExtractionRule , i) => rule.headName -> (rule , i) + }) + val relationNamesUsedInProgram = program collect { + case decl: SchemaDeclaration => decl.a.name + case fncall: FunctionCallRule => fncall.output + case rule: ExtractionRule => rule.headName + } toSet + + // find names that have multiple function calls or mixed type rules + val relationsToDesugar = rulesWithIndexByName flatMap { + case (name, allRules) => + val (fncalls, rules) = allRules map {_._1} partition {_.isInstanceOf[FunctionCallRule]} + if ((fncalls size) > 1 || ((fncalls size) > 0 && (rules size) > 0)) { + Some(name) + } else None + } + val rulesToRewrite = relationsToDesugar flatMap rulesWithIndexByName map { + _._1} filter {_.isInstanceOf[FunctionCallRule]} toList + + // determine a separator that does not create name clashes with existing heads for each relation to rewrite + val prefixForRelation: Map[String, String] = relationsToDesugar map { name => + name -> ( + Stream.from(1) map { n => s"${name}${"_" * n}" + } dropWhile { prefix => + relationNamesUsedInProgram exists {_ startsWith prefix} + } head + ) + } toMap + + // how to make names unique + def makeUnique(name: String, ordLocal: Int, ordGlobal: Int): String = { + s"${prefixForRelation(name)}${ordLocal}" + } + + // plan the rewrite + val rewritePlan : Map[Statement, List[Statement]] = + program collect { + // only function call rule needs to be rewritten + case fncall: FunctionCallRule if rulesToRewrite contains fncall => + val relationName: String = fncall.output + val rulesForTheRelationToRewriteOrdered = rulesWithIndexByName(relationName + ) sortBy {_._2} filter {_._1.isInstanceOf[FunctionCallRule]} + val orderAmongRulesToRewrite = rulesForTheRelationToRewriteOrdered map {_._1} indexOf(fncall) + val orderInProgram = rulesForTheRelationToRewriteOrdered(orderAmongRulesToRewrite)._2 + val nameUnique: String = makeUnique(relationName, orderAmongRulesToRewrite, orderInProgram) + val schema = schemaByName(relationName) + fncall -> List( + schema.copy(a = schema.a.copy(name = nameUnique)), + fncall.copy(output = nameUnique), + ExtractionRule( + headName = relationName, + q = ConjunctiveQuery( + headTerms = schema.a.terms map VarExpr, + bodies = List(List(BodyAtom(name = nameUnique, terms = schema.a.terms map VarPattern))) + ) + ) + ) + // TODO add union after the last or first or somewhere + } toMap + + // apply rewrite plan + program flatMap { case rule => rewritePlan getOrElse(rule, List(rule)) } + } + + + def derive(program: DeepDiveLog.Program): DeepDiveLog.Program = { + (List( + desugarUnionsImpliedByFunctionCallRules(_) + ) reduce (_.compose(_)) + )(program) + } +} diff --git a/src/main/scala/org/deepdive/ddlog/DeepDiveLogParser.scala b/src/main/scala/org/deepdive/ddlog/DeepDiveLogParser.scala index bced9a0..4524831 100644 --- a/src/main/scala/org/deepdive/ddlog/DeepDiveLogParser.scala +++ b/src/main/scala/org/deepdive/ddlog/DeepDiveLogParser.scala @@ -23,6 +23,7 @@ case class NullConst extends ConstExpr case class FuncExpr(function: String, args: List[Expr], isAggregation: Boolean) extends Expr case class BinaryOpExpr(lhs: Expr, op: String, rhs: Expr) extends Expr case class TypecastExpr(lhs: Expr, rhs: String) extends Expr +case class IfThenElseExpr(ifCondThenExprPairs: List[(Cond, Expr)], elseExpr: Option[Expr]) extends Expr sealed trait Pattern case class VarPattern(name: String) extends Pattern @@ -39,7 +40,7 @@ case class OuterModifier extends BodyModifier case class AllModifier extends BodyModifier case class Attribute(name : String, terms : List[String], types : List[String], annotations : List[List[Annotation]]) -case class ConjunctiveQuery(headTerms: List[Expr], bodies: List[List[Body]], isDistinct: Boolean, limit: Option[Int], +case class ConjunctiveQuery(headTerms: List[Expr], bodies: List[List[Body]], isDistinct: Boolean = false, limit: Option[Int] = None, // optional annotations for head terms headTermAnnotations: List[List[Annotation]] = List.empty, // XXX This flag is not ideal, but minimizes the impact of query treatment when compared to creating another case class @@ -57,7 +58,7 @@ case class RuleAnnotation(name: String, args: List[String]) // condition sealed trait Cond extends Body -case class ComparisonCond(lhs: Expr, op: String, rhs: Expr) extends Cond +case class ExprCond(expr: Expr) extends Cond case class NegationCond(cond: Cond) extends Cond case class CompoundCond(lhs: Cond, op: LogicOperator.LogicOperator, rhs: Cond) extends Cond @@ -105,7 +106,7 @@ case class SchemaDeclaration( a : Attribute ) extends Statement // atom and whether this is a query relation. case class FunctionDeclaration( functionName: String, inputType: FunctionInputOutputType, outputType: FunctionInputOutputType, implementations: List[FunctionImplementationDeclaration]) extends Statement -case class ExtractionRule(headName: String, q : ConjunctiveQuery, supervision: Option[String] = None) extends Statement // Extraction rule +case class ExtractionRule(headName: String, q : ConjunctiveQuery, supervision: Option[Expr] = None) extends Statement // Extraction rule case class FunctionCallRule(output: String, function: String, q : ConjunctiveQuery, mode: Option[String], parallelism: Option[Int]) extends Statement // Extraction rule case class InferenceRule(head: InferenceRuleHead, q : ConjunctiveQuery, weights : FactorWeight, mode: Option[String] = None) extends Statement // Weighted rule @@ -186,7 +187,8 @@ class DeepDiveLogParser extends JavaTokenParsers { } } - def operator = "||" | "+" | "-" | "*" | "/" | "&" | "%" + def operator = + ( "||" | "+" | "-" | "*" | "/" | "&" | "%" ) def typeOperator = "::" val aggregationFunctions = Set("MAX", "SUM", "MIN", "ARRAY_ACCUM", "ARRAY_AGG", "COUNT") @@ -197,21 +199,35 @@ class DeepDiveLogParser extends JavaTokenParsers { | lexpr ) - def lexpr : Parser[Expr] = - ( functionName ~ "(" ~ rep1sep(expr, ",") ~ ")" ^^ { - case (name ~ _ ~ args ~ _) => FuncExpr(name, args, (aggregationFunctions contains name)) + def cexpr = + ( expr ~ compareOperator ~ expr ^^ { case (lhs ~ op ~ rhs) => BinaryOpExpr(lhs, op, rhs) } + | expr + ) + + def lexpr = + ( "if" ~> (cond ~ ("then" ~> expr) ~ rep(elseIfExprs) ~ opt("else" ~> expr)) <~ "end" ^^ { + case (ifCond ~ thenExpr ~ elseIfs ~ optElseExpr) => + IfThenElseExpr((ifCond, thenExpr) :: elseIfs, optElseExpr) } | stringLiteralAsString ^^ { StringConst(_) } | double ^^ { DoubleConst(_) } | integer ^^ { IntConst(_) } | ("TRUE" | "FALSE") ^^ { x => BooleanConst(x.toBoolean) } | "NULL" ^^ { _ => new NullConst } + | functionName ~ "(" ~ rep1sep(expr, ",") ~ ")" ^^ { + case (name ~ _ ~ args ~ _) => FuncExpr(name, args, (aggregationFunctions contains name)) + } | variableName ^^ { VarExpr(_) } | "(" ~> expr <~ ")" ) + def elseIfExprs = + ("else" ~> "if" ~> cond) ~ ("then" ~> expr) ^^ { + case (ifCond ~ thenExpr) => (ifCond, thenExpr) + } + // conditional expressions - def compareOperator = "LIKE" | ">" | "<" | ">=" | "<=" | "!=" | "=" | "IS NOT" | "IS" + def compareOperator = "LIKE" | ">" | "<" | ">=" | "<=" | "!=" | "=" | "IS" ~ "NOT" ^^ { _ => "IS NOT" } | "IS" def cond : Parser[Cond] = ( acond ~ (";") ~ cond ^^ { case (lhs ~ op ~ rhs) => @@ -229,9 +245,7 @@ class DeepDiveLogParser extends JavaTokenParsers { | bcond ) def bcond : Parser[Cond] = - ( expr ~ compareOperator ~ expr ^^ { case (lhs ~ op ~ rhs) => - ComparisonCond(lhs, op, rhs) - } + ( cexpr ^^ ExprCond | "[" ~> cond <~ "]" ) @@ -292,18 +306,23 @@ class DeepDiveLogParser extends JavaTokenParsers { FunctionDeclaration(a, inTy, outTy, implementationDecls) } - def cqBody: Parser[Body] = cond | quantifiedBody | atom + def cqBody: Parser[Body] = quantifiedBody | atom | cond def cqConjunctiveBody: Parser[List[Body]] = rep1sep(cqBody, ",") def cqHeadTerms = "(" ~> rep1sep(expr, ",") <~ ")" + def conjunctiveQueryBody : Parser[ConjunctiveQuery] = + opt("*") ~ opt("|" ~> decimalNumber) ~ ":-" ~ rep1sep(cqConjunctiveBody, ";") ^^ { + case (isDistinct ~ limit ~ ":-" ~ disjunctiveBodies) => + ConjunctiveQuery(List.empty, disjunctiveBodies, isDistinct != None, limit map (_.toInt)) + } + def conjunctiveQuery : Parser[ConjunctiveQuery] = // TODO fill headTermAnnotations as done in queryWithOptionalHeadTerms to support @order_by - cqHeadTerms ~ opt("*") ~ opt("|" ~> decimalNumber) ~ ":-" ~ rep1sep(cqConjunctiveBody, ";") ^^ { - case (head ~ isDistinct ~ limit ~ ":-" ~ disjunctiveBodies) => - ConjunctiveQuery(head, disjunctiveBodies, isDistinct != None, limit map (_.toInt)) - } + cqHeadTerms ~ conjunctiveQueryBody ^^ { + case (head ~ cq) => cq.copy(headTerms = head) + } def functionMode = "@mode" ~> commit("(" ~> functionModeType <~ ")" ^? ({ case "inc" => "inc" @@ -311,29 +330,28 @@ class DeepDiveLogParser extends JavaTokenParsers { def parallelism = "@parallelism" ~> "(" ~> integer <~ ")" - def supervision = "=" ~> (variableName | "TRUE" | "FALSE") - - def conjunctiveQueryWithSupervision = // returns Parser[String], Parser[ConjunctiveQuery] - cqHeadTerms ~ opt("*") ~ opt("|" ~> decimalNumber) ~ opt(supervision) ~ ":-" ~ rep1sep(cqConjunctiveBody, ";") ^^ { - case (head ~ isDistinct ~ limit ~ sup ~ ":-" ~ disjunctiveBodies) => - (sup, ConjunctiveQuery(head, disjunctiveBodies, isDistinct != None, limit map (_.toInt))) - } - def functionCallRule : Parser[FunctionCallRule] = opt(functionMode) ~ opt(parallelism) ~ relationName ~ "+=" ~ functionName ~ conjunctiveQuery ^^ { case (mode ~ parallelism ~ out ~ _ ~ func ~ cq) => FunctionCallRule(out, func, cq, mode, parallelism) } - def oldstyleSupervision = "@label" ~> "(" ~> (variableName | "TRUE" | "FALSE") <~ ")" + def supervisionAnnotation = "@label" ~> "(" ~> expr <~ ")" + + def conjunctiveQueryWithSupervision : Parser[ConjunctiveQuery] = + cqHeadTerms ~ opt("*") ~ opt("|" ~> decimalNumber) ~ ":-" ~ rep1sep(cqConjunctiveBody, ";") ^^ { + case (head ~ isDistinct ~ limit ~ ":-" ~ disjunctiveBodies) => + ConjunctiveQuery(head, disjunctiveBodies, isDistinct != None, limit map (_.toInt)) + } def extractionRule = - ( relationName ~ conjunctiveQueryWithSupervision ^^ { - case (head ~ cq) => ExtractionRule(head, cq._2, cq._1) - } - | oldstyleSupervision ~ relationName ~ conjunctiveQuery ^^ { - case (sup ~ head ~ cq) => ExtractionRule(head, cq, Some(sup)) - } - ) + ( opt(supervisionAnnotation) ~ relationName ~ conjunctiveQuery ^^ { + case (sup ~ head ~ cq) => ExtractionRule(head, cq, sup) + } + | relationName ~ cqHeadTerms ~ ("=" ~> expr) ~ conjunctiveQueryBody ^^ { + case (head ~ headTerms ~ sup ~ cq) => + ExtractionRule(head, cq.copy(headTerms = headTerms), Some(sup)) + } + ) def factorWeight = "@weight" ~> "(" ~> rep1sep(expr, ",") <~ ")" ^^ { FactorWeight(_) } def inferenceMode = "@mode" ~> commit("(" ~> inferenceModeType <~ ")" ^? ({ diff --git a/src/main/scala/org/deepdive/ddlog/DeepDiveLogPrettyPrinter.scala b/src/main/scala/org/deepdive/ddlog/DeepDiveLogPrettyPrinter.scala index ad134b3..f1d602c 100644 --- a/src/main/scala/org/deepdive/ddlog/DeepDiveLogPrettyPrinter.scala +++ b/src/main/scala/org/deepdive/ddlog/DeepDiveLogPrettyPrinter.scala @@ -73,7 +73,8 @@ object DeepDiveLogPrettyPrinter extends DeepDiveLogHandler { } // print an expression - def print(e: Expr) : String = { + def print(e: Expr) : String = print(e, 0) + def print(e: Expr, level: Int) : String = { e match { case VarExpr(name) => name case NullConst() => "NULL" @@ -85,8 +86,16 @@ object DeepDiveLogPrettyPrinter extends DeepDiveLogHandler { val resolvedArgs = args map (x => print(x)) s"${function}(${resolvedArgs.mkString(", ")})" } - case BinaryOpExpr(lhs, op, rhs) => s"(${print(lhs)} ${op} ${print(rhs)})" + case BinaryOpExpr(lhs, op, rhs) => { + val p = s"${print(lhs, level + 1)} ${op} ${print(rhs, level + 1)}" + if (level == 0) p else s"(${p})" + } case TypecastExpr(lhs, rhs) => s"(${print(lhs)} :: ${rhs})" + case IfThenElseExpr(ifCondExprPairs, optElseExpr) => ( + (ifCondExprPairs map { + case (ifCond, thenExpr) => s"if ${print(ifCond)} then ${print(thenExpr)}" + }) ++ (optElseExpr map print toList) + ) mkString("", "\n\telse ", "\n\tend") } } @@ -114,7 +123,7 @@ object DeepDiveLogPrettyPrinter extends DeepDiveLogHandler { // print a condition def print(cond: Cond) : String = { cond match { - case ComparisonCond(lhs, op, rhs) => s"${print(lhs)} ${op} ${print(rhs)}" + case ExprCond(e) => print(e) case NegationCond(c) => s"[!${print(c)}]" case CompoundCond(lhs, op, rhs) => { op match { @@ -131,7 +140,7 @@ object DeepDiveLogPrettyPrinter extends DeepDiveLogHandler { case b: QuantifiedBody => print(b) } - def print(cq: ConjunctiveQuery, supervision: String = ""): String = { + def print(cq: ConjunctiveQuery, optSupervision: Option[Expr] = None): String = { def printBodyList(b: List[Body]) = { s"${(b map print).mkString(",\n ")}" @@ -145,7 +154,9 @@ object DeepDiveLogPrettyPrinter extends DeepDiveLogHandler { val headStr = if (headStrTmp isEmpty) "" else s"(${headStrTmp})" val headBodySeparator = if (cq.isForQuery) "?-" else ":-" - headStr + distinctStr + limitStr + supervision + s" ${headBodySeparator}\n " + bodyStr + headStr + ( + optSupervision map { sv => s" = ${print(sv)}" } getOrElse("") + ) + distinctStr + limitStr + s" ${headBodySeparator}\n " + bodyStr } def print(a: HeadAtom) : String = { @@ -168,8 +179,7 @@ object DeepDiveLogPrettyPrinter extends DeepDiveLogHandler { } def print(stmt: ExtractionRule): String = { - var supervision = stmt.supervision map (s => s" = ${s}") getOrElse(""); - stmt.headName + print(stmt.q, supervision) + ".\n" + stmt.headName + print(stmt.q, stmt.supervision) + ".\n" } def print(stmt: FunctionCallRule): String = { diff --git a/src/main/scala/org/deepdive/ddlog/DeepDiveLogSchemaExporter.scala b/src/main/scala/org/deepdive/ddlog/DeepDiveLogSchemaExporter.scala index 173f41d..639c9c7 100644 --- a/src/main/scala/org/deepdive/ddlog/DeepDiveLogSchemaExporter.scala +++ b/src/main/scala/org/deepdive/ddlog/DeepDiveLogSchemaExporter.scala @@ -58,16 +58,16 @@ object DeepDiveLogSchemaExporter extends DeepDiveLogHandler { } override def run(parsedProgram: DeepDiveLog.Program, config: DeepDiveLog.Config) = { - val program = parsedProgram // TODO derive the program based on config.mode? + var programToExport = parsedProgram // TODO derive the program based on config.mode? // first find out names of the relations that have SchemaDeclaration - val declaredNames = program collect { + val declaredNames = programToExport collect { case decl: SchemaDeclaration => decl.a.name } // then print schema in JSON println(JSONObject(Map( - "relations" -> JSONObject(program collect { + "relations" -> JSONObject(programToExport collect { case decl: SchemaDeclaration => export(decl) case rule: ExtractionRule if ! (declaredNames contains rule.headName) => // for extraction rules whose head is not declared already, diff --git a/src/main/scala/org/deepdive/ddlog/DeepDiveLogSemanticChecker.scala b/src/main/scala/org/deepdive/ddlog/DeepDiveLogSemanticChecker.scala index 65bb47b..e8058a4 100644 --- a/src/main/scala/org/deepdive/ddlog/DeepDiveLogSemanticChecker.scala +++ b/src/main/scala/org/deepdive/ddlog/DeepDiveLogSemanticChecker.scala @@ -189,18 +189,19 @@ object DeepDiveLogSemanticChecker extends DeepDiveLogHandler { stmt match { case s: ExtractionRule if (schemaDeclaration contains s.headName) && (schemaDeclaration(s.headName).variableType nonEmpty) => { val headType = schemaDeclaration(s.headName).variableType.get - s.supervision match { - case Some("TRUE") | Some("FALSE") => { + s.supervision foreach { + case b: BooleanConst => { if (headType != BooleanType) { error(s, s"Supervision column ${s.supervision} should be boolean type but is ${headType} type") } } - case Some(varname) => + case VarExpr(varname) => s.q.bodies.foreach { bodies: List[Body] => bodies.foreach { b => checkSupervisionLabelType(s, headType, VarPattern(varname), b) } } - case None => + case _ => + // XXX assume the rest of the expressions are correct } } case _ => @@ -213,6 +214,10 @@ object DeepDiveLogSemanticChecker extends DeepDiveLogHandler { case FuncExpr(function, args, agg) => args flatMap collectUsedVars toSet case BinaryOpExpr(lhs, op, rhs) => collectUsedVars(lhs) ++ collectUsedVars(rhs) case TypecastExpr(lhs, rhs) => collectUsedVars(lhs) + case IfThenElseExpr(ifCondThenExprs, optElseExpr) => + (ifCondThenExprs flatMap { case (ifCond, thenExpr) => + collectUsedVars(ifCond) ++ collectUsedVars(thenExpr) + } toSet) ++ (optElseExpr.toSet flatMap { e:Expr => collectUsedVars(e) }) case _ => Set() } @@ -228,7 +233,7 @@ object DeepDiveLogSemanticChecker extends DeepDiveLogHandler { } def collectUsedVars(cond: Cond) : Set[String] = cond match { - case ComparisonCond(lhs, op, rhs) => collectUsedVars(lhs) ++ collectUsedVars(rhs) + case ExprCond(e) => collectUsedVars(e) case CompoundCond(lhs, op, rhs) => collectUsedVars(lhs) ++ collectUsedVars(rhs) case NegationCond(c) => collectUsedVars(c) } @@ -258,7 +263,7 @@ object DeepDiveLogSemanticChecker extends DeepDiveLogHandler { if (varUndefined nonEmpty) error(stmt, s"Variable ${varUndefined mkString(", ")} must have bindings") } stmt match { - case s: ExtractionRule => checkCq(s.q) + case s: ExtractionRule => checkCq(s.q, (s.supervision.toSet flatMap { e:Expr => collectUsedVars(e) })) case s: InferenceRule => checkCq(s.q, (s.weights.variables flatMap collectUsedVars toSet)) case _ => } diff --git a/test/expected-output-test.bats.template b/test/expected-output-test.bats.template index a6ac995..325b634 100644 --- a/test/expected-output-test.bats.template +++ b/test/expected-output-test.bats.template @@ -11,6 +11,12 @@ setup() { [ -e "$TESTDIR" ] } +expectedOutputExistsFor() { + expectedOutput="$TESTDIR"/"$1".expected + actualOutput="${expectedOutput%.expected}".actual + [ -e "$expectedOutput" ] +} + ## tests for basic compilation and pretty-printing # check if example can be parsed @@ -29,7 +35,7 @@ setup() { # check if print is idempotent @test "$it parses and prints what it prints (idempotent)" { - printed=$TESTDIR/printed.actual + printed="$TESTDIR"/printed.actual ddlog print "$TESTDIR"/input.ddlog >"$printed" || skip ddlog print "$printed" >"$TESTDIR"/printed-printed.actual diff "$printed" "$TESTDIR"/printed-printed.actual @@ -38,27 +44,28 @@ setup() { # compare the compiled output with what's expected @test "$it compiles input as expected" { - expectedOutput=$TESTDIR/compile.expected - actualOutput=${expectedOutput%.expected}.actual - [ -e "$expectedOutput" ] || skip + expectedOutputExistsFor "compile" || skip ddlog compile "$TESTDIR"/input.ddlog >"$actualOutput" diff "$expectedOutput" "$actualOutput" } # compare the pretty-printed output with what's expected @test "$it prints input as expected" { - expectedOutput=$TESTDIR/print.expected - actualOutput=${expectedOutput%.expected}.actual - [ -e "$expectedOutput" ] || skip + expectedOutputExistsFor "print" || skip + ddlog print --skip-desugar "$TESTDIR"/input.ddlog >"$actualOutput" + diff "$expectedOutput" "$actualOutput" +} + +# compare the desugared output with what's expected +@test "$it desugars as expected" { + expectedOutputExistsFor "desugar" || skip ddlog print "$TESTDIR"/input.ddlog >"$actualOutput" diff "$expectedOutput" "$actualOutput" } # compare the export-schema output with what's expected @test "$it exports schema and annotations as expected" { - expectedOutput=$TESTDIR/export-schema.expected - actualOutput=${expectedOutput%.expected}.actual - [ -e "$expectedOutput" ] || skip + expectedOutputExistsFor "export-schema" || skip ddlog export-schema "$TESTDIR"/input.ddlog >"$actualOutput" diff "$expectedOutput" "$actualOutput" } @@ -68,18 +75,14 @@ setup() { # compare the compiled output of the incremental version with what's expected @test "$it compiles --incremental input as expected" { - expectedOutput=$TESTDIR/compile-incremental.expected - actualOutput=${expectedOutput%.expected}.actual - [ -e "$expectedOutput" ] || skip + expectedOutputExistsFor "compile-incremental" || skip ddlog compile --incremental "$TESTDIR"/input.ddlog >"$actualOutput" diff "$expectedOutput" "$actualOutput" } # compare the pretty-printed output of the incremental version with what's expected @test "$it prints --incremental input as expected" { - expectedOutput=$TESTDIR/print-incremental.expected - actualOutput=${expectedOutput%.expected}.actual - [ -e "$expectedOutput" ] || skip + expectedOutputExistsFor "print-incremental" || skip ddlog print --incremental "$TESTDIR"/input.ddlog >"$actualOutput" diff "$expectedOutput" "$actualOutput" } @@ -88,9 +91,7 @@ setup() { # compare the compiled output of the merge version with what's expected @test "$it compiles --merge input as expected" { - expectedOutput=$TESTDIR/compile-merge.expected - actualOutput=${expectedOutput%.expected}.actual - [ -e "$expectedOutput" ] || skip + expectedOutputExistsFor "compile-merge" || skip ddlog compile --merge "$TESTDIR"/input.ddlog >"$actualOutput" diff "$expectedOutput" "$actualOutput" } @@ -99,9 +100,7 @@ setup() { # compare the compiled output of the merge version with what's expected @test "$it compiles --materialization input as expected" { - expectedOutput=$TESTDIR/compile-materialization.expected - actualOutput=${expectedOutput%.expected}.actual - [ -e "$expectedOutput" ] || skip + expectedOutputExistsFor "compile-materialization" || skip ddlog compile --materialization "$TESTDIR"/input.ddlog >"$actualOutput" diff "$expectedOutput" "$actualOutput" } diff --git a/test/expected-output-test/expressions/compile.expected b/test/expected-output-test/expressions/compile.expected index 861115e..19f082a 100644 --- a/test/expected-output-test/expressions/compile.expected +++ b/test/expected-output-test/expressions/compile.expected @@ -93,7 +93,7 @@ FROM b R0 LEFT OUTER JOIN c R1_0 ON R1_0.n = R0.k AND R1_0.s > R0.p deepdive create view J as 'SELECT R0.k AS column_0 FROM b R0 - WHERE ((R0.k + R0.r) = 100 OR (NOT R0.k > 50))' + WHERE (R0.k + R0.r) = 100 OR NOT R0.k > 50' """ output_relation: "J" @@ -126,7 +126,7 @@ FROM b R0_0 ON FULL OUTER JOIN c R1_0 ON R1_0.n = R0_0.k deepdive.extraction.extractors.ext_F { cmd: """ - deepdive create view F as 'SELECT ((R0.k :: INT) + (R0.r :: INT)) AS column_0 + deepdive create view F as 'SELECT (R0.k :: INT) + (R0.r :: INT) AS column_0 FROM b R0 ' @@ -143,7 +143,7 @@ FROM b R0 deepdive.extraction.extractors.ext_A { cmd: """ - deepdive create view A as 'SELECT (R0.k + (R0.r * (R0.k + R0.r))) AS column_0 + deepdive create view A as 'SELECT R0.k + (R0.r * (R0.k + R0.r)) AS column_0 FROM b R0 ' @@ -180,7 +180,7 @@ FROM b R0 deepdive create view I as 'SELECT R0.k AS column_0 FROM b R0 - WHERE ((R0.k + R0.r) = 100 OR (R0.k > 50 AND R0.r < 10))' + WHERE (R0.k + R0.r) = 100 OR (R0.k > 50 AND R0.r < 10)' """ output_relation: "I" @@ -232,7 +232,7 @@ FROM c R0, a R1 deepdive create view Q as 'SELECT DISTINCT ('\''test'\'' :: TEXT) AS column_0, 123 AS column_1, R0.k AS column_2, unnest(R1.q) AS column_3 FROM a R0, b R1, c R2 - WHERE R1.k = R0.k AND R2.s = (R1.p || R1.q) AND R2.n = 10 AND R2.t = '\''foo'\'' AND (R1.r > 100 OR ((NOT R1.r < 20) AND R1.r < 50))' + WHERE R1.k = R0.k AND R2.s = R1.p || R1.q AND R2.n = 10 AND R2.t = '\''foo'\'' AND R1.r > 100 OR (NOT R1.r < 20 AND R1.r < 50)' """ output_relation: "Q" @@ -267,7 +267,7 @@ LIMIT 100' deepdive.extraction.extractors.ext_B { cmd: """ - deepdive create view B as 'SELECT func((R0.k + (R0.r + R0.k))) AS column_0 + deepdive create view B as 'SELECT func(R0.k + (R0.r + R0.k)) AS column_0 FROM b R0 ' @@ -322,7 +322,7 @@ FROM b R0 deepdive create view H as 'SELECT R0.k AS column_0 FROM b R0 - WHERE ((R0.k + R0.r) = 100 AND R0.k > 50)' + WHERE (R0.k + R0.r) = 100 AND R0.k > 50' """ output_relation: "H" @@ -357,7 +357,7 @@ FROM a R0 deepdive create view K as 'SELECT R0.k AS column_0 FROM b R0 - WHERE ((R0.k + R0.r) = 100 AND ((NOT R0.k > 50) OR R0.k = 40))' + WHERE (R0.k + R0.r) = 100 AND (NOT R0.k > 50 OR R0.k = 40)' """ output_relation: "K" @@ -391,7 +391,7 @@ FROM b R0 deepdive.extraction.extractors.ext_D { cmd: """ - deepdive create view D as 'SELECT (func((R0.k * func2((R0.r + R0.k)))) + R0.k) AS column_0 + deepdive create view D as 'SELECT func(R0.k * func2(R0.r + R0.k)) + R0.k AS column_0 FROM b R0 ' diff --git a/test/expected-output-test/expressions/print.expected b/test/expected-output-test/expressions/print.expected index b5f2669..798da86 100644 --- a/test/expected-output-test/expressions/print.expected +++ b/test/expected-output-test/expressions/print.expected @@ -12,29 +12,29 @@ c(s text, Q(("test" :: TEXT), 123, id, unnest(y)) * :- a(id), b(id, x, y, z), - c((x || y), 10, "foo"), + c(x || y, 10, "foo"), [z > 100; [[!z < 20], z < 50]]. P(y, z, MAX(w)) :- a(x), b(x, y, z, w). -A((x + (w * (x + w)))) :- +A(x + (w * (x + w))) :- b(x, y, z, w). -B(func((x + (w + x)))) :- +B(func(x + (w + x))) :- b(x, y, z, w). C(func(func(func(x)))) :- b(x, y, z, w). -D((func((x * func2((w + x)))) + x)) :- +D(func(x * func2(w + x)) + x) :- b(x, y, z, w). E((x :: INT)) :- b(x, y, z, w). -F(((x :: INT) + (w :: INT))) :- +F((x :: INT) + (w :: INT)) :- b(x, y, z, w). G(x) :- diff --git a/test/expected-output-test/if-then-else/compile.expected b/test/expected-output-test/if-then-else/compile.expected new file mode 100644 index 0000000..33fa69f --- /dev/null +++ b/test/expected-output-test/if-then-else/compile.expected @@ -0,0 +1,106 @@ + + deepdive.db.default { + driver: "org.postgresql.Driver" + url: "jdbc:postgresql://"${PGHOST}":"${PGPORT}"/"${DBNAME} + user: ${PGUSER} + password: ${PGPASSWORD} + dbname: ${DBNAME} + host: ${PGHOST} + port: ${PGPORT} + incremental_mode: ORIGINAL + } + + + + deepdive.schema.variables { + q.label: Boolean + } + + + deepdive.extraction.extractors.ext_labels_resolved { + cmd: """ + + # TODO use temporary table + deepdive create table "labels_resolved" + deepdive sql 'INSERT INTO labels_resolved SELECT R0.x AS "labels.R0.x", SUM( +CASE WHEN R0.l IS NULL THEN 0 + WHEN R0.l = true THEN 1 + ELSE -1 +END) AS column_1 +FROM labels R0 + + GROUP BY R0.x' + # TODO rename temporary table to replace output_relation + + """ + output_relation: "labels_resolved" + style: "cmd_extractor" + + input_relations: [ + labels + ] + } + + + deepdive.extraction.extractors.ext_q { + cmd: """ + + # TODO use temporary table + deepdive create table "q" + deepdive sql 'INSERT INTO q SELECT DISTINCT R0.x, 0 AS id, +CASE WHEN R0.l > 0 THEN true + WHEN R0.l < 0 THEN false + ELSE NULL +END AS label + FROM labels_resolved R0 + + ' + # TODO rename temporary table to replace output_relation + + """ + output_relation: "q" + style: "cmd_extractor" + dependencies: [ "ext_labels_resolved" ] + input_relations: [ + labels_resolved + ] + } + + + deepdive.extraction.extractors.ext_p { + cmd: """ + + deepdive create view p as 'SELECT R0.x AS column_0, +CASE WHEN abs(R0.l) > 10 THEN true + ELSE false +END AS column_1 +FROM labels R0 + +UNION ALL +SELECT R0.x AS column_0, +CASE WHEN (R0.x % 10) = 0 THEN false + ELSE +END AS column_1 +FROM labels R0 + ' + + """ + output_relation: "p" + style: "cmd_extractor" + + input_relations: [ + labels + ] + } + +deepdive.pipeline.run: ${PIPELINE} +deepdive.pipeline.pipelines.extraction: [ + ext_labels_resolved + ext_q + ext_p +] +deepdive.pipeline.pipelines.endtoend: [ + ext_labels_resolved + ext_q + ext_p +] diff --git a/test/expected-output-test/if-then-else/input.ddlog b/test/expected-output-test/if-then-else/input.ddlog new file mode 100644 index 0000000..9397459 --- /dev/null +++ b/test/expected-output-test/if-then-else/input.ddlog @@ -0,0 +1,14 @@ +labels_resolved(x INT, l INT). +q?(x INT). + +q(x) = if l > 0 then TRUE + else if l < 0 then FALSE + else NULL + end :- labels_resolved(x, l). + +labels(x INT, l BOOLEAN). +p(x, if abs(l) > 10 then TRUE else FALSE end) :- labels(x, l). +p(x, if x % 10 = 0 then FALSE end) :- labels(x, l). + +labels_resolved(x, SUM(if label IS NULL then 0 else if label = TRUE then 1 else -1 end)) :- + labels(x, label). diff --git a/test/expected-output-test/if-then-else/print.expected b/test/expected-output-test/if-then-else/print.expected new file mode 100644 index 0000000..3571723 --- /dev/null +++ b/test/expected-output-test/if-then-else/print.expected @@ -0,0 +1,29 @@ +labels_resolved(x INT, + l INT). + +q?(x INT). + +q(x) = if l > 0 then true + else if l < 0 then false + else NULL + end :- + labels_resolved(x, l). + +labels(x INT, + l BOOLEAN). + +p(x, if abs(l) > 10 then true + else false + end) :- + labels(x, l). + +p(x, if (x % 10) = 0 then false + end) :- + labels(x, l). + +labels_resolved(x, SUM(if label IS NULL then 0 + else if label = true then 1 + else -1 + end)) :- + labels(x, label). + diff --git a/test/expected-output-test/multiple_rules_for_same_head/compile-incremental.expected b/test/expected-output-test/multiple_rules_for_same_head/compile-incremental.expected index 9330135..854fe18 100644 --- a/test/expected-output-test/multiple_rules_for_same_head/compile-incremental.expected +++ b/test/expected-output-test/multiple_rules_for_same_head/compile-incremental.expected @@ -31,6 +31,22 @@ dd_new_Q.label: Boolean style: "sql_extractor" } + deepdive.extraction.extractors.init_dd_delta_S_0 { + sql: """ DROP TABLE IF EXISTS dd_delta_S_0 CASCADE; + CREATE TABLE + dd_delta_S_0(a int) + """ + style: "sql_extractor" + } + + deepdive.extraction.extractors.init_dd_new_T___0 { + sql: """ DROP TABLE IF EXISTS dd_new_T___0 CASCADE; + CREATE TABLE + dd_new_T___0(a int) + """ + style: "sql_extractor" + } + deepdive.extraction.extractors.init_dd_delta_T { sql: """ DROP TABLE IF EXISTS dd_delta_T CASCADE; CREATE TABLE @@ -39,6 +55,30 @@ dd_new_Q.label: Boolean style: "sql_extractor" } + deepdive.extraction.extractors.init_dd_new_T___1 { + sql: """ DROP TABLE IF EXISTS dd_new_T___1 CASCADE; + CREATE TABLE + dd_new_T___1(a int) + """ + style: "sql_extractor" + } + + deepdive.extraction.extractors.init_dd_delta_T_2 { + sql: """ DROP TABLE IF EXISTS dd_delta_T_2 CASCADE; + CREATE TABLE + dd_delta_T_2(a int) + """ + style: "sql_extractor" + } + + deepdive.extraction.extractors.init_dd_new_S_2 { + sql: """ DROP TABLE IF EXISTS dd_new_S_2 CASCADE; + CREATE TABLE + dd_new_S_2(a int) + """ + style: "sql_extractor" + } + deepdive.extraction.extractors.init_dd_new_T { sql: """ DROP TABLE IF EXISTS dd_new_T CASCADE; CREATE TABLE @@ -55,6 +95,22 @@ dd_new_Q.label: Boolean style: "sql_extractor" } + deepdive.extraction.extractors.init_dd_new_T___3 { + sql: """ DROP TABLE IF EXISTS dd_new_T___3 CASCADE; + CREATE TABLE + dd_new_T___3(a int) + """ + style: "sql_extractor" + } + + deepdive.extraction.extractors.init_dd_delta_T___0 { + sql: """ DROP TABLE IF EXISTS dd_delta_T___0 CASCADE; + CREATE TABLE + dd_delta_T___0(a int) + """ + style: "sql_extractor" + } + deepdive.extraction.extractors.init_dd_new_R { sql: """ DROP TABLE IF EXISTS dd_new_R CASCADE; CREATE TABLE @@ -64,6 +120,22 @@ dd_new_Q.label: Boolean style: "sql_extractor" } + deepdive.extraction.extractors.init_dd_delta_S_2 { + sql: """ DROP TABLE IF EXISTS dd_delta_S_2 CASCADE; + CREATE TABLE + dd_delta_S_2(a int) + """ + style: "sql_extractor" + } + + deepdive.extraction.extractors.init_dd_new_S_1 { + sql: """ DROP TABLE IF EXISTS dd_new_S_1 CASCADE; + CREATE TABLE + dd_new_S_1(a int) + """ + style: "sql_extractor" + } + deepdive.extraction.extractors.init_dd_delta_R { sql: """ DROP TABLE IF EXISTS dd_delta_R CASCADE; CREATE TABLE @@ -73,6 +145,38 @@ dd_new_Q.label: Boolean style: "sql_extractor" } + deepdive.extraction.extractors.init_dd_new_T___2 { + sql: """ DROP TABLE IF EXISTS dd_new_T___2 CASCADE; + CREATE TABLE + dd_new_T___2(a int) + """ + style: "sql_extractor" + } + + deepdive.extraction.extractors.init_dd_delta_T___3 { + sql: """ DROP TABLE IF EXISTS dd_delta_T___3 CASCADE; + CREATE TABLE + dd_delta_T___3(a int) + """ + style: "sql_extractor" + } + + deepdive.extraction.extractors.init_dd_delta_S_1 { + sql: """ DROP TABLE IF EXISTS dd_delta_S_1 CASCADE; + CREATE TABLE + dd_delta_S_1(a int) + """ + style: "sql_extractor" + } + + deepdive.extraction.extractors.init_dd_delta_T___1 { + sql: """ DROP TABLE IF EXISTS dd_delta_T___1 CASCADE; + CREATE TABLE + dd_delta_T___1(a int) + """ + style: "sql_extractor" + } + deepdive.extraction.extractors.init_dd_delta_Q { sql: """ DROP TABLE IF EXISTS dd_delta_Q CASCADE; CREATE TABLE @@ -83,15 +187,55 @@ dd_new_Q.label: Boolean style: "sql_extractor" } + deepdive.extraction.extractors.init_dd_new_T_2 { + sql: """ DROP TABLE IF EXISTS dd_new_T_2 CASCADE; + CREATE TABLE + dd_new_T_2(a int) + """ + style: "sql_extractor" + } + + deepdive.extraction.extractors.init_dd_new_S_0 { + sql: """ DROP TABLE IF EXISTS dd_new_S_0 CASCADE; + CREATE TABLE + dd_new_S_0(a int) + """ + style: "sql_extractor" + } + + deepdive.extraction.extractors.init_dd_delta_T___2 { + sql: """ DROP TABLE IF EXISTS dd_delta_T___2 CASCADE; + CREATE TABLE + dd_delta_T___2(a int) + """ + style: "sql_extractor" + } + deepdive.extraction.extractors.cleanup { sql: """ TRUNCATE dd_new_S; + TRUNCATE dd_delta_S_0; + TRUNCATE dd_new_T___0; TRUNCATE dd_delta_T; + TRUNCATE dd_new_T___1; + TRUNCATE dd_delta_T_2; + TRUNCATE dd_new_S_2; TRUNCATE dd_new_T; TRUNCATE dd_delta_S; + TRUNCATE dd_new_T___3; + TRUNCATE dd_delta_T___0; TRUNCATE dd_new_R; + TRUNCATE dd_delta_S_2; + TRUNCATE dd_new_S_1; TRUNCATE dd_delta_R; + TRUNCATE dd_new_T___2; + TRUNCATE dd_delta_T___3; + TRUNCATE dd_delta_S_1; + TRUNCATE dd_delta_T___1; TRUNCATE dd_delta_Q; + TRUNCATE dd_new_T_2; + TRUNCATE dd_new_S_0; + TRUNCATE dd_delta_T___2; """ style: "sql_extractor" } @@ -113,7 +257,7 @@ FROM dd_delta_S R0 """ output_relation: "dd_new_S" style: "cmd_extractor" - dependencies: [ "ext_dd_delta_S" , "ext_dd_delta_S_by_f_1" , "ext_dd_delta_S_by_f" , "ext1_dd_delta_S_by_f" ] + dependencies: [ "ext_dd_delta_S" ] input_relations: [ S dd_delta_S @@ -121,6 +265,116 @@ FROM dd_delta_S R0 } + deepdive.extraction.extractors.ext_dd_new_T___0 { + cmd: """ + + # TODO use temporary table + deepdive create table "dd_new_T___0" + deepdive sql 'INSERT INTO dd_new_T___0 SELECT R0.a +FROM T___0 R0 + +UNION ALL +SELECT R0.a +FROM dd_delta_T___0 R0 + ' + # TODO rename temporary table to replace output_relation + + """ + output_relation: "dd_new_T___0" + style: "cmd_extractor" + dependencies: [ "ext_dd_delta_T___0_by_f" ] + input_relations: [ + T___0 + dd_delta_T___0 + ] + } + + + deepdive.extraction.extractors.ext_dd_delta_T { + cmd: """ + + # TODO use temporary table + deepdive create table "dd_delta_T" + deepdive sql 'INSERT INTO dd_delta_T SELECT R0.a AS "dd_delta_T___0.R0.a" +FROM dd_delta_T___0 R0 + +UNION ALL +SELECT R0.a AS "dd_delta_T___1.R0.a" +FROM dd_delta_T___1 R0 + +UNION ALL +SELECT R0.a AS "dd_delta_T___2.R0.a" +FROM dd_delta_T___2 R0 + +UNION ALL +SELECT R0.a AS "dd_delta_T___3.R0.a" +FROM dd_delta_T___3 R0 + ' + # TODO rename temporary table to replace output_relation + + """ + output_relation: "dd_delta_T" + style: "cmd_extractor" + dependencies: [ "ext_dd_delta_T___0_by_f" , "ext_dd_delta_T___1_by_f" , "ext_dd_delta_T___2_by_f" , "ext_dd_delta_T___3_by_f_1" ] + input_relations: [ + dd_delta_T___0 + dd_delta_T___1 + dd_delta_T___2 + dd_delta_T___3 + ] + } + + + deepdive.extraction.extractors.ext_dd_new_T___1 { + cmd: """ + + # TODO use temporary table + deepdive create table "dd_new_T___1" + deepdive sql 'INSERT INTO dd_new_T___1 SELECT R0.a +FROM T___1 R0 + +UNION ALL +SELECT R0.a +FROM dd_delta_T___1 R0 + ' + # TODO rename temporary table to replace output_relation + + """ + output_relation: "dd_new_T___1" + style: "cmd_extractor" + dependencies: [ "ext_dd_delta_T___1_by_f" ] + input_relations: [ + T___1 + dd_delta_T___1 + ] + } + + + deepdive.extraction.extractors.ext_dd_new_S_2 { + cmd: """ + + # TODO use temporary table + deepdive create table "dd_new_S_2" + deepdive sql 'INSERT INTO dd_new_S_2 SELECT R0.a +FROM S_2 R0 + +UNION ALL +SELECT R0.a +FROM dd_delta_S_2 R0 + ' + # TODO rename temporary table to replace output_relation + + """ + output_relation: "dd_new_S_2" + style: "cmd_extractor" + dependencies: [ "ext_dd_delta_S_2_by_f_1" ] + input_relations: [ + S_2 + dd_delta_S_2 + ] + } + + deepdive.extraction.extractors.ext_dd_new_T { cmd: """ @@ -138,7 +392,7 @@ FROM dd_delta_T R0 """ output_relation: "dd_new_T" style: "cmd_extractor" - dependencies: [ "ext_dd_delta_T_by_f" , "ext1_dd_delta_T_by_f" , "ext2_dd_delta_T_by_f" , "ext_dd_delta_T_by_f_1" ] + dependencies: [ "ext_dd_delta_T" ] input_relations: [ T dd_delta_T @@ -173,17 +427,57 @@ FROM dd_delta_R R0, R R1 UNION ALL SELECT R0.a AS "dd_new_R.R0.a" FROM dd_new_R R0, dd_delta_R R1 - WHERE R1.a = R0.b AND R0.a = 0' + WHERE R1.a = R0.b AND R0.a = 0 +UNION ALL +SELECT R0.a AS "dd_delta_S_0.R0.a" +FROM dd_delta_S_0 R0 + +UNION ALL +SELECT R0.a AS "dd_delta_S_1.R0.a" +FROM dd_delta_S_1 R0 + +UNION ALL +SELECT R0.a AS "dd_delta_S_2.R0.a" +FROM dd_delta_S_2 R0 + ' # TODO rename temporary table to replace output_relation """ output_relation: "dd_delta_S" style: "cmd_extractor" - dependencies: [ "ext_dd_new_R" ] + dependencies: [ "ext_dd_new_R" , "ext_dd_delta_S_0_by_f" , "ext_dd_delta_S_1_by_f" , "ext_dd_delta_S_2_by_f_1" ] input_relations: [ dd_delta_R R dd_new_R + dd_delta_S_0 + dd_delta_S_1 + dd_delta_S_2 + ] + } + + + deepdive.extraction.extractors.ext_dd_new_T___3 { + cmd: """ + + # TODO use temporary table + deepdive create table "dd_new_T___3" + deepdive sql 'INSERT INTO dd_new_T___3 SELECT R0.a +FROM T___3 R0 + +UNION ALL +SELECT R0.a +FROM dd_delta_T___3 R0 + ' + # TODO rename temporary table to replace output_relation + + """ + output_relation: "dd_new_T___3" + style: "cmd_extractor" + dependencies: [ "ext_dd_delta_T___3_by_f_1" ] + input_relations: [ + T___3 + dd_delta_T___3 ] } @@ -213,6 +507,73 @@ FROM dd_delta_R R0 } + deepdive.extraction.extractors.ext_dd_new_S_1 { + cmd: """ + + # TODO use temporary table + deepdive create table "dd_new_S_1" + deepdive sql 'INSERT INTO dd_new_S_1 SELECT R0.a +FROM S_1 R0 + +UNION ALL +SELECT R0.a +FROM dd_delta_S_1 R0 + ' + # TODO rename temporary table to replace output_relation + + """ + output_relation: "dd_new_S_1" + style: "cmd_extractor" + dependencies: [ "ext_dd_delta_S_1_by_f" ] + input_relations: [ + S_1 + dd_delta_S_1 + ] + } + + + deepdive.extraction.extractors.ext_dd_new_T___2 { + cmd: """ + + # TODO use temporary table + deepdive create table "dd_new_T___2" + deepdive sql 'INSERT INTO dd_new_T___2 SELECT R0.a +FROM T___2 R0 + +UNION ALL +SELECT R0.a +FROM dd_delta_T___2 R0 + ' + # TODO rename temporary table to replace output_relation + + """ + output_relation: "dd_new_T___2" + style: "cmd_extractor" + dependencies: [ "ext_dd_delta_T___2_by_f" ] + input_relations: [ + T___2 + dd_delta_T___2 + ] + } + + + deepdive.extraction.extractors.ext_dd_delta_T__3 { + cmd: """ + + deepdive create view dd_delta_T__3 as 'SELECT R0.a AS column_0 +FROM dd_delta_T R0 + ' + + """ + output_relation: "dd_delta_T__3" + style: "cmd_extractor" + dependencies: [ "ext_dd_delta_T" ] + input_relations: [ + dd_delta_T + ] + } + + deepdive.extraction.extractors.ext_dd_new_Q { cmd: """ @@ -237,12 +598,62 @@ SELECT DISTINCT R0.x, id, label } - deepdive.extraction.extractors.ext_dd_delta_T_by_f { + deepdive.extraction.extractors.ext_dd_new_T_2 { + cmd: """ + + # TODO use temporary table + deepdive create table "dd_new_T_2" + deepdive sql 'INSERT INTO dd_new_T_2 SELECT R0.a +FROM T_2 R0 + +UNION ALL +SELECT R0.a +FROM dd_delta_T_2 R0 + ' + # TODO rename temporary table to replace output_relation + + """ + output_relation: "dd_new_T_2" + style: "cmd_extractor" + + input_relations: [ + T_2 + dd_delta_T_2 + ] + } + + + deepdive.extraction.extractors.ext_dd_new_S_0 { + cmd: """ + + # TODO use temporary table + deepdive create table "dd_new_S_0" + deepdive sql 'INSERT INTO dd_new_S_0 SELECT R0.a +FROM S_0 R0 + +UNION ALL +SELECT R0.a +FROM dd_delta_S_0 R0 + ' + # TODO rename temporary table to replace output_relation + + """ + output_relation: "dd_new_S_0" + style: "cmd_extractor" + dependencies: [ "ext_dd_delta_S_0_by_f" ] + input_relations: [ + S_0 + dd_delta_S_0 + ] + } + + + deepdive.extraction.extractors.ext_dd_delta_T___0_by_f { input: """ SELECT R0.a AS "dd_delta_R.R0.a", R0.b AS "dd_delta_R.R0.b" FROM dd_delta_R R0 WHERE R0.a > 1000 """ - output_relation: "dd_delta_T" + output_relation: "dd_delta_T___0" udf: ${APP_HOME}"//bin/false" style: "tsv_extractor" @@ -254,12 +665,12 @@ FROM dd_delta_R R0 } - deepdive.extraction.extractors.ext1_dd_delta_T_by_f { + deepdive.extraction.extractors.ext_dd_delta_T___1_by_f { input: """ SELECT R0.a AS "dd_delta_R.R0.a", R0.b AS "dd_delta_R.R0.b" FROM dd_delta_R R0 - WHERE (10 < R0.a AND R0.a < 20) + WHERE 10 < R0.a AND R0.a < 20 """ - output_relation: "dd_delta_T" + output_relation: "dd_delta_T___1" udf: ${APP_HOME}"//bin/false" style: "tsv_extractor" @@ -271,12 +682,12 @@ FROM dd_delta_R R0 } - deepdive.extraction.extractors.ext2_dd_delta_T_by_f { + deepdive.extraction.extractors.ext_dd_delta_T___2_by_f { input: """ SELECT R0.a AS "dd_delta_R.R0.a", R0.b AS "dd_delta_R.R0.b" FROM dd_delta_R R0 - WHERE (100 < R0.a AND R0.a < 200) + WHERE 100 < R0.a AND R0.a < 200 """ - output_relation: "dd_delta_T" + output_relation: "dd_delta_T___2" udf: ${APP_HOME}"//bin/false" style: "tsv_extractor" @@ -288,12 +699,12 @@ FROM dd_delta_R R0 } - deepdive.extraction.extractors.ext_dd_delta_S_by_f { + deepdive.extraction.extractors.ext_dd_delta_S_0_by_f { input: """ SELECT R0.a AS "dd_delta_R.R0.a", R0.b AS "dd_delta_R.R0.b" FROM dd_delta_R R0 WHERE R0.a < 0 """ - output_relation: "dd_delta_S" + output_relation: "dd_delta_S_0" udf: ${APP_HOME}"//bin/false" style: "tsv_extractor" @@ -305,12 +716,12 @@ FROM dd_delta_R R0 } - deepdive.extraction.extractors.ext1_dd_delta_S_by_f { + deepdive.extraction.extractors.ext_dd_delta_S_1_by_f { input: """ SELECT R0.a AS "dd_delta_R.R0.a", R0.b AS "dd_delta_R.R0.b" FROM dd_delta_R R0 WHERE R0.a > 0 """ - output_relation: "dd_delta_S" + output_relation: "dd_delta_S_1" udf: ${APP_HOME}"//bin/false" style: "tsv_extractor" @@ -322,12 +733,12 @@ FROM dd_delta_R R0 } - deepdive.extraction.extractors.ext_dd_delta_T_by_f_1 { + deepdive.extraction.extractors.ext_dd_delta_T___3_by_f_1 { input: """ SELECT R0.a AS "dd_delta_R.R0.a", R0.b AS "dd_delta_R.R0.b" FROM dd_delta_R R0 """ - output_relation: "dd_delta_T" + output_relation: "dd_delta_T___3" udf: ${APP_HOME}"//bin/false" style: "tsv_extractor" @@ -339,12 +750,12 @@ FROM dd_delta_R R0 } - deepdive.extraction.extractors.ext_dd_delta_S_by_f_1 { + deepdive.extraction.extractors.ext_dd_delta_S_2_by_f_1 { input: """ SELECT R0.a AS "dd_delta_R.R0.a", R0.b AS "dd_delta_R.R0.b" FROM dd_delta_R R0 WHERE R0.a > 0 """ - output_relation: "dd_delta_S" + output_relation: "dd_delta_S_2" udf: ${APP_HOME}"//bin/false" style: "tsv_extractor" @@ -363,7 +774,7 @@ FROM dd_delta_R R0 WHERE R1.a = R0.x AND R0.x > 1000""" function: "Imply(dd_new_Q.R0.label)" weight: "?(dd_weight_column_0)" - dependencies: [ "ext_dd_delta_S" , "ext_dd_delta_S_by_f_1" , "ext_dd_delta_S_by_f" , "ext1_dd_delta_S_by_f" ] + dependencies: [ "ext_dd_delta_S" ] input_relations: [ dd_new_Q dd_delta_S @@ -378,7 +789,7 @@ FROM dd_delta_R R0 WHERE R1.a = R0.x AND R0.x = 0""" function: "Imply(dd_new_Q.R0.label)" weight: "1.0" - dependencies: [ "ext_dd_delta_S" , "ext_dd_delta_S_by_f_1" , "ext_dd_delta_S_by_f" , "ext1_dd_delta_S_by_f" ] + dependencies: [ "ext_dd_delta_S" ] input_relations: [ dd_new_Q dd_delta_S @@ -399,7 +810,7 @@ FROM dd_delta_R R0 WHERE R2.a = R0.x AND R3.a = R0.x AND R3.b = R1.x AND R4.a = R1.x """ function: "Imply(dd_new_Q.R0.label, dd_new_Q.R1.label)" weight: "?(dd_weight_column_0)" - dependencies: [ "ext_dd_new_R" , "ext_dd_delta_S_by_f" , "ext1_dd_delta_S_by_f" , "ext_dd_new_S" , "ext_dd_delta_S_by_f_1" , "ext_dd_delta_S" ] + dependencies: [ "ext_dd_new_R" , "ext_dd_new_S" , "ext_dd_delta_S" ] input_relations: [ dd_new_Q dd_delta_S @@ -425,7 +836,7 @@ FROM dd_delta_R R0 WHERE R2.a = R0.x AND R3.a = R0.x AND R3.b = R1.x AND R4.a = R1.x AND (R0.x + R1.x) < 1000""" function: "Imply(dd_new_Q.R0.label, dd_new_Q.R1.label)" weight: "-10.0" - dependencies: [ "ext_dd_new_R" , "ext_dd_delta_S_by_f" , "ext1_dd_delta_S_by_f" , "ext_dd_new_S" , "ext_dd_delta_S_by_f_1" , "ext_dd_delta_S" ] + dependencies: [ "ext_dd_new_R" , "ext_dd_new_S" , "ext_dd_delta_S" ] input_relations: [ dd_new_Q dd_delta_S @@ -439,18 +850,28 @@ FROM dd_delta_R R0 deepdive.pipeline.run: ${PIPELINE} deepdive.pipeline.pipelines.extraction: [ + ext_dd_delta_T___1_by_f + ext_dd_new_T___3 ext_dd_new_R - ext_dd_delta_T_by_f - ext1_dd_delta_T_by_f - ext2_dd_delta_T_by_f + ext_dd_new_T_2 + ext_dd_delta_S_0_by_f + ext_dd_new_T___0 + ext_dd_delta_T__3 + ext_dd_new_S_2 + ext_dd_delta_S_2_by_f_1 + ext_dd_delta_S_1_by_f + ext_dd_delta_T ext_dd_delta_S ext_dd_new_T + ext_dd_new_S_0 + ext_dd_new_T___2 + ext_dd_delta_T___3_by_f_1 ext_dd_new_S - ext_dd_delta_S_by_f ext_dd_new_Q - ext_dd_delta_S_by_f_1 - ext_dd_delta_T_by_f_1 - ext1_dd_delta_S_by_f + ext_dd_delta_T___0_by_f + ext_dd_new_S_1 + ext_dd_new_T___1 + ext_dd_delta_T___2_by_f ] deepdive.pipeline.pipelines.inference: [ dd_delta_inf_istrue_Q @@ -459,18 +880,28 @@ deepdive.pipeline.pipelines.inference: [ dd_delta_inf1_imply_Q_Q ] deepdive.pipeline.pipelines.endtoend: [ + ext_dd_delta_T___1_by_f + ext_dd_new_T___3 ext_dd_new_R - ext_dd_delta_T_by_f - ext1_dd_delta_T_by_f - ext2_dd_delta_T_by_f + ext_dd_new_T_2 + ext_dd_delta_S_0_by_f + ext_dd_new_T___0 + ext_dd_delta_T__3 + ext_dd_new_S_2 + ext_dd_delta_S_2_by_f_1 + ext_dd_delta_S_1_by_f + ext_dd_delta_T ext_dd_delta_S ext_dd_new_T + ext_dd_new_S_0 + ext_dd_new_T___2 + ext_dd_delta_T___3_by_f_1 ext_dd_new_S - ext_dd_delta_S_by_f ext_dd_new_Q - ext_dd_delta_S_by_f_1 - ext_dd_delta_T_by_f_1 - ext1_dd_delta_S_by_f + ext_dd_delta_T___0_by_f + ext_dd_new_S_1 + ext_dd_new_T___1 + ext_dd_delta_T___2_by_f dd_delta_inf_istrue_Q dd_delta_inf1_istrue_Q dd_delta_inf_imply_Q_Q @@ -479,13 +910,29 @@ deepdive.pipeline.pipelines.endtoend: [ deepdive.pipeline.base_dir: ${BASEDIR} deepdive.pipeline.pipelines.initdb: [ init_dd_new_S + init_dd_delta_S_0 + init_dd_new_T___0 init_dd_delta_T + init_dd_new_T___1 + init_dd_delta_T_2 + init_dd_new_S_2 init_dd_new_T init_dd_delta_S + init_dd_new_T___3 + init_dd_delta_T___0 init_dd_new_R + init_dd_delta_S_2 + init_dd_new_S_1 init_dd_delta_R + init_dd_new_T___2 + init_dd_delta_T___3 + init_dd_delta_S_1 init_dd_new_Q + init_dd_delta_T___1 init_dd_delta_Q + init_dd_new_T_2 + init_dd_new_S_0 + init_dd_delta_T___2 ] deepdive.pipeline.pipelines.cleanup: [ cleanup diff --git a/test/expected-output-test/multiple_rules_for_same_head/compile.expected b/test/expected-output-test/multiple_rules_for_same_head/compile.expected index b353643..ad81226 100644 --- a/test/expected-output-test/multiple_rules_for_same_head/compile.expected +++ b/test/expected-output-test/multiple_rules_for_same_head/compile.expected @@ -32,25 +32,92 @@ FROM R R0, R R1 UNION ALL SELECT R0.a AS "R.R0.a" FROM R R0, R R1 - WHERE R1.a = R0.b AND R0.a = 0' + WHERE R1.a = R0.b AND R0.a = 0 +UNION ALL +SELECT R0.a AS "S_0.R0.a" +FROM S_0 R0 + +UNION ALL +SELECT R0.a AS "S_1.R0.a" +FROM S_1 R0 + +UNION ALL +SELECT R0.a AS "S_2.R0.a" +FROM S_2 R0 + ' # TODO rename temporary table to replace output_relation """ output_relation: "S" style: "cmd_extractor" - + dependencies: [ "ext_S_0_by_f" , "ext_S_1_by_f" , "ext_S_2_by_f_1" ] input_relations: [ R + S_0 + S_1 + S_2 + ] + } + + + deepdive.extraction.extractors.ext_T__3 { + cmd: """ + + deepdive create view T__3 as 'SELECT R0.a AS column_0 +FROM T R0 + ' + + """ + output_relation: "T__3" + style: "cmd_extractor" + dependencies: [ "ext_T" ] + input_relations: [ + T + ] + } + + + deepdive.extraction.extractors.ext_T { + cmd: """ + + # TODO use temporary table + deepdive create table "T" + deepdive sql 'INSERT INTO T SELECT R0.a AS "T___0.R0.a" +FROM T___0 R0 + +UNION ALL +SELECT R0.a AS "T___1.R0.a" +FROM T___1 R0 + +UNION ALL +SELECT R0.a AS "T___2.R0.a" +FROM T___2 R0 + +UNION ALL +SELECT R0.a AS "T___3.R0.a" +FROM T___3 R0 + ' + # TODO rename temporary table to replace output_relation + + """ + output_relation: "T" + style: "cmd_extractor" + dependencies: [ "ext_T___0_by_f" , "ext_T___1_by_f" , "ext_T___2_by_f" , "ext_T___3_by_f_1" ] + input_relations: [ + T___0 + T___1 + T___2 + T___3 ] } - deepdive.extraction.extractors.ext_T_by_f { + deepdive.extraction.extractors.ext_T___0_by_f { input: """ SELECT R0.a AS "R.R0.a", R0.b AS "R.R0.b" FROM R R0 WHERE R0.a > 1000 """ - output_relation: "T" + output_relation: "T___0" udf: ${APP_HOME}"//bin/false" style: "tsv_extractor" @@ -62,12 +129,12 @@ FROM R R0 } - deepdive.extraction.extractors.ext1_T_by_f { + deepdive.extraction.extractors.ext_T___1_by_f { input: """ SELECT R0.a AS "R.R0.a", R0.b AS "R.R0.b" FROM R R0 - WHERE (10 < R0.a AND R0.a < 20) + WHERE 10 < R0.a AND R0.a < 20 """ - output_relation: "T" + output_relation: "T___1" udf: ${APP_HOME}"//bin/false" style: "tsv_extractor" @@ -79,12 +146,12 @@ FROM R R0 } - deepdive.extraction.extractors.ext2_T_by_f { + deepdive.extraction.extractors.ext_T___2_by_f { input: """ SELECT R0.a AS "R.R0.a", R0.b AS "R.R0.b" FROM R R0 - WHERE (100 < R0.a AND R0.a < 200) + WHERE 100 < R0.a AND R0.a < 200 """ - output_relation: "T" + output_relation: "T___2" udf: ${APP_HOME}"//bin/false" style: "tsv_extractor" @@ -96,12 +163,12 @@ FROM R R0 } - deepdive.extraction.extractors.ext_S_by_f { + deepdive.extraction.extractors.ext_S_0_by_f { input: """ SELECT R0.a AS "R.R0.a", R0.b AS "R.R0.b" FROM R R0 WHERE R0.a < 0 """ - output_relation: "S" + output_relation: "S_0" udf: ${APP_HOME}"//bin/false" style: "tsv_extractor" @@ -113,12 +180,12 @@ FROM R R0 } - deepdive.extraction.extractors.ext1_S_by_f { + deepdive.extraction.extractors.ext_S_1_by_f { input: """ SELECT R0.a AS "R.R0.a", R0.b AS "R.R0.b" FROM R R0 WHERE R0.a > 0 """ - output_relation: "S" + output_relation: "S_1" udf: ${APP_HOME}"//bin/false" style: "tsv_extractor" @@ -130,12 +197,12 @@ FROM R R0 } - deepdive.extraction.extractors.ext_T_by_f_1 { + deepdive.extraction.extractors.ext_T___3_by_f_1 { input: """ SELECT R0.a AS "R.R0.a", R0.b AS "R.R0.b" FROM R R0 """ - output_relation: "T" + output_relation: "T___3" udf: ${APP_HOME}"//bin/false" style: "tsv_extractor" @@ -147,12 +214,12 @@ FROM R R0 } - deepdive.extraction.extractors.ext_S_by_f_1 { + deepdive.extraction.extractors.ext_S_2_by_f_1 { input: """ SELECT R0.a AS "R.R0.a", R0.b AS "R.R0.b" FROM R R0 WHERE R0.a > 0 """ - output_relation: "S" + output_relation: "S_2" udf: ${APP_HOME}"//bin/false" style: "tsv_extractor" @@ -171,7 +238,7 @@ FROM R R0 WHERE R1.a = R0.x AND R0.x > 1000""" function: "Imply(Q.R0.label)" weight: "?(dd_weight_column_0)" - dependencies: [ "ext_S_by_f_1" , "ext_S" , "ext1_S_by_f" , "ext_S_by_f" ] + dependencies: [ "ext_S" ] input_relations: [ Q S @@ -186,7 +253,7 @@ FROM R R0 WHERE R1.a = R0.x AND R0.x = 0""" function: "Imply(Q.R0.label)" weight: "1.0" - dependencies: [ "ext_S_by_f_1" , "ext_S" , "ext1_S_by_f" , "ext_S_by_f" ] + dependencies: [ "ext_S" ] input_relations: [ Q S @@ -201,7 +268,7 @@ FROM R R0 WHERE R2.a = R0.x AND R3.a = R0.x AND R3.b = R1.x AND R4.a = R1.x """ function: "Imply(Q.R0.label, Q.R1.label)" weight: "?(dd_weight_column_0)" - dependencies: [ "ext_S_by_f_1" , "ext_S" , "ext1_S_by_f" , "ext_S_by_f" ] + dependencies: [ "ext_S" ] input_relations: [ Q S @@ -217,7 +284,7 @@ FROM R R0 WHERE R2.a = R0.x AND R3.a = R0.x AND R3.b = R1.x AND R4.a = R1.x AND (R0.x + R1.x) < 1000""" function: "Imply(Q.R0.label, Q.R1.label)" weight: "-10.0" - dependencies: [ "ext_S_by_f_1" , "ext_S" , "ext1_S_by_f" , "ext_S_by_f" ] + dependencies: [ "ext_S" ] input_relations: [ Q S @@ -227,14 +294,16 @@ FROM R R0 deepdive.pipeline.run: ${PIPELINE} deepdive.pipeline.pipelines.extraction: [ - ext2_T_by_f - ext_S_by_f_1 - ext_S_by_f - ext1_S_by_f - ext_T_by_f_1 - ext1_T_by_f + ext_S_1_by_f + ext_T + ext_S_2_by_f_1 + ext_S_0_by_f + ext_T___3_by_f_1 + ext_T__3 + ext_T___2_by_f + ext_T___0_by_f ext_S - ext_T_by_f + ext_T___1_by_f ] deepdive.pipeline.pipelines.inference: [ inf_istrue_Q @@ -243,14 +312,16 @@ deepdive.pipeline.pipelines.inference: [ inf1_imply_Q_Q ] deepdive.pipeline.pipelines.endtoend: [ - ext2_T_by_f - ext_S_by_f_1 - ext_S_by_f - ext1_S_by_f - ext_T_by_f_1 - ext1_T_by_f + ext_S_1_by_f + ext_T + ext_S_2_by_f_1 + ext_S_0_by_f + ext_T___3_by_f_1 + ext_T__3 + ext_T___2_by_f + ext_T___0_by_f ext_S - ext_T_by_f + ext_T___1_by_f inf_istrue_Q inf1_istrue_Q inf_imply_Q_Q diff --git a/test/expected-output-test/multiple_rules_for_same_head/desugar.expected b/test/expected-output-test/multiple_rules_for_same_head/desugar.expected new file mode 100644 index 0000000..a1767b4 --- /dev/null +++ b/test/expected-output-test/multiple_rules_for_same_head/desugar.expected @@ -0,0 +1,126 @@ +R(a int, + b int). + +S(a int). + +T(a int). + +Q?(x int). + +S(a) :- + R(a, b), + R(b, _), + a > 100. + +S(a) :- + R(a, b), + R(b, _), + a < -100. + +S(a) :- + R(a, b), + R(b, _), + a = 0. + +function f + over (a int, b int) + returns (a int) + implementation "/bin/false" + handles tsv lines. + +T___0(a int). + +T___0 += f(a, b) :- + R(a, b), + a > 1000. + +T(a) :- + T___0(a). + +T___1(a int). + +T___1 += f(a, b) :- + R(a, b), + [10 < a, a < 20]. + +T(a) :- + T___1(a). + +T___2(a int). + +T___2 += f(a, b) :- + R(a, b), + [100 < a, a < 200]. + +T(a) :- + T___2(a). + +S_0(a int). + +S_0 += f(a, b) :- + R(a, b), + a < 0. + +S(a) :- + S_0(a). + +S_1(a int). + +S_1 += f(a, b) :- + R(a, b), + a > 0. + +S(a) :- + S_1(a). + +function f_1 + over (a int, b int) + returns (a int) + implementation "/bin/false" + handles tsv lines. + +T___3(a int). + +T___3 += f_1(a, b) :- + R(a, b). + +T(a) :- + T___3(a). + +S_2(a int). + +S_2 += f_1(a, b) :- + R(a, b), + a > 0. + +S(a) :- + S_2(a). + +@weight(a) +Q(a) :- + S(a), + a > 1000. + +@weight(1.0) +Q(a) :- + S(a), + a = 0. + +@weight(a) +Q(a) => Q(b) :- + S(a), + R(a, b), + S(b). + +@weight(-10.0) +Q(a) => Q(b) :- + S(a), + R(a, b), + S(b), + (a + b) < 1000. + +T_2(a int). + +T__3(x) :- + T(x). + diff --git a/test/expected-output-test/multiple_rules_for_same_head/input.ddlog b/test/expected-output-test/multiple_rules_for_same_head/input.ddlog index 690e532..c0dfdfc 100644 --- a/test/expected-output-test/multiple_rules_for_same_head/input.ddlog +++ b/test/expected-output-test/multiple_rules_for_same_head/input.ddlog @@ -34,3 +34,7 @@ Q(a) :- S(a), a = 0. Q(a) => Q(b) :- S(a), R(a, b), S(b). @weight(-10.0) Q(a) => Q(b) :- S(a), R(a, b), S(b), a + b < 1000. + +# make sure desugars without name clashes +T_2(a int). +T__3(x) :- T(x). diff --git a/test/expected-output-test/views/compile.expected b/test/expected-output-test/views/compile.expected index 985dbaa..f738ff7 100644 --- a/test/expected-output-test/views/compile.expected +++ b/test/expected-output-test/views/compile.expected @@ -20,7 +20,7 @@ deepdive.extraction.extractors.ext_A { cmd: """ - deepdive create view A as 'SELECT R0.a AS column_0, (R0.a + R0.b) AS column_1 + deepdive create view A as 'SELECT R0.a AS column_0, R0.a + R0.b AS column_1 FROM R R0 ' diff --git a/test/expected-output-test/views/print.expected b/test/expected-output-test/views/print.expected index 734e24d..7e6ceff 100644 --- a/test/expected-output-test/views/print.expected +++ b/test/expected-output-test/views/print.expected @@ -1,7 +1,7 @@ R(a int, b int). -A(x, (x + y)) :- +A(x, x + y) :- R(x, y). B(x, z) :- diff --git a/test/expected-output-test/weights/compile.expected b/test/expected-output-test/weights/compile.expected index b6595ae..cd650b0 100644 --- a/test/expected-output-test/weights/compile.expected +++ b/test/expected-output-test/weights/compile.expected @@ -23,7 +23,7 @@ c.label: Boolean deepdive.inference.factors.inf_istrue_a { input_query: """ - SELECT R0.id AS "a.R0.id" , (R0.k + R1.p) AS "dd_weight_column_0" + SELECT R0.id AS "a.R0.id" , R0.k + R1.p AS "dd_weight_column_0" FROM a R0, b R1 WHERE R1.k = R0.k """ function: "Imply(a.R0.label)" diff --git a/test/expected-output-test/weights/print.expected b/test/expected-output-test/weights/print.expected index cb35dea..b9f7bbe 100644 --- a/test/expected-output-test/weights/print.expected +++ b/test/expected-output-test/weights/print.expected @@ -12,7 +12,7 @@ b(k int, p int, q text). -@weight((x + y)) +@weight(x + y) a(x) :- b(x, y, _).