Skip to content

Commit

Permalink
Basis for upserts seems to be working...
Browse files Browse the repository at this point in the history
  • Loading branch information
Stephen Carman committed Oct 18, 2016
1 parent c6a5956 commit 6f379c9
Show file tree
Hide file tree
Showing 22 changed files with 154 additions and 13 deletions.
1 change: 1 addition & 0 deletions build.sbt
Original file line number Diff line number Diff line change
Expand Up @@ -237,6 +237,7 @@ lazy val commonSettings = ReleasePlugin.extraReleaseCommands ++ Seq(
"-Xfatal-warnings",
"-deprecation",
"-encoding", "UTF-8",
"-Xmax-classfile-name", "140",
"-feature",
"-unchecked",
"-Xlint",
Expand Down
3 changes: 3 additions & 0 deletions quill-core/src/main/scala/io/getquill/MirrorContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,9 @@ class MirrorContext[Idiom <: BaseIdiom, Naming <: NamingStrategy]
def executeActionReturning[O](string: String, prepare: Row => Row = identity, extractor: Row => O, returningColumn: String) =
ActionReturningMirror[O](string, prepare(Row()), extractor, returningColumn)

def executeActionConflict[O](string: String, prepare: Row => Row = identity, extractor: Row => O, returningColumn: String) =
ActionReturningMirror[O](string, prepare(Row()), extractor, returningColumn)

def executeBatchAction(groups: List[BatchGroup]) =
BatchActionMirror {
groups.map {
Expand Down
2 changes: 2 additions & 0 deletions quill-core/src/main/scala/io/getquill/MirrorIdiom.scala
Original file line number Diff line number Diff line change
Expand Up @@ -167,6 +167,8 @@ class MirrorIdiom extends Idiom {
implicit def actionTokenizer(implicit liftTokenizer: Tokenizer[Lift]): Tokenizer[Action] = Tokenizer[Action] {
case Update(query, assignments) => stmt"${query.token}.update(${assignments.token})"
case Insert(query, assignments) => stmt"${query.token}.insert(${assignments.token})"
case Upsert(query, assignments) => stmt"${query.token}.upsert(${assignments.token})"
case Conflict(query, alias, body) => stmt"${query.token}.conflict((${alias.token}) => ${body.token})"
case Delete(query) => stmt"${query.token}.delete"
case Returning(query, alias, body) => stmt"${query.token}.returning((${alias.token}) => ${body.token})"
case Foreach(query, alias, body) => stmt"${query.token}.forach((${alias.token}) => ${body.token})"
Expand Down
3 changes: 2 additions & 1 deletion quill-core/src/main/scala/io/getquill/ast/Ast.scala
Original file line number Diff line number Diff line change
Expand Up @@ -103,8 +103,9 @@ sealed trait Action extends Ast

case class Update(query: Ast, assignments: List[Assignment]) extends Action
case class Insert(query: Ast, assignments: List[Assignment]) extends Action
case class Upsert(query: Ast, assignments: List[Assignment]) extends Action
case class Delete(query: Ast) extends Action

case class Conflict(query: Ast, alias: Ident, property: Ast) extends Action
case class Returning(action: Ast, alias: Ident, property: Ast) extends Action

case class Foreach(query: Ast, alias: Ident, body: Ast) extends Action
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -152,6 +152,14 @@ trait StatefulTransformer[T] {
val (at, att) = apply(a)
val (bt, btt) = att.apply(b)(_.apply)
(Update(at, bt), btt)
case Upsert(a, b) =>
val (at, att) = apply(a)
val (bt, btt) = att.apply(b)(_.apply)
(Upsert(at, bt), btt)
case Conflict(a, b, c) =>
val (at, att) = apply(a)
val (ct, ctt) = att.apply(c)
(Conflict(at, b, ct), ctt)
case Delete(a) =>
val (at, att) = apply(a)
(Delete(at), att)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,8 @@ trait StatelessTransformer {
e match {
case Update(query, assignments) => Update(apply(query), assignments.map(apply))
case Insert(query, assignments) => Insert(apply(query), assignments.map(apply))
case Upsert(query, assignments) => Upsert(apply(query), assignments.map(apply))
case Conflict(query, alias, property) => Conflict(apply(query), alias, apply(property))
case Delete(query) => Delete(apply(query))
case Returning(query, alias, property) => Returning(apply(query), alias, apply(property))
case Foreach(query, alias, body) => Foreach(apply(query), alias, apply(body))
Expand Down
25 changes: 25 additions & 0 deletions quill-core/src/main/scala/io/getquill/context/ActionMacro.scala
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,18 @@ class ActionMacro(val c: MacroContext)
"""
}

def runActionConflict[T](quoted: Tree)(implicit t: WeakTypeTag[T]): Tree =
c.untypecheck{
q"""
val expanded = ${expand(extractAst(quoted))}
${c.prefix}.executeActionConflict(
expanded.string,
expanded.prepare,
${conflictExtractor[T]},
$conflictColumn
)
"""
}
def runActionReturning[T](quoted: Tree)(implicit t: WeakTypeTag[T]): Tree =
c.untypecheck {
q"""
Expand Down Expand Up @@ -101,6 +113,19 @@ class ActionMacro(val c: MacroContext)
}
"""

private def conflictColumn =
q"""
expanded.ast match {
case io.getquill.ast.Conflict(_, _, io.getquill.ast.Property(_, property)) =>
property
case ast =>
io.getquill.util.Messages.fail(s"Can't find conflict column. Ast: '$$ast'")
}
"""

private def returningExtractor[T](implicit t: WeakTypeTag[T]) =
q"(row: ${c.prefix}.ResultRow) => implicitly[Decoder[$t]].apply(0, row)"

private def conflictExtractor[T](implicit t: WeakTypeTag[T]) =
q"(row: ${c.prefix}.ResultRow) => implicitly[Decoder[$t]].apply(0, row)"
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ trait Context[Idiom <: io.getquill.idiom.Idiom, Naming <: NamingStrategy]
def run[T](quoted: Quoted[Query[T]]): RunQueryResult[T] = macro QueryMacro.runQuery[T]
def run(quoted: Quoted[Action[_]]): RunActionResult = macro ActionMacro.runAction
def run[T](quoted: Quoted[ActionReturning[_, T]]): RunActionReturningResult[T] = macro ActionMacro.runActionReturning[T]
def run[T](quoted: Quoted[ConflictAction[_, T]]): RunActionReturningResult[T] = macro ActionMacro.runActionConflict[T]
def run(quoted: Quoted[BatchAction[Action[_]]]): RunBatchActionResult = macro ActionMacro.runBatchAction
def run[T](quoted: Quoted[BatchAction[ActionReturning[_, T]]]): RunBatchActionReturningResult[T] = macro ActionMacro.runBatchActionReturning[T]

Expand Down
6 changes: 6 additions & 0 deletions quill-core/src/main/scala/io/getquill/dsl/MetaDsl.scala
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@ trait MetaDslLowPriorityImplicits {
implicit def materializeUpdateMeta[T]: UpdateMeta[T] = macro MetaDslMacro.materializeUpdateMeta[T]
implicit def materializeInsertMeta[T]: InsertMeta[T] = macro MetaDslMacro.materializeInsertMeta[T]
implicit def materializeSchemaMeta[T]: SchemaMeta[T] = macro MetaDslMacro.materializeSchemaMeta[T]
implicit def materializeUpsertMeta[T]: UpsertMeta[T] = macro MetaDslMacro.materializeUpsertMeta[T]
}

trait MetaDsl extends MetaDslLowPriorityImplicits {
Expand All @@ -20,6 +21,7 @@ trait MetaDsl extends MetaDslLowPriorityImplicits {
def queryMeta[T, R](expand: Quoted[Query[T] => Query[R]])(extract: R => T): QueryMeta[T] = macro MetaDslMacro.queryMeta[T, R]
def updateMeta[T](exclude: (T => Any)*): UpdateMeta[T] = macro MetaDslMacro.updateMeta[T]
def insertMeta[T](exclude: (T => Any)*): InsertMeta[T] = macro MetaDslMacro.insertMeta[T]
def upsertMeta[T](exclude: (T => Any)*): UpsertMeta[T] = macro MetaDslMacro.upsertMeta[T]

trait SchemaMeta[T] {
val entity: Quoted[EntityQuery[T]]
Expand All @@ -37,4 +39,8 @@ trait MetaDsl extends MetaDslLowPriorityImplicits {
trait InsertMeta[T] {
val expand: Quoted[(EntityQuery[T], T) => Insert[T]]
}

trait UpsertMeta[T] {
val expand: Quoted[(EntityQuery[T], T) => Upsert[T]]
}
}
6 changes: 6 additions & 0 deletions quill-core/src/main/scala/io/getquill/dsl/MetaDslMacro.scala
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,9 @@ class MetaDslMacro(val c: MacroContext) {
def updateMeta[T](exclude: Tree*)(implicit t: WeakTypeTag[T]): Tree =
actionMeta[T](value("Encoder", t.tpe, exclude: _*), "update")

def upsertMeta[T](exclude: Tree*)(implicit t: WeakTypeTag[T]): Tree =
actionMeta[T](value("Encoder", t.tpe, exclude: _*), "upsert")

def materializeQueryMeta[T](implicit t: WeakTypeTag[T]): Tree = {
val value = this.value("Decoder", t.tpe)
q"""
Expand All @@ -52,6 +55,9 @@ class MetaDslMacro(val c: MacroContext) {
def materializeInsertMeta[T](implicit t: WeakTypeTag[T]): Tree =
actionMeta[T](value("Encoder", t.tpe), "insert")

def materializeUpsertMeta[T](implicit t: WeakTypeTag[T]): Tree =
actionMeta[T](value("Encoder", t.tpe), "upsert")

def materializeSchemaMeta[T](implicit t: WeakTypeTag[T]): Tree =
(t.tpe.typeSymbol.isClass && t.tpe.typeSymbol.asClass.isCaseClass) match {
case true =>
Expand Down
19 changes: 19 additions & 0 deletions quill-core/src/main/scala/io/getquill/dsl/QueryDsl.scala
Original file line number Diff line number Diff line change
Expand Up @@ -73,6 +73,9 @@ private[dsl] trait QueryDsl {
def update(value: T): Update[T] = macro QueryDslMacro.expandUpdate[T]
def update(f: (T => (Any, Any)), f2: (T => (Any, Any))*): Update[T]

def upsert(value: T): Upsert[T] = macro QueryDslMacro.expandUpsert[T]
def upsert(f: (T => (Any, Any)), f2: (T => (Any, Any))*): Upsert[T]

def delete: Delete[T]
}

Expand All @@ -84,8 +87,24 @@ private[dsl] trait QueryDsl {
}

sealed trait ActionReturning[E, Output] extends Action[E]
sealed trait ConflictAction[E, Output] extends Action[E]

sealed trait Update[E] extends Action[E]
sealed trait Delete[E] extends Action[E]

sealed trait Upsert[E] extends Action[E]{
@compileTimeOnly(NonQuotedException.message)
def update(value: E): Action[E] = NonQuotedException()

@compileTimeOnly(NonQuotedException.message)
def update(f: (E => (Any, Any)), f2: (E => (Any, Any))*): Action[E] = NonQuotedException()

@compileTimeOnly(NonQuotedException.message)
def conflict[R](f: E => R): ConflictAction[E, R] = NonQuotedException()

@compileTimeOnly(NonQuotedException.message)
def returning[R](f: E => R): ActionReturning[E, R] = NonQuotedException()
}

sealed trait BatchAction[+A <: Action[_]]
}
3 changes: 3 additions & 0 deletions quill-core/src/main/scala/io/getquill/dsl/QueryDslMacro.scala
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,9 @@ class QueryDslMacro(val c: MacroContext) {
def expandUpdate[T](value: Tree)(implicit t: WeakTypeTag[T]): Tree =
expandAction(value, "Update")

def expandUpsert[T](value: Tree)(implicit t: WeakTypeTag[T]): Tree =
expandAction(value, "Upsert")

private def expandAction[T](value: Tree, prefix: String)(implicit t: WeakTypeTag[T]) =
q"${meta(prefix)}.expand(${c.prefix}, $value)"

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,10 @@ case class BetaReduction(map: collection.Map[Ast, Ast])
val t = BetaReduction(map - alias)
Returning(apply(action), alias, t(prop))

case Conflict(action, alias, prop) =>
val t = BetaReduction(map - alias)
Conflict(apply(action), alias, t(prop))

case other =>
super.apply(other)
}
Expand Down
11 changes: 6 additions & 5 deletions quill-core/src/main/scala/io/getquill/norm/Normalize.scala
Original file line number Diff line number Diff line change
@@ -1,18 +1,19 @@
package io.getquill.norm

import io.getquill.ast.Ast
import io.getquill.ast.Query
import io.getquill.ast.StatelessTransformer
import io.getquill.ast._
import io.getquill.norm.capture.AvoidCapture
import io.getquill.ast.Action

object Normalize extends StatelessTransformer {

override def apply(q: Ast): Ast =
super.apply(BetaReduction(q))

override def apply(q: Action): Action =
NormalizeReturning(super.apply(q))
q match {
case Returning(_, _, _) => NormalizeReturning(super.apply(q))
case Conflict(_, _, _) => NormalizeConflict(super.apply(q))
case _ => NormalizeReturning(super.apply(q))
}

override def apply(q: Query): Query =
norm(AvoidCapture(q))
Expand Down
24 changes: 24 additions & 0 deletions quill-core/src/main/scala/io/getquill/norm/NormalizeConflict.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,24 @@
package io.getquill.norm

import io.getquill.ast._

object NormalizeConflict {

def apply(e: Action): Action = {
e match {
case Conflict(Upsert(query, assignments), alias, body) =>
Conflict(Upsert(query, filterReturnedColumn(assignments, body)), alias, body)
case e => e
}
}
private def filterReturnedColumn(assignments: List[Assignment], column: Ast): List[Assignment] =
assignments.map(filterReturnedColumn(_, column)).flatten

private def filterReturnedColumn(assignment: Assignment, column: Ast): Option[Assignment] =
(assignment, column) match {
case (Assignment(_, Property(_, p1), _), Property(_, p2)) if (p1 == p2) =>
None
case (assignment, column) =>
Some(assignment)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,13 @@ object RenameProperties extends StatelessTransformer {
val bodyr = BetaReduction(body, replace: _*)
(Returning(action, alias, bodyr), schema)
}
case Conflict(action: Action, alias, body) =>
applySchema(action) match {
case (action, schema) =>
val replace = replacements(alias, schema)
val bodyr = BetaReduction(body, replace: _*)
(Conflict(action, alias, bodyr), schema)
}
case q => (q, Tuple(List.empty))
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,8 @@ case class FreeVariables(state: State)
action match {
case q @ Returning(a, b, c) =>
(q, free(a, b, c))
case q @ Conflict(a, b, c) =>
(q, free(a, b, c))
case other =>
super.apply(other)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -118,6 +118,8 @@ trait Liftables {
implicit val actionLiftable: Liftable[Action] = Liftable[Action] {
case Update(a, b) => q"$pack.Update($a, $b)"
case Insert(a, b) => q"$pack.Insert($a, $b)"
case Upsert(a, b) => q"$pack.Upsert($a, $b)"
case Conflict(a, b, c) => q"$pack.Conflict($a, $b, $c)"
case Delete(a) => q"$pack.Delete($a)"
case Returning(a, b, c) => q"$pack.Returning($a, $b, $c)"
case Foreach(a, b, c) => q"$pack.Foreach($a, $b, $c)"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -123,6 +123,8 @@ trait Unliftables {
implicit val actionUnliftable: Unliftable[Action] = Unliftable[Action] {
case q"$pack.Update.apply(${ a: Ast }, ${ b: List[Assignment] })" => Update(a, b)
case q"$pack.Insert.apply(${ a: Ast }, ${ b: List[Assignment] })" => Insert(a, b)
case q"$pack.Upsert.apply(${ a: Ast }, ${ b: List[Assignment] })" => Upsert(a, b)
case q"$pack.Conflict.apply(${ a: Ast }, ${ b: Ident }, ${ c: Ast })" => Conflict(a, b, c)
case q"$pack.Delete.apply(${ a: Ast })" => Delete(a)
case q"$pack.Returning.apply(${ a: Ast }, ${ b: Ident }, ${ c: Ast })" => Returning(a, b, c)
case q"$pack.Foreach.apply(${ a: Ast }, ${ b: Ident }, ${ c: Ast })" => Foreach(a, b, c)
Expand Down
20 changes: 13 additions & 7 deletions quill-sql/src/main/scala/io/getquill/PostgresDialect.scala
Original file line number Diff line number Diff line change
@@ -1,13 +1,10 @@
package io.getquill

import io.getquill.idiom.StatementInterpolator._
import java.util.concurrent.atomic.AtomicInteger
import io.getquill.context.sql.idiom.SqlIdiom
import io.getquill.ast.UnaryOperation
import io.getquill.ast.Operation
import io.getquill.ast.Property
import io.getquill.ast.StringOperator
import io.getquill.context.sql.idiom.QuestionMarkBindVariables

import io.getquill.ast._
import io.getquill.context.sql.idiom.{QuestionMarkBindVariables, SqlIdiom}
import io.getquill.idiom.StatementInterpolator._

trait PostgresDialect
extends SqlIdiom
Expand All @@ -24,6 +21,15 @@ trait PostgresDialect

override def prepareForProbing(string: String) =
s"PREPARE p${preparedStatementId.incrementAndGet.toString.token} AS $string"

/*
override implicit def actionTokenizer(implicit strategy: NamingStrategy): Tokenizer[Action] = {
Tokenizer[Action] {
case Upsert(table: Entity, assignments) => super.actionTokenizer.token(Upsert(table, assignments))
case action => super.actionTokenizer.token(action)
}
}
*/
}

object PostgresDialect extends PostgresDialect
Original file line number Diff line number Diff line change
Expand Up @@ -280,6 +280,8 @@ trait SqlIdiom extends Idiom {
val values = assignments.map(_.value)
stmt"INSERT INTO ${table.token} (${columns.mkStmt(",")}) VALUES (${values.map(scopedTokenizer(_)).mkStmt(", ")})"

case Upsert(table: Entity, assignments) => actionTokenizer.token(Insert(table, assignments))

case Update(table: Entity, assignments) =>
stmt"UPDATE ${table.token} SET ${assignments.token}"

Expand All @@ -292,6 +294,9 @@ trait SqlIdiom extends Idiom {
case Delete(table: Entity) =>
stmt"DELETE FROM ${table.token}"

case Conflict(action, prop, value) =>
stmt"${action.token} ON CONFLICT(${value.token}) UPDATE ${prop.name.token}"

case Returning(action, prop, value) =>
action.token

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,5 +24,16 @@ class PostgresDialectSpec extends Spec {
}
context.run(q).string mustEqual "SELECT t.s::integer FROM TestEntity t"
}
"upsert" in {
val e = TestEntity("", 1, 1L, Some(1))
val q = quote {
qr1.upsert(lift(e)).conflict(_.i)
}
val i = quote {
qr1.insert(lift(TestEntity("", 1, 1L, Some(1))))
}
println(context.run(i).string)
println(context.run(q).string)
}
}
}

0 comments on commit 6f379c9

Please sign in to comment.