From 2f64c795bc0a7131d3df5b684adb767433043b34 Mon Sep 17 00:00:00 2001 From: Stefan Zeiger Date: Fri, 30 Jan 2015 16:32:01 +0100 Subject: [PATCH] Rename Action to DBIO: - Package `scala.slick.action` becomes `scala.slick.dbio`. - The generic `EffectfulAction` type with effect tracking becomes `DBIOAction` with type aliases `DBIO` and `StreamingDBIO` without effect tracking. - The effect type parameter is placed last instead of first in all types and methods. Effect types can be verbose and are usually irrelevant when looking at type error messages. Fixes #1049. --- scaladoc-root.txt | 59 +--- .../scala/scala/slick/blocking/Blocking.scala | 6 +- .../scala/slick/direct/SlickBackend.scala | 2 +- .../src/codegen/scala/CodeGeneratorTest.scala | 6 +- .../slick/testkit/tests/JdbcMiscTest.scala | 4 +- .../slick/testkit/tests/TransactionTest.scala | 2 +- .../slick/testkit/util/StandardTestDBs.scala | 12 +- .../typesafe/slick/testkit/util/TestDB.scala | 6 +- .../typesafe/slick/testkit/util/Testkit.scala | 16 +- .../scala/scala/slick/action/package.scala | 7 - .../slick/backend/DatabaseComponent.scala | 34 +-- .../slick/backend/DatabasePublisher.scala | 6 +- .../slick/backend/RelationalBackend.scala | 2 +- .../Action.scala => dbio/DBIOAction.scala} | 267 ++++++++---------- src/main/scala/scala/slick/dbio/Effect.scala | 26 ++ .../scala/scala/slick/dbio/NoStream.scala | 9 + src/main/scala/scala/slick/dbio/package.scala | 11 + .../scala/slick/driver/DerbyDriver.scala | 4 +- .../scala/slick/driver/HsqldbDriver.scala | 4 +- .../slick/driver/JdbcActionComponent.scala | 18 +- .../slick/driver/JdbcModelComponent.scala | 6 +- .../scala/slick/driver/JdbcProfile.scala | 6 +- .../scala/slick/driver/PostgresDriver.scala | 4 +- .../scala/slick/driver/SQLiteDriver.scala | 4 +- .../scala/scala/slick/jdbc/JdbcBackend.scala | 7 +- .../scala/slick/jdbc/JdbcModelBuilder.scala | 16 +- .../scala/slick/jdbc/ResultSetInvoker.scala | 2 +- .../scala/scala/slick/jdbc/StaticQuery.scala | 2 +- .../slick/jdbc/StreamingInvokerAction.scala | 2 +- .../scala/slick/jdbc/meta/MPrimaryKey.scala | 2 +- .../scala/scala/slick/jdbc/meta/MTable.scala | 2 +- .../scala/scala/slick/lifted/Aliases.scala | 16 +- .../slick/memory/DistributedBackend.scala | 2 +- .../slick/memory/DistributedProfile.scala | 2 +- .../scala/slick/memory/HeapBackend.scala | 2 +- .../scala/slick/memory/MemoryProfile.scala | 2 +- .../scala/slick/profile/BasicProfile.scala | 4 +- .../slick/profile/RelationalProfile.scala | 2 +- .../scala/slick/profile/SqlProfile.scala | 2 +- src/sphinx/code/Connection.scala | 2 +- src/sphinx/database.rst | 75 ++--- src/sphinx/queries.rst | 2 +- src/sphinx/upgrade.rst | 8 +- 43 files changed, 313 insertions(+), 360 deletions(-) delete mode 100644 src/main/scala/scala/slick/action/package.scala rename src/main/scala/scala/slick/{action/Action.scala => dbio/DBIOAction.scala} (52%) create mode 100644 src/main/scala/scala/slick/dbio/Effect.scala create mode 100644 src/main/scala/scala/slick/dbio/NoStream.scala create mode 100644 src/main/scala/scala/slick/dbio/package.scala diff --git a/scaladoc-root.txt b/scaladoc-root.txt index 34383b6087..37abb948d9 100644 --- a/scaladoc-root.txt +++ b/scaladoc-root.txt @@ -1,66 +1,11 @@ edit this text on github Slick logo

Scala Language-Integrated Connection Kit

-This is the documentation for the Slick database library. -Slick is a joint effort by Typesafe and -LAMP, EPFL Lausanne. +This is the API documentation for the Slick database library. +It should be used as an additional resource to the user manual. Further documentation for Slick can be found on the documentation pages. To the slick package list... - -

Important places

- -

Type-safe query operators

- - -Plain SQL queries - - -

Driver / Connection related

- - -

Mapping related

- - -

Model / Table class related

