Skip to content

Commit

Permalink
Remove dependency on linearizers from constraint handling and templates.
Browse files Browse the repository at this point in the history
Index/constraint handling used to rely on linearizers for getting a list
of referenced columns. Instead we now use the Shape to pack() the original
value, grab the packed Node and extract the linear sequence from that
(which can be done independently of the Shape -- see
ExtraUtil.linearizeFieldRefs).

QueryTemplates not only do not need the linearizers any more, they can
even be unified with QueryInvokers (so that a QueryInvoker[P] is simply a
QueryTemplate[Unit, P]) with the additional unit invoker mix-ins.
  • Loading branch information
szeiger committed Feb 6, 2013
1 parent de5c5b1 commit 518a498
Show file tree
Hide file tree
Showing 12 changed files with 36 additions and 53 deletions.
11 changes: 11 additions & 0 deletions src/main/scala/scala/slick/ast/Util.scala
Expand Up @@ -138,4 +138,15 @@ object ExtraUtil {
case r: RefNode => s.contains(r.nodeReference) || n.nodeChildren.exists(ch => hasRefToOneOf(ch, s))
case n => n.nodeChildren.exists(ch => hasRefToOneOf(ch, s))
}

def linearizeFieldRefs(n: Node): IndexedSeq[Node] = {
val sels = new ArrayBuffer[Node]
def f(n: Node): Unit = n match {
case Path(_) => sels += n
case _: ProductNode | _: OptionApply | _: GetOrElse | _: TypeMapping | _: ClientSideOp =>
n.nodeChildren.foreach(f)
}
f(n)
sels
}
}
17 changes: 11 additions & 6 deletions src/main/scala/scala/slick/driver/JdbcInvokerComponent.scala
Expand Up @@ -15,23 +15,28 @@ trait JdbcInvokerComponent { driver: JdbcDriver =>
def createMappedKeysInsertInvoker[U, RU, R](tree: Node, keys: Node, tr: (U, RU) => R) = new MappedKeysInsertInvoker[U, RU, R](tree, keys, tr)
def createQueryInvoker[R](tree: Node) = new QueryInvoker[R](tree)
def createUpdateInvoker[T](tree: Node) = new UpdateInvoker[T](tree)
def createQueryTemplate[P,R](tree: Node): QueryTemplate[P,R] = new QueryTemplate[P,R](tree)

/** Invoker for executing queries. */
class QueryInvoker[R](protected val tree: Node)
extends MutatingStatementInvoker[Unit, R] with UnitInvokerMixin[R] with MutatingUnitInvoker[R] {

/** A parameterized query invoker. */
class QueryTemplate[P, R](protected val tree: Node) extends MutatingStatementInvoker[P, R] {
protected[this] val ResultSetMapping(_,
CompiledStatement(_, sres: SQLBuilder.Result, _),
CompiledMapping(converter, _)) = tree

override protected val delegate = this
def selectStatement = getStatement
protected def getStatement = sres.sql
protected def setParam(param: Unit, st: PreparedStatement): Unit = sres.setter(new PositionedParameters(st), null)
protected def setParam(param: P, st: PreparedStatement): Unit = sres.setter(new PositionedParameters(st), param)
protected def extractValue(pr: PositionedResult): R = converter.read(pr).asInstanceOf[R]
protected def updateRowValues(pr: PositionedResult, value: R) = converter.update(value, pr)
def invoker: this.type = this
}

/** Invoker for executing queries. */
class QueryInvoker[R](tree: Node) extends QueryTemplate[Unit, R](tree)
with UnitInvokerMixin[R] with MutatingUnitInvoker[R] {
override protected val delegate = this
}

/** Pseudo-invoker for running DDL statements. */
class DDLInvoker(ddl: DDL) {
/** Create the entities described by this DDL object */
Expand Down
3 changes: 3 additions & 0 deletions src/main/scala/scala/slick/driver/JdbcProfile.scala
Expand Up @@ -29,6 +29,9 @@ trait JdbcProfile extends SqlProfile with JdbcTableComponent
final def buildTableDDL(table: Table[_]): DDL = createTableDDLBuilder(table).buildDDL
final def buildSequenceDDL(seq: Sequence[_]): DDL = createSequenceDDLBuilder(seq).buildDDL

def queryToQueryTemplate[P,R](q: Query[_, R]): QueryTemplate[P,R] =
createQueryTemplate[P,R](selectStatementCompiler.run(Node(q)).tree)

class Implicits extends ImplicitJdbcTypes with ExtensionMethodConversions {
implicit val slickDriver: driver.type = driver
implicit def columnToOptionColumn[T : BaseTypedType](c: Column[T]): Column[Option[T]] = c.?
Expand Down
22 changes: 0 additions & 22 deletions src/main/scala/scala/slick/driver/JdbcQueryTemplate.scala

This file was deleted.

Expand Up @@ -16,8 +16,6 @@ import scala.slick.jdbc.{ResultConverter, CompiledMapping, Insert}
trait JdbcStatementBuilderComponent { driver: JdbcDriver =>

// Create the different builders -- these methods should be overridden by drivers as needed
def createQueryTemplate[P,R](q: Query[_, R]): JdbcQueryTemplate[P,R] =
new JdbcQueryTemplate[P,R](selectStatementCompiler.run(Node(q)).tree, q, this)
def createQueryBuilder(n: Node, state: CompilerState): QueryBuilder = new QueryBuilder(n, state)
def createInsertBuilder(node: Node): InsertBuilder = new InsertBuilder(node)
def createTableDDLBuilder(table: Table[_]): TableDDLBuilder = new TableDDLBuilder(table)
Expand Down
2 changes: 1 addition & 1 deletion src/main/scala/scala/slick/driver/JdbcTableComponent.scala
Expand Up @@ -36,7 +36,7 @@ trait JdbcTableComponent { driver: JdbcDriver =>
}) + "." + n
}

def createFinderBy[P](f: (this.type => Column[P]))(implicit tm: TypedType[P]): JdbcQueryTemplate[P,T] = {
def createFinderBy[P](f: (this.type => Column[P]))(implicit tm: TypedType[P]): QueryTemplate[P,T] = {
import driver.Implicit._
val thisQ = tableToQuery(this).asInstanceOf[Query[this.type, this.type]]
for {
Expand Down
4 changes: 2 additions & 2 deletions src/main/scala/scala/slick/lifted/AbstractTable.scala
Expand Up @@ -29,7 +29,7 @@ abstract class AbstractTable[T](val schemaName: Option[String], val tableName: S
new ForeignKeyQuery[TT, U](Filter(generator, Node(q), fv), q.unpackable, IndexedSeq(fk), q, generator, aliased.value)
}

def primaryKey[T](name: String, sourceColumns: T)(implicit unpack: Shape[T, _, _]): PrimaryKey = PrimaryKey(name, unpack.linearizer(sourceColumns).narrowedLinearizer.getLinearizedNodes)
def primaryKey[T](name: String, sourceColumns: T)(implicit shape: Shape[T, _, _]): PrimaryKey = PrimaryKey(name, ExtraUtil.linearizeFieldRefs(Node(shape.pack(sourceColumns))))

def tableConstraints: Iterator[Constraint] = for {
m <- getClass().getMethods.iterator
Expand All @@ -43,7 +43,7 @@ abstract class AbstractTable[T](val schemaName: Option[String], val tableName: S
final def primaryKeys: Iterable[PrimaryKey] =
tableConstraints.collect{ case k: PrimaryKey => k }.toIndexedSeq

def index[T](name: String, on: T, unique: Boolean = false)(implicit shape: Shape[T, _, _]) = new Index(name, this, shape.linearizer(on).narrowedLinearizer.getLinearizedNodes, unique)
def index[T](name: String, on: T, unique: Boolean = false)(implicit shape: Shape[T, _, _]) = new Index(name, this, ExtraUtil.linearizeFieldRefs(Node(shape.pack(on))), unique)

def indexes: Iterable[Index] = (for {
m <- getClass().getMethods.view
Expand Down
6 changes: 3 additions & 3 deletions src/main/scala/scala/slick/lifted/Constraint.scala
Expand Up @@ -39,9 +39,9 @@ object ForeignKey {
onDelete,
originalSourceColumns,
originalTargetColumns,
pShape.linearizer(originalSourceColumns).narrowedLinearizer.getLinearizedNodes,
pShape.linearizer(originalTargetColumns(targetTableShaped.value)).narrowedLinearizer.getLinearizedNodes,
pShape.linearizer(originalTargetColumns(originalTargetTable)).narrowedLinearizer.getLinearizedNodes,
ExtraUtil.linearizeFieldRefs(Node(pShape.pack(originalSourceColumns))),
ExtraUtil.linearizeFieldRefs(Node(pShape.pack(originalTargetColumns(targetTableShaped.value)))),
ExtraUtil.linearizeFieldRefs(Node(pShape.pack(originalTargetColumns(originalTargetTable)))),
targetTableShaped.value
)
}
Expand Down
10 changes: 5 additions & 5 deletions src/main/scala/scala/slick/lifted/Parameters.scala
@@ -1,16 +1,16 @@
package scala.slick.lifted

import scala.slick.SlickException
import scala.slick.driver.{JdbcDriver, JdbcProfile, JdbcQueryTemplate}
import scala.slick.driver.{JdbcDriver, JdbcProfile}
import scala.slick.util.NaturalTransformation2
import scala.slick.ast.TypedType

final class Parameters[PU, PP](c: PP) {
def flatMap[QU](f: PP => Query[_, QU])(implicit profile: JdbcProfile): JdbcQueryTemplate[PU, QU] =
profile.asInstanceOf[JdbcDriver].createQueryTemplate[PU, QU](f(c))
def flatMap[QU](f: PP => Query[_, QU])(implicit profile: JdbcProfile): profile.QueryTemplate[PU, QU] =
profile.queryToQueryTemplate[PU, QU](f(c))

def map[QM, QU](f: PP => QM)(implicit profile: JdbcProfile, shape: Shape[QM, QU, _]): JdbcQueryTemplate[PU, QU] =
profile.asInstanceOf[JdbcDriver].createQueryTemplate[PU, QU](Query(f(c)))
def map[QM, QU](f: PP => QM)(implicit profile: JdbcProfile, shape: Shape[QM, QU, _]): profile.QueryTemplate[PU, QU] =
profile.queryToQueryTemplate[PU, QU](Query(f(c)))

def filter(f: PP => Boolean): Parameters[PU, PP] =
if (!f(c)) throw new SlickException("Match failed when unpacking Parameters")
Expand Down
1 change: 0 additions & 1 deletion src/main/scala/scala/slick/lifted/Query.scala
Expand Up @@ -15,7 +15,6 @@ abstract class Query[+E, U] extends Rep[Seq[U]] with CollectionLinearizer[Seq, U

def unpackable: ShapedValue[_ <: E, U]
final lazy val packed = unpackable.packedNode
final lazy val elementLinearizer = unpackable.linearizer
final val canBuildFrom: CanBuildFrom[Nothing, U, Seq[U]] = implicitly

def flatMap[F, T](f: E => Query[F, T]): Query[F, T] = {
Expand Down
7 changes: 0 additions & 7 deletions src/main/scala/scala/slick/lifted/Shape.scala
Expand Up @@ -24,7 +24,6 @@ abstract class Shape[-Mixed_, Unpacked_, Packed_] {
type Packed = Packed_
def pack(from: Mixed): Packed
def packedShape: Shape[Packed, Unpacked, Packed]
def linearizer(from: Mixed): ValueLinearizer[Unpacked]

/** Build a packed representation from the shape and the TypedTypes alone.
* This method is not available for shapes where Mixed and Unpacked are
Expand All @@ -44,12 +43,10 @@ object Shape extends ShapeLowPriority {
selfLinearizingShape.asInstanceOf[Shape[Column[T], T, Column[T]]]

val selfLinearizingShape: Shape[ValueLinearizer[_], Any, ValueLinearizer[_]] = new IdentityShape[ValueLinearizer[_], Any] {
def linearizer(from: Mixed) = from.asInstanceOf[ValueLinearizer[Unpacked]]
def buildPacked(f: NaturalTransformation2[TypedType, ({ type L[X] = Unpacked => X })#L, Column]) = impureShape
}

val sharedTableShape: Shape[TableNode, Any, TableNode] = new IdentityShape[TableNode, Any] {
def linearizer(from: Mixed) = from.nodeShaped_*.value.asInstanceOf[ValueLinearizer[Unpacked]]
def buildPacked(f: NaturalTransformation2[TypedType, ({ type L[X] = Unpacked => X })#L, Column]) = impureShape
}
}
Expand All @@ -66,7 +63,6 @@ class ShapeLowPriority extends ShapeLowPriority2 {
implicit final def unpackPrimitive[T](implicit tm: TypedType[T]): Shape[T, T, Column[T]] = new Shape[T, T, Column[T]] {
def pack(from: Mixed) = ConstColumn(from)
def packedShape: Shape[Packed, Unpacked, Packed] = unpackColumnBase[T, Column[T]]
def linearizer(from: Mixed) = ConstColumn(from)
def buildPacked(f: NaturalTransformation2[TypedType, ({ type L[X] = Unpacked => X })#L, Column]): Packed =
f(tm, identity)
}
Expand All @@ -77,8 +73,6 @@ final class TupleShape[M <: Product, U <: Product, P <: Product](ps: Shape[_, _,
TupleSupport.buildTuple(ps.iterator.zip(from.productIterator).map{case (p, f) => p.pack(f.asInstanceOf[p.Mixed])}.toIndexedSeq).asInstanceOf[Packed]
def packedShape: Shape[Packed, Unpacked, Packed] =
new TupleShape(ps.map(_.packedShape): _*)
def linearizer(from: Mixed) =
new ProductLinearizer(ps.iterator.zip(from.productIterator).map{case (p, f) => p.linearizer(f.asInstanceOf[p.Mixed]).asInstanceOf[RecordLinearizer[_]]}.toIndexedSeq)
def buildPacked(f: NaturalTransformation2[TypedType, ({ type L[X] = Unpacked => X })#L, Column]): Packed =
TupleSupport.buildTuple(ps.iterator.zipWithIndex.map{ case (p, i) => p.buildPacked(productTf(i, f)) }.toIndexedSeq).asInstanceOf[Packed]

Expand All @@ -98,7 +92,6 @@ case class ShapedValue[T, U](value: T, shape: Shape[T, U, _]) {
}
def packedNode = Node(shape.pack(value))
def packedValue[R](implicit ev: Shape[T, _, R]): ShapedValue[R, U] = ShapedValue(shape.pack(value).asInstanceOf[R], shape.packedShape.asInstanceOf[Shape[R, U, _]])
def linearizer = shape.linearizer(value).asInstanceOf[ValueLinearizer[U]]
def zip[T2, U2](s2: ShapedValue[T2, U2]) = new ShapedValue[(T, T2), (U, U2)]((value, s2.value), Shape.tuple2Shape(shape, s2.shape))
}

Expand Down
4 changes: 0 additions & 4 deletions src/main/scala/scala/slick/util/ValueLinearizer.scala
Expand Up @@ -11,16 +11,13 @@ import scala.slick.ast.Node
* sequence of columns) form of values.
*/
sealed trait ValueLinearizer[T] {
def narrowedLinearizer: RecordLinearizer[_]
}

/**
* A linearizer for collection values.
*/
trait CollectionLinearizer[F[+_], T] extends ValueLinearizer[F[T]] {
def elementLinearizer: ValueLinearizer[T]
def canBuildFrom: CanBuildFrom[Nothing, T, F[T]]
final def narrowedLinearizer = elementLinearizer.narrowedLinearizer
}

/**
Expand All @@ -31,7 +28,6 @@ trait RecordLinearizer[T] extends ValueLinearizer[T] {
def updateResult(driver: JdbcDriver, rs: PositionedResult, value: T): Unit
def setParameter(driver: JdbcDriver, ps: PositionedParameters, value: Option[T]): Unit
def getLinearizedNodes: IndexedSeq[Node]
final def narrowedLinearizer = this
}

trait DelegateRecordLinearizer[T] extends RecordLinearizer[T] {
Expand Down

0 comments on commit 518a498

Please sign in to comment.