diff --git a/build.sbt b/build.sbt
index a18382b9c..dd1165045 100644
--- a/build.sbt
+++ b/build.sbt
@@ -26,7 +26,8 @@ addCommandAlias("check", "all scalafmtSbtCheck scalafmtCheck test:scalafmtCheck"
val zioVersion = "2.0.0-RC6"
val zioSchemaVersion = "0.1.9"
val testcontainersVersion = "1.17.2"
-val testcontainersScalaVersion = "0.40.7"
+val testcontainersScalaVersion = "0.40.8"
+val logbackVersion = "1.2.11"
lazy val root = project
.in(file("."))
@@ -161,7 +162,8 @@ lazy val mysql = project
"org.testcontainers" % "jdbc" % testcontainersVersion % Test,
"org.testcontainers" % "mysql" % testcontainersVersion % Test,
"mysql" % "mysql-connector-java" % "8.0.29" % Test,
- "com.dimafeng" %% "testcontainers-scala-mysql" % testcontainersScalaVersion % Test
+ "com.dimafeng" %% "testcontainers-scala-mysql" % testcontainersScalaVersion % Test,
+ "ch.qos.logback" % "logback-classic" % logbackVersion % Test
)
)
.settings(testFrameworks += new TestFramework("zio.test.sbt.ZTestFramework"))
@@ -179,7 +181,8 @@ lazy val oracle = project
"org.testcontainers" % "oracle-xe" % testcontainersVersion % Test,
"org.testcontainers" % "jdbc" % testcontainersVersion % Test,
"com.oracle.database.jdbc" % "ojdbc8" % "21.5.0.0" % Test,
- "com.dimafeng" %% "testcontainers-scala-oracle-xe" % testcontainersScalaVersion % Test
+ "com.dimafeng" %% "testcontainers-scala-oracle-xe" % testcontainersScalaVersion % Test,
+ "ch.qos.logback" % "logback-classic" % logbackVersion % Test
)
)
.settings(testFrameworks += new TestFramework("zio.test.sbt.ZTestFramework"))
@@ -197,7 +200,8 @@ lazy val postgres = project
"org.testcontainers" % "postgresql" % testcontainersVersion % Test,
"org.testcontainers" % "jdbc" % testcontainersVersion % Test,
"org.postgresql" % "postgresql" % "42.3.6" % Compile,
- "com.dimafeng" %% "testcontainers-scala-postgresql" % testcontainersScalaVersion % Test
+ "com.dimafeng" %% "testcontainers-scala-postgresql" % testcontainersScalaVersion % Test,
+ "ch.qos.logback" % "logback-classic" % logbackVersion % Test
)
)
.settings(testFrameworks += new TestFramework("zio.test.sbt.ZTestFramework"))
@@ -215,7 +219,8 @@ lazy val sqlserver = project
"org.testcontainers" % "mssqlserver" % testcontainersVersion % Test,
"org.testcontainers" % "jdbc" % testcontainersVersion % Test,
"com.microsoft.sqlserver" % "mssql-jdbc" % "9.4.0.jre8" % Test,
- "com.dimafeng" %% "testcontainers-scala-mssqlserver" % testcontainersScalaVersion % Test
+ "com.dimafeng" %% "testcontainers-scala-mssqlserver" % testcontainersScalaVersion % Test,
+ "ch.qos.logback" % "logback-classic" % logbackVersion % Test
)
)
.settings(testFrameworks += new TestFramework("zio.test.sbt.ZTestFramework"))
diff --git a/jdbc/src/main/scala/zio/sql/SqlDriverLiveModule.scala b/jdbc/src/main/scala/zio/sql/SqlDriverLiveModule.scala
index 1b47c5af7..716bb6cce 100644
--- a/jdbc/src/main/scala/zio/sql/SqlDriverLiveModule.scala
+++ b/jdbc/src/main/scala/zio/sql/SqlDriverLiveModule.scala
@@ -9,6 +9,12 @@ import zio.schema.Schema
trait SqlDriverLiveModule { self: Jdbc =>
private[sql] trait SqlDriverCore {
+ def deleteOnBatch(delete: List[Delete[_]], conn: Connection): IO[Exception, List[Int]]
+
+ def updateOnBatch(update: List[Update[_]], conn: Connection): IO[Exception, List[Int]]
+
+ def insertOnBatch[A: Schema](insert: List[Insert[_, A]], conn: Connection): IO[Exception, List[Int]]
+
def deleteOn(delete: Delete[_], conn: Connection): IO[Exception, Int]
def updateOn(update: Update[_], conn: Connection): IO[Exception, Int]
@@ -22,6 +28,9 @@ trait SqlDriverLiveModule { self: Jdbc =>
def delete(delete: Delete[_]): IO[Exception, Int] =
ZIO.scoped(pool.connection.flatMap(deleteOn(delete, _)))
+ def delete(delete: List[Delete[_]]): IO[Exception, List[Int]] =
+ ZIO.scoped(pool.connection.flatMap(deleteOnBatch(delete, _)))
+
def deleteOn(delete: Delete[_], conn: Connection): IO[Exception, Int] =
ZIO.attemptBlocking {
val query = renderDelete(delete)
@@ -29,18 +38,31 @@ trait SqlDriverLiveModule { self: Jdbc =>
statement.executeUpdate(query)
}.refineToOrDie[Exception]
+ def deleteOnBatch(delete: List[Delete[_]], conn: Connection): IO[Exception, List[Int]] =
+ ZIO.attemptBlocking {
+ val statement = conn.createStatement()
+ delete.map(delete_ => statement.addBatch(renderDelete(delete_)))
+ statement.executeBatch().toList
+ }.refineToOrDie[Exception]
+
def update(update: Update[_]): IO[Exception, Int] =
ZIO.scoped(pool.connection.flatMap(updateOn(update, _)))
def updateOn(update: Update[_], conn: Connection): IO[Exception, Int] =
ZIO.attemptBlocking {
-
- val query = renderUpdate(update)
-
+ val query = renderUpdate(update)
val statement = conn.createStatement()
-
statement.executeUpdate(query)
+ }.refineToOrDie[Exception]
+
+ def update(update: List[Update[_]]): IO[Exception, List[Int]] =
+ ZIO.scoped(pool.connection.flatMap(updateOnBatch(update, _)))
+ def updateOnBatch(update: List[Update[_]], conn: Connection): IO[Exception, List[Int]] =
+ ZIO.attemptBlocking {
+ val statement = conn.createStatement()
+ update.map(update_ => statement.addBatch(renderUpdate(update_)))
+ statement.executeBatch().toList
}.refineToOrDie[Exception]
def read[A](read: Read[A]): Stream[Exception, A] =
@@ -93,9 +115,19 @@ trait SqlDriverLiveModule { self: Jdbc =>
statement.executeUpdate(query)
}.refineToOrDie[Exception]
+ override def insertOnBatch[A: Schema](insert: List[Insert[_, A]], conn: Connection): IO[Exception, List[Int]] =
+ ZIO.attemptBlocking {
+ val statement = conn.createStatement()
+ insert.map(insert_ => statement.addBatch(renderInsert(insert_)))
+ statement.executeBatch().toList
+ }.refineToOrDie[Exception]
+
override def insert[A: Schema](insert: Insert[_, A]): IO[Exception, Int] =
ZIO.scoped(pool.connection.flatMap(insertOn(insert, _)))
+ def insert[A: Schema](insert: List[Insert[_, A]]): IO[Exception, List[Int]] =
+ ZIO.scoped(pool.connection.flatMap(insertOnBatch(insert, _)))
+
override def transact[R, A](tx: ZTransaction[R, Exception, A]): ZIO[R, Throwable, A] =
ZIO.scoped[R] {
for {
diff --git a/jdbc/src/main/scala/zio/sql/TransactionModule.scala b/jdbc/src/main/scala/zio/sql/TransactionModule.scala
index e4c0876d7..c33e60bb6 100644
--- a/jdbc/src/main/scala/zio/sql/TransactionModule.scala
+++ b/jdbc/src/main/scala/zio/sql/TransactionModule.scala
@@ -87,16 +87,31 @@ trait TransactionModule { self: Jdbc =>
ZTransaction.fromEffect(coreDriver.updateOn(update, connection))
}
+ def batchUpdate(update: List[self.Update[_]]): ZTransaction[Any, Exception, List[Int]] =
+ txn.flatMap { case Txn(connection, coreDriver) =>
+ ZTransaction.fromEffect(coreDriver.updateOnBatch(update, connection))
+ }
+
def apply[Z: Schema](insert: self.Insert[_, Z]): ZTransaction[Any, Exception, Int] =
txn.flatMap { case Txn(connection, coreDriver) =>
ZTransaction.fromEffect(coreDriver.insertOn(insert, connection))
}
+ def batchInsert[Z: Schema](insert: List[self.Insert[_, Z]]): ZTransaction[Any, Exception, List[Int]] =
+ txn.flatMap { case Txn(connection, coreDriver) =>
+ ZTransaction.fromEffect(coreDriver.insertOnBatch(insert, connection))
+ }
+
def apply(delete: self.Delete[_]): ZTransaction[Any, Exception, Int] =
txn.flatMap { case Txn(connection, coreDriver) =>
ZTransaction.fromEffect(coreDriver.deleteOn(delete, connection))
}
+ def batchDelete(delete: List[self.Delete[_]]): ZTransaction[Any, Exception, List[Int]] =
+ txn.flatMap { case Txn(connection, coreDriver) =>
+ ZTransaction.fromEffect(coreDriver.deleteOnBatch(delete, connection))
+ }
+
def succeed[A](a: => A): ZTransaction[Any, Nothing, A] = fromEffect(ZIO.succeed(a))
def fail[E](e: => E): ZTransaction[Any, E, Nothing] = fromEffect(ZIO.fail(e))
diff --git a/jdbc/src/main/scala/zio/sql/jdbc.scala b/jdbc/src/main/scala/zio/sql/jdbc.scala
index e1a314506..cd4f6c044 100644
--- a/jdbc/src/main/scala/zio/sql/jdbc.scala
+++ b/jdbc/src/main/scala/zio/sql/jdbc.scala
@@ -8,15 +8,22 @@ trait Jdbc extends zio.sql.Sql with TransactionModule with JdbcInternalModule wi
trait SqlDriver {
def delete(delete: Delete[_]): IO[Exception, Int]
+ def delete(delete: List[Delete[_]]): IO[Exception, List[Int]]
+
def update(update: Update[_]): IO[Exception, Int]
+ def update(update: List[Update[_]]): IO[Exception, List[Int]]
+
def read[A](read: Read[A]): Stream[Exception, A]
def transact[R, A](tx: ZTransaction[R, Exception, A]): ZIO[R, Throwable, A]
def insert[A: Schema](insert: Insert[_, A]): IO[Exception, Int]
+
+ def insert[A: Schema](insert: List[Insert[_, A]]): IO[Exception, List[Int]]
}
object SqlDriver {
+
val live: ZLayer[ConnectionPool, Nothing, SqlDriver] =
ZLayer(ZIO.serviceWith[ConnectionPool](new SqlDriverLive(_)))
}
@@ -32,9 +39,18 @@ trait Jdbc extends zio.sql.Sql with TransactionModule with JdbcInternalModule wi
def execute(delete: Delete[_]): ZIO[SqlDriver, Exception, Int] =
ZIO.serviceWithZIO(_.delete(delete))
+ def executeBatchDelete(delete: List[Delete[_]]): ZIO[SqlDriver, Exception, List[Int]] =
+ ZIO.serviceWithZIO(_.delete(delete))
+
def execute[A: Schema](insert: Insert[_, A]): ZIO[SqlDriver, Exception, Int] =
ZIO.serviceWithZIO(_.insert(insert))
+ def executeBatchInsert[A: Schema](insert: List[Insert[_, A]]): ZIO[SqlDriver, Exception, List[Int]] =
+ ZIO.serviceWithZIO(_.insert(insert))
+
def execute(update: Update[_]): ZIO[SqlDriver, Exception, Int] =
ZIO.serviceWithZIO(_.update(update))
+
+ def executeBatchUpdate(update: List[Update[_]]): ZIO[SqlDriver, Exception, List[Int]] =
+ ZIO.serviceWithZIO(_.update(update))
}
diff --git a/jdbc/src/test/scala/zio/sql/JdbcRunnableSpec.scala b/jdbc/src/test/scala/zio/sql/JdbcRunnableSpec.scala
index 59116b1c4..86ccfb653 100644
--- a/jdbc/src/test/scala/zio/sql/JdbcRunnableSpec.scala
+++ b/jdbc/src/test/scala/zio/sql/JdbcRunnableSpec.scala
@@ -1,19 +1,68 @@
package zio.sql
+import com.dimafeng.testcontainers.JdbcDatabaseContainer
import zio.test.TestEnvironment
-import zio.ZLayer
+import zio.{ Scope, ZIO, ZLayer }
import zio.test.ZIOSpecDefault
+import com.dimafeng.testcontainers.SingleContainer
+import java.util.Properties
+import zio.test.Spec
+/**
+ * Base trait for integration-style tests running on Testcontainers.
+ * Extending classes are expected to provide the container implementation
+ * this test suite will work on by implementing {@link getContainer}.
+ *
+ * Test suite should be implemented in {@link specLayered} and
+ * particular tests can depend on {@link SQLDriver} in the environment.
+ */
trait JdbcRunnableSpec extends ZIOSpecDefault with Jdbc {
type JdbcEnvironment = TestEnvironment with SqlDriver
- val poolConfigLayer: ZLayer[Any, Throwable, ConnectionPoolConfig]
+ def specLayered: Spec[JdbcEnvironment, Object]
- final lazy val jdbcLayer: ZLayer[Any, Any, SqlDriver] =
+ protected def getContainer: SingleContainer[_] with JdbcDatabaseContainer
+
+ protected val autoCommit = false
+
+ override def spec: Spec[TestEnvironment, Any] =
+ specLayered.provideCustomShared(jdbcLayer)
+
+ private[this] def connProperties(user: String, password: String): Properties = {
+ val props = new Properties
+ props.setProperty("user", user)
+ props.setProperty("password", password)
+ props
+ }
+
+ private[this] val poolConfigLayer: ZLayer[Any, Throwable, ConnectionPoolConfig] =
+ ZLayer.scoped {
+ testContainer
+ .map(a =>
+ ConnectionPoolConfig(
+ url = a.jdbcUrl,
+ properties = connProperties(a.username, a.password),
+ autoCommit = autoCommit
+ )
+ )
+ }
+
+ private[this] final lazy val jdbcLayer: ZLayer[Any, Any, SqlDriver] =
ZLayer.make[SqlDriver](
poolConfigLayer.orDie,
ConnectionPool.live.orDie,
SqlDriver.live
)
+
+ private[this] def testContainer: ZIO[Scope, Throwable, SingleContainer[_] with JdbcDatabaseContainer] =
+ ZIO.acquireRelease {
+ ZIO.attemptBlocking {
+ val c = getContainer
+ c.start()
+ c
+ }
+ } { container =>
+ ZIO.attemptBlocking(container.stop()).orDie
+ }
}
diff --git a/mysql/src/test/resources/logback.xml b/mysql/src/test/resources/logback.xml
new file mode 100644
index 000000000..1378a823a
--- /dev/null
+++ b/mysql/src/test/resources/logback.xml
@@ -0,0 +1,14 @@
+
+
+
+ %d{HH:mm:ss.SSS} [%thread] %-5level %logger - %msg%n
+
+
+
+
+
+
+
+
+
+
diff --git a/mysql/src/test/scala/zio/sql/mysql/MysqlRunnableSpec.scala b/mysql/src/test/scala/zio/sql/mysql/MysqlRunnableSpec.scala
index 3878172b5..1a97a1a2e 100644
--- a/mysql/src/test/scala/zio/sql/mysql/MysqlRunnableSpec.scala
+++ b/mysql/src/test/scala/zio/sql/mysql/MysqlRunnableSpec.scala
@@ -1,29 +1,17 @@
package zio.sql.mysql
-import java.util.Properties
-import zio.sql.{ ConnectionPoolConfig, JdbcRunnableSpec }
-import zio.test._
-import zio.ZLayer
+import com.dimafeng.testcontainers.{ JdbcDatabaseContainer, MySQLContainer, SingleContainer }
+import org.testcontainers.utility.DockerImageName
+import zio.sql.JdbcRunnableSpec
trait MysqlRunnableSpec extends JdbcRunnableSpec with MysqlJdbcModule {
- private def connProperties(user: String, password: String): Properties = {
- val props = new Properties
- props.setProperty("user", user)
- props.setProperty("password", password)
- props
- }
-
- val poolConfigLayer: ZLayer[Any, Throwable, ConnectionPoolConfig] =
- ZLayer.scoped {
- TestContainer
- .mysql()
- .map(a => ConnectionPoolConfig(a.jdbcUrl, connProperties(a.username, a.password)))
+ override protected def getContainer: SingleContainer[_] with JdbcDatabaseContainer =
+ new MySQLContainer(
+ mysqlImageVersion = Option("mysql").map(DockerImageName.parse)
+ ).configure { a =>
+ a.withInitScript("shop_schema.sql")
+ ()
}
- override def spec: Spec[TestEnvironment, Any] =
- specLayered.provideCustomLayerShared(jdbcLayer)
-
- def specLayered: Spec[JdbcEnvironment, Object]
-
}
diff --git a/mysql/src/test/scala/zio/sql/mysql/TestContainer.scala b/mysql/src/test/scala/zio/sql/mysql/TestContainer.scala
deleted file mode 100644
index 67d778806..000000000
--- a/mysql/src/test/scala/zio/sql/mysql/TestContainer.scala
+++ /dev/null
@@ -1,22 +0,0 @@
-package zio.sql.mysql
-
-import com.dimafeng.testcontainers.MySQLContainer
-import org.testcontainers.utility.DockerImageName
-import zio._
-
-object TestContainer {
-
- def mysql(imageName: String = "mysql"): ZIO[Scope, Throwable, MySQLContainer] =
- ZIO.acquireRelease {
- ZIO.attemptBlocking {
- val c = new MySQLContainer(
- mysqlImageVersion = Option(imageName).map(DockerImageName.parse)
- ).configure { a =>
- a.withInitScript("shop_schema.sql")
- ()
- }
- c.start()
- c
- }
- }(container => ZIO.attemptBlocking(container.stop()).orDie)
-}
diff --git a/oracle/src/test/resources/logback.xml b/oracle/src/test/resources/logback.xml
new file mode 100644
index 000000000..1378a823a
--- /dev/null
+++ b/oracle/src/test/resources/logback.xml
@@ -0,0 +1,14 @@
+
+
+
+ %d{HH:mm:ss.SSS} [%thread] %-5level %logger - %msg%n
+
+
+
+
+
+
+
+
+
+
diff --git a/oracle/src/test/scala/zio/sql/oracle/OracleRunnableSpec.scala b/oracle/src/test/scala/zio/sql/oracle/OracleRunnableSpec.scala
index a964bf708..fcf4a2c2f 100644
--- a/oracle/src/test/scala/zio/sql/oracle/OracleRunnableSpec.scala
+++ b/oracle/src/test/scala/zio/sql/oracle/OracleRunnableSpec.scala
@@ -1,34 +1,17 @@
package zio.sql.oracle
-import zio.ZLayer
-import zio.sql.{ ConnectionPoolConfig, JdbcRunnableSpec }
-import zio.test.{ Spec, TestEnvironment }
-
-import java.util.Properties
+import com.dimafeng.testcontainers.{ JdbcDatabaseContainer, OracleContainer, SingleContainer }
+import org.testcontainers.utility.DockerImageName
+import zio.sql.JdbcRunnableSpec
trait OracleRunnableSpec extends JdbcRunnableSpec with OracleJdbcModule {
- private def connProperties(user: String, password: String): Properties = {
- val props = new Properties
- props.setProperty("user", user)
- props.setProperty("password", password)
- props
- }
-
- val poolConfigLayer = ZLayer.scoped {
- TestContainer
- .oracle()
- .map(container =>
- ConnectionPoolConfig(
- url = container.jdbcUrl,
- properties = connProperties(container.username, container.password)
- )
- )
- }
-
- override def spec: Spec[TestEnvironment, Any] =
- specLayered.provideCustomShared(jdbcLayer)
-
- def specLayered: Spec[JdbcEnvironment, Object]
+ override protected def getContainer: SingleContainer[_] with JdbcDatabaseContainer =
+ new OracleContainer(
+ dockerImageName = DockerImageName.parse("gvenzl/oracle-xe")
+ ).configure { container =>
+ container.withInitScript("shop_schema.sql")
+ ()
+ }
}
diff --git a/oracle/src/test/scala/zio/sql/oracle/TestContainer.scala b/oracle/src/test/scala/zio/sql/oracle/TestContainer.scala
deleted file mode 100644
index 74577d433..000000000
--- a/oracle/src/test/scala/zio/sql/oracle/TestContainer.scala
+++ /dev/null
@@ -1,22 +0,0 @@
-package zio.sql.oracle
-
-import com.dimafeng.testcontainers.OracleContainer
-import org.testcontainers.utility.DockerImageName
-import zio.{ Scope, ZIO }
-
-object TestContainer {
-
- def oracle(imageName: String = "gvenzl/oracle-xe"): ZIO[Scope, Throwable, OracleContainer] =
- ZIO.acquireRelease {
- ZIO.attemptBlocking {
- val c = new OracleContainer(
- dockerImageName = DockerImageName.parse(imageName)
- ).configure { container =>
- container.withInitScript("shop_schema.sql")
- ()
- }
- c.start()
- c
- }
- }(container => ZIO.attemptBlocking(container.stop()).orDie)
-}
diff --git a/postgres/src/test/resources/logback.xml b/postgres/src/test/resources/logback.xml
new file mode 100644
index 000000000..1378a823a
--- /dev/null
+++ b/postgres/src/test/resources/logback.xml
@@ -0,0 +1,14 @@
+
+
+
+ %d{HH:mm:ss.SSS} [%thread] %-5level %logger - %msg%n
+
+
+
+
+
+
+
+
+
+
diff --git a/postgres/src/test/scala/zio/sql/postgresql/DbSchema.scala b/postgres/src/test/scala/zio/sql/postgresql/DbSchema.scala
index e9510c911..37adbeff6 100644
--- a/postgres/src/test/scala/zio/sql/postgresql/DbSchema.scala
+++ b/postgres/src/test/scala/zio/sql/postgresql/DbSchema.scala
@@ -2,6 +2,9 @@ package zio.sql.postgresql
import zio.sql.Jdbc
+import java.time.{ LocalDate, ZonedDateTime }
+import java.util.UUID
+
trait DbSchema extends Jdbc { self =>
import self.ColumnSet._
@@ -17,6 +20,16 @@ trait DbSchema extends Jdbc { self =>
}
object Customers {
+ case class Customer(
+ id: UUID,
+ fname: String,
+ lname: String,
+ verified: Boolean,
+ dateOfBirth: LocalDate,
+ created: String,
+ created2: ZonedDateTime
+ )
+
// https://github.com/zio/zio-sql/issues/320 Once Insert is supported, we can remove created_timestamp_string
val customers =
(uuid("Id") ++ localDate("Dob") ++ string("First_name") ++ string("Last_name") ++ boolean(
@@ -26,6 +39,8 @@ trait DbSchema extends Jdbc { self =>
val (customerId, dob, fName, lName, verified, createdString, createdTimestamp) =
customers.columns
+
+ val ALL = customerId ++ fName ++ lName ++ verified ++ dob ++ createdString ++ createdTimestamp
}
object Orders {
val orders = (uuid("id") ++ uuid("customer_id") ++ localDate("order_date")).table("orders")
diff --git a/postgres/src/test/scala/zio/sql/postgresql/DeleteBatchSpec.scala b/postgres/src/test/scala/zio/sql/postgresql/DeleteBatchSpec.scala
new file mode 100644
index 000000000..5aeb77ac4
--- /dev/null
+++ b/postgres/src/test/scala/zio/sql/postgresql/DeleteBatchSpec.scala
@@ -0,0 +1,94 @@
+package zio.sql.postgresql
+
+import zio.Cause
+import zio.test.Assertion._
+import zio.test._
+
+import java.time.{ LocalDate, ZonedDateTime }
+import java.util.UUID
+
+object DeleteBatchSpec extends PostgresRunnableSpec with DbSchema {
+
+ import Customers._
+
+ private def delete_(c: Customer): Delete[customers.TableType] =
+ deleteFrom(customers).where((verified.isTrue) && (customerId === c.id))
+
+ override def specLayered = suite("Postgres module batch delete")(
+ test("Can delete more than one customer from single table with a condition") {
+ val query = deleteFrom(customers).where(verified.isNotTrue)
+
+ val result = executeBatchDelete(List(query))
+
+ val assertion = for {
+ r <- result
+ } yield assert(r)(equalTo(List(1)))
+
+ assertion.mapErrorCause(cause => Cause.stackless(cause.untraced))
+ },
+ test("Can insert more than one customer into single table prior to deleting them") {
+ val id1 = UUID.randomUUID()
+ val id2 = UUID.randomUUID()
+ val id3 = UUID.randomUUID()
+ val id4 = UUID.randomUUID()
+ val c1 = Customer(
+ id1,
+ "fnameCustomer1",
+ "lnameCustomer1",
+ true,
+ LocalDate.now(),
+ LocalDate.now().toString,
+ ZonedDateTime.now()
+ )
+ val c2 = Customer(
+ id2,
+ "fnameCustomer2",
+ "lnameCustomer2",
+ true,
+ LocalDate.now(),
+ LocalDate.now().toString,
+ ZonedDateTime.now()
+ )
+ val c3 = Customer(
+ id3,
+ "fnameCustomer3",
+ "lnameCustomer3",
+ true,
+ LocalDate.now(),
+ LocalDate.now().toString,
+ ZonedDateTime.now()
+ )
+ val c4 = Customer(
+ id4,
+ "fnameCustomer4",
+ "lnameCustomer4",
+ false,
+ LocalDate.now(),
+ LocalDate.now().toString,
+ ZonedDateTime.now()
+ )
+
+ val allCustomer = List(c1, c2, c3, c4)
+ val data = allCustomer.map(Customer.unapply(_).get)
+ val query = insertInto(customers)(ALL).values(data)
+ val resultInsert = execute(query)
+
+ val insertAssertion = for {
+ r <- resultInsert
+ } yield assert(r)(equalTo(4))
+ insertAssertion.mapErrorCause(cause => Cause.stackless(cause.untraced))
+
+ val selectAll = select(ALL).from(customers)
+ val result_ = execute(selectAll.to((Customer.apply _).tupled)).runCollect
+
+ val assertion_ = for {
+ x <- result_
+ updated = x.toList.map(delete_)
+ result <- executeBatchDelete(updated).map(l => l.fold(0)((a, b) => a + b))
+ } yield assert(result)(equalTo(4))
+ assertion_.mapErrorCause(cause => Cause.stackless(cause.untraced))
+
+ }
+ )
+
+}
diff --git a/postgres/src/test/scala/zio/sql/postgresql/InsertBatchSpec.scala b/postgres/src/test/scala/zio/sql/postgresql/InsertBatchSpec.scala
new file mode 100644
index 000000000..767e51f31
--- /dev/null
+++ b/postgres/src/test/scala/zio/sql/postgresql/InsertBatchSpec.scala
@@ -0,0 +1,70 @@
+package zio.sql.postgresql
+
+import zio.Cause
+import zio.test.Assertion._
+import zio.test._
+
+import java.time.{ LocalDate, ZonedDateTime }
+import java.util.UUID
+
+object InsertBatchSpec extends PostgresRunnableSpec with DbSchema {
+
+ import Customers._
+
+ override def specLayered = suite("Postgres module batch insert")(
+ test("Can insert more than one customer into a table with a condition") {
+ val id1 = UUID.randomUUID()
+ val id2 = UUID.randomUUID()
+ val id3 = UUID.randomUUID()
+ val id4 = UUID.randomUUID()
+ val c1 = Customer(
+ id1,
+ "fnameCustomer1",
+ "lnameCustomer1",
+ true,
+ LocalDate.now(),
+ LocalDate.now().toString,
+ ZonedDateTime.now()
+ )
+ val c2 = Customer(
+ id2,
+ "fnameCustomer2",
+ "lnameCustomer2",
+ true,
+ LocalDate.now(),
+ LocalDate.now().toString,
+ ZonedDateTime.now()
+ )
+ val c3 = Customer(
+ id3,
+ "fnameCustomer3",
+ "lnameCustomer3",
+ true,
+ LocalDate.now(),
+ LocalDate.now().toString,
+ ZonedDateTime.now()
+ )
+ val c4 = Customer(
+ id4,
+ "fnameCustomer4",
+ "lnameCustomer4",
+ false,
+ LocalDate.now(),
+ LocalDate.now().toString,
+ ZonedDateTime.now()
+ )
+
+ val allCustomer = List(c1, c2, c3, c4)
+ val data = allCustomer.map(Customer.unapply(_).get)
+ val query = insertInto(customers)(ALL).values(data)
+
+ val resultInsert = execute(query)
+
+ val insertAssertion = for {
+ result <- resultInsert
+ } yield assert(result)(equalTo(4))
+ insertAssertion.mapErrorCause(cause => Cause.stackless(cause.untraced))
+ }
+ )
+
+}
diff --git a/postgres/src/test/scala/zio/sql/postgresql/PostgresRunnableSpec.scala b/postgres/src/test/scala/zio/sql/postgresql/PostgresRunnableSpec.scala
index f4a26efae..b5b4f7e83 100644
--- a/postgres/src/test/scala/zio/sql/postgresql/PostgresRunnableSpec.scala
+++ b/postgres/src/test/scala/zio/sql/postgresql/PostgresRunnableSpec.scala
@@ -1,36 +1,17 @@
package zio.sql.postgresql
-import zio._
-import zio.test._
-import java.util.Properties
-import zio.sql.{ ConnectionPoolConfig, JdbcRunnableSpec }
+import com.dimafeng.testcontainers.{ JdbcDatabaseContainer, PostgreSQLContainer, SingleContainer }
+import org.testcontainers.utility.DockerImageName
+import zio.sql.JdbcRunnableSpec
trait PostgresRunnableSpec extends JdbcRunnableSpec with PostgresJdbcModule {
- def autoCommit: Boolean = true
-
- private def connProperties(user: String, password: String): Properties = {
- val props = new Properties
- props.setProperty("user", user)
- props.setProperty("password", password)
- props
- }
-
- val poolConfigLayer = ZLayer.scoped {
- TestContainer
- .postgres()
- .map(a =>
- ConnectionPoolConfig(
- url = a.jdbcUrl,
- properties = connProperties(a.username, a.password),
- autoCommit = autoCommit
- )
- )
- }
-
- override def spec: Spec[TestEnvironment, Any] =
- specLayered.provideCustomShared(jdbcLayer)
-
- def specLayered: Spec[JdbcEnvironment, Object]
+ override protected def getContainer: SingleContainer[_] with JdbcDatabaseContainer =
+ new PostgreSQLContainer(
+ dockerImageNameOverride = Option("postgres:alpine").map(DockerImageName.parse)
+ ).configure { a =>
+ a.withInitScript("db_schema.sql")
+ ()
+ }
}
diff --git a/postgres/src/test/scala/zio/sql/postgresql/TestContainer.scala b/postgres/src/test/scala/zio/sql/postgresql/TestContainer.scala
deleted file mode 100644
index 42704c031..000000000
--- a/postgres/src/test/scala/zio/sql/postgresql/TestContainer.scala
+++ /dev/null
@@ -1,25 +0,0 @@
-package zio.sql.postgresql
-
-import com.dimafeng.testcontainers.PostgreSQLContainer
-import org.testcontainers.utility.DockerImageName
-import zio._
-
-object TestContainer {
-
- def postgres(imageName: String = "postgres:alpine"): ZIO[Scope, Throwable, PostgreSQLContainer] =
- ZIO.acquireRelease {
- ZIO.attemptBlocking {
- val c = new PostgreSQLContainer(
- dockerImageNameOverride = Option(imageName).map(DockerImageName.parse)
- ).configure { a =>
- a.withInitScript("db_schema.sql")
- ()
- }
- c.start()
- c
- }
- } { container =>
- ZIO.attemptBlocking(container.stop()).orDie
- }
-
-}
diff --git a/postgres/src/test/scala/zio/sql/postgresql/UpdateBatchSpec.scala b/postgres/src/test/scala/zio/sql/postgresql/UpdateBatchSpec.scala
new file mode 100644
index 000000000..823b0fd6a
--- /dev/null
+++ b/postgres/src/test/scala/zio/sql/postgresql/UpdateBatchSpec.scala
@@ -0,0 +1,85 @@
+package zio.sql.postgresql
+
+import zio.Cause
+import zio.test.Assertion._
+import zio.test._
+
+import java.time.{ LocalDate, ZonedDateTime }
+import java.util.UUID
+
+object UpdateBatchSpec extends PostgresRunnableSpec with DbSchema {
+
+ import Customers._
+
+ private def update_(c: Customer): Update[customers.TableType] =
+ update(customers)
+ .set(verified, !c.verified)
+ .where(customerId === c.id)
+
+ override def specLayered = suite("Postgres module batch update")(
+ test("Can update more than one customer from single table with a condition") {
+ val id1 = UUID.randomUUID()
+ val id2 = UUID.randomUUID()
+ val id3 = UUID.randomUUID()
+ val id4 = UUID.randomUUID()
+ val c1 = Customer(
+ id1,
+ "fnameCustomer1",
+ "lnameCustomer1",
+ true,
+ LocalDate.now(),
+ LocalDate.now().toString,
+ ZonedDateTime.now()
+ )
+ val c2 = Customer(
+ id2,
+ "fnameCustomer2",
+ "lnameCustomer2",
+ true,
+ LocalDate.now(),
+ LocalDate.now().toString,
+ ZonedDateTime.now()
+ )
+ val c3 = Customer(
+ id3,
+ "fnameCustomer3",
+ "lnameCustomer3",
+ true,
+ LocalDate.now(),
+ LocalDate.now().toString,
+ ZonedDateTime.now()
+ )
+ val c4 = Customer(
+ id4,
+ "fnameCustomer4",
+ "lnameCustomer4",
+ false,
+ LocalDate.now(),
+ LocalDate.now().toString,
+ ZonedDateTime.now()
+ )
+
+ val allCustomer = List(c1, c2, c3, c4)
+ val data = allCustomer.map(Customer.unapply(_).get)
+ val query = insertInto(customers)(ALL).values(data)
+
+ val resultInsert = execute(query)
+
+ val insertAssertion = for {
+ r <- resultInsert
+ } yield assert(r)(equalTo(4))
+ insertAssertion.mapErrorCause(cause => Cause.stackless(cause.untraced))
+
+ val selectAll = select(ALL).from(customers)
+ val result_ = execute(selectAll.to((Customer.apply _).tupled)).runCollect
+
+ val assertion_ = for {
+ x <- result_
+ updated = x.toList.map(update_)
+ result <- executeBatchUpdate(updated).map(l => l.reduce((a, b) => a + b))
+ } yield assert(result)(equalTo(5))
+ assertion_.mapErrorCause(cause => Cause.stackless(cause.untraced))
+ }
+ )
+
+}
diff --git a/project/BuildHelper.scala b/project/BuildHelper.scala
index 321d4883e..991416b57 100644
--- a/project/BuildHelper.scala
+++ b/project/BuildHelper.scala
@@ -162,6 +162,7 @@ object BuildHelper {
compilerPlugin(("com.github.ghik" % "silencer-plugin" % SilencerVersion).cross(CrossVersion.full))
)
},
+ resolvers += Resolver.sonatypeRepo("snapshots"),
Test / parallelExecution := true,
incOptions ~= (_.withLogRecompileOnMacro(false)),
autoAPIMappings := true,
diff --git a/sqlserver/src/test/resources/logback.xml b/sqlserver/src/test/resources/logback.xml
new file mode 100644
index 000000000..1378a823a
--- /dev/null
+++ b/sqlserver/src/test/resources/logback.xml
@@ -0,0 +1,14 @@
+
+
+
+ %d{HH:mm:ss.SSS} [%thread] %-5level %logger - %msg%n
+
+
+
+
+
+
+
+
+
+
diff --git a/sqlserver/src/test/scala/zio/sql/sqlserver/SqlServerRunnableSpec.scala b/sqlserver/src/test/scala/zio/sql/sqlserver/SqlServerRunnableSpec.scala
index d415d6487..a28092daa 100644
--- a/sqlserver/src/test/scala/zio/sql/sqlserver/SqlServerRunnableSpec.scala
+++ b/sqlserver/src/test/scala/zio/sql/sqlserver/SqlServerRunnableSpec.scala
@@ -1,34 +1,19 @@
package zio.sql.sqlserver
-import zio._
-import zio.test._
-import java.util.Properties
-import zio.sql.{ ConnectionPoolConfig, JdbcRunnableSpec }
+import com.dimafeng.testcontainers.{ JdbcDatabaseContainer, MSSQLServerContainer, SingleContainer }
+import org.testcontainers.utility.DockerImageName
+import zio.sql.JdbcRunnableSpec
trait SqlServerRunnableSpec extends JdbcRunnableSpec with SqlServerJdbcModule {
- def autoCommit: Boolean = true
+ override protected def getContainer: SingleContainer[_] with JdbcDatabaseContainer =
+ new MSSQLServerContainer(
+ dockerImageName = DockerImageName
+ .parse("mcr.microsoft.com/azure-sql-edge:latest")
+ .asCompatibleSubstituteFor("mcr.microsoft.com/mssql/server")
+ ).configure { a =>
+ a.withInitScript("db_schema.sql")
+ ()
+ }
- private def connProperties(user: String, password: String): Properties = {
- val props = new Properties
- props.setProperty("user", user)
- props.setProperty("password", password)
- props
- }
-
- val poolConfigLayer = ZLayer.scoped {
- TestContainer.sqlServer
- .map(a =>
- ConnectionPoolConfig(
- url = a.jdbcUrl,
- properties = connProperties(a.username, a.password),
- autoCommit = autoCommit
- )
- )
- }
-
- override def spec: Spec[TestEnvironment, Any] =
- specLayered.provideCustomLayerShared(jdbcLayer)
-
- def specLayered: Spec[JdbcEnvironment, Object]
}
diff --git a/sqlserver/src/test/scala/zio/sql/sqlserver/TestContainer.scala b/sqlserver/src/test/scala/zio/sql/sqlserver/TestContainer.scala
deleted file mode 100644
index 8834755dd..000000000
--- a/sqlserver/src/test/scala/zio/sql/sqlserver/TestContainer.scala
+++ /dev/null
@@ -1,30 +0,0 @@
-package zio.sql.sqlserver
-
-import com.dimafeng.testcontainers.MSSQLServerContainer
-import org.testcontainers.utility.DockerImageName
-import zio._
-
-object TestContainer {
-
- /**
- * We are using Azure sql edge because MS Sql Server image won't run on ARM.
- */
- val sqlServer: ZIO[Scope, Throwable, MSSQLServerContainer] =
- ZIO.acquireRelease {
- ZIO.attemptBlocking {
- val c = new MSSQLServerContainer(
- dockerImageName = DockerImageName
- .parse("mcr.microsoft.com/azure-sql-edge:latest")
- .asCompatibleSubstituteFor("mcr.microsoft.com/mssql/server")
- ).configure { a =>
- a.withInitScript("db_schema.sql")
- ()
- }
- c.start()
- c
- }
- } { container =>
- ZIO.attemptBlocking(container.stop()).orDie
- }
-
-}