- - -
Other
- diff --git a/slick-blocking/src/main/scala/scala/slick/blocking/Blocking.scala b/slick-blocking/src/main/scala/scala/slick/blocking/Blocking.scala index 5fca3ff842..0530d83ed7 100644 --- a/slick-blocking/src/main/scala/scala/slick/blocking/Blocking.scala +++ b/slick-blocking/src/main/scala/scala/slick/blocking/Blocking.scala @@ -2,7 +2,7 @@ package scala.slick.blocking import scala.language.implicitConversions -import scala.slick.action._ +import scala.slick.dbio._ import scala.slick.backend.DatabaseComponent import scala.slick.util.{CloseableIterator, ignoreFollowOnError} @@ -14,12 +14,12 @@ object Blocking { /** Run an Action and block the current thread until the result is ready. If the Database uses * synchronous, blocking excution, it is performed on the current thread in order to avoid any * context switching, otherwise execution happens asynchronously. */ - def run[R](db: DatabaseComponent#DatabaseDef, a: Action[R]): R = db.runInternal(a, true).value.get.get + def run[R](db: DatabaseComponent#DatabaseDef, a: DBIO[R]): R = db.runInternal(a, true).value.get.get /** Run a streaming Action and return an `Iterator` which performs blocking I/O on the current * thread (if supported by the Database) or blocks the current thread while waiting for the * next result. */ - def iterator[S](db: DatabaseComponent#DatabaseDef, a: StreamingAction[Any, S]): CloseableIterator[S] = new CloseableIterator[S] { + def iterator[S](db: DatabaseComponent#DatabaseDef, a: StreamingDBIO[Any, S]): CloseableIterator[S] = new CloseableIterator[S] { val p = db.streamInternal(a, true) var error: Throwable = null var sub: Subscription = null diff --git a/slick-direct/src/main/scala/scala/slick/direct/SlickBackend.scala b/slick-direct/src/main/scala/scala/slick/direct/SlickBackend.scala index b18ecfed76..ef19c25506 100644 --- a/slick-direct/src/main/scala/scala/slick/direct/SlickBackend.scala +++ b/slick-direct/src/main/scala/scala/slick/direct/SlickBackend.scala @@ -3,7 +3,7 @@ package scala.slick.direct import language.existentials import scala.slick.SlickException import scala.language.implicitConversions -import scala.slick.action.{Streaming, Effect, EffectfulAction, NoStream, ActionContext} +import scala.slick.dbio.{Streaming, Effect, NoStream, ActionContext} import scala.slick.driver._ import scala.slick.profile.{SqlProfile, SqlStreamingAction, SqlAction} import scala.slick.relational.CompiledMapping diff --git a/slick-testkit/src/codegen/scala/CodeGeneratorTest.scala b/slick-testkit/src/codegen/scala/CodeGeneratorTest.scala index f817c5ed64..2d84d05c4b 100644 --- a/slick-testkit/src/codegen/scala/CodeGeneratorTest.scala +++ b/slick-testkit/src/codegen/scala/CodeGeneratorTest.scala @@ -3,7 +3,7 @@ package scala.slick.test.codegen import scala.concurrent.{Future, Await} import scala.concurrent.duration.Duration import scala.concurrent.ExecutionContext.Implicits.global -import scala.slick.action.Action +import scala.slick.dbio.DBIO import scala.slick.codegen.SourceCodeGenerator import scala.slick.driver._ import scala.slick.jdbc.JdbcBackend @@ -157,12 +157,12 @@ lazy val database = Database.forURL(url=""\"$url""\",driver="$jdbcDriver",user=" slickDriverObject: JdbcDriver, slickDriver: String, jdbcDriver: String, - generator: Config => Action[SourceCodeGenerator] + generator: Config => DBIO[SourceCodeGenerator] ) class H2Config( objectName: String, inits: Seq[String], - generator: Config => Action[SourceCodeGenerator] + generator: Config => DBIO[SourceCodeGenerator] = config => H2Driver.createModel(ignoreInvalidDefaults=false).map(m => new MySourceCodeGenerator(m, config)) ) extends Config( objectName, diff --git a/slick-testkit/src/main/scala/com/typesafe/slick/testkit/tests/JdbcMiscTest.scala b/slick-testkit/src/main/scala/com/typesafe/slick/testkit/tests/JdbcMiscTest.scala index 6d48cf4085..7846e5037d 100644 --- a/slick-testkit/src/main/scala/com/typesafe/slick/testkit/tests/JdbcMiscTest.scala +++ b/slick-testkit/src/main/scala/com/typesafe/slick/testkit/tests/JdbcMiscTest.scala @@ -54,8 +54,8 @@ class JdbcMiscTest extends AsyncTest[JdbcTestDB] { Action.successful(()).flatMap { _ => TableQuery[Foo].schema.create }.failed.map(_.shouldBeA[SlickException]) } - def testSimpleAction = { - val getAutoCommit = SimpleAction[Boolean](_.session.conn.getAutoCommit) + def testSimpleDBIO = { + val getAutoCommit = SimpleDBIO[Boolean](_.connection.getAutoCommit) getAutoCommit.map(_ shouldBe true) } diff --git a/slick-testkit/src/main/scala/com/typesafe/slick/testkit/tests/TransactionTest.scala b/slick-testkit/src/main/scala/com/typesafe/slick/testkit/tests/TransactionTest.scala index b223c45a28..1e9eda5210 100644 --- a/slick-testkit/src/main/scala/com/typesafe/slick/testkit/tests/TransactionTest.scala +++ b/slick-testkit/src/main/scala/com/typesafe/slick/testkit/tests/TransactionTest.scala @@ -14,7 +14,7 @@ class TransactionTest extends AsyncTest[JdbcTestDB] { } val ts = TableQuery[T] - val getTI = SimpleAction(_.session.conn.getTransactionIsolation) + val getTI = SimpleDBIO(_.connection.getTransactionIsolation) class ExpectedException extends RuntimeException diff --git a/slick-testkit/src/main/scala/com/typesafe/slick/testkit/util/StandardTestDBs.scala b/slick-testkit/src/main/scala/com/typesafe/slick/testkit/util/StandardTestDBs.scala index dde311db46..3364983232 100644 --- a/slick-testkit/src/main/scala/com/typesafe/slick/testkit/util/StandardTestDBs.scala +++ b/slick-testkit/src/main/scala/com/typesafe/slick/testkit/util/StandardTestDBs.scala @@ -4,7 +4,7 @@ import java.io.File import java.util.logging.{Level, Logger} import java.sql.SQLException import scala.concurrent.ExecutionContext -import scala.slick.action._ +import scala.slick.dbio._ import scala.slick.driver._ import scala.slick.memory.MemoryDriver import scala.slick.jdbc.{StaticQuery => Q, ResultSetAction, ResultSetInvoker} @@ -85,7 +85,7 @@ object StandardTestDBs { lazy val Postgres = new ExternalJdbcTestDB("postgres") { val driver = PostgresDriver - override def localTables(implicit ec: ExecutionContext): Action[Vector[String]] = + override def localTables(implicit ec: ExecutionContext): DBIO[Vector[String]] = ResultSetAction[(String,String,String, String)](_.conn.getMetaData().getTables("", "public", null, null)).map { ts => ts.filter(_._4.toUpperCase == "TABLE").map(_._3).sorted } @@ -142,7 +142,7 @@ class SQLiteTestDB(dburl: String, confName: String) extends InternalJdbcTestDB(c val driver = SQLiteDriver val url = dburl val jdbcDriver = "org.sqlite.JDBC" - override def localTables(implicit ec: ExecutionContext): Action[Vector[String]] = + override def localTables(implicit ec: ExecutionContext): DBIO[Vector[String]] = super.localTables.map(_.filter(s => !s.toLowerCase.contains("sqlite_"))) override def dropUserArtifacts(implicit session: profile.Backend#Session) = { for(t <- getLocalTables) @@ -171,7 +171,7 @@ class AccessDB(confName: String) extends ExternalJdbcTestDB(confName) { } /* Works in some situations but fails with "Optional feature not implemented" in others */ override def canGetLocalTables = false - override def localTables(implicit ec: ExecutionContext): Action[Vector[String]] = + override def localTables(implicit ec: ExecutionContext): DBIO[Vector[String]] = MTable.getTables.map(_.map(_.name.name).sorted) override def capabilities = super.capabilities - TestDB.capabilities.jdbcMeta } @@ -180,7 +180,7 @@ abstract class DerbyDB(confName: String) extends InternalJdbcTestDB(confName) { val driver = DerbyDriver System.setProperty("derby.stream.error.method", classOf[DerbyDB].getName + ".DEV_NULL") val jdbcDriver = "org.apache.derby.jdbc.EmbeddedDriver" - override def localTables(implicit ec: ExecutionContext): Action[Vector[String]] = + override def localTables(implicit ec: ExecutionContext): DBIO[Vector[String]] = ResultSetAction[(String,String,String, String)](_.conn.getMetaData().getTables(null, "APP", null, null)).map { ts => ts.map(_._3).sorted } @@ -215,7 +215,7 @@ object DerbyDB { abstract class HsqlDB(confName: String) extends InternalJdbcTestDB(confName) { val driver = HsqldbDriver val jdbcDriver = "org.hsqldb.jdbcDriver" - override def localTables(implicit ec: ExecutionContext): Action[Vector[String]] = + override def localTables(implicit ec: ExecutionContext): DBIO[Vector[String]] = ResultSetAction[(String,String,String, String)](_.conn.getMetaData().getTables(null, "PUBLIC", null, null)).map { ts => ts.map(_._3).sorted } diff --git a/slick-testkit/src/main/scala/com/typesafe/slick/testkit/util/TestDB.scala b/slick-testkit/src/main/scala/com/typesafe/slick/testkit/util/TestDB.scala index 696e364976..e05b24a3c8 100644 --- a/slick-testkit/src/main/scala/com/typesafe/slick/testkit/util/TestDB.scala +++ b/slick-testkit/src/main/scala/com/typesafe/slick/testkit/util/TestDB.scala @@ -8,7 +8,7 @@ import java.util.zip.GZIPInputStream import scala.collection.mutable import scala.concurrent.ExecutionContext import scala.slick.SlickException -import scala.slick.action.{NoStream, EffectfulAction, Action} +import scala.slick.dbio.{NoStream, DBIOAction, DBIO} import scala.slick.jdbc.{StaticQuery => Q, ResultSetAction, JdbcDataSource, SimpleJdbcAction, ResultSetInvoker} import scala.slick.jdbc.GetResult._ import scala.slick.driver._ @@ -149,7 +149,7 @@ abstract class JdbcTestDB(val confName: String) extends SqlTestDB { final def getLocalTables(implicit session: profile.Backend#Session) = blockingRunOnSession(ec => localTables(ec)) def canGetLocalTables = true - def localTables(implicit ec: ExecutionContext): Action[Vector[String]] = + def localTables(implicit ec: ExecutionContext): DBIO[Vector[String]] = ResultSetAction[(String,String,String, String)](_.conn.getMetaData().getTables("", "", null, null)).map { ts => ts.filter(_._4.toUpperCase == "TABLE").map(_._3).sorted } @@ -187,7 +187,7 @@ abstract class JdbcTestDB(val confName: String) extends SqlTestDB { def close(): Unit = () }, executor) } - final def blockingRunOnSession[R](f: ExecutionContext => EffectfulAction[Nothing, R, NoStream])(implicit session: profile.Backend#Session): R = { + final def blockingRunOnSession[R](f: ExecutionContext => DBIOAction[R, NoStream, Nothing])(implicit session: profile.Backend#Session): R = { val ec = new ExecutionContext { def execute(runnable: Runnable): Unit = runnable.run() def reportFailure(t: Throwable): Unit = throw t diff --git a/slick-testkit/src/main/scala/com/typesafe/slick/testkit/util/Testkit.scala b/slick-testkit/src/main/scala/com/typesafe/slick/testkit/util/Testkit.scala index 2fc72a0e8f..ef5ce51641 100644 --- a/slick-testkit/src/main/scala/com/typesafe/slick/testkit/util/Testkit.scala +++ b/slick-testkit/src/main/scala/com/typesafe/slick/testkit/util/Testkit.scala @@ -12,7 +12,7 @@ import java.lang.reflect.Method import java.util.concurrent.{LinkedBlockingQueue, ThreadPoolExecutor, ExecutionException, TimeUnit} import java.util.concurrent.atomic.AtomicInteger -import scala.slick.action._ +import scala.slick.dbio._ import scala.slick.jdbc.JdbcBackend import scala.slick.util.DumpInfo import scala.slick.profile.{RelationalProfile, SqlProfile, Capability} @@ -94,7 +94,7 @@ case class TestMethod(name: String, desc: Description, method: Method, cl: Class case testObject: AsyncTest[_] => if(r == classOf[Future[_]]) await(method.invoke(testObject).asInstanceOf[Future[Any]]) - else if(r == classOf[EffectfulAction[_, _, _]]) await(testObject.db.run(method.invoke(testObject).asInstanceOf[Action[Any]])) + else if(r == classOf[DBIOAction[_, _, _]]) await(testObject.db.run(method.invoke(testObject).asInstanceOf[DBIO[Any]])) else throw new RuntimeException(s"Illegal return type: '${r.getName}' in test method '$name' -- AsyncTest methods must return Future or Action") } } @@ -212,23 +212,23 @@ abstract class AsyncTest[TDB >: Null <: TestDB](implicit TdbClass: ClassTag[TDB] def getDumpInfo = DumpInfo(name = "") } - def ifCap[E <: Effect, R](caps: Capability*)(f: => EffectfulAction[E, R, NoStream]): EffectfulAction[E, Unit, NoStream] = - if(caps.forall(c => tdb.capabilities.contains(c))) f.andThen(Action.successful(())) else Action.successful(()) - def ifNotCap[E <: Effect, R](caps: Capability*)(f: => EffectfulAction[E, R, NoStream]): EffectfulAction[E, Unit, NoStream] = - if(!caps.forall(c => tdb.capabilities.contains(c))) f.andThen(Action.successful(())) else Action.successful(()) + def ifCap[E <: Effect, R](caps: Capability*)(f: => DBIOAction[R, NoStream, E]): DBIOAction[Unit, NoStream, E] = + if(caps.forall(c => tdb.capabilities.contains(c))) f.andThen(DBIO.successful(())) else DBIO.successful(()) + def ifNotCap[E <: Effect, R](caps: Capability*)(f: => DBIOAction[R, NoStream, E]): DBIOAction[Unit, NoStream, E] = + if(!caps.forall(c => tdb.capabilities.contains(c))) f.andThen(DBIO.successful(())) else DBIO.successful(()) def ifCapF[R](caps: Capability*)(f: => Future[R]): Future[Unit] = if(caps.forall(c => tdb.capabilities.contains(c))) f.map(_ => ()) else Future.successful(()) def ifNotCapF[R](caps: Capability*)(f: => Future[R]): Future[Unit] = if(!caps.forall(c => tdb.capabilities.contains(c))) f.map(_ => ()) else Future.successful(()) - def asAction[R](f: tdb.profile.Backend#Session => R): EffectfulAction[Effect, R, NoStream] = + def asAction[R](f: tdb.profile.Backend#Session => R): DBIOAction[R, NoStream, Effect] = new SynchronousDatabaseAction[tdb.profile.Backend, Effect, R, NoStream] { def run(context: tdb.profile.Backend#Context): R = f(context.session) def getDumpInfo = DumpInfo(name = "") } - def seq[E <: Effect](actions: EffectfulAction[E, _, NoStream]*): EffectfulAction[E, Unit, NoStream] = Action.seq[E](actions: _*) + def seq[E <: Effect](actions: DBIOAction[_, NoStream, E]*): DBIOAction[Unit, NoStream, E] = DBIO.seq[E](actions: _*) /** Synchronously consume a Reactive Stream and materialize it as a Vector. */ def materialize[T](p: Publisher[T]): Future[Vector[T]] = { diff --git a/src/main/scala/scala/slick/action/package.scala b/src/main/scala/scala/slick/action/package.scala deleted file mode 100644 index 6619fb6a49..0000000000 --- a/src/main/scala/scala/slick/action/package.scala +++ /dev/null @@ -1,7 +0,0 @@ -package scala.slick - -package object action { - type StreamingAction[+R, +T] = EffectfulAction[Effect.All, R, Streaming[T]] - - type Action[+R] = EffectfulAction[Effect.All, R, NoStream] -} diff --git a/src/main/scala/scala/slick/backend/DatabaseComponent.scala b/src/main/scala/scala/slick/backend/DatabaseComponent.scala index 22c76dbba1..4cf9d6b7e6 100644 --- a/src/main/scala/scala/slick/backend/DatabaseComponent.scala +++ b/src/main/scala/scala/slick/backend/DatabaseComponent.scala @@ -13,7 +13,7 @@ import org.slf4j.LoggerFactory import org.reactivestreams._ import scala.slick.SlickException -import scala.slick.action._ +import scala.slick.dbio._ import scala.slick.util.{GlobalConfig, DumpInfo, TreeDump, SlickLogger, ignoreFollowOnError} /** Backend for the basic database and session handling features. @@ -47,9 +47,9 @@ trait DatabaseComponent { self => def close(): Unit /** Run an Action asynchronously and return the result as a Future. */ - final def run[R](a: EffectfulAction[Nothing, R, NoStream]): Future[R] = runInternal(a, false) + final def run[R](a: DBIOAction[R, NoStream, Nothing]): Future[R] = runInternal(a, false) - private[slick] final def runInternal[R](a: EffectfulAction[Nothing, R, NoStream], useSameThread: Boolean): Future[R] = + private[slick] final def runInternal[R](a: DBIOAction[R, NoStream, Nothing], useSameThread: Boolean): Future[R] = runInContext(a, createDatabaseActionContext(useSameThread), false) /** Create a `Publisher` for Reactive Streams which, when subscribed to, will run the specified @@ -73,13 +73,13 @@ trait DatabaseComponent { self => * from within `onNext`. If streaming is interrupted due to back-pressure signaling, the next * row will be prefetched (in order to buffer the next result page from the server when a page * boundary has been reached). */ - final def stream[T](a: EffectfulAction[Nothing, _, Streaming[T]]): DatabasePublisher[T] = streamInternal(a, false) + final def stream[T](a: DBIOAction[_, Streaming[T], Nothing]): DatabasePublisher[T] = streamInternal(a, false) - private[slick] final def streamInternal[T](a: EffectfulAction[Nothing, _, Streaming[T]], useSameThread: Boolean): DatabasePublisher[T] = + private[slick] final def streamInternal[T](a: DBIOAction[_, Streaming[T], Nothing], useSameThread: Boolean): DatabasePublisher[T] = createPublisher(a, s => createStreamingDatabaseActionContext(s, useSameThread)) /** Create a Reactive Streams `Publisher` using the given context factory. */ - protected[this] def createPublisher[T](a: EffectfulAction[Nothing, _, Streaming[T]], createCtx: Subscriber[_ >: T] => StreamingContext): DatabasePublisher[T] = new DatabasePublisherSupport[T] { + protected[this] def createPublisher[T](a: DBIOAction[_, Streaming[T], Nothing], createCtx: Subscriber[_ >: T] => StreamingContext): DatabasePublisher[T] = new DatabasePublisherSupport[T] { def subscribe(s: Subscriber[_ >: T]) = if(allowSubscriber(s)) { val ctx = createCtx(s) if(streamLogger.isDebugEnabled) streamLogger.debug(s"Signaling onSubscribe($ctx)") @@ -93,7 +93,7 @@ trait DatabaseComponent { self => runInContext(a, ctx, true).onComplete { case Success(_) => ctx.tryOnComplete case Failure(t) => ctx.tryOnError(t) - }(Action.sameThreadExecutionContext) + }(DBIO.sameThreadExecutionContext) } catch { case NonFatal(ex) => streamLogger.warn("Database.streamInContext failed unexpectedly", ex) @@ -117,7 +117,7 @@ trait DatabaseComponent { self => * be a `StreamingDatabaseActionContext` and the Future result should be * completed with `null` or failed after streaming has finished. This * method should not call any `Subscriber` method other than `onNext`. */ - protected[this] def runInContext[R](a: EffectfulAction[Nothing, R, NoStream], ctx: Context, streaming: Boolean): Future[R] = { + protected[this] def runInContext[R](a: DBIOAction[R, NoStream, Nothing], ctx: Context, streaming: Boolean): Future[R] = { logAction(a, ctx) a match { case SuccessAction(v) => Future.successful(v) @@ -126,13 +126,13 @@ trait DatabaseComponent { self => case FlatMapAction(base, f, ec) => runInContext(base, ctx, false).flatMap(v => runInContext(f(v), ctx, streaming))(ctx.getEC(ec)) case AndThenAction(a1, a2) => - runInContext(a1, ctx, false).flatMap(_ => runInContext(a2, ctx, streaming))(Action.sameThreadExecutionContext) + runInContext(a1, ctx, false).flatMap(_ => runInContext(a2, ctx, streaming))(DBIO.sameThreadExecutionContext) case ZipAction(a1, a2) => runInContext(a1, ctx, false).flatMap { r1 => runInContext(a2, ctx, false).map { r2 => (r1, r2) - }(Action.sameThreadExecutionContext) - }(Action.sameThreadExecutionContext).asInstanceOf[Future[R]] + }(DBIO.sameThreadExecutionContext) + }(DBIO.sameThreadExecutionContext).asInstanceOf[Future[R]] case CleanUpAction(base, f, keepFailure, ec) => val p = Promise[R]() runInContext(base, ctx, streaming).onComplete { t1 => @@ -144,7 +144,7 @@ trait DatabaseComponent { self => runInContext(a2, ctx, false).onComplete { t2 => if(t2.isFailure && (t1.isSuccess || !keepFailure)) p.complete(t2.asInstanceOf[Failure[R]]) else p.complete(t1) - } (Action.sameThreadExecutionContext) + } (DBIO.sameThreadExecutionContext) } catch { case NonFatal(ex) => throw (t1 match { @@ -158,14 +158,14 @@ trait DatabaseComponent { self => runInContext(a, ctx, false).failed.asInstanceOf[Future[R]] case AsTryAction(a) => val p = Promise[R]() - runInContext(a, ctx, false).onComplete(v => p.success(v.asInstanceOf[R]))(Action.sameThreadExecutionContext) + runInContext(a, ctx, false).onComplete(v => p.success(v.asInstanceOf[R]))(DBIO.sameThreadExecutionContext) p.future case NamedAction(a, _) => runInContext(a, ctx, streaming) case a: SynchronousDatabaseAction[_, _, _, _] => if(streaming) { if(a.supportsStreaming) streamSynchronousDatabaseAction(a.asInstanceOf[SynchronousDatabaseAction[This, _ <: Effect, _, _ <: NoStream]], ctx.asInstanceOf[StreamingContext]).asInstanceOf[Future[R]] - else runInContext(CleanUpAction(AndThenAction(Action.Pin, a.nonFusedEquivalentAction), _ => Action.Unpin, true, Action.sameThreadExecutionContext), ctx, streaming) + else runInContext(CleanUpAction(AndThenAction(DBIO.Pin, a.nonFusedEquivalentAction), _ => DBIO.Unpin, true, DBIO.sameThreadExecutionContext), ctx, streaming) } else runSynchronousDatabaseAction(a.asInstanceOf[SynchronousDatabaseAction[This, _, R, NoStream]], ctx) case a: DatabaseAction[_, _, _] => throw new SlickException(s"Unsupported database action $a for $this") @@ -279,13 +279,13 @@ trait DatabaseComponent { self => * SynchronousDatabaseActions for asynchronous execution. */ protected[this] def synchronousExecutionContext: ExecutionContext - protected[this] def logAction(a: EffectfulAction[Nothing, _, NoStream], ctx: Context): Unit = { + protected[this] def logAction(a: DBIOAction[_, NoStream, Nothing], ctx: Context): Unit = { if(actionLogger.isDebugEnabled && a.isLogged) { ctx.sequenceCounter += 1 val logA = a.nonFusedEquivalentAction val aPrefix = if(a eq logA) "" else "[fused] " val dump = TreeDump.get(logA, prefix = " ", firstPrefix = aPrefix, narrow = { - case a: EffectfulAction[_, _, _] => a.nonFusedEquivalentAction + case a: DBIOAction[_, _, _] => a.nonFusedEquivalentAction case o => o }) val msg = DumpInfo.highlight("#" + ctx.sequenceCounter) + ": " + dump.substring(0, dump.length-1) @@ -385,7 +385,7 @@ trait DatabaseComponent { self => /** Return the specified ExecutionContext unless running in same-thread mode, in which case * `Action.sameThreadExecutionContext` is returned instead. */ private[DatabaseComponent] def getEC(ec: ExecutionContext): ExecutionContext = - if(useSameThread) Action.sameThreadExecutionContext else ec + if(useSameThread) DBIO.sameThreadExecutionContext else ec /** A volatile variable to enforce the happens-before relationship (see * [[https://docs.oracle.com/javase/specs/jls/se7/html/jls-17.html]] and diff --git a/src/main/scala/scala/slick/backend/DatabasePublisher.scala b/src/main/scala/scala/slick/backend/DatabasePublisher.scala index f3a826b77f..6acc4728ba 100644 --- a/src/main/scala/scala/slick/backend/DatabasePublisher.scala +++ b/src/main/scala/scala/slick/backend/DatabasePublisher.scala @@ -5,7 +5,7 @@ import java.util.concurrent.atomic.AtomicBoolean import org.reactivestreams._ import scala.concurrent.{Promise, Future, ExecutionContext} -import scala.slick.action.Action +import scala.slick.dbio.DBIO import scala.util.{Failure, Success} /** A Reactive Streams `Publisher` for database Actions. */ @@ -38,12 +38,12 @@ abstract class DatabasePublisher[T] extends Publisher[T] { self => if(l ne null) l.onComplete { case Success(_) => p.trySuccess(()) case Failure(t) => p.tryFailure(t) - }(Action.sameThreadExecutionContext) + }(DBIO.sameThreadExecutionContext) else p.trySuccess(()) } def onError(t: Throwable): Unit = { val l = lastMsg - if(l ne null) l.onComplete(_ => p.tryFailure(t))(Action.sameThreadExecutionContext) + if(l ne null) l.onComplete(_ => p.tryFailure(t))(DBIO.sameThreadExecutionContext) else p.tryFailure(t) } def onNext(t: T): Unit = { diff --git a/src/main/scala/scala/slick/backend/RelationalBackend.scala b/src/main/scala/scala/slick/backend/RelationalBackend.scala index 0abde6451f..fcd7f1f7ab 100644 --- a/src/main/scala/scala/slick/backend/RelationalBackend.scala +++ b/src/main/scala/scala/slick/backend/RelationalBackend.scala @@ -1,6 +1,6 @@ package scala.slick.backend -import scala.slick.action.Effect +import scala.slick.dbio.Effect /** The required backend level for RelationalProfile. */ trait RelationalBackend extends DatabaseComponent diff --git a/src/main/scala/scala/slick/action/Action.scala b/src/main/scala/scala/slick/dbio/DBIOAction.scala similarity index 52% rename from src/main/scala/scala/slick/action/Action.scala rename to src/main/scala/scala/slick/dbio/DBIOAction.scala index f7f745a8bb..b14f47a2e5 100644 --- a/src/main/scala/scala/slick/action/Action.scala +++ b/src/main/scala/scala/slick/dbio/DBIOAction.scala @@ -1,4 +1,4 @@ -package scala.slick.action +package scala.slick.dbio import org.reactivestreams.Subscription @@ -13,57 +13,56 @@ import scala.slick.util.{DumpInfo, Dumpable, ignoreFollowOnError} import scala.util.{Try, Failure, Success} import scala.util.control.NonFatal -/** An Action that can be executed on a database. The Action type allows a separation of execution - * logic and resource usage management logic from composition logic. Actions can be composed with - * methods such as `andThen`, `andFinally` and `flatMap`. Individual parts of a composite Action - * are always executed serially on a single database, but possibly in different database sessions, - * unless the session is pinned either explicitly (using `withPinnedSession`) or implicitly (e.g. - * through a transaction). +/** A Database I/O Action that can be executed on a database. The DBIOAction type allows a + * separation of execution logic and resource usage management logic from composition logic. + * DBIOActions can be composed with methods such as `andThen`, `andFinally` and `flatMap`. + * Individual parts of a composite DBIOAction are always executed serially on a single database, + * but possibly in different database sessions, unless the session is pinned either explicitly + * (using `withPinnedSession`) or implicitly (e.g. through a transaction). * - * The actual implementation base type for all Actions is `EffectfulAction`. `StreamingAction` and - * `Action` are type aliases which discard the effect type (and the streaming result type in the - * latter case) to make Action types easier to write when these features are not needed. All - * primitive Actions and all Actions produced by the standard combinators in Slick have correct - * Effect types and are streaming (if possible). + * The actual implementation base type for all Actions is `DBIOAction`. `StreamingDBIO` and + * `DBIO` are type aliases which discard the effect type (and the streaming result type in the + * latter case) to make DBIOAction types easier to write when these features are not needed. All + * primitive DBIOActions and all DBIOActions produced by the standard combinators in Slick have + * correct Effect types and are streaming (if possible). * - * @tparam E The Action's effect type, e.g. `Effect.Read with Effect.Write`. When composing - * Actions, the correct combined effect type will be inferred. Effects are used to tie - * an Action to a specific back-end type and they can also be used in user code, e.g. - * to automatically direct all read-only Actions to a slave database and write Actions - * to the master copy. - * @tparam R The result type when executing the Action and fully materializing the result. + * @tparam R The result type when executing the DBIOAction and fully materializing the result. * @tparam S An encoding of the result type for streaming results. If this action is capable of * streaming, it is `Streaming[T]` for an element type `T`. For non-streaming - * Actions it is `NoStream`. + * DBIOActions it is `NoStream`. + * @tparam E The DBIOAction's effect type, e.g. `Effect.Read with Effect.Write`. When composing + * actions, the correct combined effect type will be inferred. Effects can be used in + * user code, e.g. to automatically direct all read-only Actions to a slave database + * and write Actions to the master copy. */ -sealed trait EffectfulAction[-E <: Effect, +R, +S <: NoStream] extends Dumpable { +sealed trait DBIOAction[+R, +S <: NoStream, -E <: Effect] extends Dumpable { /** Transform the result of a successful execution of this action. If this action fails, the * resulting action also fails. */ - def map[R2](f: R => R2)(implicit executor: ExecutionContext): EffectfulAction[E, R2, NoStream] = - flatMap[E, R2, NoStream](r => SuccessAction[R2](f(r))) + def map[R2](f: R => R2)(implicit executor: ExecutionContext): DBIOAction[R2, NoStream, E] = + flatMap[R2, NoStream, E](r => SuccessAction[R2](f(r))) /** Use the result produced by the successful execution of this action to compute and then * run the next action in sequence. The resulting action fails if either this action, the * computation, or the computed action fails. */ - def flatMap[E2 <: Effect, R2, S2 <: NoStream](f: R => EffectfulAction[E2, R2, S2])(implicit executor: ExecutionContext): EffectfulAction[E with E2, R2, S2] = - FlatMapAction[E with E2, R2, S2, R](this, f, executor) + def flatMap[R2, S2 <: NoStream, E2 <: Effect](f: R => DBIOAction[R2, S2, E2])(implicit executor: ExecutionContext): DBIOAction[R2, S2, E with E2] = + FlatMapAction[R2, S2, R, E with E2](this, f, executor) /** Run another action after this action, if it completed successfully, and return the result * of the second action. If either of the two actions fails, the resulting action also fails. */ - def andThen[E2 <: Effect, R2, S2 <: NoStream](a: EffectfulAction[E2, R2, S2]): EffectfulAction[E with E2, R2, S2] = - AndThenAction[E with E2, R2, S2](this, a) + def andThen[R2, S2 <: NoStream, E2 <: Effect](a: DBIOAction[R2, S2, E2]): DBIOAction[R2, S2, E with E2] = + AndThenAction[R2, S2, E with E2](this, a) /** Run another action after this action, if it completed successfully, and return the result * of both actions. If either of the two actions fails, the resulting action also fails. */ - def zip[E2 <: Effect, R2](a: EffectfulAction[E2, R2, NoStream]): EffectfulAction[E with E2, (R, R2), NoStream] = - ZipAction[E with E2, R, R2](this, a) + def zip[R2, E2 <: Effect](a: DBIOAction[R2, NoStream, E2]): DBIOAction[(R, R2), NoStream, E with E2] = + ZipAction[R, R2, E with E2](this, a) /** Run another action after this action, whether it succeeds or fails, and then return the * result of the first action. If the first action fails, its failure is propagated, whether * the second action fails or succeeds. If the first action succeeds, a failure of the second * action is propagated. */ - def andFinally[E2 <: Effect](a: EffectfulAction[E2, _, NoStream]): EffectfulAction[E with E2, R, S] = - cleanUp[E2](_ => a)(Action.sameThreadExecutionContext) + def andFinally[E2 <: Effect](a: DBIOAction[_, NoStream, E2]): DBIOAction[R, S, E with E2] = + cleanUp[E2](_ => a)(DBIO.sameThreadExecutionContext) /** Run another action after this action, whether it succeeds or fails, in order to clean up or * transform an error produced by this action. The clean-up action is computed from the failure @@ -75,91 +74,91 @@ sealed trait EffectfulAction[-E <: Effect, +R, +S <: NoStream] extends Dumpable * with the same error, no matter whether the clean-up action succeeds or * fails. If `keepFailure` is set to `false`, an error from the clean-up * action will override the error from this action. */ - def cleanUp[E2 <: Effect](f: Option[Throwable] => EffectfulAction[E2, _, NoStream], keepFailure: Boolean = true)(implicit executor: ExecutionContext): EffectfulAction[E with E2, R, S] = - CleanUpAction[E with E2, R, S](this, f, keepFailure, executor) + def cleanUp[E2 <: Effect](f: Option[Throwable] => DBIOAction[_, NoStream, E2], keepFailure: Boolean = true)(implicit executor: ExecutionContext): DBIOAction[R, S, E with E2] = + CleanUpAction[R, S, E with E2](this, f, keepFailure, executor) /** A shortcut for `andThen`. */ - final def >> [E2 <: Effect, R2, S2 <: NoStream](a: EffectfulAction[E2, R2, S2]): EffectfulAction[E with E2, R2, S2] = - andThen[E2, R2, S2](a) + final def >> [R2, S2 <: NoStream, E2 <: Effect](a: DBIOAction[R2, S2, E2]): DBIOAction[R2, S2, E with E2] = + andThen[R2, S2, E2](a) - /** Filter the result of this Action with the given predicate. If the predicate matches, the - * original result is returned, otherwise the resulting Action fails with a + /** Filter the result of this action with the given predicate. If the predicate matches, the + * original result is returned, otherwise the resulting action fails with a * NoSuchElementException. */ - final def filter(p: R => Boolean)(implicit executor: ExecutionContext): EffectfulAction[E, R, NoStream] = + final def filter(p: R => Boolean)(implicit executor: ExecutionContext): DBIOAction[R, NoStream, E] = withFilter(p) - def withFilter(p: R => Boolean)(implicit executor: ExecutionContext): EffectfulAction[E, R, NoStream] = + def withFilter(p: R => Boolean)(implicit executor: ExecutionContext): DBIOAction[R, NoStream, E] = flatMap(v => if(p(v)) SuccessAction(v) else throw new NoSuchElementException("Action.withFilter failed")) - /** Return an Action which contains the Throwable with which this Action failed as its result. - * If this Action succeeded, the resulting Action fails with a NoSuchElementException. */ - def failed: EffectfulAction[E, Throwable, NoStream] = FailedAction[E](this) + /** Return an action which contains the Throwable with which this action failed as its result. + * If this action succeeded, the resulting action fails with a NoSuchElementException. */ + def failed: DBIOAction[Throwable, NoStream, E] = FailedAction[E](this) - /** Convert a successful result `v` of this Action into a successful result `Success(v)` and a + /** Convert a successful result `v` of this action into a successful result `Success(v)` and a * failure `t` into a successful result `Failure(t)`. This is the most generic combinator that * can be used for error recovery. If possible, use [[andFinally]] or [[cleanUp]] instead, * because those combinators, unlike `asTry`, support streaming. */ - def asTry: EffectfulAction[E, Try[R], NoStream] = AsTryAction[E, R](this) + def asTry: DBIOAction[Try[R], NoStream, E] = AsTryAction[R, E](this) - /** Run this Action with a pinned database session. If this action is composed of multiple + /** Use a pinned database session when running this action. If it is composed of multiple * database actions, they will all use the same session, even when sequenced with non-database * actions. For non-composite or non-database actions, this has no effect. */ - def withPinnedSession: EffectfulAction[E, R, S] = Action.Pin andThen this andFinally Action.Unpin + def withPinnedSession: DBIOAction[R, S, E] = DBIO.Pin andThen this andFinally DBIO.Unpin - /** Get a wrapping Action which has a name that will be included in log output. */ - def named(name: String): EffectfulAction[E, R, S] = - NamedAction[E, R, S](this, name) + /** Get a wrapping action which has a name that will be included in log output. */ + def named(name: String): DBIOAction[R, S, E] = + NamedAction[R, S, E](this, name) - /** Get the equivalent non-fused Action if this Action has been fused, otherwise this - * Action is returned. */ - def nonFusedEquivalentAction: EffectfulAction[E, R, S] = this + /** Get the equivalent non-fused action if this action has been fused, otherwise this + * action is returned. */ + def nonFusedEquivalentAction: DBIOAction[R, S, E] = this - /** Whether or not this Action should be included in log output by default. */ + /** Whether or not this action should be included in log output by default. */ def isLogged: Boolean = false } -object Action { - /** Convert a `Future` to an [[Action]]. */ - def from[R](f: Future[R]): EffectfulAction[Effect, R, NoStream] = FutureAction[R](f) +object DBIO { + /** Convert a `Future` to a [[DBIOAction]]. */ + def from[R](f: Future[R]): DBIOAction[R, NoStream, Effect] = FutureAction[R](f) - /** Lift a constant value to an [[Action]]. */ - def successful[R](v: R): EffectfulAction[Effect, R, NoStream] = SuccessAction[R](v) + /** Lift a constant value to a [[DBIOAction]]. */ + def successful[R](v: R): DBIOAction[R, NoStream, Effect] = SuccessAction[R](v) - /** Create an [[Action]] that always fails. */ - def failed(t: Throwable): EffectfulAction[Effect, Nothing, NoStream] = FailureAction(t) + /** Create a [[DBIOAction]] that always fails. */ + def failed(t: Throwable): DBIOAction[Nothing, NoStream, Effect] = FailureAction(t) - /** Transform a `TraversableOnce[EffectfulAction[E, R, NoStream]]` into an `EffectfulAction[E, TraversableOnce[R], NoStream]`. */ - def sequence[E <: Effect, R, M[+_] <: TraversableOnce[_]](in: M[EffectfulAction[E, R, NoStream]])(implicit cbf: CanBuildFrom[M[EffectfulAction[E, R, NoStream]], R, M[R]]): EffectfulAction[E, M[R], NoStream] = { - implicit val ec = Action.sameThreadExecutionContext - in.foldLeft(Action.successful(cbf(in)): EffectfulAction[E, mutable.Builder[R, M[R]], NoStream]) { (ar, ae) => - for (r <- ar; e <- ae.asInstanceOf[EffectfulAction[E, R, NoStream]]) yield (r += e) + /** Transform a `TraversableOnce[ DBIO[R] ]` into a `DBIO[ TraversableOnce[R] ]`. */ + def sequence[R, M[+_] <: TraversableOnce[_], E <: Effect](in: M[DBIOAction[R, NoStream, E]])(implicit cbf: CanBuildFrom[M[DBIOAction[R, NoStream, E]], R, M[R]]): DBIOAction[M[R], NoStream, E] = { + implicit val ec = DBIO.sameThreadExecutionContext + in.foldLeft(DBIO.successful(cbf(in)): DBIOAction[mutable.Builder[R, M[R]], NoStream, E]) { (ar, ae) => + for (r <- ar; e <- ae.asInstanceOf[DBIOAction[R, NoStream, E]]) yield (r += e) } map (_.result) } - /** A simpler version of `sequence` that takes a number of Actions with any return type as - * varargs and returns an Action that performs the individual Actions in sequence (using + /** A simpler version of `sequence` that takes a number of DBIOActions with any return type as + * varargs and returns a DBIOAction that performs the individual actions in sequence (using * `andThen`), returning `()` in the end. */ - def seq[E <: Effect](actions: EffectfulAction[E, _, NoStream]*): EffectfulAction[E, Unit, NoStream] = - (actions :+ SuccessAction(())).reduceLeft(_ andThen _).asInstanceOf[EffectfulAction[E, Unit, NoStream]] + def seq[E <: Effect](actions: DBIOAction[_, NoStream, E]*): DBIOAction[Unit, NoStream, E] = + (actions :+ SuccessAction(())).reduceLeft(_ andThen _).asInstanceOf[DBIOAction[Unit, NoStream, E]] - /** Create an Action that runs some other actions in sequence and combines their results + /** Create a DBIOAction that runs some other actions in sequence and combines their results * with the given function. */ - def fold[E <: Effect, T](actions: Seq[EffectfulAction[E, T, NoStream]], zero: T)(f: (T, T) => T)(implicit ec: ExecutionContext): EffectfulAction[E, T, NoStream] = - actions.foldLeft[EffectfulAction[E, T, NoStream]](Action.successful(zero)) { (za, va) => za.flatMap(z => va.map(v => f(z, v))) } + def fold[T, E <: Effect](actions: Seq[DBIOAction[T, NoStream, E]], zero: T)(f: (T, T) => T)(implicit ec: ExecutionContext): DBIOAction[T, NoStream, E] = + actions.foldLeft[DBIOAction[T, NoStream, E]](DBIO.successful(zero)) { (za, va) => za.flatMap(z => va.map(v => f(z, v))) } - /** An Action that pins the current session */ + /** A DBIOAction that pins the current session */ private[slick] object Pin extends SynchronousDatabaseAction[DatabaseComponent, Effect, Unit, NoStream] { def run(context: DatabaseComponent#Context): Unit = context.pin def getDumpInfo = DumpInfo(name = "SynchronousDatabaseAction.Pin") } - /** An Action that unpins the current session */ + /** A DBIOAction that unpins the current session */ private[slick] object Unpin extends SynchronousDatabaseAction[DatabaseComponent, Effect, Unit, NoStream] { def run(context: DatabaseComponent#Context): Unit = context.unpin def getDumpInfo = DumpInfo(name = "SynchronousDatabaseAction.Unpin") } - /** An ExecutionContext used internally for executing plumbing operations during Action + /** An ExecutionContext used internally for executing plumbing operations during DBIOAction * composition. */ private[slick] object sameThreadExecutionContext extends ExecutionContext { override def execute(runnable: Runnable): Unit = runnable.run() @@ -167,61 +166,61 @@ object Action { } } -/** An Action that represents a database operation. Concrete implementations are backend-specific. */ -trait DatabaseAction[-E <: Effect, +R, +S <: NoStream] extends EffectfulAction[E, R, S] { +/** A DBIOAction that represents a database operation. Concrete implementations are backend-specific. */ +trait DatabaseAction[+R, +S <: NoStream, -E <: Effect] extends DBIOAction[R, S, E] { override def isLogged = true } -/** An Action that returns a constant value. */ +/** A DBIOAction that returns a constant value. */ case class SuccessAction[+R](value: R) extends SynchronousDatabaseAction[DatabaseComponent, Effect, R, NoStream] { def getDumpInfo = DumpInfo("success", String.valueOf(value)) def run(ctx: DatabaseComponent#Context): R = value } -/** An Action that fails. */ +/** A DBIOAction that fails. */ case class FailureAction(t: Throwable) extends SynchronousDatabaseAction[DatabaseComponent, Effect, Nothing, NoStream] { def getDumpInfo = DumpInfo("failure", String.valueOf(t)) def run(ctx: DatabaseComponent#Context): Nothing = throw t } -/** An asynchronous Action that returns the result of a Future. */ -case class FutureAction[+R](f: Future[R]) extends EffectfulAction[Effect, R, NoStream] { +/** An asynchronous DBIOAction that returns the result of a Future. */ +case class FutureAction[+R](f: Future[R]) extends DBIOAction[R, NoStream, Effect] { def getDumpInfo = DumpInfo("future", String.valueOf(f)) override def isLogged = true } -/** An Action that represents a `flatMap` operation for sequencing in the Action monad. */ -case class FlatMapAction[-E <: Effect, +R, +S <: NoStream, P](base: EffectfulAction[E, P, NoStream], f: P => EffectfulAction[E, R, S], executor: ExecutionContext) extends EffectfulAction[E, R, S] { +/** A DBIOAction that represents a `flatMap` operation for sequencing in the DBIOAction monad. */ +case class FlatMapAction[+R, +S <: NoStream, P, -E <: Effect](base: DBIOAction[P, NoStream, E], f: P => DBIOAction[R, S, E], executor: ExecutionContext) extends DBIOAction[R, S, E] { def getDumpInfo = DumpInfo("flatMap", String.valueOf(f), children = Vector(("base", base))) } -/** An Action that represents an `andThen` operation for sequencing in the Action monad. */ -case class AndThenAction[-E <: Effect, +R, +S <: NoStream](a1: EffectfulAction[E, _, NoStream], a2: EffectfulAction[E, R, S]) extends EffectfulAction[E, R, S] { +/** A DBIOAction that represents an `andThen` operation for sequencing in the DBIOAction monad. */ +case class AndThenAction[+R, +S <: NoStream, -E <: Effect](a1: DBIOAction[_, NoStream, E], a2: DBIOAction[R, S, E]) extends DBIOAction[R, S, E] { def getDumpInfo = DumpInfo("andThen", children = Vector(("1", a1), ("2", a2))) } -/** An Action that represents a `zip` operation for sequencing in the Action monad. */ -case class ZipAction[-E <: Effect, +R1, +R2](a1: EffectfulAction[E, R1, NoStream], a2: EffectfulAction[E, R2, NoStream]) extends EffectfulAction[E, (R1, R2), NoStream] { +/** A DBIOAction that represents a `zip` operation for sequencing in the DBIOAction monad. */ +case class ZipAction[+R1, +R2, -E <: Effect](a1: DBIOAction[R1, NoStream, E], a2: DBIOAction[R2, NoStream, E]) extends DBIOAction[(R1, R2), NoStream, E] { def getDumpInfo = DumpInfo("zip", children = Vector(("1", a1), ("2", a2))) } -/** An Action that represents a `cleanUp` operation for sequencing in the Action monad. */ -case class CleanUpAction[-E <: Effect, +R, +S <: NoStream](base: EffectfulAction[E, R, S], f: Option[Throwable] => EffectfulAction[E, _, NoStream], keepFailure: Boolean, executor: ExecutionContext) extends EffectfulAction[E, R, S] { +/** A DBIOAction that represents a `cleanUp` operation for sequencing in the DBIOAction monad. */ +case class CleanUpAction[+R, +S <: NoStream, -E <: Effect](base: DBIOAction[R, S, E], f: Option[Throwable] => DBIOAction[_, NoStream, E], keepFailure: Boolean, executor: ExecutionContext) extends DBIOAction[R, S, E] { def getDumpInfo = DumpInfo("cleanUp", children = Vector(("try", base))) } -/** An Action that represents a `failed` operation. */ -case class FailedAction[-E <: Effect](a: EffectfulAction[E, _, NoStream]) extends EffectfulAction[E, Throwable, NoStream] { +/** A DBIOAction that represents a `failed` operation. */ +case class FailedAction[-E <: Effect](a: DBIOAction[_, NoStream, E]) extends DBIOAction[Throwable, NoStream, E] { def getDumpInfo = DumpInfo("failed", children = Vector(("base", a))) } -/** An Action that represents an `asTry` operation. */ -case class AsTryAction[-E <: Effect, +R](a: EffectfulAction[E, R, NoStream]) extends EffectfulAction[E, Try[R], NoStream] { +/** A DBIOAction that represents an `asTry` operation. */ +case class AsTryAction[+R, -E <: Effect](a: DBIOAction[R, NoStream, E]) extends DBIOAction[Try[R], NoStream, E] { def getDumpInfo = DumpInfo("asTry") } -/** An Action that attaches a name for logging purposes to another action. */ -case class NamedAction[-E <: Effect, +R, +S <: NoStream](a: EffectfulAction[E, R, S], name: String) extends EffectfulAction[E, R, S] { +/** A DBIOAction that attaches a name for logging purposes to another action. */ +case class NamedAction[+R, +S <: NoStream, -E <: Effect](a: DBIOAction[R, S, E], name: String) extends DBIOAction[R, S, E] { def getDumpInfo = DumpInfo("named", mainInfo = DumpInfo.highlight(name)) override def isLogged = true } @@ -246,7 +245,7 @@ trait ActionContext { final def unpin: Unit = stickiness -= 1 } -/** An ActionContext with extra functionality required for streaming Actions. */ +/** An ActionContext with extra functionality required for streaming DBIOActions. */ trait StreamingActionContext extends ActionContext { /** Emit a single result of the stream. Any Exception thrown by this method should be passed on * to the caller. */ @@ -266,7 +265,7 @@ trait StreamingActionContext extends ActionContext { * The execution engine ensures that an [[ActionContext]] is never used concurrently and that * all state changes performed by one invocation of a SynchronousDatabaseAction are visible * to the next invocation of the same or a different SynchronousDatabaseAction. */ -trait SynchronousDatabaseAction[-B <: DatabaseComponent, -E <: Effect, +R, +S <: NoStream] extends DatabaseAction[E, R, S] { self => +trait SynchronousDatabaseAction[-B <: DatabaseComponent, -E <: Effect, +R, +S <: NoStream] extends DatabaseAction[R, S, E] { self => /** The type used by this action for the state of a suspended stream. A call to `emitStream` * produces such a state which is then fed back into the next call. */ type StreamState >: Null <: AnyRef @@ -291,38 +290,38 @@ trait SynchronousDatabaseAction[-B <: DatabaseComponent, -E <: Effect, +R, +S <: * method will not be called in these cases. */ def cancelStream(context: B#StreamingContext, state: StreamState): Unit = () - /** Whether or not this Action supports streaming results. An Action with a `Streaming` result + /** Whether or not this action supports streaming results. An action with a `Streaming` result * type must either support streaming directly or have a [[nonFusedEquivalentAction]] which - * supports streaming. This flag is ignored if the Action has a `NoStream` result type. */ + * supports streaming. This flag is not used if the Action has a `NoStream` result type. */ def supportsStreaming: Boolean = true - private[this] def superAndThen[E2 <: Effect, R2, S2 <: NoStream](a: EffectfulAction[E2, R2, S2]) = super.andThen[E2, R2, S2](a) - override def andThen[E2 <: Effect, R2, S2 <: NoStream](a: EffectfulAction[E2, R2, S2]): EffectfulAction[E with E2, R2, S2] = a match { + private[this] def superAndThen[R2, S2 <: NoStream, E2 <: Effect](a: DBIOAction[R2, S2, E2]) = super.andThen[R2, S2, E2](a) + override def andThen[R2, S2 <: NoStream, E2 <: Effect](a: DBIOAction[R2, S2, E2]): DBIOAction[R2, S2, E with E2] = a match { case a: SynchronousDatabaseAction[_, _, _, _] => new SynchronousDatabaseAction.Fused[B, E with E2, R2, S2] { def run(context: B#Context): R2 = { self.run(context) a.asInstanceOf[SynchronousDatabaseAction[B, E2, R2, S2]].run(context) } - override def nonFusedEquivalentAction: EffectfulAction[E with E2, R2, S2] = superAndThen(a) + override def nonFusedEquivalentAction: DBIOAction[R2, S2, E with E2] = superAndThen(a) } case a => superAndThen(a) } - private[this] def superZip[E2 <: Effect, R2](a: EffectfulAction[E2, R2, NoStream]) = super.zip[E2, R2](a) - override def zip[E2 <: Effect, R2](a: EffectfulAction[E2, R2, NoStream]): EffectfulAction[E with E2, (R, R2), NoStream] = a match { + private[this] def superZip[R2, E2 <: Effect](a: DBIOAction[R2, NoStream, E2]) = super.zip[R2, E2](a) + override def zip[R2, E2 <: Effect](a: DBIOAction[R2, NoStream, E2]): DBIOAction[(R, R2), NoStream, E with E2] = a match { case a: SynchronousDatabaseAction[_, _, _, _] => new SynchronousDatabaseAction.Fused[B, E with E2, (R, R2), NoStream] { def run(context: B#Context): (R, R2) = { val r1 = self.run(context) val r2 = a.asInstanceOf[SynchronousDatabaseAction[B, E2, R2, NoStream]].run(context) (r1, r2) } - override def nonFusedEquivalentAction: EffectfulAction[E with E2, (R, R2), NoStream] = superZip(a) + override def nonFusedEquivalentAction: DBIOAction[(R, R2), NoStream, E with E2] = superZip(a) } case a => superZip(a) } - private[this] def superAndFinally[E2 <: Effect](a: EffectfulAction[E2, _, NoStream]) = super.andFinally[E2](a) - override def andFinally[E2 <: Effect](a: EffectfulAction[E2, _, NoStream]): EffectfulAction[E with E2, R, S] = a match { + private[this] def superAndFinally[E2 <: Effect](a: DBIOAction[_, NoStream, E2]) = super.andFinally[E2](a) + override def andFinally[E2 <: Effect](a: DBIOAction[_, NoStream, E2]): DBIOAction[R, S, E with E2] = a match { case a: SynchronousDatabaseAction[_, _, _, _] => new SynchronousDatabaseAction.Fused[B, E with E2, R, S] { def run(context: B#Context): R = { val res = try self.run(context) catch { @@ -333,13 +332,13 @@ trait SynchronousDatabaseAction[-B <: DatabaseComponent, -E <: Effect, +R, +S <: a.asInstanceOf[SynchronousDatabaseAction[B, E2, Any, S]].run(context) res } - override def nonFusedEquivalentAction: EffectfulAction[E with E2, R, S] = superAndFinally(a) + override def nonFusedEquivalentAction: DBIOAction[R, S, E with E2] = superAndFinally(a) } case a => superAndFinally(a) } private[this] def superWithPinnedSession = super.withPinnedSession - override def withPinnedSession: EffectfulAction[E, R, S] = new SynchronousDatabaseAction.Fused[B, E, R, S] { + override def withPinnedSession: DBIOAction[R, S, E] = new SynchronousDatabaseAction.Fused[B, E, R, S] { def run(context: B#Context): R = { context.pin val res = try self.run(context) catch { @@ -353,8 +352,8 @@ trait SynchronousDatabaseAction[-B <: DatabaseComponent, -E <: Effect, +R, +S <: override def nonFusedEquivalentAction = superWithPinnedSession } - private[this] def superFailed: EffectfulAction[E, Throwable, NoStream] = super.failed - override def failed: EffectfulAction[E, Throwable, NoStream] = new SynchronousDatabaseAction.Fused[B, E, Throwable, NoStream] { + private[this] def superFailed: DBIOAction[Throwable, NoStream, E] = super.failed + override def failed: DBIOAction[Throwable, NoStream, E] = new SynchronousDatabaseAction.Fused[B, E, Throwable, NoStream] { def run(context: B#Context): Throwable = { var ok = false try { @@ -368,8 +367,8 @@ trait SynchronousDatabaseAction[-B <: DatabaseComponent, -E <: Effect, +R, +S <: override def nonFusedEquivalentAction = superFailed } - private[this] def superAsTry: EffectfulAction[E, Try[R], NoStream] = super.asTry - override def asTry: EffectfulAction[E, Try[R], NoStream] = new SynchronousDatabaseAction.Fused[B, E, Try[R], NoStream] { + private[this] def superAsTry: DBIOAction[Try[R], NoStream, E] = super.asTry + override def asTry: DBIOAction[Try[R], NoStream, E] = new SynchronousDatabaseAction.Fused[B, E, Try[R], NoStream] { def run(context: B#Context): Try[R] = { try Success(self.run(context)) catch { case NonFatal(ex) => Failure(ex) @@ -387,12 +386,12 @@ object SynchronousDatabaseAction { } /** Fuse `flatMap` / `map`, `cleanUp` and `filter` / `withFilter` combinators if they use - * `Action.sameThreadExecutionContext` and produce a `SynchronousDatabaseAction` in their + * `DBIO.sameThreadExecutionContext` and produce a `SynchronousDatabaseAction` in their * evaluation function (where applicable). This cannot be verified at fusion time, so a wrongly - * fused Action can fail with a `ClassCastException` during evaluation. */ - private[slick] def fuseUnsafe[E <: Effect, R, S <: NoStream](a: EffectfulAction[E, R, S]): EffectfulAction[E, R, S] = { + * fused action can fail with a `ClassCastException` during evaluation. */ + private[slick] def fuseUnsafe[R, S <: NoStream, E <: Effect](a: DBIOAction[R, S, E]): DBIOAction[R, S, E] = { a match { - case FlatMapAction(base: SynchronousDatabaseAction[_, _, _, _], f, ec) if ec eq Action.sameThreadExecutionContext => + case FlatMapAction(base: SynchronousDatabaseAction[_, _, _, _], f, ec) if ec eq DBIO.sameThreadExecutionContext => new SynchronousDatabaseAction.Fused[DatabaseComponent, E, R, S] { def run(context: DatabaseComponent#Context): R = { val b = base.asInstanceOf[SynchronousDatabaseAction[DatabaseComponent, Effect, Any, NoStream]].run(context) @@ -402,7 +401,7 @@ object SynchronousDatabaseAction { override def nonFusedEquivalentAction = a } - case CleanUpAction(base: SynchronousDatabaseAction[_, _, _, _], f, keepFailure, ec) if ec eq Action.sameThreadExecutionContext => + case CleanUpAction(base: SynchronousDatabaseAction[_, _, _, _], f, keepFailure, ec) if ec eq DBIO.sameThreadExecutionContext => new SynchronousDatabaseAction.Fused[DatabaseComponent, E, R, S] { def run(context: DatabaseComponent#Context): R = { val res = try { @@ -425,37 +424,3 @@ object SynchronousDatabaseAction { } } } - -/** A phantom type for annotating database Actions with specific effects (e.g. `Write` or - * `Transactional`). Effects can be composed through intersection types (e.g. - * `Write with Transactional`. Backends declare all supported types to prevent execution - * of unsupported features (e.g. [[scala.slick.memory.HeapBackend]] does not support - * transactions). Further restrictions (like ensuring that all writes go to a master - * database but reads can also be performed by a slave) can be enforced in user-level code. */ -trait Effect - -object Effect { - /** Effect for Actions that read from the database ("DQL") */ - trait Read extends Effect - /** Effect for Actions that write to the database ("DML") */ - trait Write extends Effect - /** Effect for Actions that manipulate a database schema ("DDL") */ - trait Schema extends Effect - /** Effect for transactional Actions ("DTL") */ - trait Transactional extends Effect - - /** The bottom type of all standard effects. It is used by the `Action` and `StreamingAction` - * type aliases instead of `Nothing` because the compiler does not properly infer `Nothing` - * where needed. You can still introduce your own custom effect types but they will not be - * used by `Action` and `StreamingAction`, so you either have to define your own type aliases - * or spell out the proper `EffectfulAction` types in type annotations. */ - trait All extends Read with Write with Schema with Transactional -} - -/** A phantom type used as the streaming result type for Actions that do not support streaming. - * Note that this is a supertype of `Streaming` (and it is used in covariant position), - * so that any streaming Action can be used where a non-streaming Action is expected. */ -sealed trait NoStream - -/** A phantom type used as the streaming result type for Actions that do support streaming. */ -sealed trait Streaming[+T] extends NoStream diff --git a/src/main/scala/scala/slick/dbio/Effect.scala b/src/main/scala/scala/slick/dbio/Effect.scala new file mode 100644 index 0000000000..7930c6e7dd --- /dev/null +++ b/src/main/scala/scala/slick/dbio/Effect.scala @@ -0,0 +1,26 @@ +package scala.slick.dbio + +/** A phantom type for annotating DBIOActions with specific effects (e.g. `Write` or + * `Transactional`). Effects can be composed through intersection types (e.g. + * `Write with Transactional`. The standard Slick back-ends do not restrict the evaluation of + * actions based on effects but they can be used in user-level code (e.g. for ensuring that all + * writes go to a master database but reads can also be performed by a slave). */ +trait Effect + +object Effect { + /** Effect for DBIOActions that read from the database ("DQL") */ + trait Read extends Effect + /** Effect for DBIOActions that write to the database ("DML") */ + trait Write extends Effect + /** Effect for DBIOActions that manipulate a database schema ("DDL") */ + trait Schema extends Effect + /** Effect for transactional DBIOActions ("DTL") */ + trait Transactional extends Effect + + /** The bottom type of all standard effects. It is used by the `DBIO` and `StreamingDBIO` + * type aliases instead of `Nothing` because the compiler does not properly infer `Nothing` + * where needed. You can still introduce your own custom effect types but they will not be + * used by `DBIO` and `StreamingDBIO`, so you either have to define your own type aliases + * or spell out the proper `DBIOAction` types in type annotations. */ + trait All extends Read with Write with Schema with Transactional +} diff --git a/src/main/scala/scala/slick/dbio/NoStream.scala b/src/main/scala/scala/slick/dbio/NoStream.scala new file mode 100644 index 0000000000..a13c53f71f --- /dev/null +++ b/src/main/scala/scala/slick/dbio/NoStream.scala @@ -0,0 +1,9 @@ +package scala.slick.dbio + +/** A phantom type used as the streaming result type for DBIOActions that do not support streaming. + * Note that this is a supertype of `Streaming` (and it is used in covariant position), + * so that any streaming action can be used where a non-streaming action is expected. */ +sealed trait NoStream + +/** A phantom type used as the streaming result type for DBIOActions that do support streaming. */ +sealed trait Streaming[+T] extends NoStream diff --git a/src/main/scala/scala/slick/dbio/package.scala b/src/main/scala/scala/slick/dbio/package.scala new file mode 100644 index 0000000000..6ff7666a3f --- /dev/null +++ b/src/main/scala/scala/slick/dbio/package.scala @@ -0,0 +1,11 @@ +package scala.slick + +/** The `dbio` package contains the Database I/O Action implementation. + * See [[DBIOAction]] for details. */ +package object dbio { + /** Simplified type for a streaming [[DBIOAction]] without effect tracking */ + type StreamingDBIO[+R, +T] = DBIOAction[R, Streaming[T], Effect.All] + + /** Simplified type for a [[DBIOAction]] without streaming or effect tracking */ + type DBIO[+R] = DBIOAction[R, NoStream, Effect.All] +} diff --git a/src/main/scala/scala/slick/driver/DerbyDriver.scala b/src/main/scala/scala/slick/driver/DerbyDriver.scala index e876ebf38c..693655c260 100644 --- a/src/main/scala/scala/slick/driver/DerbyDriver.scala +++ b/src/main/scala/scala/slick/driver/DerbyDriver.scala @@ -3,7 +3,7 @@ package scala.slick.driver import scala.concurrent.ExecutionContext import scala.slick.SlickException import scala.slick.lifted._ -import scala.slick.action._ +import scala.slick.dbio._ import scala.slick.ast._ import scala.slick.ast.TypeUtil._ import scala.slick.util.MacroSupport.macroSupportInterpolation @@ -93,7 +93,7 @@ trait DerbyDriver extends JdbcDriver { driver => override def createModelBuilder(tables: Seq[MTable], ignoreInvalidDefaults: Boolean)(implicit ec: ExecutionContext): JdbcModelBuilder = new ModelBuilder(tables, ignoreInvalidDefaults) - override def defaultTables(implicit ec: ExecutionContext): Action[Seq[MTable]] = + override def defaultTables(implicit ec: ExecutionContext): DBIO[Seq[MTable]] = MTable.getTables(None, None, None, Some(Seq("TABLE"))) override protected def computeQueryCompiler = super.computeQueryCompiler + Phase.rewriteBooleans + Phase.specializeParameters diff --git a/src/main/scala/scala/slick/driver/HsqldbDriver.scala b/src/main/scala/scala/slick/driver/HsqldbDriver.scala index ae05c5c605..732e17b1a9 100644 --- a/src/main/scala/scala/slick/driver/HsqldbDriver.scala +++ b/src/main/scala/scala/slick/driver/HsqldbDriver.scala @@ -3,7 +3,7 @@ package scala.slick.driver import java.sql.Types import scala.concurrent.ExecutionContext import scala.slick.SlickException -import scala.slick.action._ +import scala.slick.dbio._ import scala.slick.lifted._ import scala.slick.ast._ import scala.slick.util.MacroSupport.macroSupportInterpolation @@ -47,7 +47,7 @@ trait HsqldbDriver extends JdbcDriver { driver => override def createModelBuilder(tables: Seq[MTable], ignoreInvalidDefaults: Boolean)(implicit ec: ExecutionContext): JdbcModelBuilder = new ModelBuilder(tables, ignoreInvalidDefaults) - override def defaultTables(implicit ec: ExecutionContext): Action[Seq[MTable]] = + override def defaultTables(implicit ec: ExecutionContext): DBIO[Seq[MTable]] = MTable.getTables(None, None, None, Some(Seq("TABLE"))) override protected def computeQueryCompiler = super.computeQueryCompiler + Phase.specializeParameters diff --git a/src/main/scala/scala/slick/driver/JdbcActionComponent.scala b/src/main/scala/scala/slick/driver/JdbcActionComponent.scala index cd8f0b909d..2b9f34bde2 100644 --- a/src/main/scala/scala/slick/driver/JdbcActionComponent.scala +++ b/src/main/scala/scala/slick/driver/JdbcActionComponent.scala @@ -10,7 +10,7 @@ import scala.util.Try import scala.util.control.NonFatal import scala.slick.SlickException -import scala.slick.action._ +import scala.slick.dbio._ import scala.slick.ast._ import scala.slick.ast.Util._ import scala.slick.ast.TypeUtil.:@ @@ -75,26 +75,26 @@ trait JdbcActionComponent extends SqlActionComponent { driver: JdbcDriver => def getDumpInfo = DumpInfo(name = "SetTransactionIsolation") } - class JdbcActionExtensionMethods[E <: Effect, R, S <: NoStream](a: EffectfulAction[E, R, S]) { + class JdbcActionExtensionMethods[E <: Effect, R, S <: NoStream](a: DBIOAction[R, S, E]) { /** Run this Action transactionally. This does not guarantee failures to be atomic in the * presence of error handling combinators. If multiple `transactionally` combinators are * nested, only the outermost one will be backed by an actual database transaction. Depending * on the outcome of running the Action it surrounds, the transaction is committed if the * wrapped Action succeeds, or rolled back if the wrapped Action fails. When called on a - * [[scala.slick.action.SynchronousDatabaseAction]], this combinator gets fused into the + * [[scala.slick.dbio.SynchronousDatabaseAction]], this combinator gets fused into the * action. */ - def transactionally: EffectfulAction[E with Effect.Transactional, R, S] = SynchronousDatabaseAction.fuseUnsafe( - StartTransaction.andThen(a).cleanUp(eo => if(eo.isEmpty) Commit else Rollback)(Action.sameThreadExecutionContext) - .asInstanceOf[EffectfulAction[E with Effect.Transactional, R, S]] + def transactionally: DBIOAction[R, S, E with Effect.Transactional] = SynchronousDatabaseAction.fuseUnsafe( + StartTransaction.andThen(a).cleanUp(eo => if(eo.isEmpty) Commit else Rollback)(DBIO.sameThreadExecutionContext) + .asInstanceOf[DBIOAction[R, S, E with Effect.Transactional]] ) /** Run this Action with the specified transaction isolation level. This should be used around * the outermost `transactionally` Action. The semantics of using it inside a transaction are * database-dependent. It does not create a transaction by itself but it pins the session. */ - def withTransactionIsolation(ti: TransactionIsolation): EffectfulAction[E, R, S] = { + def withTransactionIsolation(ti: TransactionIsolation): DBIOAction[R, S, E] = { val isolated = - (new SetTransactionIsolation(ti.intValue)).flatMap(old => a.andFinally(new SetTransactionIsolation(old)))(Action.sameThreadExecutionContext) + (new SetTransactionIsolation(ti.intValue)).flatMap(old => a.andFinally(new SetTransactionIsolation(old)))(DBIO.sameThreadExecutionContext) val fused = if(a.isInstanceOf[SynchronousDatabaseAction[_, _, _, _]]) SynchronousDatabaseAction.fuseUnsafe(isolated) else isolated @@ -115,7 +115,7 @@ trait JdbcActionComponent extends SqlActionComponent { driver: JdbcDriver => def withStatementParameters(rsType: ResultSetType = null, rsConcurrency: ResultSetConcurrency = null, rsHoldability: ResultSetHoldability = null, - statementInit: Statement => Unit = null): EffectfulAction[E, R, S] = + statementInit: Statement => Unit = null): DBIOAction[R, S, E] = (new PushStatementParameters(JdbcBackend.StatementParameters(rsType, rsConcurrency, rsHoldability, statementInit))). andThen(a).andFinally(PopStatementParameters) } diff --git a/src/main/scala/scala/slick/driver/JdbcModelComponent.scala b/src/main/scala/scala/slick/driver/JdbcModelComponent.scala index 2b7a06c257..d5ec3f9075 100644 --- a/src/main/scala/scala/slick/driver/JdbcModelComponent.scala +++ b/src/main/scala/scala/slick/driver/JdbcModelComponent.scala @@ -1,19 +1,19 @@ package scala.slick.driver import scala.concurrent.{ExecutionContext, Future} -import scala.slick.action.Action +import scala.slick.dbio.DBIO import scala.slick.jdbc.JdbcModelBuilder import scala.slick.jdbc.meta.MTable import scala.slick.model.Model trait JdbcModelComponent { driver: JdbcDriver => /** Jdbc meta data for all tables included in the Slick model by default */ - def defaultTables(implicit ec: ExecutionContext): Action[Seq[MTable]] = MTable.getTables + def defaultTables(implicit ec: ExecutionContext): DBIO[Seq[MTable]] = MTable.getTables /** Gets the Slick data model describing this data source * @param tables used to build the model, uses defaultTables if None given * @param ignoreInvalidDefaults logs unrecognized default values instead of throwing an exception */ - def createModel(tables: Option[Action[Seq[MTable]]] = None, ignoreInvalidDefaults: Boolean = true)(implicit ec: ExecutionContext): Action[Model] = { + def createModel(tables: Option[DBIO[Seq[MTable]]] = None, ignoreInvalidDefaults: Boolean = true)(implicit ec: ExecutionContext): DBIO[Model] = { val tablesA = tables.getOrElse(defaultTables) tablesA.flatMap(t => createModelBuilder(t, ignoreInvalidDefaults).buildModel) } diff --git a/src/main/scala/scala/slick/driver/JdbcProfile.scala b/src/main/scala/scala/slick/driver/JdbcProfile.scala index 6b7a8f277c..3fa88fd810 100644 --- a/src/main/scala/scala/slick/driver/JdbcProfile.scala +++ b/src/main/scala/scala/slick/driver/JdbcProfile.scala @@ -71,8 +71,8 @@ trait JdbcProfile extends SqlProfile with JdbcActionComponent trait API extends LowPriorityAPI with super.API with CommonImplicits { type FastPath[T] = JdbcFastPath[T] - type SimpleAction[+R] = SimpleJdbcAction[R] - val SimpleAction = SimpleJdbcAction + type SimpleDBIO[+R] = SimpleJdbcAction[R] + val SimpleDBIO = SimpleJdbcAction implicit def queryDeleteActionExtensionMethods[C[_]](q: Query[_ <: Table[_], _, C]): DeleteActionExtensionMethods = createDeleteActionExtensionMethods(deleteCompiler.run(q.toNode).tree, ()) @@ -82,7 +82,7 @@ trait JdbcProfile extends SqlProfile with JdbcActionComponent implicit def runnableCompiledUpdateActionExtensionMethods[RU, C[_]](c: RunnableCompiled[_ <: Query[_, _, C], C[RU]]): UpdateActionExtensionMethods[RU] = createUpdateActionExtensionMethods(c.compiledUpdate, c.param) - implicit def jdbcActionExtensionMethods[E <: Effect, R, S <: NoStream](a: EffectfulAction[E, R, S]): JdbcActionExtensionMethods[E, R, S] = + implicit def jdbcActionExtensionMethods[E <: Effect, R, S <: NoStream](a: DBIOAction[R, S, E]): JdbcActionExtensionMethods[E, R, S] = new JdbcActionExtensionMethods[E, R, S](a) implicit def actionBasedSQLInterpolation(s: StringContext) = new ActionBasedSQLInterpolation(s) diff --git a/src/main/scala/scala/slick/driver/PostgresDriver.scala b/src/main/scala/scala/slick/driver/PostgresDriver.scala index 4177af35d1..0882889623 100644 --- a/src/main/scala/scala/slick/driver/PostgresDriver.scala +++ b/src/main/scala/scala/slick/driver/PostgresDriver.scala @@ -3,7 +3,7 @@ package scala.slick.driver import java.util.UUID import java.sql.{PreparedStatement, ResultSet} import scala.concurrent.ExecutionContext -import scala.slick.action._ +import scala.slick.dbio._ import scala.slick.lifted._ import scala.slick.profile.{SqlProfile, RelationalProfile, Capability} import scala.slick.ast.{SequenceNode, Library, FieldSymbol, Node, Insert, InsertColumn, Select, ElementSymbol, ColumnOption } @@ -85,7 +85,7 @@ trait PostgresDriver extends JdbcDriver { driver => override def createModelBuilder(tables: Seq[MTable], ignoreInvalidDefaults: Boolean)(implicit ec: ExecutionContext): JdbcModelBuilder = new ModelBuilder(tables, ignoreInvalidDefaults) - override def defaultTables(implicit ec: ExecutionContext): Action[Seq[MTable]] = + override def defaultTables(implicit ec: ExecutionContext): DBIO[Seq[MTable]] = MTable.getTables(None, None, None, Some(Seq("TABLE"))) override val columnTypes = new JdbcTypes diff --git a/src/main/scala/scala/slick/driver/SQLiteDriver.scala b/src/main/scala/scala/slick/driver/SQLiteDriver.scala index 38418a4b54..8bb716b538 100644 --- a/src/main/scala/scala/slick/driver/SQLiteDriver.scala +++ b/src/main/scala/scala/slick/driver/SQLiteDriver.scala @@ -3,7 +3,7 @@ package scala.slick.driver import java.sql.{Timestamp, Time, Date} import scala.concurrent.ExecutionContext import scala.slick.SlickException -import scala.slick.action._ +import scala.slick.dbio._ import scala.slick.lifted._ import scala.slick.ast._ import scala.slick.util.MacroSupport.macroSupportInterpolation @@ -120,7 +120,7 @@ trait SQLiteDriver extends JdbcDriver { driver => override def createModelBuilder(tables: Seq[MTable], ignoreInvalidDefaults: Boolean)(implicit ec: ExecutionContext): JdbcModelBuilder = new ModelBuilder(tables, ignoreInvalidDefaults) - override def defaultTables(implicit ec: ExecutionContext): Action[Seq[MTable]] = + override def defaultTables(implicit ec: ExecutionContext): DBIO[Seq[MTable]] = MTable.getTables(Some(""), Some(""), None, Some(Seq("TABLE"))) .map(_.filter(_.name.name.toLowerCase != "sqlite_sequence")) diff --git a/src/main/scala/scala/slick/jdbc/JdbcBackend.scala b/src/main/scala/scala/slick/jdbc/JdbcBackend.scala index ad37849da9..d898ce35a8 100644 --- a/src/main/scala/scala/slick/jdbc/JdbcBackend.scala +++ b/src/main/scala/scala/slick/jdbc/JdbcBackend.scala @@ -9,7 +9,7 @@ import java.sql.{Array => _, _} import javax.sql.DataSource import javax.naming.InitialContext -import scala.slick.action._ +import scala.slick.dbio._ import scala.slick.backend.{DatabasePublisher, DatabaseComponent, RelationalBackend} import scala.slick.SlickException import scala.slick.util.{LogUtil, GlobalConfig, SlickLogger, AsyncExecutor} @@ -47,7 +47,7 @@ trait JdbcBackend extends RelationalBackend { * `request()` more data. This allows you to process LOBs asynchronously by requesting only * one single element at a time after processing the current one, so that the proper * sequencing is preserved even though processing may happen on a different thread. */ - final def stream[T](a: StreamingAction[_, T], bufferNext: Boolean): DatabasePublisher[T] = + final def stream[T](a: StreamingDBIO[_, T], bufferNext: Boolean): DatabasePublisher[T] = createPublisher(a, s => new JdbcStreamingActionContext(s, false, DatabaseDef.this, bufferNext)) override protected[this] def createDatabaseActionContext[T](_useSameThread: Boolean): Context = @@ -467,6 +467,9 @@ trait JdbcBackend extends RelationalBackend { val p = statementParameters.head super.session.internalForParameters(p.rsType, p.rsConcurrency, p.rsHoldability, p.statementInit) } + + /** The current JDBC Connection */ + def connection: Connection = session.conn } class JdbcStreamingActionContext(subscriber: Subscriber[_], useSameThread: Boolean, database: Database, val bufferNext: Boolean) extends BasicStreamingActionContext(subscriber, useSameThread, database) with JdbcActionContext diff --git a/src/main/scala/scala/slick/jdbc/JdbcModelBuilder.scala b/src/main/scala/scala/slick/jdbc/JdbcModelBuilder.scala index 3ed309569e..406542f6e5 100644 --- a/src/main/scala/scala/slick/jdbc/JdbcModelBuilder.scala +++ b/src/main/scala/scala/slick/jdbc/JdbcModelBuilder.scala @@ -9,7 +9,7 @@ import scala.util.{Failure, Success} import java.sql.DatabaseMetaData import scala.slick.SlickException -import scala.slick.action._ +import scala.slick.dbio._ import scala.slick.ast.ColumnOption import scala.slick.jdbc.meta._ import scala.slick.{model => m} @@ -32,13 +32,13 @@ class JdbcModelBuilder(mTables: Seq[MTable], ignoreInvalidDefaults: Boolean)(imp ////////////////////////////////////////////////////////////////////// Actions for reading the required JDBC metadata /** Read the column metadata for a table in ordinal position order */ - def readColumns(t: MTable): Action[Vector[MColumn]] = t.getColumns.map(_.sortBy(_.ordinalPosition)) + def readColumns(t: MTable): DBIO[Vector[MColumn]] = t.getColumns.map(_.sortBy(_.ordinalPosition)) /** Read the primary key metadata for a table in key sequence order */ - def readPrimaryKeys(t: MTable): Action[Vector[MPrimaryKey]] = t.getPrimaryKeys.map(_.sortBy(_.keySeq)) + def readPrimaryKeys(t: MTable): DBIO[Vector[MPrimaryKey]] = t.getPrimaryKeys.map(_.sortBy(_.keySeq)) /** Read the foreign key metadata for a table grouped by name and in key sequence order */ - def readForeignKeys(t: MTable): Action[Seq[Seq[MForeignKey]]] = t.getImportedKeys.map( + def readForeignKeys(t: MTable): DBIO[Seq[Seq[MForeignKey]]] = t.getImportedKeys.map( // remove foreign keys pointing to tables which were not included _.filter(fk => tableNamersByQName.isDefinedAt(fk.pkTable)) .groupBy(fk => (fk.pkTable,fk.fkName,fk.pkName,fk.fkTable)) @@ -48,7 +48,7 @@ class JdbcModelBuilder(mTables: Seq[MTable], ignoreInvalidDefaults: Boolean)(imp ) /** Read the index metadata grouped by name and in ordinal position order */ - def readIndices(t: MTable): Action[Seq[Seq[MIndexInfo]]] = t.getIndexInfo().asTry.map { + def readIndices(t: MTable): DBIO[Seq[Seq[MIndexInfo]]] = t.getIndexInfo().asTry.map { case Success(iis) => iis.groupBy(_.indexName).toSeq.sortBy(_._1).map(_._2.sortBy(_.ordinalPosition)) // respect order case Failure(e: java.sql.SQLException) => // TODO: this needs a test! @@ -78,7 +78,7 @@ class JdbcModelBuilder(mTables: Seq[MTable], ignoreInvalidDefaults: Boolean)(imp /** Table model builder factory. Override for customization. * @group Basic customization overrides */ - def createTableBuilder(namer: TableNamer): Action[TableBuilder] = for { + def createTableBuilder(namer: TableNamer): DBIO[TableBuilder] = for { cs <- readColumns(namer.meta) pks <- readPrimaryKeys(namer.meta) fks <- readForeignKeys(namer.meta) @@ -87,8 +87,8 @@ class JdbcModelBuilder(mTables: Seq[MTable], ignoreInvalidDefaults: Boolean)(imp /** Creates a Slick data model from jdbc meta data. Foreign keys pointing out of the given tables * are not included. */ - def buildModel: Action[m.Model] = for { - ts <- Action.sequence(tableNamers.map(createTableBuilder)) + def buildModel: DBIO[m.Model] = for { + ts <- DBIO.sequence(tableNamers.map(createTableBuilder)) tablesByQName = ts.map(t => t.meta.name -> t).toMap builders = createBuilders(tablesByQName) } yield m.Model(ts.sortBy(_.meta.name.name).map(_.buildModel(builders))) diff --git a/src/main/scala/scala/slick/jdbc/ResultSetInvoker.scala b/src/main/scala/scala/slick/jdbc/ResultSetInvoker.scala index 462f888804..983173fbb1 100644 --- a/src/main/scala/scala/slick/jdbc/ResultSetInvoker.scala +++ b/src/main/scala/scala/slick/jdbc/ResultSetInvoker.scala @@ -1,7 +1,7 @@ package scala.slick.jdbc import java.sql.ResultSet -import scala.slick.action.{Effect, NoStream, SynchronousDatabaseAction} +import scala.slick.dbio.{Effect, NoStream, SynchronousDatabaseAction} import scala.slick.profile.BasicStreamingAction import scala.slick.util.CloseableIterator diff --git a/src/main/scala/scala/slick/jdbc/StaticQuery.scala b/src/main/scala/scala/slick/jdbc/StaticQuery.scala index 6f8bd54a9c..de1e7b97cd 100644 --- a/src/main/scala/scala/slick/jdbc/StaticQuery.scala +++ b/src/main/scala/scala/slick/jdbc/StaticQuery.scala @@ -5,7 +5,7 @@ import scala.collection.mutable.ArrayBuffer import java.sql.PreparedStatement -import scala.slick.action.Effect +import scala.slick.dbio.Effect import scala.slick.profile.SqlStreamingAction diff --git a/src/main/scala/scala/slick/jdbc/StreamingInvokerAction.scala b/src/main/scala/scala/slick/jdbc/StreamingInvokerAction.scala index af7bdd4b18..54e47af11f 100644 --- a/src/main/scala/scala/slick/jdbc/StreamingInvokerAction.scala +++ b/src/main/scala/scala/slick/jdbc/StreamingInvokerAction.scala @@ -3,7 +3,7 @@ package scala.slick.jdbc import scala.collection.mutable.Builder import scala.util.control.NonFatal -import scala.slick.action._ +import scala.slick.dbio._ import scala.slick.profile.{FixedSqlAction, FixedSqlStreamingAction} import scala.slick.util.{DumpInfo, CloseableIterator, ignoreFollowOnError} diff --git a/src/main/scala/scala/slick/jdbc/meta/MPrimaryKey.scala b/src/main/scala/scala/slick/jdbc/meta/MPrimaryKey.scala index 10e8f02e24..58b99dce16 100644 --- a/src/main/scala/scala/slick/jdbc/meta/MPrimaryKey.scala +++ b/src/main/scala/scala/slick/jdbc/meta/MPrimaryKey.scala @@ -1,6 +1,6 @@ package scala.slick.jdbc.meta -import scala.slick.action.Effect +import scala.slick.dbio.Effect import scala.slick.jdbc.{ResultSetAction, Invoker} import scala.slick.profile.BasicStreamingAction diff --git a/src/main/scala/scala/slick/jdbc/meta/MTable.scala b/src/main/scala/scala/slick/jdbc/meta/MTable.scala index e1926b86ad..b693ac1037 100644 --- a/src/main/scala/scala/slick/jdbc/meta/MTable.scala +++ b/src/main/scala/scala/slick/jdbc/meta/MTable.scala @@ -1,6 +1,6 @@ package scala.slick.jdbc.meta -import scala.slick.action.Effect +import scala.slick.dbio.Effect import scala.slick.jdbc.{ResultSetAction, Invoker} import scala.slick.profile.BasicStreamingAction diff --git a/src/main/scala/scala/slick/lifted/Aliases.scala b/src/main/scala/scala/slick/lifted/Aliases.scala index 6150f1ef88..e3734210ad 100644 --- a/src/main/scala/scala/slick/lifted/Aliases.scala +++ b/src/main/scala/scala/slick/lifted/Aliases.scala @@ -50,14 +50,14 @@ trait Aliases { @deprecated("Use Rep[T : TypedType] instead of Column[T]", "3.0") val Column = lifted.Rep - type Action[+R] = action.Action[R] - type StreamingAction[+R, +T] = action.StreamingAction[R, T] - type EffectfulAction[-E <: action.Effect, +R, +S <: action.NoStream] = action.EffectfulAction[E, R, S] - val Action = action.Action - type Effect = action.Effect - val Effect = action.Effect - type NoStream = action.NoStream - type Streaming[+T] = action.Streaming[T] + type Action[+R] = dbio.DBIO[R] + type StreamingAction[+R, +T] = dbio.StreamingDBIO[R, T] + type DBIOAction[+R, +S <: dbio.NoStream, -E <: dbio.Effect] = dbio.DBIOAction[R, S, E] + val Action = dbio.DBIO + type Effect = dbio.Effect + val Effect = dbio.Effect + type NoStream = dbio.NoStream + type Streaming[+T] = dbio.Streaming[T] type AsyncExecutor = util.AsyncExecutor val AsyncExecutor = util.AsyncExecutor } diff --git a/src/main/scala/scala/slick/memory/DistributedBackend.scala b/src/main/scala/scala/slick/memory/DistributedBackend.scala index 5cc1180260..ea3f58fae5 100644 --- a/src/main/scala/scala/slick/memory/DistributedBackend.scala +++ b/src/main/scala/scala/slick/memory/DistributedBackend.scala @@ -4,7 +4,7 @@ import org.reactivestreams.Subscriber import scala.concurrent.{ExecutionContext, Future, blocking} import scala.slick.SlickException -import scala.slick.action._ +import scala.slick.dbio._ import scala.slick.backend.{RelationalBackend, DatabaseComponent} import scala.slick.util.Logging import scala.collection.mutable.ArrayBuffer diff --git a/src/main/scala/scala/slick/memory/DistributedProfile.scala b/src/main/scala/scala/slick/memory/DistributedProfile.scala index dc0c24b49f..b85bcdd765 100644 --- a/src/main/scala/scala/slick/memory/DistributedProfile.scala +++ b/src/main/scala/scala/slick/memory/DistributedProfile.scala @@ -3,7 +3,7 @@ package scala.slick.memory import scala.language.{implicitConversions, existentials} import scala.collection.mutable.{Builder, HashMap} import scala.slick.SlickException -import scala.slick.action._ +import scala.slick.dbio._ import scala.slick.ast._ import scala.slick.ast.TypeUtil._ import scala.slick.compiler._ diff --git a/src/main/scala/scala/slick/memory/HeapBackend.scala b/src/main/scala/scala/slick/memory/HeapBackend.scala index 3da3da1082..a9098ad394 100644 --- a/src/main/scala/scala/slick/memory/HeapBackend.scala +++ b/src/main/scala/scala/slick/memory/HeapBackend.scala @@ -6,7 +6,7 @@ import org.reactivestreams.Subscriber import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.concurrent.{Future, ExecutionContext} import scala.slick.SlickException -import scala.slick.action._ +import scala.slick.dbio._ import scala.slick.ast._ import scala.slick.backend.{DatabaseComponent, RelationalBackend} import scala.slick.lifted.{PrimaryKey, Constraint, Index} diff --git a/src/main/scala/scala/slick/memory/MemoryProfile.scala b/src/main/scala/scala/slick/memory/MemoryProfile.scala index e16023c744..b895142622 100644 --- a/src/main/scala/scala/slick/memory/MemoryProfile.scala +++ b/src/main/scala/scala/slick/memory/MemoryProfile.scala @@ -5,7 +5,7 @@ import scala.collection.mutable.Builder import scala.reflect.ClassTag import scala.util.control.NonFatal -import scala.slick.action._ +import scala.slick.dbio._ import scala.slick.ast._ import TypeUtil._ import scala.slick.compiler._ diff --git a/src/main/scala/scala/slick/profile/BasicProfile.scala b/src/main/scala/scala/slick/profile/BasicProfile.scala index 3a4c61543c..fa19538e97 100644 --- a/src/main/scala/scala/slick/profile/BasicProfile.scala +++ b/src/main/scala/scala/slick/profile/BasicProfile.scala @@ -4,7 +4,7 @@ import scala.language.{higherKinds, implicitConversions, existentials} import scala.slick.SlickException import scala.slick.compiler.QueryCompiler import scala.slick.backend.DatabaseComponent -import scala.slick.action._ +import scala.slick.dbio._ import scala.slick.ast._ import scala.slick.lifted._ import com.typesafe.config.{ConfigFactory, Config} @@ -239,7 +239,7 @@ trait BasicActionComponent { driver: BasicDriver => } } -trait BasicAction[-E <: Effect, +R, +S <: NoStream] extends DatabaseAction[E, R, S] { +trait BasicAction[-E <: Effect, +R, +S <: NoStream] extends DatabaseAction[R, S, E] { type ResultAction[-E <: Effect, +R, +S <: NoStream] <: BasicAction[E, R, S] } diff --git a/src/main/scala/scala/slick/profile/RelationalProfile.scala b/src/main/scala/scala/slick/profile/RelationalProfile.scala index e1e9929472..ed8a2050a9 100644 --- a/src/main/scala/scala/slick/profile/RelationalProfile.scala +++ b/src/main/scala/scala/slick/profile/RelationalProfile.scala @@ -1,7 +1,7 @@ package scala.slick.profile import scala.language.{implicitConversions, higherKinds, existentials} -import scala.slick.action._ +import scala.slick.dbio._ import scala.slick.ast._ import scala.slick.backend.RelationalBackend import scala.slick.lifted._ diff --git a/src/main/scala/scala/slick/profile/SqlProfile.scala b/src/main/scala/scala/slick/profile/SqlProfile.scala index 125f0c65fd..9f2db023c9 100644 --- a/src/main/scala/scala/slick/profile/SqlProfile.scala +++ b/src/main/scala/scala/slick/profile/SqlProfile.scala @@ -1,7 +1,7 @@ package scala.slick.profile import scala.language.higherKinds -import scala.slick.action._ +import scala.slick.dbio._ import scala.slick.ast.{TableNode, Symbol, SymbolNamer, Node, ColumnOption} import scala.slick.lifted.AbstractTable import scala.slick.util.DumpInfo diff --git a/src/sphinx/code/Connection.scala b/src/sphinx/code/Connection.scala index 031f724b80..192d211b4f 100644 --- a/src/sphinx/code/Connection.scala +++ b/src/sphinx/code/Connection.scala @@ -101,6 +101,6 @@ object Connection extends App { } finally db.close //#simpleaction - val getAutoCommit = SimpleAction[Boolean](_.session.conn.getAutoCommit) + val getAutoCommit = SimpleDBIO[Boolean](_.connection.getAutoCommit) //#simpleaction } diff --git a/src/sphinx/database.rst b/src/sphinx/database.rst index f4a3e839f3..ebc48ac08c 100644 --- a/src/sphinx/database.rst +++ b/src/sphinx/database.rst @@ -1,17 +1,21 @@ -Databases & Actions -=================== +Databases & I/O Actions +======================= Anything that you can execute on a database, whether it is a getting the result of a query ("``myQuery.result``"), creating a table ("``myTable.schema.create``"), inserting data ("``myTable += item``") or something else, is an instance of -:api:`scala.slick.action.EffectfulAction`, parameterized by the result type it will produce when you +:api:`scala.slick.dbio.DBIOAction`, parameterized by the result type it will produce when you execute it. -Actions can be combined with several different combinators (see the -:api:`EffectfulAction class ` and :api:`Action object ` +*Database I/O Actions* can be combined with several different combinators (see the +:api:`DBIOAction class ` and :api:`DBIO object ` for details), but they will always be executed strictly sequentially and (at least conceptually) in a single database session. +In most cases you will want to use the type aliases :api:`DBIO ` +and :api:`StreamingDBIO ` for non-streaming and +streaming Database I/O Actions. They omit the optional *effect types* supported by :api:`scala.slick.dbio.DBIOAction`. + .. index:: pair: database; configuration pair: database; pool @@ -45,25 +49,23 @@ of this method for details on the configuration parameters). Using a JDBC URL ________________ -You can provide a JDBC URL to +You can pass a JDBC URL to :api:`forURL `. (see your database's JDBC driver's documentation for the correct URL syntax). .. includecode:: code/Connection.scala#forURL -Here we are connecting to a new, empty, in-memory H2 database called ``test1`` -and keep it resident until the JVM ends (``DB_CLOSE_DELAY=-1``, which is H2 -specific). +Here we are connecting to a new, empty, in-memory H2 database called ``test1`` and keep it resident +until the JVM ends (``DB_CLOSE_DELAY=-1``, which is H2 specific). .. index:: DataSource Using a DataSource __________________ -You can provide a :javaapi:`DataSource ` object to +You can pass a :javaapi:`DataSource ` object to :api:`forDataSource `. -If you got it from the connection pool of your application framework, this -plugs the pool into Slick. +If you got it from the connection pool of your application framework, this plugs the pool into Slick. .. includecode:: code/Connection.scala#forDataSource @@ -72,10 +74,9 @@ plugs the pool into Slick. Using a JNDI Name _________________ -If you are using :wikipedia:`JNDI` you can provide a JNDI name to +If you are using :wikipedia:`JNDI` you can pass a JNDI name to :api:`forName ` -under which a -:javaapi:`DataSource ` object can be looked up. +under which a :javaapi:`DataSource ` object can be looked up. .. includecode:: code/Connection.scala#forName @@ -86,7 +87,7 @@ Database thread pool -------------------- Every ``Database`` contains an :api:`scala.slick.util.AsyncExecutor` that manages the thread pool -for asynchronous execution of database Actions. Its size is the main parameter to tune for the best +for asynchronous execution of Database I/O Actions. Its size is the main parameter to tune for the best performance of the ``Database`` object. It should be set to the value that you would use for the size of the *connection pool* in a traditional, blocking application (see `About Pool Sizing`_ in the HikariCP_ documentation for further information). When using @@ -122,10 +123,10 @@ should therefore enable prepared statement caching in the connection pool's conf .. _executing-actions: -Executing Actions ------------------ +Executing Database I/O Actions +------------------------------ -Actions can be executed either with the goal of producing a fully materialized result or streaming +DBIOActions can be executed either with the goal of producing a fully materialized result or streaming data back from the database. .. index:: materialize @@ -133,12 +134,12 @@ data back from the database. Materialized ____________ -You can use ``run`` to execute an Action on a Database and produce a materialized result. This can +You can use ``run`` to execute a DBIOAction on a Database and produce a materialized result. This can be, for example, a scalar query result ("``myTable.length.result``"), a collection-valued query -result ("``myTable.to[Set].result``"), or any other Action. Every Action supports this mode of +result ("``myTable.to[Set].result``"), or any other action. Every DBIOAction supports this mode of execution. -Execution of the Action starts when ``run`` is called, and the materialized result is returned as a +Execution of the DBIOAction starts when ``run`` is called, and the materialized result is returned as a ``Future`` which is completed asynchronously as soon as the result is available: .. includecode:: code/Connection.scala#materialize @@ -152,9 +153,9 @@ Collection-valued queries also support streaming results. In this case, the actu is ignored and elements are streamed directly from the result set through a `Reactive Streams`_ ``Publisher``, which can be processed and consumed by `Akka Streams`_. -Execution of the Action does not start until a Subscriber is attached to the stream. Only a single +Execution of the DBIOAction does not start until a ``Subscriber`` is attached to the stream. Only a single Subscriber is supported, and any further attempts to subscribe again will fail. Stream elements are -signaled as soon as they become available in the streaming part of the Action. The end of the +signaled as soon as they become available in the streaming part of the DBIOAction. The end of the stream is signaled only after the entire Action has completed. For example, when streaming inside a transaction and all elements have been delivered successfully, the stream can still fail afterwards if the transaction cannot be committed. @@ -177,26 +178,26 @@ convenience method ``mapResult`` is provided for this purpose: Transactions and Pinned Sessions ________________________________ -When executing an Action that is composed of several smaller Actions, Slick acquires sessions from +When executing a DBIOAction that is composed of several smaller actions, Slick acquires sessions from the connection pool and releases them again as needed so that a session is not kept in use unnecessarily while waiting for the result from a non-database computation (e.g. the function passed to -:api:`flatMap ` -that determines the next Action to run). All :api:`Action combinators ` -which combine two database Actions without any non-database computations in between (e.g. -:api:`andThen ` -or :api:`zip `) -can fuse these Actions for more efficient execution, with the side-effect that the fused Action +:api:`flatMap ` +that determines the next Action to run). All :api:`DBIOAction combinators ` +which combine two database actions without any non-database computations in between (e.g. +:api:`andThen ` +or :api:`zip `) +can fuse these actions for more efficient execution, with the side-effect that the fused action runs inside a single session. You can use -:api:`withPinnedSession ` to force the +:api:`withPinnedSession ` to force the use of a single session, keeping the existing session open even when waiting for non-database computations. -There is a similar combinator -:api:`transactionally ` -to force the use of a transaction. This guarantees that the entire Action that is executed will +There is a similar combinator called +:api:`transactionally ` +to force the use of a transaction. This guarantees that the entire DBIOAction that is executed will either succeed or fail atomically. Note that failure is not guaranteed to be atomic at the level -of an individual Action that is wrapped with ``.transactionally``, so you should not apply error +of an individual DBIOAction that is wrapped with ``.transactionally``, so you should not apply error recovery combinators at that point. .. includecode:: code/Connection.scala#transaction @@ -208,6 +209,6 @@ JDBC Interoperability --------------------- In order to drop down to the JDBC level for functionality that is not available in Slick, you can -use a ``SimpleAction`` which is run on a database thread and gets access to the JDBC ``Connection``: +use a ``SimpleDBIO`` action which is run on a database thread and gets access to the JDBC ``Connection``: .. includecode:: code/Connection.scala#simpleaction diff --git a/src/sphinx/queries.rst b/src/sphinx/queries.rst index d6966dbe12..978ab531af 100644 --- a/src/sphinx/queries.rst +++ b/src/sphinx/queries.rst @@ -150,7 +150,7 @@ as done in ``q2``. Querying -------- -A Query can be converted into an :api:`Action ` by calling its +A Query can be converted into an :api:`Action ` by calling its ``result`` method. The Action can then be :ref:`executed ` directly in a streaming or fully materialized way, or composed further with other Actions: diff --git a/src/sphinx/upgrade.rst b/src/sphinx/upgrade.rst index c72933939d..79141fc056 100644 --- a/src/sphinx/upgrade.rst +++ b/src/sphinx/upgrade.rst @@ -6,13 +6,13 @@ Upgrade guides Upgrade from 2.1 to 3.0 ======================= -Actions -------- +Database I/O Actions +-------------------- The ``simple`` and ``Implicits`` imports from drivers are deprecated. You should use ``api`` instead, which will give you the same features, except for the old ``Invoker`` and ``Executor`` APIs for blocking execution of database calls. -These have been replaced by a new monadic database actions API. See :doc:`Databases & Actions ` for details -of the new API. +These have been replaced by a new monadic database I/O actions API. See :doc:`Databases & I/O Actions ` for +details of the new API. Join Operators --------------