From 5704ca2cba32dc4e42abefba553259377b853cd7 Mon Sep 17 00:00:00 2001 From: Andrea Passaglia Date: Sat, 26 Dec 2020 16:35:26 +0000 Subject: [PATCH 01/15] compiles --- build.sbt | 22 +++- core/src/main/scala/tamer/Setup.scala | 48 +------- core/src/main/scala/tamer/TamerApp.scala | 28 ----- core/src/main/scala/tamer/db/Db.scala | 80 ------------- core/src/main/scala/tamer/db/package.scala | 12 -- core/src/main/scala/tamer/kafka/Kafka.scala | 8 +- doobie/src/main/scala/tamer/db/Db.scala | 17 +++ doobie/src/main/scala/tamer/db/Setup.scala | 41 +++++++ doobie/src/main/scala/tamer/db/package.scala | 105 ++++++++++++++++++ .../src/main/scala/tamer/example/Main.scala | 61 +++++----- 10 files changed, 217 insertions(+), 205 deletions(-) delete mode 100644 core/src/main/scala/tamer/TamerApp.scala delete mode 100644 core/src/main/scala/tamer/db/Db.scala delete mode 100644 core/src/main/scala/tamer/db/package.scala create mode 100644 doobie/src/main/scala/tamer/db/Db.scala create mode 100644 doobie/src/main/scala/tamer/db/Setup.scala create mode 100644 doobie/src/main/scala/tamer/db/package.scala diff --git a/build.sbt b/build.sbt index 0b55f095..9e937e9b 100644 --- a/build.sbt +++ b/build.sbt @@ -19,6 +19,7 @@ lazy val V = new { val scalatest = "3.2.3" val silencer = "1.7.1" val zio = "1.0.3" + val `zio-s3` = "latest.integration" val `zio-interop` = "2.2.0.1" val `zio-kafka` = "0.13.0" } @@ -78,7 +79,9 @@ lazy val D = new { val zio = Seq( "dev.zio" %% "zio-interop-cats" % V.`zio-interop`, "dev.zio" %% "zio-kafka" % V.`zio-kafka`, - "dev.zio" %% "zio-streams" % V.zio + "dev.zio" %% "zio-s3" % V.`zio-s3`, + "dev.zio" %% "zio-streams" % V.zio, + "dev.zio" %% "zio-test" % V.zio ) } @@ -135,7 +138,7 @@ lazy val commonSettings = Seq( licenses += "MIT" -> url("http://opensource.org/licenses/MIT"), developers += Developer("sirocchj", "Julien Sirocchi", "julien.sirocchi@gmail.com", url("https://github.com/sirocchj")), scalacOptions ++= versionDependent(scalaVersion.value), - resolvers += "confluent" at "https://packages.confluent.io/maven/" + resolvers ++= Seq("confluent" at "https://packages.confluent.io/maven/", "snapshots" at "https://oss.sonatype.org/content/repositories/snapshots") ) lazy val tamer = project @@ -143,7 +146,7 @@ lazy val tamer = project .settings(commonSettings) .settings( name := "tamer", - libraryDependencies ++= (D.cats ++ D.config ++ D.doobie ++ D.kafka ++ D.logs ++ D.refined ++ D.serialization ++ D.silencer ++ D.tests ++ D.zio) + libraryDependencies ++= (D.cats ++ D.config ++ D.kafka ++ D.logs ++ D.refined ++ D.serialization ++ D.silencer ++ D.tests ++ D.zio) .map(_.withSources) .map(_.withJavadoc), libraryDependencies ++= D.avro, @@ -152,10 +155,19 @@ lazy val tamer = project Test / console / scalacOptions := (Compile / console / scalacOptions).value ) +lazy val doobie = project + .in(file("doobie")) + .dependsOn(tamer) + .settings(commonSettings) + .settings( + name := "doobie", + libraryDependencies ++= D.doobie + ) + lazy val example = project .in(file("example")) .enablePlugins(JavaAppPackaging) - .dependsOn(tamer) + .dependsOn(tamer, doobie) .settings(commonSettings) .settings( libraryDependencies ++= D.postgres, @@ -164,7 +176,7 @@ lazy val example = project lazy val root = project .in(file(".")) - .aggregate(tamer, example) + .aggregate(tamer, example, doobie) .settings(commonSettings) .settings( publish / skip := true, diff --git a/core/src/main/scala/tamer/Setup.scala b/core/src/main/scala/tamer/Setup.scala index 60f2decb..05b77f04 100644 --- a/core/src/main/scala/tamer/Setup.scala +++ b/core/src/main/scala/tamer/Setup.scala @@ -1,48 +1,12 @@ package tamer -import com.sksamuel.avro4s._ -import doobie.util.query.Query0 import tamer.registry.{Registry, Topic} -import zio.UIO import zio.kafka.serde.Serializer -final case class ResultMetadata(queryExecutionTime: Long) -final case class QueryResult[V](metadata: ResultMetadata, results: List[V]) - -final case class Setup[K, V, State]( - keySerializer: Serializer[Registry with Topic, K], - valueSerializer: Serializer[Registry with Topic, V], - stateSerde: ZSerde[Registry with Topic, State], - valueToKey: V => K, - defaultState: State, - buildQuery: State => Query0[V], - stateFoldM: State => QueryResult[V] => UIO[State] +abstract class Setup[-K, -V, S]( + val keySerializer: Serializer[Registry with Topic, K], + val valueSerializer: Serializer[Registry with Topic, V], + val stateSerde: ZSerde[Registry with Topic, S], + val defaultState: S, + val queryHash: Int ) - -object Setup { - final def avro[K <: Product: Decoder: Encoder: SchemaFor, V <: Product: Decoder: Encoder: SchemaFor, State <: Product: Decoder: Encoder: SchemaFor]( - defaultState: State - )( - buildQuery: State => Query0[V] - )( - valueToKey: V => K, - stateFoldM: State => QueryResult[V] => UIO[State] - ): Setup[K, V, State] = - Setup(Serde[K](isKey = true).serializer, Serde[V]().serializer, Serde[State]().serde, valueToKey, defaultState, buildQuery, stateFoldM) - - final def avroSimple[K <: Product: Decoder: Encoder: SchemaFor, V <: Product: Decoder: Encoder: SchemaFor]( - defaultState: V - )( - buildQuery: V => Query0[V], - valueToKey: V => K - ): Setup[K, V, V] = - Setup( - Serde[K](isKey = true).serializer, - Serde[V]().serializer, - Serde[V]().serde, - valueToKey, - defaultState, - buildQuery, - _ => r => UIO(r.results.last) - ) -} diff --git a/core/src/main/scala/tamer/TamerApp.scala b/core/src/main/scala/tamer/TamerApp.scala deleted file mode 100644 index 5b7ef5eb..00000000 --- a/core/src/main/scala/tamer/TamerApp.scala +++ /dev/null @@ -1,28 +0,0 @@ -package tamer - -import tamer.config._ -import tamer.db.{Db, DbTransactor} -import tamer.kafka.Kafka -import zio.{App, ExitCode, IO, Layer, URIO, ZEnv} -import zio.blocking.Blocking -import zio.clock.Clock -import zio.console._ - -abstract class TamerApp[K, V, State](private val setup: IO[TamerError, Setup[K, V, State]]) extends App { - final val run = for { - setup <- setup - program <- kafka.runLoop(setup)(db.runQuery(setup)) - } yield program - - override final def run(args: List[String]): URIO[ZEnv, ExitCode] = { - val transactorLayer: Layer[TamerError, DbTransactor] = (Blocking.live ++ Config.live) >>> Db.hikariLayer - val dbLayer: Layer[TamerError, Db] = (Config.live ++ transactorLayer) >>> Db.live - val kafkaLayer: Layer[TamerError, Kafka] = Config.live >>> Kafka.live - run - .provideLayer(Blocking.live ++ Clock.live ++ dbLayer ++ kafkaLayer) - .foldM( - err => putStrLn(s"Execution failed with: $err") *> IO.succeed(ExitCode.failure), - _ => IO.succeed(ExitCode.success) - ) - } -} diff --git a/core/src/main/scala/tamer/db/Db.scala b/core/src/main/scala/tamer/db/Db.scala deleted file mode 100644 index 0b0d9b53..00000000 --- a/core/src/main/scala/tamer/db/Db.scala +++ /dev/null @@ -1,80 +0,0 @@ -package tamer -package db - -import java.sql.SQLException -import java.time.Instant - -import cats.effect.Blocker -import doobie.hikari.HikariTransactor -import doobie.implicits._ -import doobie.util.transactor.Transactor -import eu.timepit.refined.auto._ -import fs2.{Chunk, Stream} -import log.effect.LogWriter -import log.effect.zio.ZioLogWriter.log4sFromName -import tamer.config._ -import zio._ -import zio.blocking.Blocking -import zio.interop.catz._ -import tamer.db.Compat.toIterable - -import scala.concurrent.ExecutionContext - -object Db { - implicit class InstantOps(ours: Instant) { - def -(theirs: Instant): Long = ours.toEpochMilli - theirs.toEpochMilli - } - - case class ChunkWithMetadata[V](chunk: Chunk[V], pulledAt: Instant = Instant.now()) - case class ValueWithMetadata[V](value: V, pulledAt: Instant = Instant.now()) - - trait Service { - def runQuery[K, V, State](setup: Setup[K, V, State])(state: State, q: Queue[(K, V)]): IO[TamerError, State] - } - - // https://github.com/zio/zio/issues/2949 - val live: URLayer[DbTransactor with QueryConfig, Db] = ZLayer.fromServices[Transactor[Task], Config.Query, Service] { (tnx, cfg) => - new Service { - private[this] val logTask: Task[LogWriter[Task]] = log4sFromName.provide("tamer.db") - override final def runQuery[K, V, State](setup: Setup[K, V, State])(state: State, q: Queue[(K, V)]): IO[TamerError, State] = - (for { - log <- logTask - query <- UIO(setup.buildQuery(state)) - _ <- log.debug(s"running ${query.sql} with params derived from $state").ignore - start <- UIO(Instant.now()) - values <- - query - .streamWithChunkSize(cfg.fetchChunkSize) - .chunks - .transact(tnx) - .map(ChunkWithMetadata(_)) - .evalTap(c => q.offerAll(toIterable(c.chunk).map(v => setup.valueToKey(v) -> v))) - .flatMap(c => Stream.chunk(c.chunk).map(ValueWithMetadata(_, c.pulledAt))) - .compile - .toList - newState <- setup.stateFoldM(state)( - QueryResult( - ResultMetadata(values.headOption.fold(Instant.now())(_.pulledAt) - start), - values.map(_.value) - ) - ) - } yield newState).mapError(e => TamerError(e.getLocalizedMessage, e)) - } - } - - val hikariLayer: ZLayer[Blocking with DbConfig, TamerError, DbTransactor] = ZLayer.fromManaged { - for { - cfg <- dbConfig.toManaged_ - connectEC <- ZIO.descriptor.map(_.executor.asEC).toManaged_ - blockingEC <- blocking.blocking(ZIO.descriptor.map(_.executor.asEC)).toManaged_ - managedTransactor <- mkTransactor(cfg, connectEC, blockingEC) - } yield managedTransactor - } - - def mkTransactor(db: Config.Db, connectEC: ExecutionContext, transactEC: ExecutionContext): Managed[TamerError, HikariTransactor[Task]] = - HikariTransactor - .newHikariTransactor[Task](db.driver, db.uri, db.username, db.password, connectEC, Blocker.liftExecutionContext(transactEC)) - .toManagedZIO - .refineToOrDie[SQLException] - .mapError(sqle => TamerError(sqle.getLocalizedMessage, sqle)) -} diff --git a/core/src/main/scala/tamer/db/package.scala b/core/src/main/scala/tamer/db/package.scala deleted file mode 100644 index ac93af9b..00000000 --- a/core/src/main/scala/tamer/db/package.scala +++ /dev/null @@ -1,12 +0,0 @@ -package tamer - -import doobie.util.transactor.Transactor -import zio.{Has, Queue, Task, ZIO} - -package object db { - type Db = Has[Db.Service] - type DbTransactor = Has[Transactor[Task]] - - def runQuery[K, V, State](setup: Setup[K, V, State])(state: State, q: Queue[(K, V)]): ZIO[Db, TamerError, State] = - ZIO.accessM(_.get.runQuery(setup)(state, q)) -} diff --git a/core/src/main/scala/tamer/kafka/Kafka.scala b/core/src/main/scala/tamer/kafka/Kafka.scala index 1057f7e0..cb481ba3 100644 --- a/core/src/main/scala/tamer/kafka/Kafka.scala +++ b/core/src/main/scala/tamer/kafka/Kafka.scala @@ -1,8 +1,6 @@ package tamer package kafka -import java.security.MessageDigest - import eu.timepit.refined.auto._ import io.confluent.kafka.schemaregistry.client.{CachedSchemaRegistryClient, SchemaRegistryClient} import log.effect.LogWriter @@ -37,10 +35,6 @@ object Kafka { val live: URLayer[KafkaConfig, Kafka] = ZLayer.fromService { cfg => new Service { private[this] val logTask: Task[LogWriter[Task]] = log4sFromName.provide("tamer.kafka") - private[this] def stateKeyTask[A](a: A, s: String)(f: A => String): Task[StateKey] = - Task(MessageDigest.getInstance("SHA-1")).map { md => - StateKey(md.digest(f(a).getBytes).take(7).map(b => f"$b%02x").mkString, s) - } override final def runLoop[K, V, State, R](setup: Setup[K, V, State])( f: (State, Queue[(K, V)]) => ZIO[R, TamerError, State] @@ -83,7 +77,7 @@ object Kafka { sp: Producer.Service[Registry with Topic, StateKey, State], layer: ULayer[Registry with Topic] ) = - ZStream.fromEffect(logTask <*> stateKeyTask(setup.defaultState, cfg.state.groupId)(setup.buildQuery(_).sql)).flatMap { + ZStream.fromEffect(logTask <*> UIO(StateKey(setup.queryHash.toHexString, cfg.state.groupId))).flatMap { // TODO: no need for UIO, it's pure case (log, stateKey) => ZStream .fromEffect(subscribe(sc)) diff --git a/doobie/src/main/scala/tamer/db/Db.scala b/doobie/src/main/scala/tamer/db/Db.scala new file mode 100644 index 00000000..14359937 --- /dev/null +++ b/doobie/src/main/scala/tamer/db/Db.scala @@ -0,0 +1,17 @@ +package tamer +package db + +import fs2.Chunk + +import java.time.Instant + +object Db { + abstract class Datable(val instant: Instant) + + case class ChunkWithMetadata[V](chunk: Chunk[V], pulledAt: Instant = Instant.now()) + case class ValueWithMetadata[V](value: V, pulledAt: Instant = Instant.now()) + + case class TimeSegment(from: Instant, to: Instant) extends State { + override lazy val stateId: Int = this.hashCode() + } +} diff --git a/doobie/src/main/scala/tamer/db/Setup.scala b/doobie/src/main/scala/tamer/db/Setup.scala new file mode 100644 index 00000000..2b95af56 --- /dev/null +++ b/doobie/src/main/scala/tamer/db/Setup.scala @@ -0,0 +1,41 @@ +package tamer.db + +import com.sksamuel.avro4s.{Decoder, Encoder, SchemaFor} +import doobie.Query0 +import tamer.Serde +import zio.UIO + +trait QueryBuilder[V, -S] { + + /** Used for hashing purposes + */ + val queryId: Int + def query(state: S): Query0[V] +} + +trait State { + + /** Used for hashing purposes1 + */ + val stateId: Int +} + +final case class ResultMetadata(queryExecutionTime: Long) +final case class QueryResult[V](metadata: ResultMetadata, results: List[V]) + +case class Setup[ + K <: Product: Encoder: Decoder: SchemaFor, + V <: Product: Encoder: Decoder: SchemaFor, + S <: Product with State: Encoder: Decoder: SchemaFor +]( + queryBuilder: QueryBuilder[V, S], + override val defaultState: S, + keyExtract: V => K, + stateFoldM: S => QueryResult[V] => UIO[S] +) extends tamer.Setup[K, V, S]( + Serde[K](isKey = true).serializer, + Serde[V]().serializer, + Serde[S]().serde, + defaultState, + queryBuilder.queryId + defaultState.stateId + ) diff --git a/doobie/src/main/scala/tamer/db/package.scala b/doobie/src/main/scala/tamer/db/package.scala new file mode 100644 index 00000000..b0765b5e --- /dev/null +++ b/doobie/src/main/scala/tamer/db/package.scala @@ -0,0 +1,105 @@ +package tamer + +import cats.effect.Blocker +import com.sksamuel.avro4s.{Decoder, Encoder, SchemaFor} +import doobie.Query0 +import doobie.hikari.HikariTransactor +import doobie.implicits.{toDoobieStreamOps, _} +import doobie.util.transactor.Transactor +import eu.timepit.refined.auto._ +import fs2.Stream +import tamer.config.{Config, DbConfig, dbConfig} +import tamer.db.Compat.toIterable +import tamer.db.Db.{Datable, TimeSegment, _} +import tamer.kafka.Kafka +import zio.blocking.Blocking +import zio.interop.catz._ +import zio.{Queue, Task, UIO, ZIO, _} + +import java.sql.SQLException +import java.time.{Duration, Instant} +import scala.concurrent.ExecutionContext + +package object db { + type DbTransactor = Has[Transactor[Task]] + type TamerDBConfig = DbTransactor with Has[Config.Query] + + private[this] implicit final class InstantOps(private val instant: Instant) extends AnyVal { + def orNow: UIO[Instant] = + UIO(Instant.now).map { + case now if instant.isAfter(now) => now + case _ => instant + } + } + + final def mkSetup[K <: Product: Encoder: Decoder: SchemaFor, V <: Product with Datable: Ordering: Encoder: Decoder: SchemaFor]( + queryBuilder: TimeSegment => Query0[V] + )(earliest: Instant, tumblingStep: Duration, keyExtract: V => K): Setup[K, V, TimeSegment] = { + + val timeSegment = TimeSegment(earliest, earliest.plus(tumblingStep)) + + val qBuilder = new QueryBuilder[V, TimeSegment] { + override val queryId: Int = queryBuilder(timeSegment).sql.hashCode + override def query(state: TimeSegment): Query0[V] = queryBuilder(state) + } + + def stateFold(timeSegment: TimeSegment)(queryResult: QueryResult[V]): UIO[TimeSegment] = + if (queryResult.results.isEmpty) timeSegment.to.plus(tumblingStep).orNow.map(TimeSegment(timeSegment.from, _)) + else { + val mostRecent = queryResult.results.max.instant + mostRecent.plus(tumblingStep).orNow.map(TimeSegment(mostRecent, _)) + } + + Setup[K, V, TimeSegment](qBuilder, timeSegment, keyExtract, stateFold) + } + + final def iteration[K <: Product, V <: Product]( + setup: Setup[K, V, TimeSegment] + )(state: TimeSegment, q: Queue[(K, V)]): ZIO[TamerDBConfig, TamerError, TimeSegment] = + (for { + cfg <- ZIO.service[Config.Query] + tnx <- ZIO.service[Transactor[Task]] + query <- UIO(setup.queryBuilder.query(state)) + start <- UIO(Instant.now()) + values <- + query + .streamWithChunkSize(cfg.fetchChunkSize) + .chunks + .transact(tnx) + .map(ChunkWithMetadata(_)) + .evalTap(c => q.offerAll(toIterable(c.chunk).map(v => setup.keyExtract(v) -> v))) + .flatMap(c => Stream.chunk(c.chunk).map(ValueWithMetadata(_, c.pulledAt))) + .compile + .toList + pulledTimeOrNow = values.headOption.map(_.pulledAt).getOrElse(Instant.now()) + newState <- setup.stateFoldM(state)( + QueryResult( + ResultMetadata( + Duration.between(start, pulledTimeOrNow).toMillis + ), + values.map(_.value) + ) + ) + } yield newState).mapError(e => TamerError(e.getLocalizedMessage, e)) + + final def fetchWithTimeSegment[K <: Product, V <: Product]( + setup: Setup[K, V, TimeSegment] + ): ZIO[Kafka with TamerDBConfig with ZEnv, TamerError, Unit] = + tamer.kafka.runLoop(setup)(iteration(setup)) + + val hikariLayer: ZLayer[Blocking with DbConfig, TamerError, DbTransactor] = ZLayer.fromManaged { + for { + cfg <- dbConfig.toManaged_ + connectEC <- ZIO.descriptor.map(_.executor.asEC).toManaged_ + blockingEC <- blocking.blocking(ZIO.descriptor.map(_.executor.asEC)).toManaged_ + managedTransactor <- mkTransactor(cfg, connectEC, blockingEC) + } yield managedTransactor + } + + def mkTransactor(db: Config.Db, connectEC: ExecutionContext, transactEC: ExecutionContext): Managed[TamerError, HikariTransactor[Task]] = + HikariTransactor + .newHikariTransactor[Task](db.driver, db.uri, db.username, db.password, connectEC, Blocker.liftExecutionContext(transactEC)) + .toManagedZIO + .refineToOrDie[SQLException] + .mapError(sqle => TamerError(sqle.getLocalizedMessage, sqle)) +} diff --git a/example/src/main/scala/tamer/example/Main.scala b/example/src/main/scala/tamer/example/Main.scala index 9d2b63da..3b8f1fe1 100644 --- a/example/src/main/scala/tamer/example/Main.scala +++ b/example/src/main/scala/tamer/example/Main.scala @@ -1,41 +1,40 @@ package tamer package example -import java.time.Instant -import java.time.temporal.ChronoUnit._ - +import tamer.config.{Config, TamerConfig} +import tamer.db.Db.Datable +import tamer.db.{DbTransactor, TamerDBConfig} +import tamer.kafka.Kafka +import zio.blocking.Blocking +import zio._ import doobie.implicits.legacy.instant._ import doobie.syntax.string._ -import zio.UIO -final case class State(from: Instant, to: Instant) +import java.time.temporal.ChronoUnit._ +import java.time.{Duration, Instant} + final case class Key(id: String) -final case class Value(id: String, name: String, description: Option[String], modifiedAt: Instant) +final case class Value(id: String, name: String, description: Option[String], modifiedAt: Instant) extends Datable(modifiedAt) +object Value { + implicit val ordering: Ordering[Value] = (x: Value, y: Value) => x.modifiedAt.compareTo(y.modifiedAt) +} -object Source { - private[this] implicit final class InstantOps(private val instant: Instant) extends AnyVal { - def plus5Minutes: Instant = instant.plus(5, MINUTES) - def minus60Days: Instant = instant.minus(60, DAYS) - def orNow: UIO[Instant] = - UIO(Instant.now).map { - case now if instant.isAfter(now) => now - case _ => instant - } - } - final val setup = UIO(Instant.now.truncatedTo(DAYS)).map { bootTime => - val sixtyDaysAgo = bootTime.minus60Days - Setup.avro( - State(sixtyDaysAgo, sixtyDaysAgo.plus5Minutes) - )(s => sql"""SELECT id, name, description, modified_at FROM users WHERE modified_at > ${s.from} AND modified_at <= ${s.to}""".query[Value])( - v => Key(v.id), - s => { - case QueryResult(_, Nil) => s.to.plus5Minutes.orNow.map(State(s.from, _)) - case QueryResult(_, values) => - val max = values.map(_.modifiedAt).max // if we can't order in the query we need to do it here... - max.plus5Minutes.orNow.map(State(max, _)) - } +object Main extends zio.App { + val transactorLayer: Layer[TamerError, DbTransactor] = (Blocking.live ++ Config.live) >>> db.hikariLayer + val kafkaLayer: Layer[TamerError, Kafka] = Config.live >>> Kafka.live + val layer: Layer[TamerError, DbTransactor with Kafka with TamerConfig] = transactorLayer ++ kafkaLayer ++ Config.live + def keyExtract(value: Value): Key = Key(value.id) + val program: ZIO[Kafka with TamerDBConfig with ZEnv, TamerError, Unit] = for { + boot <- UIO(Instant.now()) + setup = tamer.db.mkSetup(ts => + sql"""SELECT id, name, description, modified_at FROM users WHERE modified_at > ${ts.from} AND modified_at <= ${ts.to}""".query[Value] + )( + earliest = boot.minus(60, DAYS), + tumblingStep = Duration.of(5, MINUTES), + keyExtract = keyExtract ) - } -} + _ <- tamer.db.fetchWithTimeSegment(setup) + } yield () -object Main extends TamerApp(Source.setup) + override final def run(args: List[String]): URIO[ZEnv, ExitCode] = program.provideCustomLayer(layer).exitCode +} From b981d3a8449d6ddb01701890e9f3869483d02fd0 Mon Sep 17 00:00:00 2001 From: Andrea Passaglia Date: Sat, 26 Dec 2020 19:42:01 +0000 Subject: [PATCH 02/15] config extracted barely compiles --- core/src/main/scala/tamer/config/Config.scala | 21 +-------- .../src/main/scala/tamer/config/package.scala | 5 --- doobie/src/main/scala/tamer/db/ConfigDb.scala | 44 +++++++++++++++++++ doobie/src/main/scala/tamer/db/package.scala | 10 ++--- .../src/main/scala/tamer/example/Main.scala | 10 +++-- 5 files changed, 57 insertions(+), 33 deletions(-) create mode 100644 doobie/src/main/scala/tamer/db/ConfigDb.scala diff --git a/core/src/main/scala/tamer/config/Config.scala b/core/src/main/scala/tamer/config/Config.scala index 1e7dc104..f28eb44f 100644 --- a/core/src/main/scala/tamer/config/Config.scala +++ b/core/src/main/scala/tamer/config/Config.scala @@ -13,8 +13,6 @@ import zio.interop.catz._ import scala.concurrent.duration.FiniteDuration object Config { - final case class Db(driver: NonEmptyString, uri: UriString, username: NonEmptyString, password: Password) - final case class Query(fetchChunkSize: PosInt) final case class KafkaSink(topic: NonEmptyString) final case class KafkaState(topic: NonEmptyString, groupId: NonEmptyString, clientId: NonEmptyString) final case class Kafka( @@ -25,20 +23,10 @@ object Config { sink: KafkaSink, state: KafkaState ) - final case class Tamer(db: Db, query: Query, kafka: Kafka) private[this] implicit final val hostListConfigDecoder: ConfigDecoder[String, HostList] = ConfigDecoder.identity[String].map(_.split(",").toList.map(_.trim)).mapEither(ConfigDecoder[List[String], HostList].decode) - private[this] val dbConfigValue = ( - env("DATABASE_DRIVER").as[NonEmptyString], - env("DATABASE_URL").as[UriString], - env("DATABASE_USERNAME").as[NonEmptyString], - env("DATABASE_PASSWORD").as[Password].redacted - ).parMapN(Db) - - private[this] val queryConfigValue = env("QUERY_FETCH_CHUNK_SIZE").as[PosInt].map(Query) - private[this] val kafkaSinkConfigValue = env("KAFKA_SINK_TOPIC").as[NonEmptyString].map(KafkaSink) private[this] val kafkaStateConfigValue = ( env("KAFKA_STATE_TOPIC").as[NonEmptyString], @@ -53,17 +41,12 @@ object Config { kafkaSinkConfigValue, kafkaStateConfigValue ).parMapN(Kafka) - private[this] val tamerConfigValue: ConfigValue[Tamer] = (dbConfigValue, queryConfigValue, kafkaConfigValue).parMapN(Tamer.apply) trait Service { - val dbConfig: URIO[DbConfig, Db] - val queryConfig: URIO[QueryConfig, Query] val kafkaConfig: URIO[KafkaConfig, Kafka] } - val live: Layer[TamerError, TamerConfig] = ZLayer.fromEffectMany { - tamerConfigValue.load[Task].refineToOrDie[ConfigException].mapError(ce => TamerError(ce.error.redacted.show, ce)).map { - case Tamer(db, query, kafka) => Has(db) ++ Has(query) ++ Has(kafka) - } + val live: Layer[TamerError, KafkaConfig] = ZLayer.fromEffect { + kafkaConfigValue.load[Task].refineToOrDie[ConfigException].mapError(ce => TamerError(ce.error.redacted.show, ce)) } } diff --git a/core/src/main/scala/tamer/config/package.scala b/core/src/main/scala/tamer/config/package.scala index 30dbf51e..0e8abd55 100644 --- a/core/src/main/scala/tamer/config/package.scala +++ b/core/src/main/scala/tamer/config/package.scala @@ -12,12 +12,7 @@ package object config { type UriString = String Refined Uri type UrlString = String Refined Url - type DbConfig = Has[Config.Db] - type QueryConfig = Has[Config.Query] type KafkaConfig = Has[Config.Kafka] - type TamerConfig = DbConfig with QueryConfig with KafkaConfig - val dbConfig: URIO[DbConfig, Config.Db] = ZIO.access(_.get) - val queryConfig: URIO[QueryConfig, Config.Query] = ZIO.access(_.get) val kafkaConfig: URIO[KafkaConfig, Config.Kafka] = ZIO.access(_.get) } diff --git a/doobie/src/main/scala/tamer/db/ConfigDb.scala b/doobie/src/main/scala/tamer/db/ConfigDb.scala new file mode 100644 index 00000000..e6e2f9be --- /dev/null +++ b/doobie/src/main/scala/tamer/db/ConfigDb.scala @@ -0,0 +1,44 @@ +package tamer.db + +import ciris.{ConfigException, env} +import ciris.refined.refTypeConfigDecoder +import cats.implicits._ +import eu.timepit.refined.types.numeric.PosInt +import eu.timepit.refined.types.string.NonEmptyString +import tamer.TamerError +import tamer.config.{Password, UriString} +import zio.interop.catz.{taskConcurrentInstance, zioContextShift} +import zio.{Has, Layer, Task, URIO, ZIO, ZLayer} + +object ConfigDb { + type DbConfig = Has[Db] + type QueryConfig = Has[Query] + + val dbConfig: URIO[DbConfig, Db] = ZIO.service + val queryConfig: URIO[QueryConfig, Query] = ZIO.service + + final case class Db(driver: NonEmptyString, uri: UriString, username: NonEmptyString, password: Password) + final case class Query(fetchChunkSize: PosInt) + final case class DeleteMe(db: Db, query: Query) + + private[this] val dbConfigValue = ( + env("DATABASE_DRIVER").as[NonEmptyString], + env("DATABASE_URL").as[UriString], + env("DATABASE_USERNAME").as[NonEmptyString], + env("DATABASE_PASSWORD").as[Password].redacted + ).parMapN(Db) + private[this] val queryConfigValue = env("QUERY_FETCH_CHUNK_SIZE").as[PosInt].map(Query) + + private[this] val deleteMeConfigValue = (dbConfigValue, queryConfigValue).parMapN(DeleteMe.apply) + + trait Service { + val dbConfig: URIO[DbConfig, Db] + val queryConfig: URIO[QueryConfig, Query] + } + + val live: Layer[TamerError, Has[Db] with Has[Query]] = ZLayer.fromEffectMany { + deleteMeConfigValue.load[Task].refineToOrDie[ConfigException].mapError(ce => TamerError(ce.error.redacted.show, ce)).map { + case DeleteMe(db, query) => Has(db) ++ Has(query) + } + } +} diff --git a/doobie/src/main/scala/tamer/db/package.scala b/doobie/src/main/scala/tamer/db/package.scala index b0765b5e..e3341b0f 100644 --- a/doobie/src/main/scala/tamer/db/package.scala +++ b/doobie/src/main/scala/tamer/db/package.scala @@ -8,8 +8,8 @@ import doobie.implicits.{toDoobieStreamOps, _} import doobie.util.transactor.Transactor import eu.timepit.refined.auto._ import fs2.Stream -import tamer.config.{Config, DbConfig, dbConfig} import tamer.db.Compat.toIterable +import tamer.db.ConfigDb.{DbConfig, QueryConfig} import tamer.db.Db.{Datable, TimeSegment, _} import tamer.kafka.Kafka import zio.blocking.Blocking @@ -22,7 +22,7 @@ import scala.concurrent.ExecutionContext package object db { type DbTransactor = Has[Transactor[Task]] - type TamerDBConfig = DbTransactor with Has[Config.Query] + type TamerDBConfig = DbTransactor with QueryConfig private[this] implicit final class InstantOps(private val instant: Instant) extends AnyVal { def orNow: UIO[Instant] = @@ -57,7 +57,7 @@ package object db { setup: Setup[K, V, TimeSegment] )(state: TimeSegment, q: Queue[(K, V)]): ZIO[TamerDBConfig, TamerError, TimeSegment] = (for { - cfg <- ZIO.service[Config.Query] + cfg <- ConfigDb.queryConfig tnx <- ZIO.service[Transactor[Task]] query <- UIO(setup.queryBuilder.query(state)) start <- UIO(Instant.now()) @@ -89,14 +89,14 @@ package object db { val hikariLayer: ZLayer[Blocking with DbConfig, TamerError, DbTransactor] = ZLayer.fromManaged { for { - cfg <- dbConfig.toManaged_ + cfg <- ConfigDb.dbConfig.toManaged_ connectEC <- ZIO.descriptor.map(_.executor.asEC).toManaged_ blockingEC <- blocking.blocking(ZIO.descriptor.map(_.executor.asEC)).toManaged_ managedTransactor <- mkTransactor(cfg, connectEC, blockingEC) } yield managedTransactor } - def mkTransactor(db: Config.Db, connectEC: ExecutionContext, transactEC: ExecutionContext): Managed[TamerError, HikariTransactor[Task]] = + def mkTransactor(db: ConfigDb.Db, connectEC: ExecutionContext, transactEC: ExecutionContext): Managed[TamerError, HikariTransactor[Task]] = HikariTransactor .newHikariTransactor[Task](db.driver, db.uri, db.username, db.password, connectEC, Blocker.liftExecutionContext(transactEC)) .toManagedZIO diff --git a/example/src/main/scala/tamer/example/Main.scala b/example/src/main/scala/tamer/example/Main.scala index 3b8f1fe1..46d29744 100644 --- a/example/src/main/scala/tamer/example/Main.scala +++ b/example/src/main/scala/tamer/example/Main.scala @@ -1,14 +1,15 @@ package tamer package example -import tamer.config.{Config, TamerConfig} +import tamer.config.{Config} import tamer.db.Db.Datable -import tamer.db.{DbTransactor, TamerDBConfig} +import tamer.db.{ConfigDb, DbTransactor, TamerDBConfig} import tamer.kafka.Kafka import zio.blocking.Blocking import zio._ import doobie.implicits.legacy.instant._ import doobie.syntax.string._ +import tamer.db.ConfigDb.{DbConfig, QueryConfig} import java.time.temporal.ChronoUnit._ import java.time.{Duration, Instant} @@ -20,9 +21,10 @@ object Value { } object Main extends zio.App { - val transactorLayer: Layer[TamerError, DbTransactor] = (Blocking.live ++ Config.live) >>> db.hikariLayer + val transactorLayer: Layer[TamerError, DbTransactor] = (Blocking.live ++ ConfigDb.live) >>> db.hikariLayer val kafkaLayer: Layer[TamerError, Kafka] = Config.live >>> Kafka.live - val layer: Layer[TamerError, DbTransactor with Kafka with TamerConfig] = transactorLayer ++ kafkaLayer ++ Config.live + val queryConfigLayer: Layer[TamerError, DbConfig with QueryConfig] = ConfigDb.live + val layer: Layer[TamerError, DbTransactor with Kafka with QueryConfig] = transactorLayer ++ kafkaLayer ++ queryConfigLayer def keyExtract(value: Value): Key = Key(value.id) val program: ZIO[Kafka with TamerDBConfig with ZEnv, TamerError, Unit] = for { boot <- UIO(Instant.now()) From 8b5c151f5b9c65c8f71fc37a4cf906ed0437949c Mon Sep 17 00:00:00 2001 From: Andrea Passaglia Date: Mon, 28 Dec 2020 20:02:45 +0000 Subject: [PATCH 03/15] working (apparently) --- core/src/main/scala/tamer/kafka/Kafka.scala | 2 +- doobie/src/main/scala/tamer/db/ConfigDb.scala | 6 +- doobie/src/main/scala/tamer/db/package.scala | 6 + .../src/main/scala/tamer/example/Main.scala | 12 +- local/data/users.sql | 1006 +++++++++++++++++ local/docker-compose.yml | 157 +++ 6 files changed, 1182 insertions(+), 7 deletions(-) create mode 100644 local/data/users.sql create mode 100644 local/docker-compose.yml diff --git a/core/src/main/scala/tamer/kafka/Kafka.scala b/core/src/main/scala/tamer/kafka/Kafka.scala index cb481ba3..f0e3cab2 100644 --- a/core/src/main/scala/tamer/kafka/Kafka.scala +++ b/core/src/main/scala/tamer/kafka/Kafka.scala @@ -61,7 +61,7 @@ object Kafka { def sink(q: Queue[(K, V)], p: Producer.Service[Registry with Topic, K, V], layer: ULayer[Registry with Topic]) = logTask.flatMap { log => q.takeAll.flatMap { - case Nil => log.trace("no data to push") *> ZIO.unit + case Nil => log.trace("no data to push").unit case kvs => p.produceChunkAsync(mkRecordChunk(kvs)).provideSomeLayer[Blocking](layer).retry(tenTimes).flatten.unit <* log.info(s"pushed ${kvs.size} messages to ${cfg.sink.topic}") diff --git a/doobie/src/main/scala/tamer/db/ConfigDb.scala b/doobie/src/main/scala/tamer/db/ConfigDb.scala index e6e2f9be..3f53d0b8 100644 --- a/doobie/src/main/scala/tamer/db/ConfigDb.scala +++ b/doobie/src/main/scala/tamer/db/ConfigDb.scala @@ -19,7 +19,7 @@ object ConfigDb { final case class Db(driver: NonEmptyString, uri: UriString, username: NonEmptyString, password: Password) final case class Query(fetchChunkSize: PosInt) - final case class DeleteMe(db: Db, query: Query) + final case class DatabaseConfig(db: Db, query: Query) private[this] val dbConfigValue = ( env("DATABASE_DRIVER").as[NonEmptyString], @@ -29,7 +29,7 @@ object ConfigDb { ).parMapN(Db) private[this] val queryConfigValue = env("QUERY_FETCH_CHUNK_SIZE").as[PosInt].map(Query) - private[this] val deleteMeConfigValue = (dbConfigValue, queryConfigValue).parMapN(DeleteMe.apply) + private[this] val deleteMeConfigValue = (dbConfigValue, queryConfigValue).parMapN(DatabaseConfig.apply) trait Service { val dbConfig: URIO[DbConfig, Db] @@ -38,7 +38,7 @@ object ConfigDb { val live: Layer[TamerError, Has[Db] with Has[Query]] = ZLayer.fromEffectMany { deleteMeConfigValue.load[Task].refineToOrDie[ConfigException].mapError(ce => TamerError(ce.error.redacted.show, ce)).map { - case DeleteMe(db, query) => Has(db) ++ Has(query) + case DatabaseConfig(db, query) => Has(db) ++ Has(query) } } } diff --git a/doobie/src/main/scala/tamer/db/package.scala b/doobie/src/main/scala/tamer/db/package.scala index e3341b0f..b4058583 100644 --- a/doobie/src/main/scala/tamer/db/package.scala +++ b/doobie/src/main/scala/tamer/db/package.scala @@ -8,6 +8,8 @@ import doobie.implicits.{toDoobieStreamOps, _} import doobie.util.transactor.Transactor import eu.timepit.refined.auto._ import fs2.Stream +import log.effect.LogWriter +import log.effect.zio.ZioLogWriter.log4sFromName import tamer.db.Compat.toIterable import tamer.db.ConfigDb.{DbConfig, QueryConfig} import tamer.db.Db.{Datable, TimeSegment, _} @@ -53,13 +55,17 @@ package object db { Setup[K, V, TimeSegment](qBuilder, timeSegment, keyExtract, stateFold) } + private[this] val logTask: Task[LogWriter[Task]] = log4sFromName.provide("tamer.db") + final def iteration[K <: Product, V <: Product]( setup: Setup[K, V, TimeSegment] )(state: TimeSegment, q: Queue[(K, V)]): ZIO[TamerDBConfig, TamerError, TimeSegment] = (for { + log <- logTask cfg <- ConfigDb.queryConfig tnx <- ZIO.service[Transactor[Task]] query <- UIO(setup.queryBuilder.query(state)) + _ <- log.info(s"running ${query.sql} with params derived from $state") // TODO: make this debug start <- UIO(Instant.now()) values <- query diff --git a/example/src/main/scala/tamer/example/Main.scala b/example/src/main/scala/tamer/example/Main.scala index 46d29744..6c29ac0c 100644 --- a/example/src/main/scala/tamer/example/Main.scala +++ b/example/src/main/scala/tamer/example/Main.scala @@ -1,7 +1,7 @@ package tamer package example -import tamer.config.{Config} +import tamer.config.Config import tamer.db.Db.Datable import tamer.db.{ConfigDb, DbTransactor, TamerDBConfig} import tamer.kafka.Kafka @@ -9,6 +9,8 @@ import zio.blocking.Blocking import zio._ import doobie.implicits.legacy.instant._ import doobie.syntax.string._ +import log.effect.LogWriter +import log.effect.zio.ZioLogWriter.log4sFromName import tamer.db.ConfigDb.{DbConfig, QueryConfig} import java.time.temporal.ChronoUnit._ @@ -26,7 +28,10 @@ object Main extends zio.App { val queryConfigLayer: Layer[TamerError, DbConfig with QueryConfig] = ConfigDb.live val layer: Layer[TamerError, DbTransactor with Kafka with QueryConfig] = transactorLayer ++ kafkaLayer ++ queryConfigLayer def keyExtract(value: Value): Key = Key(value.id) - val program: ZIO[Kafka with TamerDBConfig with ZEnv, TamerError, Unit] = for { + private[this] val logTask: Task[LogWriter[Task]] = log4sFromName.provide("tamer.example") + val program: ZIO[Kafka with TamerDBConfig with ZEnv, TamerError, Unit] = (for { + log <- logTask + _ <- log.info("Starting tamer...") boot <- UIO(Instant.now()) setup = tamer.db.mkSetup(ts => sql"""SELECT id, name, description, modified_at FROM users WHERE modified_at > ${ts.from} AND modified_at <= ${ts.to}""".query[Value] @@ -35,8 +40,9 @@ object Main extends zio.App { tumblingStep = Duration.of(5, MINUTES), keyExtract = keyExtract ) + _ <- log.info(s"Tamer initialized with setup $setup") _ <- tamer.db.fetchWithTimeSegment(setup) - } yield () + } yield ()).mapError(e => TamerError("Could not run tamer example", e)) override final def run(args: List[String]): URIO[ZEnv, ExitCode] = program.provideCustomLayer(layer).exitCode } diff --git a/local/data/users.sql b/local/data/users.sql new file mode 100644 index 00000000..16c89313 --- /dev/null +++ b/local/data/users.sql @@ -0,0 +1,1006 @@ +create table users ( + id VARCHAR(50) NOT NULL, + name VARCHAR(50) NOT NULL, + description TEXT, + modified_at TIMESTAMP NOT NULL +); +insert into users (id, name, description, modified_at) values ('3a61545eab16d91c90b009d11da8474a', 'Garçon', null, '2020-07-21T02:57:38Z'); +insert into users (id, name, description, modified_at) values ('f27a76c984bb78047efab6eebd0fcb0f', 'Mahélie', null, '2020-06-21T11:24:22Z'); +insert into users (id, name, description, modified_at) values ('59ae32730283c492a1a8f34305b27145', 'Aí', null, '2020-06-24T18:28:49Z'); +insert into users (id, name, description, modified_at) values ('074fb4cf46c0f6322c9803c35fd10940', 'Mylène', null, '2020-02-17T15:55:53Z'); +insert into users (id, name, description, modified_at) values ('05d5c30fd64d63da6cbdf493f09e609a', 'Mà', null, '2020-11-07T19:48:35Z'); +insert into users (id, name, description, modified_at) values ('21281de02f7efaa4eb70373cf73efc8b', 'Kuí', null, '2020-06-25T22:59:31Z'); +insert into users (id, name, description, modified_at) values ('3bcd9c127e5800c3512e3d0f4e24e01d', 'Ophélie', 'Laceration without foreign body of right eyelid and periocular area, sequela', '2020-06-09T09:34:20Z'); +insert into users (id, name, description, modified_at) values ('b3eea23a7f8616103ed7e5038c7b59a7', 'Sòng', null, '2020-05-27T14:21:01Z'); +insert into users (id, name, description, modified_at) values ('4ccb9a4a32c55380577df1cd5dde92bf', 'Léane', null, '2020-08-07T00:16:20Z'); +insert into users (id, name, description, modified_at) values ('54641cc748b8b333cba6eee855402da5', 'Crééz', null, '2020-02-06T01:48:36Z'); +insert into users (id, name, description, modified_at) values ('fb6417671e151b3f0b2a40a2a6328d2c', 'Táng', null, '2020-06-21T07:03:17Z'); +insert into users (id, name, description, modified_at) values ('7726dc9af4ba62a31a8fa8dd8cd89274', 'Josée', null, '2020-08-02T08:49:15Z'); +insert into users (id, name, description, modified_at) values ('4a6f3abc26b7b304d3b9a5f0c1713282', 'Hélèna', null, '2020-07-24T03:36:17Z'); +insert into users (id, name, description, modified_at) values ('e57f2a98a35f48ed0f75ed830276bdc3', 'Dafnée', 'Gastrointestinal stromal tumor, unspecified site', '2020-05-26T10:24:05Z'); +insert into users (id, name, description, modified_at) values ('c60e4a31141f17aceac6ed7395f9fba8', 'Aloïs', null, '2020-08-27T22:20:49Z'); +insert into users (id, name, description, modified_at) values ('ce2e368188f2d772e73d78df80d59085', 'Angélique', 'Underdosing of anthelminthics, sequela', '2020-01-11T12:07:08Z'); +insert into users (id, name, description, modified_at) values ('d3ab8583c27b96f7aa313c5065c8162f', 'Dafnée', null, '2020-02-05T13:02:19Z'); +insert into users (id, name, description, modified_at) values ('b011286d1db9b5a341c7208ae1b54504', 'Célestine', 'Person on outside of other special all-terrain or other off-road motor vehicles injured in nontraffic accident, sequela', '2020-07-04T16:24:56Z'); +insert into users (id, name, description, modified_at) values ('94bbdb26b624f27c845cc8c6c2f06939', 'Frédérique', 'Solitary bone cyst, right hand', '2020-05-06T09:51:16Z'); +insert into users (id, name, description, modified_at) values ('4276c4ac8991e2af6ab77716209b4fd0', 'Valérie', 'Other juvenile arthritis, unspecified ankle and foot', '2020-07-05T21:48:29Z'); +insert into users (id, name, description, modified_at) values ('17f64a94284798b21c15f0f3cdec3b0f', 'Anaëlle', null, '2020-06-10T18:06:15Z'); +insert into users (id, name, description, modified_at) values ('a06975b01ab61801fb9daf800054f225', 'Marlène', null, '2020-06-24T21:56:35Z'); +insert into users (id, name, description, modified_at) values ('739c1e9db6de540be2835713c074b689', 'Mélinda', null, '2020-05-05T14:25:15Z'); +insert into users (id, name, description, modified_at) values ('baf54fbea96f539a887b36829e7d68ce', 'Méthode', 'Unspecified injury of unspecified blood vessel at lower leg level, left leg, initial encounter', '2020-03-13T00:49:27Z'); +insert into users (id, name, description, modified_at) values ('42895f24f77617409e8540dc8fc72972', 'Aí', 'Conjoined twin pregnancy, third trimester', '2020-01-24T11:47:57Z'); +insert into users (id, name, description, modified_at) values ('eca9faee7e259c79391fe5ef99f7acd1', 'Nélie', null, '2020-01-23T03:48:27Z'); +insert into users (id, name, description, modified_at) values ('f268df39ae55aa3b6a7ec8ae8fabc8d7', 'Béatrice', null, '2020-08-28T17:26:18Z'); +insert into users (id, name, description, modified_at) values ('7a373bb05c8fabf6f6267d5f2fc6c0d2', 'Anaëlle', null, '2020-03-25T19:45:28Z'); +insert into users (id, name, description, modified_at) values ('3042043d88293e598e0f0a818847ec99', 'Personnalisée', null, '2020-08-16T19:26:36Z'); +insert into users (id, name, description, modified_at) values ('87a6119528910d45234574138526d8d7', 'Aimée', 'Chronic inflammatory disorders of orbit', '2020-02-02T10:36:51Z'); +insert into users (id, name, description, modified_at) values ('ae342d9a86b479c0acaedf1c802ec6fc', 'Méryl', 'Unspecified open wound of other finger without damage to nail', '2020-09-13T03:04:05Z'); +insert into users (id, name, description, modified_at) values ('45e163118fdf5d5dc570e80176606cbe', 'Daphnée', null, '2020-02-08T07:46:42Z'); +insert into users (id, name, description, modified_at) values ('3fc6f23507d837177345ae2357bf15a5', 'Léa', null, '2020-10-17T20:10:32Z'); +insert into users (id, name, description, modified_at) values ('fee9f33cab42bd5ee92eb870cfd7089c', 'Intéressant', null, '2020-01-27T10:36:08Z'); +insert into users (id, name, description, modified_at) values ('df80d3ce9afb4af7950602d12258bf49', 'Adèle', null, '2020-01-06T22:17:09Z'); +insert into users (id, name, description, modified_at) values ('77bdfee389c4185780793d2d1d543562', 'Béatrice', 'Other low birth weight newborn, 1500-1749 grams', '2020-11-27T09:55:01Z'); +insert into users (id, name, description, modified_at) values ('3ee8c0e9975548162e72584bf7bfb2cf', 'Mylène', 'Acute monoblastic/monocytic leukemia, in remission', '2020-11-23T14:15:46Z'); +insert into users (id, name, description, modified_at) values ('7680902fe3e76c06515062d33667c642', 'Andrée', null, '2020-07-28T13:55:36Z'); +insert into users (id, name, description, modified_at) values ('37ca0a1005aae5c091e5572692e7fbbc', 'Táng', null, '2020-01-08T07:44:29Z'); +insert into users (id, name, description, modified_at) values ('58feff392a8bbab8673f032830887479', 'Táng', null, '2020-08-22T17:21:18Z'); +insert into users (id, name, description, modified_at) values ('288976ac0f8c19413456897b50c8a446', 'Aimée', null, '2020-09-03T19:47:23Z'); +insert into users (id, name, description, modified_at) values ('02bb8bd0a80c434bb46c9af98a7f7857', 'Maëline', null, '2020-07-16T08:26:05Z'); +insert into users (id, name, description, modified_at) values ('7b62e47d5034145b53614050b009bea0', 'Desirée', null, '2020-11-25T21:48:48Z'); +insert into users (id, name, description, modified_at) values ('a7748503f22719db7516cd04b623755d', 'Thérèse', null, '2020-03-11T16:29:34Z'); +insert into users (id, name, description, modified_at) values ('f675440f2d33db076f8e3776aeaf1298', 'Adélaïde', null, '2020-08-22T04:33:53Z'); +insert into users (id, name, description, modified_at) values ('c5c4d86d72f0f390b31a1d17d2755b99', 'Adélie', 'Toxic effect of venom of other reptiles, undetermined', '2020-02-03T15:31:40Z'); +insert into users (id, name, description, modified_at) values ('bccb2b74fd5f96e248eb001e1d9f57a0', 'Táng', null, '2020-03-25T17:50:02Z'); +insert into users (id, name, description, modified_at) values ('519d9321cc32e58e037f93ab1897ae79', 'Zoé', 'Malignant melanoma of left upper limb, including shoulder', '2020-10-18T14:04:02Z'); +insert into users (id, name, description, modified_at) values ('cc90859ec697c52e11404725e3828318', 'Görel', 'Poisoning by monoamine-oxidase-inhibitor antidepressants, intentional self-harm, initial encounter', '2020-11-23T00:30:58Z'); +insert into users (id, name, description, modified_at) values ('0041bc287b0c36aef968bc5d1588c31b', 'Kuí', null, '2020-06-30T05:05:44Z'); +insert into users (id, name, description, modified_at) values ('1e568fb8e3a71639eaef41ab09864d27', 'Personnalisée', 'Displaced segmental fracture of shaft of ulna, right arm, subsequent encounter for closed fracture with nonunion', '2020-11-06T03:01:56Z'); +insert into users (id, name, description, modified_at) values ('14e0aad55761d1b01eb5cca736601c72', 'Illustrée', null, '2020-11-13T02:32:07Z'); +insert into users (id, name, description, modified_at) values ('0973249fea81d8eb829080a7c9019428', 'Magdalène', 'Labor and delivery complicated by cord around neck, without compression, fetus 1', '2020-10-01T02:50:55Z'); +insert into users (id, name, description, modified_at) values ('83ba4dff032add3e788118b7b960b9dc', 'Loïca', null, '2020-01-15T19:09:04Z'); +insert into users (id, name, description, modified_at) values ('0984399543bd0f56e505bf36c7029109', 'Léane', 'Displacement of other vascular grafts', '2020-06-25T16:12:45Z'); +insert into users (id, name, description, modified_at) values ('5cf9c28337382053f4d848245168e6bb', 'Bérangère', null, '2020-01-26T04:37:31Z'); +insert into users (id, name, description, modified_at) values ('f05973cc22387e03e386f6a96cacce8c', 'Eugénie', null, '2020-11-27T22:41:42Z'); +insert into users (id, name, description, modified_at) values ('c6b9d811d95edd4a8858611b3fa59875', 'Océane', null, '2020-05-02T19:50:36Z'); +insert into users (id, name, description, modified_at) values ('c94ad9f5af3f41b64566b32ed9f0234a', 'Renée', null, '2020-11-14T19:45:55Z'); +insert into users (id, name, description, modified_at) values ('6fa6c08d5ff929a84dc24b078b424196', 'Ophélie', 'Unspecified helicopter accident injuring occupant, subsequent encounter', '2020-01-06T18:41:50Z'); +insert into users (id, name, description, modified_at) values ('13a6942a68fb2dcfd32e5bab1228e716', 'Styrbjörn', null, '2020-01-14T13:53:41Z'); +insert into users (id, name, description, modified_at) values ('94a2796ecd378fe391130a0d37e2d4aa', 'Yè', null, '2020-07-22T05:27:03Z'); +insert into users (id, name, description, modified_at) values ('0ac5f0576805dd6b9e33e7ea2ac976e2', 'Noémie', null, '2020-08-29T15:31:52Z'); +insert into users (id, name, description, modified_at) values ('338b014371f080afa61af546f35fe1d7', 'Angélique', 'Displaced intertrochanteric fracture of unspecified femur, subsequent encounter for closed fracture with routine healing', '2020-11-30T16:53:47Z'); +insert into users (id, name, description, modified_at) values ('b0c99ced9eb72925f1d835a03b57fbf8', 'Gaëlle', null, '2020-03-02T11:38:43Z'); +insert into users (id, name, description, modified_at) values ('acc359c0f30281d28487f157ee86b249', 'Valérie', 'Nondisplaced fracture of left tibial tuberosity, subsequent encounter for closed fracture with delayed healing', '2020-07-07T12:16:28Z'); +insert into users (id, name, description, modified_at) values ('45d1824c861ec657c6b8b7c86a1f570f', 'Annotés', 'Drug-induced folate deficiency anemia', '2020-02-29T00:30:12Z'); +insert into users (id, name, description, modified_at) values ('d267d5484041a529f34b9a7f3af97278', 'Réjane', null, '2020-10-28T15:54:48Z'); +insert into users (id, name, description, modified_at) values ('1d2491674b0daaed2e5026fbaaba77d9', 'Marlène', 'Adverse effect of local antifungal, anti-infective and anti-inflammatory drugs, initial encounter', '2020-06-21T17:48:17Z'); +insert into users (id, name, description, modified_at) values ('15447e73eedf70960b2fd6154e9ae684', 'Pò', null, '2020-10-12T01:52:34Z'); +insert into users (id, name, description, modified_at) values ('6f8dbb9b049ec39b963297ad2bfdd053', 'Bérengère', null, '2020-06-17T05:30:51Z'); +insert into users (id, name, description, modified_at) values ('28e8e0ef294a5ca3b40486a924cc9878', 'Pål', 'Displaced osteochondral fracture of right patella, subsequent encounter for open fracture type I or II with malunion', '2020-10-21T02:08:07Z'); +insert into users (id, name, description, modified_at) values ('7e1255363e8f1e605c0cdd8b3c7215ce', 'Almérinda', null, '2020-05-11T19:50:44Z'); +insert into users (id, name, description, modified_at) values ('cec39527912d7b8972c0aae1be4fe745', 'Pélagie', null, '2020-07-23T17:54:51Z'); +insert into users (id, name, description, modified_at) values ('b77da4e2bec2768b7ae4279c4ac879a7', 'Crééz', null, '2020-11-17T00:46:13Z'); +insert into users (id, name, description, modified_at) values ('ad7f3a51f44530cca1e48193f7701d86', 'Inès', null, '2020-02-16T19:26:26Z'); +insert into users (id, name, description, modified_at) values ('c98519b273fa201e4b84abd97b842eec', 'Géraldine', null, '2020-02-08T01:42:28Z'); +insert into users (id, name, description, modified_at) values ('e2179c257aeab59fe01fa41769e55dac', 'Mégane', null, '2020-05-17T08:46:46Z'); +insert into users (id, name, description, modified_at) values ('20674050462688808e9fa7bdf489a6b6', 'Judicaël', null, '2020-09-16T00:43:04Z'); +insert into users (id, name, description, modified_at) values ('390e73c7aa672b63e0c0af4fafb4d00e', 'Görel', null, '2020-04-01T08:07:00Z'); +insert into users (id, name, description, modified_at) values ('2b3a00602e62d53c9772687f3e5b01e0', 'Chloé', null, '2020-05-25T12:14:26Z'); +insert into users (id, name, description, modified_at) values ('da30a79c3e2766819e78f1e70ce25800', 'Crééz', null, '2020-08-22T07:26:45Z'); +insert into users (id, name, description, modified_at) values ('7e08dbe7299b6ec60cd0be7fe6714d41', 'Méthode', 'Scratched by cat, subsequent encounter', '2020-08-04T09:23:22Z'); +insert into users (id, name, description, modified_at) values ('3840c62defc1b693b86cde7cb0011782', 'Adélaïde', null, '2020-10-12T07:35:14Z'); +insert into users (id, name, description, modified_at) values ('30842a8938a826396ac102381df3e664', 'Véronique', null, '2020-02-14T05:28:45Z'); +insert into users (id, name, description, modified_at) values ('e6ecf2c2ed21070d1b735d4a9ec116af', 'Lèi', 'Hemorrhage of anus and rectum', '2020-11-08T12:49:18Z'); +insert into users (id, name, description, modified_at) values ('a952daa13f8f9aaae459ccc6370cefa1', 'Nadège', 'Osseous stenosis of neural canal of cervical region', '2020-01-12T13:13:40Z'); +insert into users (id, name, description, modified_at) values ('78f868ecba541cf72b9d55d67b54d93e', 'Håkan', null, '2020-08-05T09:48:43Z'); +insert into users (id, name, description, modified_at) values ('afdc59b1960f98095857aaf1c66145fa', 'Anaël', null, '2020-06-04T04:59:12Z'); +insert into users (id, name, description, modified_at) values ('ea8ade603b7079e5f26478bd29ffc3d3', 'Lài', null, '2020-09-04T18:49:13Z'); +insert into users (id, name, description, modified_at) values ('ae0a9bffa642f0e821b996fb3dfe93bb', 'Solène', null, '2020-05-20T00:31:35Z'); +insert into users (id, name, description, modified_at) values ('01ab7e2c716e47abb5af57bd1aabeb31', 'Nuó', null, '2020-07-24T19:18:02Z'); +insert into users (id, name, description, modified_at) values ('71bb59d15f4c8f086ef1811da2d17945', 'Cloé', null, '2020-09-11T15:12:18Z'); +insert into users (id, name, description, modified_at) values ('47344055e714b59f2b7ceec02fb4824e', 'Esbjörn', null, '2020-11-16T08:37:22Z'); +insert into users (id, name, description, modified_at) values ('6524c8f36d50e3e28cc086aea1b733f2', 'Táng', null, '2020-08-16T06:35:20Z'); +insert into users (id, name, description, modified_at) values ('7931fcf8a78e9bad20df1b74b4d152bc', 'Fèi', null, '2020-11-28T20:33:18Z'); +insert into users (id, name, description, modified_at) values ('ee203f408a2cdaa830d64e995c85e288', 'Camélia', null, '2020-05-06T03:01:03Z'); +insert into users (id, name, description, modified_at) values ('b44d70a9d28a31ec656bb4d21ef459b3', 'Irène', null, '2020-01-10T07:30:44Z'); +insert into users (id, name, description, modified_at) values ('b8652254298c8b117c861f22fe583de1', 'Pål', null, '2020-11-04T13:12:45Z'); +insert into users (id, name, description, modified_at) values ('e91e8fdf0f3b9947ddad53ea331740ce', 'Daphnée', null, '2020-01-28T18:17:02Z'); +insert into users (id, name, description, modified_at) values ('9cc50c17732afa93777765c09948f8cd', 'Athéna', 'Malignant neoplasm of peripheral nerves of unspecified upper limb, including shoulder', '2020-05-30T21:20:17Z'); +insert into users (id, name, description, modified_at) values ('ed970edfdcc48db86f3fc8cef8bde131', 'Liè', 'Contusion of left upper arm', '2020-08-27T23:59:30Z'); +insert into users (id, name, description, modified_at) values ('0b6060a20aa2cd4ea15722ddb14811a4', 'Magdalène', 'Contact blepharoconjunctivitis, unspecified eye', '2020-09-03T19:13:44Z'); +insert into users (id, name, description, modified_at) values ('8679f0577014967b25ddbdb55384009c', 'Néhémie', 'Unspecified fracture of left calcaneus, subsequent encounter for fracture with routine healing', '2020-04-24T19:20:54Z'); +insert into users (id, name, description, modified_at) values ('14fc9cdaa9f2c3a28c88468ca50c0e80', 'Garçon', null, '2020-10-27T00:38:18Z'); +insert into users (id, name, description, modified_at) values ('eb996da5b154581cdabfe7e87919110d', 'Yè', 'Sedative, hypnotic or anxiolytic use, unspecified with other sedative, hypnotic or anxiolytic-induced disorders', '2020-06-26T23:12:08Z'); +insert into users (id, name, description, modified_at) values ('91d811ce8c1101fe1ae07a1ff84a5ae8', 'Styrbjörn', null, '2020-06-22T05:10:45Z'); +insert into users (id, name, description, modified_at) values ('fcfa4dfc9ffa6788a6d972bfc7c8d344', 'Audréanne', null, '2020-11-26T12:32:14Z'); +insert into users (id, name, description, modified_at) values ('5394ba3b39cada791da7d0032826ff1f', 'Jú', null, '2020-06-18T04:18:25Z'); +insert into users (id, name, description, modified_at) values ('7074432850f69d29ea8f7557503b9bfd', 'Rébecca', null, '2020-06-10T08:51:23Z'); +insert into users (id, name, description, modified_at) values ('27b39903c98627f5dc7058d894170365', 'Céline', 'Burn of third degree of back of unspecified hand, subsequent encounter', '2020-10-10T02:54:51Z'); +insert into users (id, name, description, modified_at) values ('5d7a557767440b4f1f93087a0c0884f7', 'Judicaël', 'Military operations involving other forms of conventional warfare, civilian, initial encounter', '2020-01-11T18:19:28Z'); +insert into users (id, name, description, modified_at) values ('3106436cf7413c7923b4b32acb8ed524', 'Nélie', null, '2020-04-24T09:58:42Z'); +insert into users (id, name, description, modified_at) values ('735c7046bd65c13b3a4d5acdfc38cc0c', 'Lóng', null, '2020-03-01T16:33:29Z'); +insert into users (id, name, description, modified_at) values ('2d968efa7040baf3aff66c714fe921ef', 'Daphnée', null, '2020-10-02T08:25:22Z'); +insert into users (id, name, description, modified_at) values ('ae270ef8b31dd9a4e6a61c8cec717da6', 'Méghane', 'Mucosal cyst of postmastoidectomy cavity, left ear', '2020-08-01T13:44:06Z'); +insert into users (id, name, description, modified_at) values ('22e3c2b491db78acbdde07a753f4de9c', 'Gaétane', 'Nondisplaced fracture of navicular [scaphoid] of left foot, sequela', '2020-07-07T00:33:33Z'); +insert into users (id, name, description, modified_at) values ('a62d334fd0eb55f0df14d8ff4705a5fd', 'Alizée', null, '2020-01-25T19:15:18Z'); +insert into users (id, name, description, modified_at) values ('c45b7095d8633603a03c92badbbbaaa2', 'Cloé', 'Unspecified injury of lower leg', '2020-11-09T04:06:32Z'); +insert into users (id, name, description, modified_at) values ('418ec80665abe88a1a4b6044c12ee902', 'Agnès', null, '2020-03-03T11:56:09Z'); +insert into users (id, name, description, modified_at) values ('d85e9cd82d8a7a3e6fe350202b22ecdd', 'Pélagie', 'Bucket-handle tear of lateral meniscus, current injury, left knee', '2020-05-25T09:28:05Z'); +insert into users (id, name, description, modified_at) values ('d8d61497d8988f6d7b28158eba64a782', 'Loïs', 'Unspecified benign mammary dysplasia of left breast', '2020-09-03T10:03:21Z'); +insert into users (id, name, description, modified_at) values ('ba9bf328bd08f47717d0104aebbc555a', 'Liè', 'Subluxation of unspecified interphalangeal joint of right ring finger, sequela', '2020-02-19T11:54:10Z'); +insert into users (id, name, description, modified_at) values ('d10b92454dc5bedf2681716e8fbcdd7f', 'Maëlla', null, '2020-04-01T14:03:13Z'); +insert into users (id, name, description, modified_at) values ('8311ae4f4696b67e27674d0540412bb2', 'Estée', 'Other noninfective disorders of pinna', '2020-05-29T17:57:56Z'); +insert into users (id, name, description, modified_at) values ('69c8abc948f09fc31967c3c09e824f16', 'Bécassine', null, '2020-05-13T17:42:11Z'); +insert into users (id, name, description, modified_at) values ('a79e068396c422cec02d8693e65afa98', 'Maëline', null, '2020-04-13T01:17:55Z'); +insert into users (id, name, description, modified_at) values ('aa7739e9c81d4e0ffa44a0e66349be4a', 'Angélique', 'Follicular lymphoma grade I, intra-abdominal lymph nodes', '2020-09-13T03:15:16Z'); +insert into users (id, name, description, modified_at) values ('885b2c9faf724210a4acb7e349aa32a5', 'Mélanie', null, '2020-08-08T09:37:49Z'); +insert into users (id, name, description, modified_at) values ('e3ac8240f33895deb250ee68e403ce78', 'Sélène', null, '2020-09-21T06:27:10Z'); +insert into users (id, name, description, modified_at) values ('9ff8f4b474507d0342c4f455a065d4e7', 'Lèi', null, '2020-09-02T17:15:18Z'); +insert into users (id, name, description, modified_at) values ('4b0440a8886aab43f34f6c7d8d2f5cdf', 'Renée', null, '2020-03-01T01:35:52Z'); +insert into users (id, name, description, modified_at) values ('4dbec809e3092a6984c40f664de4a996', 'Hélène', '3-part fracture of surgical neck of left humerus, subsequent encounter for fracture with nonunion', '2020-03-10T14:02:21Z'); +insert into users (id, name, description, modified_at) values ('acb334a013b7449122a75690c9c36d3e', 'Irène', 'Poisoning by unspecified nonopioid analgesic, antipyretic and antirheumatic, undetermined, subsequent encounter', '2020-11-18T22:20:38Z'); +insert into users (id, name, description, modified_at) values ('b041f135412c3038706357bec579469c', 'Ruò', null, '2020-06-25T23:02:06Z'); +insert into users (id, name, description, modified_at) values ('9698ff56ba3989bdb43624d3b73a8429', 'Cléopatre', null, '2020-06-01T10:45:30Z'); +insert into users (id, name, description, modified_at) values ('456afa27e49890506cdbac9697e5b442', 'Bérangère', null, '2020-08-21T21:47:31Z'); +insert into users (id, name, description, modified_at) values ('6203348c872f4f3a5c86b827114ef593', 'Marie-thérèse', null, '2020-06-08T08:53:01Z'); +insert into users (id, name, description, modified_at) values ('36a337fc29f0704aa8a89886a91d85d3', 'Crééz', null, '2020-07-12T15:39:37Z'); +insert into users (id, name, description, modified_at) values ('1d714c070b1fe1af070a5111a35be897', 'Maï', null, '2020-08-05T21:56:38Z'); +insert into users (id, name, description, modified_at) values ('c4f8ef609255588fa7bba41d2431c2bf', 'Gérald', null, '2020-06-17T04:50:39Z'); +insert into users (id, name, description, modified_at) values ('247ad3427b53f27a73513daaa1f45c9e', 'Mylène', null, '2020-09-11T04:15:40Z'); +insert into users (id, name, description, modified_at) values ('7e84d9284e184f2be0592395a688dbae', 'Maëlyss', null, '2020-07-19T03:11:10Z'); +insert into users (id, name, description, modified_at) values ('a12b5907ea935f95a143b2a00bb4a22c', 'Táng', null, '2020-04-19T14:45:33Z'); +insert into users (id, name, description, modified_at) values ('3da082afe325cbb92220d435d2c5bce9', 'Andréanne', 'Presence of artificial skin', '2020-02-15T13:36:35Z'); +insert into users (id, name, description, modified_at) values ('885f167e3f5824a4e66bf48886d1fb92', 'Estève', null, '2020-10-07T01:20:53Z'); +insert into users (id, name, description, modified_at) values ('50141aa9a0a85144f497629b487ff7db', 'Loïs', null, '2020-05-23T20:45:20Z'); +insert into users (id, name, description, modified_at) values ('aa87c90deb3040ee6677f9196f2e7e4b', 'Dà', null, '2020-07-08T15:12:57Z'); +insert into users (id, name, description, modified_at) values ('d7a4fddb58db4e00222aa6616317806b', 'Ruò', null, '2020-07-08T08:50:40Z'); +insert into users (id, name, description, modified_at) values ('e53ea5e7fc37c0626120e70f08f87801', 'Andréanne', null, '2020-04-14T19:10:48Z'); +insert into users (id, name, description, modified_at) values ('253b881ad3b51b855ad41bc00d625746', 'Aí', 'Blister (nonthermal) of unspecified part of head, sequela', '2020-06-12T11:15:37Z'); +insert into users (id, name, description, modified_at) values ('c4cacc8a0faa7235c7707432877670af', 'Sélène', null, '2020-09-26T12:33:45Z'); +insert into users (id, name, description, modified_at) values ('22bd6d5e9cebff0aa1f40652759716d7', 'Marie-thérèse', null, '2020-10-24T14:29:38Z'); +insert into users (id, name, description, modified_at) values ('19de96a38748fb71112093af219f31ee', 'Táng', null, '2020-02-26T00:16:28Z'); +insert into users (id, name, description, modified_at) values ('d6086efb7e9c8910149210412cd9a7ce', 'Néhémie', null, '2020-01-24T14:36:24Z'); +insert into users (id, name, description, modified_at) values ('b7d0af6ef9faf56808c490bf36a715e1', 'Börje', 'Encounter for screening for certain developmental disorders in childhood', '2020-10-14T19:22:49Z'); +insert into users (id, name, description, modified_at) values ('6db83ed2dd539f0573186e1a421412ea', 'Naéva', null, '2020-11-26T02:12:11Z'); +insert into users (id, name, description, modified_at) values ('3b103383be7e8c48850b6954fd76ad3d', 'Yáo', null, '2020-11-08T01:49:47Z'); +insert into users (id, name, description, modified_at) values ('bb23812fee9826cbba30fb9883e5889e', 'Adélie', 'Driver of heavy transport vehicle injured in collision with car, pick-up truck or van in nontraffic accident, sequela', '2020-11-30T07:37:59Z'); +insert into users (id, name, description, modified_at) values ('aabd9b0fec1d2680692e82ced0ca174a', 'Camélia', null, '2020-01-26T02:22:06Z'); +insert into users (id, name, description, modified_at) values ('d38475df8b197608d269163d883ac547', 'Camélia', null, '2020-04-25T19:35:27Z'); +insert into users (id, name, description, modified_at) values ('65535da8d0b80f0b9845d79de9cf4358', 'Gisèle', null, '2020-01-27T03:46:36Z'); +insert into users (id, name, description, modified_at) values ('acc6e55492f9ededede5dc54eee8ffa6', 'Solène', null, '2020-05-19T07:26:53Z'); +insert into users (id, name, description, modified_at) values ('fddb2e75d517ab6e8877917af0ec943c', 'Simplifiés', 'Person on outside of pick-up truck or van injured in collision with heavy transport vehicle or bus in nontraffic accident', '2020-10-24T00:45:12Z'); +insert into users (id, name, description, modified_at) values ('6f8dedd6620e2ae9b3b724945419c452', 'Lauréna', null, '2020-09-24T22:18:42Z'); +insert into users (id, name, description, modified_at) values ('832153576f93f7f4de69c6cf92420200', 'Réjane', null, '2020-09-24T09:20:09Z'); +insert into users (id, name, description, modified_at) values ('92e184ce329a41c0caaffde6aae7cc19', 'Stévina', null, '2020-10-31T02:47:11Z'); +insert into users (id, name, description, modified_at) values ('57c2fc4d70ddb2e6e6f7a51257570c30', 'Lorène', 'Drowning and submersion due to being washed overboard from passenger ship, subsequent encounter', '2020-01-03T06:02:56Z'); +insert into users (id, name, description, modified_at) values ('40df9567f6ef9b000d8ba4b737191209', 'Michèle', 'Unspecified fracture of third lumbar vertebra, subsequent encounter for fracture with routine healing', '2020-08-19T05:57:53Z'); +insert into users (id, name, description, modified_at) values ('7d821a1249a574c9cbb2610a6b90a302', 'Marie-françoise', null, '2020-01-24T15:06:37Z'); +insert into users (id, name, description, modified_at) values ('97a20f45b7b8c8f1a88e928bcc3c6143', 'Pò', null, '2020-07-04T14:35:28Z'); +insert into users (id, name, description, modified_at) values ('672318c6a8247e27f4b74e247fb8bff1', 'Mylène', 'Poisoning by predominantly beta-adrenoreceptor agonists, accidental (unintentional), subsequent encounter', '2020-05-31T13:53:15Z'); +insert into users (id, name, description, modified_at) values ('8970f882f563cf7987a704d9337e0d4a', 'Personnalisée', null, '2020-01-29T12:05:41Z'); +insert into users (id, name, description, modified_at) values ('7ab28ce762134295338f14f80a810d40', 'Gisèle', null, '2020-08-02T18:38:48Z'); +insert into users (id, name, description, modified_at) values ('72929c9da0d770d694868f93f0531826', 'Marlène', 'Other specified disorders of synovium and tendon, unspecified knee', '2020-03-28T06:29:54Z'); +insert into users (id, name, description, modified_at) values ('e41e212f763b9c0befa7d587d15519c8', 'Cléa', null, '2020-08-19T05:36:26Z'); +insert into users (id, name, description, modified_at) values ('c098a2f859f7e98743bd7d91e00937ba', 'Erwéi', 'Laceration of abdominal wall with foreign body, right upper quadrant without penetration into peritoneal cavity, sequela', '2020-09-21T03:47:03Z'); +insert into users (id, name, description, modified_at) values ('f77ea98108aea734185bc767cbc1b2a9', 'Camélia', null, '2020-02-22T01:16:34Z'); +insert into users (id, name, description, modified_at) values ('65e264a09010162f5f93b2370641dfd2', 'Liè', null, '2020-04-05T10:43:16Z'); +insert into users (id, name, description, modified_at) values ('ec2cf2afc11710cc686536f3141a8400', 'Mégane', 'Neurological devices associated with adverse incidents', '2020-11-11T17:27:12Z'); +insert into users (id, name, description, modified_at) values ('93af484bd02380dfd2f2010bf3997421', 'Danièle', null, '2020-08-07T13:43:38Z'); +insert into users (id, name, description, modified_at) values ('36639b1a5fcac546b0c752314f1e83fa', 'Chloé', null, '2020-09-03T10:22:49Z'); +insert into users (id, name, description, modified_at) values ('8c5ca19c75d1aeafcc239eb367bfa4f0', 'Publicité', null, '2020-02-22T00:28:35Z'); +insert into users (id, name, description, modified_at) values ('e037187a6587418c9fa4357a52a8cacc', 'Cléopatre', 'Tuberculous meningitis', '2020-11-26T13:56:35Z'); +insert into users (id, name, description, modified_at) values ('71b3c2998b32562bfa376675aa1b5341', 'Joséphine', null, '2020-10-02T09:42:39Z'); +insert into users (id, name, description, modified_at) values ('3ae189b01002f1cf7daa0f3467657d16', 'Östen', null, '2020-05-31T22:32:30Z'); +insert into users (id, name, description, modified_at) values ('6082da4a9e91f4af1a2cb99f91b371bb', 'Laïla', 'Burn of unspecified degree of multiple sites of left ankle and foot, initial encounter', '2020-01-21T07:47:02Z'); +insert into users (id, name, description, modified_at) values ('ce3a726e65234b8e1ecf8da74d57bc78', 'Réjane', 'Displaced fracture of proximal phalanx of right lesser toe(s), subsequent encounter for fracture with malunion', '2020-11-22T23:58:20Z'); +insert into users (id, name, description, modified_at) values ('1651cf8cad1076be6b7f20913794af2f', 'Néhémie', 'Multifocal and multisystemic (disseminated) Langerhans-cell histiocytosis', '2020-01-02T01:11:04Z'); +insert into users (id, name, description, modified_at) values ('ff0fb14b306fa88eb7769d995b8d1a2c', 'Méng', null, '2020-02-02T18:23:22Z'); +insert into users (id, name, description, modified_at) values ('9d40fb38bce15ded8000eb10b8abc4a7', 'Nélie', null, '2020-05-15T08:08:17Z'); +insert into users (id, name, description, modified_at) values ('19432ab169c24c2b55e66717f95f16a2', 'Naéva', null, '2020-04-13T06:29:36Z'); +insert into users (id, name, description, modified_at) values ('d070552d32fa4deb9e6e040ea8620a2f', 'Märta', null, '2020-07-17T15:54:30Z'); +insert into users (id, name, description, modified_at) values ('ec1a799f0956293c7deec3529805bcb6', 'Esbjörn', null, '2020-10-10T22:43:22Z'); +insert into users (id, name, description, modified_at) values ('9221c71698fd091f60ddb78e2b8d4820', 'Maïly', null, '2020-10-31T05:32:53Z'); +insert into users (id, name, description, modified_at) values ('6a2d7aca1fa31765322c9fbb3df74306', 'Méryl', null, '2020-09-21T06:00:42Z'); +insert into users (id, name, description, modified_at) values ('5939bf6c9d843e4987253d8b6bed2354', 'Léane', 'Exposure to other rapid changes in air pressure during descent', '2020-11-18T07:03:57Z'); +insert into users (id, name, description, modified_at) values ('80d4ee6a2420d1af65098ef34c61f3e0', 'Marie-josée', null, '2020-09-26T22:29:27Z'); +insert into users (id, name, description, modified_at) values ('3043ac9b2f22b94ed9cd0d43270ab61f', 'Åsa', null, '2020-03-17T20:14:25Z'); +insert into users (id, name, description, modified_at) values ('c06bd4616dcabb33200c8efd8bc77cf1', 'Dà', 'Transient synovitis, right ankle and foot', '2020-01-06T12:27:24Z'); +insert into users (id, name, description, modified_at) values ('dd770fdd841415854fbd9c654d2df6fe', 'Célia', 'Underdosing of diagnostic agents, sequela', '2020-08-28T14:49:06Z'); +insert into users (id, name, description, modified_at) values ('2cd5b3c97de84232aea3a652d21742f5', 'Garçon', 'Nondisplaced fracture of lateral condyle of left humerus, subsequent encounter for fracture with nonunion', '2020-11-20T17:31:24Z'); +insert into users (id, name, description, modified_at) values ('d7428ba3e87d3df42408fa8feb205ffe', 'Rachèle', null, '2020-09-06T09:32:45Z'); +insert into users (id, name, description, modified_at) values ('37537e6214251479eaf94ac166177951', 'Lucrèce', null, '2020-01-01T09:55:03Z'); +insert into users (id, name, description, modified_at) values ('a8718782bc4a562e7e9f1e660bbf76f5', 'Marie-josée', null, '2020-07-17T16:24:21Z'); +insert into users (id, name, description, modified_at) values ('c6b5d9517ebfc6fbf3ad2119a049df30', 'Bécassine', null, '2020-04-16T03:41:23Z'); +insert into users (id, name, description, modified_at) values ('d65ee004c927d7f785a841cf3e7707a8', 'Marylène', 'Displaced fracture of proximal phalanx of left middle finger, subsequent encounter for fracture with routine healing', '2020-09-16T18:07:54Z'); +insert into users (id, name, description, modified_at) values ('77eec590a120e29f9ad9ce305041ee8d', 'Jú', null, '2020-09-28T06:01:45Z'); +insert into users (id, name, description, modified_at) values ('bfabfbb54b72996cc81a65972711b2c8', 'Réjane', 'Displaced transverse fracture of shaft of left femur, subsequent encounter for open fracture type I or II with nonunion', '2020-09-14T13:09:02Z'); +insert into users (id, name, description, modified_at) values ('0bed8f89941fd3fe19ab2addf91b489f', 'Aí', null, '2020-03-24T14:15:25Z'); +insert into users (id, name, description, modified_at) values ('01a08ab3a57dce0f38e7058e0bf2a3b1', 'Angèle', null, '2020-03-06T02:28:06Z'); +insert into users (id, name, description, modified_at) values ('c6222c843642f519cf3183dce2516fbd', 'Andréa', 'Jumping or diving into other water striking bottom causing other injury, sequela', '2020-04-01T02:31:58Z'); +insert into users (id, name, description, modified_at) values ('8f88904ef203970d5d82bf988332078e', 'Kù', null, '2020-01-13T05:37:02Z'); +insert into users (id, name, description, modified_at) values ('0c2df217a7cce26ace210f4f5f3077e4', 'Östen', null, '2020-02-08T10:51:26Z'); +insert into users (id, name, description, modified_at) values ('733ebd6c01176d8b62b7ff586c4a1c17', 'Athéna', null, '2020-07-03T04:24:52Z'); +insert into users (id, name, description, modified_at) values ('b7bd62b574fe8bc5c32a18f7e0bdec0e', 'Andréa', null, '2020-10-07T09:32:57Z'); +insert into users (id, name, description, modified_at) values ('3e7f3407214d6c8f76b483c25e28727f', 'Josée', null, '2020-02-20T18:26:26Z'); +insert into users (id, name, description, modified_at) values ('628f675742df8ceae3d371c813cc9c7e', 'Liè', null, '2020-02-13T14:14:41Z'); +insert into users (id, name, description, modified_at) values ('5d4ba126fde694209c5cc71736d3101b', 'Pål', 'Adverse effect of other viral vaccines, initial encounter', '2020-03-09T22:33:38Z'); +insert into users (id, name, description, modified_at) values ('6c14eda4213150d6dd37418cf4968433', 'Chloé', null, '2020-08-17T08:25:01Z'); +insert into users (id, name, description, modified_at) values ('e324937c216110b8e80f841e3a61cfc5', 'Kallisté', null, '2020-05-25T07:34:14Z'); +insert into users (id, name, description, modified_at) values ('eb5c58ea954b19e516bc229ed1445a4f', 'Cunégonde', null, '2020-03-23T23:30:53Z'); +insert into users (id, name, description, modified_at) values ('3af8b14d1fe750324d983e3217506529', 'Françoise', null, '2020-07-24T01:05:18Z'); +insert into users (id, name, description, modified_at) values ('4dd5bd14cb9e33b2c6f4b1d0237ea6f3', 'Laurène', null, '2020-01-01T00:38:44Z'); +insert into users (id, name, description, modified_at) values ('3a07fe9b4e8651db1b268fd64abe2727', 'Béatrice', null, '2020-01-10T23:17:50Z'); +insert into users (id, name, description, modified_at) values ('bbd44ea43c1cec95bdd87602a591a265', 'Célestine', null, '2020-06-26T15:47:52Z'); +insert into users (id, name, description, modified_at) values ('3545cff03b4b8cac2396199589518209', 'Marlène', null, '2020-08-21T00:04:09Z'); +insert into users (id, name, description, modified_at) values ('e52c176dce748e0283b1f4da28236671', 'Réjane', 'Strain of other specified muscles, fascia and tendons at thigh level, right thigh, subsequent encounter', '2020-01-13T00:47:21Z'); +insert into users (id, name, description, modified_at) values ('423083155b41545d13cabdcd8cb11859', 'Tú', null, '2020-04-30T09:22:43Z'); +insert into users (id, name, description, modified_at) values ('2e8cdcb5e52ee5122f7de717aeeff13b', 'Félicie', null, '2020-07-26T06:12:27Z'); +insert into users (id, name, description, modified_at) values ('e987f654ed452a63566e3094dbcd8181', 'Tán', null, '2020-04-05T19:10:11Z'); +insert into users (id, name, description, modified_at) values ('29594a8776974bc11a964dd25b39d584', 'Joséphine', null, '2020-05-03T22:59:07Z'); +insert into users (id, name, description, modified_at) values ('4abc460187f9493f74fa456c335af6bf', 'Géraldine', null, '2020-01-19T00:15:54Z'); +insert into users (id, name, description, modified_at) values ('36c5e2858f6fd7de130611554c0338e6', 'Mà', null, '2020-05-03T23:55:46Z'); +insert into users (id, name, description, modified_at) values ('4910761502644010543067537b9ae344', 'Börje', null, '2020-04-06T10:01:05Z'); +insert into users (id, name, description, modified_at) values ('be805207cbb2b626afdc251c94c2539d', 'Jú', null, '2020-04-29T03:32:11Z'); +insert into users (id, name, description, modified_at) values ('0235a84e83b7cb543b8e29425868ab0d', 'Björn', 'Unspecified nondisplaced fracture of surgical neck of unspecified humerus, subsequent encounter for fracture with routine healing', '2020-07-14T00:27:27Z'); +insert into users (id, name, description, modified_at) values ('7bc724aafd1f617876961786ed7f6e7c', 'Océane', null, '2020-04-13T02:16:37Z'); +insert into users (id, name, description, modified_at) values ('4afcea4027993562960251f633710598', 'Clélia', 'Arthritis due to other bacteria, unspecified wrist', '2020-03-18T08:59:44Z'); +insert into users (id, name, description, modified_at) values ('6049b71662041c2a8ef92cf90295d88a', 'Geneviève', null, '2020-07-06T17:02:22Z'); +insert into users (id, name, description, modified_at) values ('5fd22cad2fab9ca00cdfed06991c958a', 'Marie-josée', 'Other and unspecified disorders of binocular vision', '2020-02-15T15:32:43Z'); +insert into users (id, name, description, modified_at) values ('db366ddbafd519d1aa8c42219ce42e40', 'Anaël', null, '2020-08-24T19:51:20Z'); +insert into users (id, name, description, modified_at) values ('0f4ee2315f50078d1a00f8618c44936c', 'Maëlyss', 'Unspecified blepharoconjunctivitis, left eye', '2020-08-21T12:41:36Z'); +insert into users (id, name, description, modified_at) values ('8803e1a24a82b2c29fc3df4b06dd0924', 'Mélinda', null, '2020-04-11T15:57:16Z'); +insert into users (id, name, description, modified_at) values ('e3c7c92c888a3037206371ab6fab7a36', 'Miléna', null, '2020-08-08T00:41:59Z'); +insert into users (id, name, description, modified_at) values ('06915fa637443550441dd5e09d8ca72c', 'Méghane', null, '2020-10-22T14:05:22Z'); +insert into users (id, name, description, modified_at) values ('85a028dc882647066e43429d77e73708', 'Léa', null, '2020-07-22T17:03:05Z'); +insert into users (id, name, description, modified_at) values ('60bb03e5dc7cb927307b74539a9ebf5f', 'Josée', null, '2020-08-03T00:26:56Z'); +insert into users (id, name, description, modified_at) values ('f2ede6ae6864d8cd54c4c6a99d1b4b64', 'Maëlys', null, '2020-03-29T20:45:52Z'); +insert into users (id, name, description, modified_at) values ('b21f0f2e881968f693533593a5c39fb7', 'Örjan', null, '2020-04-01T09:07:05Z'); +insert into users (id, name, description, modified_at) values ('6112ef94e9aad683646cc3b56857a684', 'Céline', null, '2020-05-02T19:18:37Z'); +insert into users (id, name, description, modified_at) values ('24b36d1fddd3a45a4c7de1b9848b5b1d', 'Mélinda', null, '2020-07-26T06:16:13Z'); +insert into users (id, name, description, modified_at) values ('6776323d2ef6b799692715a6023ffd75', 'Tú', null, '2020-07-09T04:18:14Z'); +insert into users (id, name, description, modified_at) values ('c2baf4fa320a1eba26eeb1903ca5ccd2', 'Noëlla', 'Major laceration of thoracic aorta, initial encounter', '2020-10-05T10:11:05Z'); +insert into users (id, name, description, modified_at) values ('4489202dfe4438ee94b4dabe5d121603', 'Mélina', null, '2020-11-10T17:34:38Z'); +insert into users (id, name, description, modified_at) values ('4129ce405020dd2173b662e2b00eec6f', 'Mårten', null, '2020-10-19T20:45:56Z'); +insert into users (id, name, description, modified_at) values ('8e0785c56ef96c21aa5e362e27d8d726', 'Zoé', null, '2020-10-27T02:04:04Z'); +insert into users (id, name, description, modified_at) values ('f8c2521378b3f660243c9e3122f1115a', 'Marie-hélène', 'Nondisplaced comminuted fracture of right patella', '2020-09-26T23:05:31Z'); +insert into users (id, name, description, modified_at) values ('f513dd91b0cad3b7d32f01501f2b4d13', 'Pò', null, '2020-02-20T07:14:40Z'); +insert into users (id, name, description, modified_at) values ('f93528e689754d3ef68e46e977a76d62', 'Ophélie', null, '2020-04-03T09:52:02Z'); +insert into users (id, name, description, modified_at) values ('5957e315cd518eb6a3f1df170addaaf0', 'Clélia', 'Fracture of unspecified part of scapula, unspecified shoulder, subsequent encounter for fracture with malunion', '2020-04-01T10:42:17Z'); +insert into users (id, name, description, modified_at) values ('d47df9cdc88967b081b724d53a5c0bc8', 'Danièle', 'Pneumoconiosis due to talc dust', '2020-11-10T13:04:14Z'); +insert into users (id, name, description, modified_at) values ('9d07279253cf13b086e73c41a0f2f205', 'Régine', null, '2020-09-08T21:56:15Z'); +insert into users (id, name, description, modified_at) values ('6b3716bdca09d8e2c95f26224817081a', 'Cécile', null, '2020-04-08T16:35:06Z'); +insert into users (id, name, description, modified_at) values ('564e91119fabd8fd9709491c5d97e945', 'Naëlle', null, '2020-11-23T06:24:23Z'); +insert into users (id, name, description, modified_at) values ('88bdcd6bd951b180a5c0ec291743f0b3', 'Léana', null, '2020-01-21T01:26:07Z'); +insert into users (id, name, description, modified_at) values ('0cb4f205e28693c8ea7d61a4fe1f483d', 'Eléonore', 'Toxic effect of fluorine gas and hydrogen fluoride, accidental (unintentional), subsequent encounter', '2020-11-28T07:58:10Z'); +insert into users (id, name, description, modified_at) values ('474e56c09abb110484cd5044bea006a8', 'Véronique', null, '2020-10-01T05:15:16Z'); +insert into users (id, name, description, modified_at) values ('ea375e9b9342139a92149f4db7846722', 'Célestine', 'Twin pregnancy, dichorionic/diamniotic, unspecified trimester', '2020-06-20T16:43:51Z'); +insert into users (id, name, description, modified_at) values ('9884b5383de3cd8365b7e307e9de6151', 'Publicité', null, '2020-09-04T01:37:10Z'); +insert into users (id, name, description, modified_at) values ('a5e7677e31b02dcb0431e3d4c2affc4a', 'Uò', null, '2020-05-07T02:18:20Z'); +insert into users (id, name, description, modified_at) values ('6d911f465edcd70d669dfb2cd700fcae', 'Adèle', null, '2020-02-28T10:39:06Z'); +insert into users (id, name, description, modified_at) values ('ce108216442b9a071681eebfdf934174', 'Lucrèce', 'Displaced fracture of medial phalanx of right little finger, subsequent encounter for fracture with delayed healing', '2020-07-16T02:17:28Z'); +insert into users (id, name, description, modified_at) values ('7ccfd6298e2f3de27594e13080fdd66b', 'Danièle', null, '2020-10-18T05:11:25Z'); +insert into users (id, name, description, modified_at) values ('69fee7932a67fcb660ed2c81de631abd', 'Ruò', null, '2020-07-22T07:26:55Z'); +insert into users (id, name, description, modified_at) values ('e1da4548dc39c562f4396c82e9daa990', 'Marie-noël', null, '2020-02-01T17:25:17Z'); +insert into users (id, name, description, modified_at) values ('ab3854077c9d26e61381db6f1c0d88e9', 'Andréanne', null, '2020-05-18T23:33:21Z'); +insert into users (id, name, description, modified_at) values ('feaf69f5877b8ad9072cc677d387a043', 'Alizée', 'Dislocation of unspecified interphalangeal joint of right middle finger, initial encounter', '2020-06-30T03:39:02Z'); +insert into users (id, name, description, modified_at) values ('6ef2e8ae9151445ee169ca058f5adba8', 'Kuí', null, '2020-04-26T00:10:49Z'); +insert into users (id, name, description, modified_at) values ('62306d2e97e1a2449a0b42a4ba6a15d8', 'Adèle', null, '2020-11-27T15:57:38Z'); +insert into users (id, name, description, modified_at) values ('dd57f30973c9bdfb46576cba6df20c3c', 'Thérèsa', null, '2020-04-16T09:12:13Z'); +insert into users (id, name, description, modified_at) values ('fff56d8dd6622fec9d9aee1371cebcdc', 'Naéva', null, '2020-06-23T16:32:49Z'); +insert into users (id, name, description, modified_at) values ('fe4d9a66732f6836084be5eba01f91a4', 'Kévina', null, '2020-10-25T23:08:04Z'); +insert into users (id, name, description, modified_at) values ('cac6d76a2f5b65fb554f9e644e42d8cc', 'Garçon', null, '2020-07-31T18:17:07Z'); +insert into users (id, name, description, modified_at) values ('7657f7647da955ae926e7d86e1c50d9a', 'Gisèle', 'Laceration without foreign body of unspecified back wall of thorax with penetration into thoracic cavity, subsequent encounter', '2020-09-20T07:23:47Z'); +insert into users (id, name, description, modified_at) values ('43cdc5d91484f368acfc1160fb05a184', 'Dorothée', null, '2020-11-10T14:51:07Z'); +insert into users (id, name, description, modified_at) values ('3cbabe8e606a9d9132e88b8cdbe127b2', 'Céline', null, '2020-04-29T20:13:14Z'); +insert into users (id, name, description, modified_at) values ('ded6f42ce65168e0c1bb5bbb340e29d5', 'Françoise', 'Poisoning by antidiarrheal drugs, assault, initial encounter', '2020-06-18T10:18:42Z'); +insert into users (id, name, description, modified_at) values ('8cbbe01ced0bc8901151a90943bf1147', 'Alizée', null, '2020-10-23T01:09:03Z'); +insert into users (id, name, description, modified_at) values ('203691c500a7d3d6fd8ee2bfe509b863', 'Régine', null, '2020-01-30T06:22:40Z'); +insert into users (id, name, description, modified_at) values ('21a471638284c26cc559c4bf72d5c6b8', 'Mélia', 'Other specified deforming dorsopathies', '2020-08-26T20:50:04Z'); +insert into users (id, name, description, modified_at) values ('c63e7f6654ac5e12949c8c20e6d56903', 'Bérangère', 'Displaced fracture of olecranon process with intraarticular extension of unspecified ulna, subsequent encounter for closed fracture with malunion', '2020-11-20T02:19:39Z'); +insert into users (id, name, description, modified_at) values ('9c1834c3f90272a056891ffb55f30fb1', 'Agnès', null, '2020-02-05T21:27:57Z'); +insert into users (id, name, description, modified_at) values ('047abd961934499b4c6d2f5a0c80dd62', 'Méghane', 'Bitten by other rodent, subsequent encounter', '2020-07-07T10:01:37Z'); +insert into users (id, name, description, modified_at) values ('c4654f1f21a0e3374740631029106e17', 'Lài', null, '2020-07-14T07:52:08Z'); +insert into users (id, name, description, modified_at) values ('2cb1cbf3ce0685d2b03fc2c8159826f4', 'Adèle', null, '2020-02-24T08:48:37Z'); +insert into users (id, name, description, modified_at) values ('3c93851a5ea313897f5591e565ca09b6', 'Célestine', null, '2020-08-23T06:59:09Z'); +insert into users (id, name, description, modified_at) values ('9ccb43e1f6d5351d686c89bc00dbe0e5', 'Mélanie', null, '2020-10-19T23:14:11Z'); +insert into users (id, name, description, modified_at) values ('dcff64a766ff21726d53fbc3aa8de820', 'Maïlys', null, '2020-08-29T01:08:20Z'); +insert into users (id, name, description, modified_at) values ('0409f217c7967fe949d95d59b39b4c6f', 'Cécile', null, '2020-05-06T03:43:32Z'); +insert into users (id, name, description, modified_at) values ('13a64d18b4c95eddb53a5d657788ed38', 'Stévina', 'Unspecified fracture of lower end of left tibia, subsequent encounter for closed fracture with delayed healing', '2020-06-14T04:02:30Z'); +insert into users (id, name, description, modified_at) values ('efb2eb7e2de00a6e250e8330ac20c01d', 'Illustrée', null, '2020-06-12T08:15:23Z'); +insert into users (id, name, description, modified_at) values ('f981fd952f517e61f8defe804b01a6a8', 'Léa', 'Acute bronchitis due to coxsackievirus', '2020-07-04T02:11:54Z'); +insert into users (id, name, description, modified_at) values ('034c7e68ff9a6b6bb8a4c0569b53b505', 'Maïlis', null, '2020-02-28T18:26:35Z'); +insert into users (id, name, description, modified_at) values ('7bbd6c46794480e8032fb18adc193288', 'Laïla', null, '2020-03-21T10:15:20Z'); +insert into users (id, name, description, modified_at) values ('0d3758e32bb92402499c02b8835aa97e', 'Méthode', null, '2020-09-08T18:00:08Z'); +insert into users (id, name, description, modified_at) values ('c592b1a46549ab3948cade04f9aea53c', 'Stévina', null, '2020-07-09T23:56:49Z'); +insert into users (id, name, description, modified_at) values ('af0431d2a4a653074016b8feef3b4a75', 'Joséphine', null, '2020-04-10T04:28:09Z'); +insert into users (id, name, description, modified_at) values ('7cb2647cca91c07ba92822ee585e326c', 'Rachèle', null, '2020-10-28T23:30:42Z'); +insert into users (id, name, description, modified_at) values ('77773e500a711827df88ee8ee192fe1d', 'Jú', null, '2020-04-10T23:57:00Z'); +insert into users (id, name, description, modified_at) values ('7a27e8fc484744a330884a7c0b5f9b2d', 'Naëlle', null, '2020-06-27T04:10:12Z'); +insert into users (id, name, description, modified_at) values ('a2abace77f374f31bddee7101b2f8966', 'Marie-noël', null, '2020-02-25T11:34:32Z'); +insert into users (id, name, description, modified_at) values ('681e47a7be07d1757646a5e0434f89c1', 'Östen', null, '2020-01-14T05:38:15Z'); +insert into users (id, name, description, modified_at) values ('181a059cdb655bdaa9656845129266b4', 'Ophélie', null, '2020-05-22T14:20:54Z'); +insert into users (id, name, description, modified_at) values ('d09a60376c3cc7d17262272afaad585a', 'Maëlys', null, '2020-10-14T06:22:10Z'); +insert into users (id, name, description, modified_at) values ('7c6dda68280ef4824e91bfe48f6a57d8', 'Personnalisée', null, '2020-11-01T13:55:27Z'); +insert into users (id, name, description, modified_at) values ('aa7120402e20eb021977d429629cb948', 'Mélys', 'Contusion of unspecified lesser toe(s) without damage to nail, initial encounter', '2020-04-10T21:58:36Z'); +insert into users (id, name, description, modified_at) values ('49b16748b10cfff591c531c871d99f0b', 'Lóng', null, '2020-02-10T08:31:59Z'); +insert into users (id, name, description, modified_at) values ('2cc284d6d030b883c46662e28d935e2a', 'Ophélie', null, '2020-01-21T07:24:18Z'); +insert into users (id, name, description, modified_at) values ('5393ce35a57bb99405998b43be1d36b2', 'Björn', 'Carcinoma in situ of skin of eyelid, including canthus', '2020-09-22T18:59:44Z'); +insert into users (id, name, description, modified_at) values ('e0061987b4613737ef49955bea297f81', 'Eléa', null, '2020-09-07T01:29:22Z'); +insert into users (id, name, description, modified_at) values ('2e609d51f48ab45d78cf6ab4bdd19ba3', 'Néhémie', null, '2020-04-04T10:06:29Z'); +insert into users (id, name, description, modified_at) values ('7a3cde847df71156256b0f329f7e0839', 'Solène', null, '2020-11-11T11:03:59Z'); +insert into users (id, name, description, modified_at) values ('5d11df78aa1c9d42a5bb8bbc86e9ca89', 'Zhì', 'Bent bone of unspecified radius, initial encounter for closed fracture', '2020-03-03T21:36:11Z'); +insert into users (id, name, description, modified_at) values ('19a6a039b1600b2d975e7a8fc78fe7a5', 'Pål', null, '2020-03-09T08:34:50Z'); +insert into users (id, name, description, modified_at) values ('a8c6bfcc23bc8b1e97e6fdd2903c5eb2', 'Eléonore', 'Fall on same level from slipping, tripping and stumbling with subsequent striking against furniture, initial encounter', '2020-08-13T20:31:51Z'); +insert into users (id, name, description, modified_at) values ('2dba1dfca252edea01985a8138377b9c', 'Clémence', 'Nicotine dependence, cigarettes, in remission', '2020-02-01T07:42:53Z'); +insert into users (id, name, description, modified_at) values ('7c5696b7398dfdb3cb4d711ed82dd20c', 'Ophélie', null, '2020-07-01T21:45:49Z'); +insert into users (id, name, description, modified_at) values ('64f238784abbde645dd4dc8f197e7c2a', 'Michèle', null, '2020-08-04T01:19:36Z'); +insert into users (id, name, description, modified_at) values ('9e524ca1736fcb0fd1a84a193260b864', 'Göran', null, '2020-01-05T23:33:49Z'); +insert into users (id, name, description, modified_at) values ('78da2a8cc448803826c2bdc849907446', 'Fèi', null, '2020-02-20T00:46:00Z'); +insert into users (id, name, description, modified_at) values ('4cd79727766bc5f5bc3fdf99281c8b8c', 'Dafnée', null, '2020-11-29T14:45:17Z'); +insert into users (id, name, description, modified_at) values ('3185e0208e6e717d978fe6f9a5b77cec', 'Miléna', null, '2020-09-12T21:55:23Z'); +insert into users (id, name, description, modified_at) values ('3e80ed9f6bb618177ae002364926c163', 'Séverine', null, '2020-06-27T01:47:15Z'); +insert into users (id, name, description, modified_at) values ('b2b111e2b48c8988e34c6ea733785572', 'Lóng', null, '2020-05-18T00:14:21Z'); +insert into users (id, name, description, modified_at) values ('6595880251a6a506607f7128f32e9ccf', 'Réjane', null, '2020-10-02T03:03:14Z'); +insert into users (id, name, description, modified_at) values ('7759547dac42b826d01ca66d801c0834', 'Stéphanie', null, '2020-03-23T16:48:04Z'); +insert into users (id, name, description, modified_at) values ('1c8f068b69f00226a5883644022d7c4b', 'Anaëlle', null, '2020-04-05T09:11:11Z'); +insert into users (id, name, description, modified_at) values ('13048c087672ccf21475193bb5dbac02', 'Anaïs', null, '2020-10-17T17:51:22Z'); +insert into users (id, name, description, modified_at) values ('018e21eaceeb8490d5bfbd3882f06dd0', 'Mélina', null, '2020-08-30T19:12:16Z'); +insert into users (id, name, description, modified_at) values ('fa24883f1298520ee3066fd1105889be', 'Mélia', 'Crushing injury of ankle', '2020-05-20T04:56:53Z'); +insert into users (id, name, description, modified_at) values ('8dc17b5b1d47702529b64dad7cb0b276', 'Réjane', null, '2020-09-07T06:00:10Z'); +insert into users (id, name, description, modified_at) values ('09174a45a444c76f7a9c44b1d9888a50', 'Gisèle', null, '2020-10-01T00:41:22Z'); +insert into users (id, name, description, modified_at) values ('fff1c50bccb78095acfb94cb6d63da71', 'Laurène', null, '2020-08-03T15:57:07Z'); +insert into users (id, name, description, modified_at) values ('d23199cec16a007bca0068b5058368b8', 'Laïla', 'Stromal corneal pigmentations, bilateral', '2020-03-30T17:24:08Z'); +insert into users (id, name, description, modified_at) values ('698cac2b884193d04dc4cf4e1d75d389', 'Almérinda', 'Dislocation of C4/C5 cervical vertebrae, initial encounter', '2020-05-24T13:10:01Z'); +insert into users (id, name, description, modified_at) values ('fb533e0b045cc80d88a80c182e7a147d', 'Ophélie', null, '2020-07-15T17:50:34Z'); +insert into users (id, name, description, modified_at) values ('82cb31e91b59f84d0f5cf9876bf72aad', 'Andréanne', null, '2020-01-19T15:48:16Z'); +insert into users (id, name, description, modified_at) values ('454bdb82872b8242860ca0d608a2516d', 'Kù', null, '2020-04-18T10:20:03Z'); +insert into users (id, name, description, modified_at) values ('d23eeacff2700ca69f24133e45ac3b94', 'Lauréna', null, '2020-06-01T07:43:09Z'); +insert into users (id, name, description, modified_at) values ('b06e97ff71efcecb41b73d5145e28224', 'Faîtes', null, '2020-10-07T06:59:10Z'); +insert into users (id, name, description, modified_at) values ('c4e80e9adbeba0c61be5978b8300c671', 'Maëline', null, '2020-02-20T18:30:50Z'); +insert into users (id, name, description, modified_at) values ('cf515c27f9042ca8b0ee95dd296c5470', 'Mégane', null, '2020-11-30T20:24:27Z'); +insert into users (id, name, description, modified_at) values ('12c7bd6f9d262523fbc9176d78954e69', 'Esbjörn', null, '2020-05-22T19:05:57Z'); +insert into users (id, name, description, modified_at) values ('d446286ab7f7aa1b39ea1d20d0b801b1', 'Lóng', null, '2020-06-27T09:52:29Z'); +insert into users (id, name, description, modified_at) values ('34aa136ca6f0712c1783efffe4e3caff', 'Gwenaëlle', 'Open bite of right shoulder, sequela', '2020-09-01T01:37:19Z'); +insert into users (id, name, description, modified_at) values ('5a77ac730795c34de925b453b35c7e35', 'Kallisté', null, '2020-11-15T06:38:11Z'); +insert into users (id, name, description, modified_at) values ('1f9bbc2777aabffd160a21c3234bd1a4', 'Camélia', null, '2020-08-01T13:52:23Z'); +insert into users (id, name, description, modified_at) values ('7116809f8dac586882bb8a8070296a6f', 'Mén', 'Unspecified amblyopia', '2020-10-28T22:21:31Z'); +insert into users (id, name, description, modified_at) values ('dc80503a32f2460e032d956bfdb441fe', 'Amélie', null, '2020-10-25T02:55:33Z'); +insert into users (id, name, description, modified_at) values ('5ea512f521adc46fa34ee827d868e4d5', 'Yóu', null, '2020-08-05T02:55:23Z'); +insert into users (id, name, description, modified_at) values ('58a14f85a80352acf72f9299603f1895', 'Amélie', null, '2020-11-29T00:44:06Z'); +insert into users (id, name, description, modified_at) values ('75739346ff8fc8551381b0dc04063d81', 'Adélie', null, '2020-04-24T12:02:54Z'); +insert into users (id, name, description, modified_at) values ('09b5f4df588bbd6467f087f7ac17568e', 'Célia', 'Rheumatoid polyneuropathy with rheumatoid arthritis of elbow', '2020-02-05T14:29:43Z'); +insert into users (id, name, description, modified_at) values ('a417439dd4523fa1301a46de6c323bc2', 'Östen', 'Unspecified fracture of upper end of unspecified humerus, subsequent encounter for fracture with nonunion', '2020-01-25T10:18:55Z'); +insert into users (id, name, description, modified_at) values ('2dbf149f9564c54eaa75fe608c99938a', 'Stévina', null, '2020-07-09T18:28:58Z'); +insert into users (id, name, description, modified_at) values ('a462d1177792e6db78a991570ac300fd', 'Tán', null, '2020-09-28T09:00:57Z'); +insert into users (id, name, description, modified_at) values ('43fc215f576b0b8f09246d8069b954e4', 'Danièle', 'Pathological fracture in neoplastic disease, unspecified hand, subsequent encounter for fracture with routine healing', '2020-03-22T05:40:15Z'); +insert into users (id, name, description, modified_at) values ('699883ee43aff56c02e722f287f3dbf7', 'Océane', null, '2020-07-14T01:29:56Z'); +insert into users (id, name, description, modified_at) values ('c79d4a6a1575e99d68d3886dae449bed', 'Illustrée', null, '2020-05-30T04:32:10Z'); +insert into users (id, name, description, modified_at) values ('8b051106db514fdf62d06641d2e9faef', 'Anaé', null, '2020-10-25T22:31:12Z'); +insert into users (id, name, description, modified_at) values ('acbfe8e6cbdc52bdb76d7100125cd13a', 'Vénus', null, '2020-04-04T11:32:25Z'); +insert into users (id, name, description, modified_at) values ('f3f28f8fc07610ddf2c64fb07380c6d7', 'Rachèle', null, '2020-08-22T17:44:41Z'); +insert into users (id, name, description, modified_at) values ('2c6fe2c137740ec1e1982181d5dc9579', 'Séréna', 'Strain of other flexor muscle, fascia and tendon at forearm level, unspecified arm, initial encounter', '2020-06-27T20:31:45Z'); +insert into users (id, name, description, modified_at) values ('1a12bcd2e722c40698036cb8821658e3', 'Béatrice', 'Unspecified open wound of right great toe without damage to nail', '2020-10-23T15:05:34Z'); +insert into users (id, name, description, modified_at) values ('2a2559c9a47b7c4b5e978ab05f2cc7f5', 'Lèi', null, '2020-03-22T03:16:37Z'); +insert into users (id, name, description, modified_at) values ('7b4c164c8a1ffffc277757b5025e4d22', 'Cloé', null, '2020-01-24T12:50:32Z'); +insert into users (id, name, description, modified_at) values ('db38843693a167cad0a812e4192b517e', 'Yáo', 'Abrasion of unspecified external genital organs, male, sequela', '2020-11-21T05:20:04Z'); +insert into users (id, name, description, modified_at) values ('10b4c66d747cdc881ee278e3e795c20b', 'Médiamass', null, '2020-11-29T15:40:07Z'); +insert into users (id, name, description, modified_at) values ('1c9715c36d758cc20e595c5580357484', 'Irène', null, '2020-09-08T13:35:07Z'); +insert into users (id, name, description, modified_at) values ('b8ce1f34447f89b930560aa52ac9e53c', 'Pélagie', null, '2020-03-06T14:53:08Z'); +insert into users (id, name, description, modified_at) values ('27b25227dc45847d5433e16e7260ab1d', 'Sòng', 'Salter-Harris Type I physeal fracture of lower end of humerus, unspecified arm, subsequent encounter for fracture with delayed healing', '2020-10-12T15:12:02Z'); +insert into users (id, name, description, modified_at) values ('f347d59f448fc7b2d16cf6a0a3c9807a', 'Mylène', null, '2020-11-20T09:41:07Z'); +insert into users (id, name, description, modified_at) values ('e563dc8177f0ef225ee2dae2f76f3999', 'Cécilia', null, '2020-01-07T04:49:05Z'); +insert into users (id, name, description, modified_at) values ('b8cd34953a556d7ed2b7b47f1b7265fb', 'Mélissandre', null, '2020-07-22T15:32:21Z'); +insert into users (id, name, description, modified_at) values ('b572871734948af7e0efeed366bd945c', 'Nélie', 'Burn of respiratory tract, part unspecified, initial encounter', '2020-06-30T03:51:00Z'); +insert into users (id, name, description, modified_at) values ('07fc1ed7fefe7138c716613929c8412a', 'Zoé', 'Displaced comminuted fracture of unspecified patella, subsequent encounter for closed fracture with delayed healing', '2020-03-27T01:12:28Z'); +insert into users (id, name, description, modified_at) values ('a5e484f4e915110282f4da8b482b0a09', 'Magdalène', null, '2020-02-11T21:33:07Z'); +insert into users (id, name, description, modified_at) values ('85ffd1f785612cfbe4d79a8c2d9c3fa8', 'Tú', null, '2020-03-17T04:22:59Z'); +insert into users (id, name, description, modified_at) values ('e4526ff9f612bc02c2f374dac404b23c', 'Eloïse', null, '2020-05-04T22:21:37Z'); +insert into users (id, name, description, modified_at) values ('554bb86e5bce1f41c8a3a31eff1507c9', 'Zhì', 'Nondisplaced simple supracondylar fracture without intercondylar fracture of unspecified humerus, subsequent encounter for fracture with nonunion', '2020-10-23T19:17:16Z'); +insert into users (id, name, description, modified_at) values ('e16d51295ece23634d0bfc12f771b6cd', 'Vérane', null, '2020-08-18T11:06:53Z'); +insert into users (id, name, description, modified_at) values ('9e26d31fde7ecf3dc452dc6f7a4f3361', 'Audréanne', 'Burn of unspecified degree of unspecified hand, unspecified site, initial encounter', '2020-05-14T21:53:10Z'); +insert into users (id, name, description, modified_at) values ('9ef74fb9bdb47e09c83f11df4df04551', 'Edmée', null, '2020-07-06T22:25:35Z'); +insert into users (id, name, description, modified_at) values ('de8651c2305bffd3b19a2bd7a99327f7', 'Françoise', null, '2020-11-21T06:50:16Z'); +insert into users (id, name, description, modified_at) values ('eeac35c7e8fdb060965f4fe624052ef0', 'Océane', null, '2020-04-07T20:49:26Z'); +insert into users (id, name, description, modified_at) values ('4da5360f5343a029e4d8658fb827eba6', 'Mà', 'Pulmonary sporotrichosis', '2020-03-10T05:31:06Z'); +insert into users (id, name, description, modified_at) values ('52c891c81a6a997c93b20acb7fe66c99', 'Thérèsa', null, '2020-02-05T11:06:05Z'); +insert into users (id, name, description, modified_at) values ('43a82b3ac080df008a7569f5402a60c0', 'Estève', null, '2020-10-26T04:33:18Z'); +insert into users (id, name, description, modified_at) values ('13fe9cbc895942c22fe04a0ab1474178', 'Tán', null, '2020-06-02T08:18:39Z'); +insert into users (id, name, description, modified_at) values ('c07346783197685379fc29e8c140a86e', 'Mà', null, '2020-06-23T02:44:11Z'); +insert into users (id, name, description, modified_at) values ('ef27941521cefeb2ae6485725cfe5882', 'Séréna', 'Other extraarticular fracture of lower end of unspecified radius, initial encounter for open fracture type I or II', '2020-08-04T20:39:29Z'); +insert into users (id, name, description, modified_at) values ('7bc02a3069f8c52d05a14d02fc327dcc', 'Maïlys', null, '2020-04-25T01:01:34Z'); +insert into users (id, name, description, modified_at) values ('6b1e7e5918404ef7c17af212fb622d99', 'Torbjörn', null, '2020-05-06T10:26:28Z'); +insert into users (id, name, description, modified_at) values ('485a925c1a6bd582ccd72412021437b7', 'Judicaël', 'Frostbite with tissue necrosis of unspecified ear', '2020-11-19T16:22:17Z'); +insert into users (id, name, description, modified_at) values ('8603711e1ed77e622091412d165d13ca', 'Marie-josée', null, '2020-10-14T01:14:30Z'); +insert into users (id, name, description, modified_at) values ('2b44de6843cdd191220f82b27045b8d1', 'Cinéma', null, '2020-09-26T02:35:10Z'); +insert into users (id, name, description, modified_at) values ('54dc9906e740f2eec7c0e8a696b9e51c', 'Ráo', 'Other mature T/NK-cell lymphomas, lymph nodes of inguinal region and lower limb', '2020-08-16T21:19:01Z'); +insert into users (id, name, description, modified_at) values ('563b1664d3ef55389ac1b8f416a2a581', 'Mårten', null, '2020-10-13T23:15:30Z'); +insert into users (id, name, description, modified_at) values ('eedaa60a399f9d9daae2a5f840ed4e74', 'Maëlann', 'Unspecified motorcycle rider injured in collision with other nonmotor vehicle in nontraffic accident, subsequent encounter', '2020-03-27T17:51:22Z'); +insert into users (id, name, description, modified_at) values ('1e9e091fa59ab0b21735fad48ea23561', 'Mylène', null, '2020-07-11T15:31:29Z'); +insert into users (id, name, description, modified_at) values ('1745ff048a3196a8833e9e7cff1fc58a', 'Pò', null, '2020-08-04T05:06:09Z'); +insert into users (id, name, description, modified_at) values ('8351f1537ebb17e72df3ec6fb837ffe9', 'Andréanne', null, '2020-09-17T16:34:37Z'); +insert into users (id, name, description, modified_at) values ('eac2dd010294f592ed39e89cfa21ffab', 'Méghane', null, '2020-09-29T05:46:16Z'); +insert into users (id, name, description, modified_at) values ('1b81262dc32bfe4fe8f21615c1493251', 'Uò', null, '2020-04-01T09:04:36Z'); +insert into users (id, name, description, modified_at) values ('48179fb62763d792bd0e3aa53a66a78f', 'Estée', 'Morquio mucopolysaccharidoses, unspecified', '2020-09-07T22:31:58Z'); +insert into users (id, name, description, modified_at) values ('9a2224a30f013bc625bf0461e28cc907', 'Célia', null, '2020-07-20T08:34:53Z'); +insert into users (id, name, description, modified_at) values ('6d5c3e97ad35314031f8ddf38fd30b33', 'Annotés', null, '2020-09-04T12:52:38Z'); +insert into users (id, name, description, modified_at) values ('e64e5abb1cf92dc31e0c0e8084d8118b', 'Gwenaëlle', null, '2020-01-01T02:02:15Z'); +insert into users (id, name, description, modified_at) values ('5e53a7979677e89c8d92237a30b7576d', 'Märta', null, '2020-03-16T20:03:25Z'); +insert into users (id, name, description, modified_at) values ('737006d980cece55c98f2acd3b792a77', 'Médiamass', null, '2020-02-18T18:43:47Z'); +insert into users (id, name, description, modified_at) values ('2cc6becd7f5bfff286cb759b96ac0699', 'Personnalisée', 'Displaced fracture of neck of scapula, unspecified shoulder, initial encounter for open fracture', '2020-09-28T11:37:21Z'); +insert into users (id, name, description, modified_at) values ('2da1bd192dc937aee399a4fc3658405c', 'Dù', 'Other injury of other part of small intestine, subsequent encounter', '2020-07-15T04:05:52Z'); +insert into users (id, name, description, modified_at) values ('f6559d21ea4c4f32ab3972852b176a87', 'Frédérique', null, '2020-08-28T11:52:51Z'); +insert into users (id, name, description, modified_at) values ('e7e364d1704983d57e8842b8fc6fded8', 'Michèle', null, '2020-01-16T07:14:31Z'); +insert into users (id, name, description, modified_at) values ('9f29cca86142eab94a6ae55c4f5d63af', 'Méghane', null, '2020-05-02T08:27:40Z'); +insert into users (id, name, description, modified_at) values ('503dad5c597489d6386053397ffb9ccb', 'Hélèna', 'Displaced fracture of medial phalanx of other finger, initial encounter for open fracture', '2020-10-05T17:17:36Z'); +insert into users (id, name, description, modified_at) values ('a723623db8fb61832c088822f452e236', 'Loïs', null, '2020-11-29T04:43:13Z'); +insert into users (id, name, description, modified_at) values ('c04327c77f4134f81e3485fb8c42b68f', 'Gaétane', null, '2020-10-15T18:30:04Z'); +insert into users (id, name, description, modified_at) values ('8350c8f168a607a1bdaa9e12de12045e', 'Eléa', null, '2020-09-18T18:33:03Z'); +insert into users (id, name, description, modified_at) values ('b5e0a662c38233d0532867030a4d5304', 'Håkan', null, '2020-04-03T16:04:01Z'); +insert into users (id, name, description, modified_at) values ('94a97c33e12589cf93bfe05291017623', 'Mén', null, '2020-04-15T03:27:18Z'); +insert into users (id, name, description, modified_at) values ('087fc3d35bc92cc66fb9fce5dfaecfbe', 'Océane', null, '2020-02-04T22:22:08Z'); +insert into users (id, name, description, modified_at) values ('6f69ad365b73ce5eaf381310101e5fff', 'Andrée', 'Nondisplaced fracture of proximal phalanx of right little finger, sequela', '2020-10-23T04:01:51Z'); +insert into users (id, name, description, modified_at) values ('9623462c09d6e8d508d0508f953b49d3', 'Bérengère', null, '2020-09-10T09:25:15Z'); +insert into users (id, name, description, modified_at) values ('3743af7d147df0c5c99a0465a4dc4c99', 'Maïwenn', null, '2020-08-25T08:27:24Z'); +insert into users (id, name, description, modified_at) values ('d81e4a3874f59e0a8ea78c55b4205142', 'Océanne', null, '2020-11-18T03:49:43Z'); +insert into users (id, name, description, modified_at) values ('a1a6690b0bb8a948214cc03bf50b1e7c', 'Örjan', null, '2020-04-30T03:45:33Z'); +insert into users (id, name, description, modified_at) values ('42bc7473582672109d501b171f594d7b', 'Annotés', null, '2020-04-11T05:44:12Z'); +insert into users (id, name, description, modified_at) values ('5a9c20f212abd8ef0cb3b672a0ad9e82', 'Agnès', null, '2020-08-29T12:41:17Z'); +insert into users (id, name, description, modified_at) values ('1ecba29743b91b2afb69002e01e25ff1', 'Laurène', null, '2020-02-02T20:27:48Z'); +insert into users (id, name, description, modified_at) values ('47ce0dc2269350e4f73df55147fa1bf5', 'Salomé', 'Nondisplaced fracture of olecranon process without intraarticular extension of unspecified ulna, subsequent encounter for open fracture type IIIA, IIIB, or IIIC with nonunion', '2020-08-20T08:47:18Z'); +insert into users (id, name, description, modified_at) values ('bf2ab1cbd76998c28608b25c33dd61c0', 'Maïly', null, '2020-03-06T21:32:10Z'); +insert into users (id, name, description, modified_at) values ('0095686e5f835dcb35c58371ee27ef64', 'Eliès', null, '2020-11-29T16:27:24Z'); +insert into users (id, name, description, modified_at) values ('ed4517a144e480c120c344591dcff1f8', 'Erwéi', 'Nondisplaced Maisonneuve''s fracture of right leg, subsequent encounter for open fracture type I or II with nonunion', '2020-01-19T13:22:27Z'); +insert into users (id, name, description, modified_at) values ('2359459dd239ebdaf63c21d0eb160b05', 'Naéva', null, '2020-10-20T23:12:46Z'); +insert into users (id, name, description, modified_at) values ('d63fc766a931f77e390b0717a9153b6b', 'Nuó', 'Poisoning by other estrogens and progestogens, intentional self-harm, subsequent encounter', '2020-10-13T00:06:26Z'); +insert into users (id, name, description, modified_at) values ('78b039c1c22ad222dc75b69e2671e2b1', 'Judicaël', null, '2020-03-16T14:01:12Z'); +insert into users (id, name, description, modified_at) values ('c9b25a13298d344370b6add986a96a4e', 'Börje', null, '2020-10-03T22:29:54Z'); +insert into users (id, name, description, modified_at) values ('c4620918518656e195b5eccab1dd042f', 'Léonore', null, '2020-05-03T07:49:43Z'); +insert into users (id, name, description, modified_at) values ('950ec0ef52ce9837f45dce629dfaae4f', 'Léone', null, '2020-04-28T20:44:51Z'); +insert into users (id, name, description, modified_at) values ('1567fa76e9289bee8c63c718502e85b6', 'Loïc', null, '2020-02-06T07:50:57Z'); +insert into users (id, name, description, modified_at) values ('99227588780d5a5c89f04deb04f03d4f', 'Bérangère', 'Displaced oblique fracture of shaft of left tibia, subsequent encounter for closed fracture with delayed healing', '2020-08-11T11:31:58Z'); +insert into users (id, name, description, modified_at) values ('a037161d7f91f31a04c42765e1e53aab', 'Anaïs', null, '2020-07-10T10:20:23Z'); +insert into users (id, name, description, modified_at) values ('5262d65cb9c3a3a118733ff85fd719ff', 'Agnès', null, '2020-02-10T23:55:59Z'); +insert into users (id, name, description, modified_at) values ('9c1aebdd542e29532d03e93e7e5bc223', 'Chloé', null, '2020-02-17T06:29:39Z'); +insert into users (id, name, description, modified_at) values ('7c181d729e98e9b9d752cb39f9a8f081', 'Garçon', null, '2020-01-12T07:05:57Z'); +insert into users (id, name, description, modified_at) values ('4165ab1ea69b0b36e25d0a2dec6ee567', 'Anaé', 'Fall into swimming pool striking wall', '2020-04-06T11:06:02Z'); +insert into users (id, name, description, modified_at) values ('162b2b396ef90e1dab67dfbaee693f5d', 'Pélagie', null, '2020-10-09T13:56:39Z'); +insert into users (id, name, description, modified_at) values ('47d0cba3fb65ea3590f8e9d7abdd1d31', 'Anaïs', 'Unspecified open wound of front wall of thorax with penetration into thoracic cavity', '2020-11-21T12:34:39Z'); +insert into users (id, name, description, modified_at) values ('fc50e8ed1f93c7750ea588737ce89633', 'Zhì', null, '2020-05-16T00:38:49Z'); +insert into users (id, name, description, modified_at) values ('7d677de99818935d026277ca8cc61a30', 'Frédérique', null, '2020-02-10T08:05:15Z'); +insert into users (id, name, description, modified_at) values ('a9e8523a0a9485028fc996cede695cda', 'Geneviève', null, '2020-09-10T19:48:16Z'); +insert into users (id, name, description, modified_at) values ('83ea37049a1e38a9d73ab3d82daa228f', 'Anaé', 'Toxic effect of contact with venomous toad, intentional self-harm, subsequent encounter', '2020-04-27T23:45:45Z'); +insert into users (id, name, description, modified_at) values ('93c03df55de2fee739829164aa9504c8', 'Hélèna', 'Hypoplasia of testis and scrotum', '2020-09-12T07:50:58Z'); +insert into users (id, name, description, modified_at) values ('63a525f80f57a70d224f8a66f082f506', 'Danièle', 'Nondisplaced Zone II fracture of sacrum, sequela', '2020-03-13T08:30:06Z'); +insert into users (id, name, description, modified_at) values ('729a9c3db248a91c54e402172f414bf7', 'Maëlann', null, '2020-05-27T22:51:55Z'); +insert into users (id, name, description, modified_at) values ('c0bfeb38d2feeed6dcc7c33307fac940', 'Börje', null, '2020-05-19T15:35:33Z'); +insert into users (id, name, description, modified_at) values ('ac5043f218b985ee12031763642482c8', 'Personnalisée', null, '2020-03-09T17:49:22Z'); +insert into users (id, name, description, modified_at) values ('05ed867c07825fcdc07b41e061581a1f', 'Félicie', null, '2020-10-11T05:17:05Z'); +insert into users (id, name, description, modified_at) values ('cb3700a1742479336a6153b5d0b46bf7', 'Stévina', null, '2020-04-15T17:38:59Z'); +insert into users (id, name, description, modified_at) values ('e00516a2de0ad0b6b37d830ab33c6d69', 'Maïlys', null, '2020-10-10T12:08:25Z'); +insert into users (id, name, description, modified_at) values ('c8a3de1077da534728b87630d44591f2', 'Léonore', 'Poisoning by, adverse effect of and underdosing of cocaine', '2020-10-07T14:51:17Z'); +insert into users (id, name, description, modified_at) values ('743443631e02d8ac90fe76458361bae4', 'Gwenaëlle', null, '2020-02-25T09:45:53Z'); +insert into users (id, name, description, modified_at) values ('731d6534738a4c6b1f67cdc74113bb60', 'Joséphine', null, '2020-05-14T02:07:12Z'); +insert into users (id, name, description, modified_at) values ('449bf04a090a34acfe36d2867f70b5e2', 'Angèle', null, '2020-07-29T18:22:04Z'); +insert into users (id, name, description, modified_at) values ('382af53d567deff0c58be862f6306825', 'Frédérique', null, '2020-03-08T01:56:47Z'); +insert into users (id, name, description, modified_at) values ('03b23c5a58bfeec5f950008265db1838', 'Laïla', null, '2020-04-07T08:22:44Z'); +insert into users (id, name, description, modified_at) values ('7d918fa32861a462e738907c967d939a', 'Görel', null, '2020-07-12T13:34:49Z'); +insert into users (id, name, description, modified_at) values ('25c98a72fa37223d749b39749cbba975', 'Solène', null, '2020-03-13T17:36:29Z'); +insert into users (id, name, description, modified_at) values ('02f989f03f0b1c95b741a0469c4aec6d', 'Bécassine', 'Displaced fracture of anterior column [iliopubic] of unspecified acetabulum, sequela', '2020-01-28T16:13:17Z'); +insert into users (id, name, description, modified_at) values ('2cd1905052a6b92bc0a955ca33ddbcd3', 'Méghane', null, '2020-06-22T23:03:01Z'); +insert into users (id, name, description, modified_at) values ('8bbad6cdeeeeaec8daf9c2fca85e857a', 'Renée', null, '2020-03-11T07:21:39Z'); +insert into users (id, name, description, modified_at) values ('f54ea9c2979d530d75776ccc575f9863', 'Danièle', null, '2020-10-11T08:54:45Z'); +insert into users (id, name, description, modified_at) values ('8fdb4e53e88b2fd2cd87c79edab0aaee', 'Dà', null, '2020-07-19T14:47:30Z'); +insert into users (id, name, description, modified_at) values ('1e9263a9e6d48511987f80a2271989d7', 'Dafnée', 'Malignant neoplasm of lacrimal gland and duct', '2020-09-02T03:42:04Z'); +insert into users (id, name, description, modified_at) values ('1b1eabc0eb26b48e4860e88eb9ffd1b8', 'Pénélope', 'Laceration of muscle and tendon of back wall of thorax, subsequent encounter', '2020-09-17T10:27:03Z'); +insert into users (id, name, description, modified_at) values ('6794dd9fd99948701109501e2a248019', 'Illustrée', 'Occupant (driver) (passenger) of heavy transport vehicle injured in unspecified traffic accident, subsequent encounter', '2020-10-05T16:27:22Z'); +insert into users (id, name, description, modified_at) values ('014201cb3a6b2bd594d919fe75ab15d5', 'Réservés', null, '2020-11-29T19:57:19Z'); +insert into users (id, name, description, modified_at) values ('21efd66dba4c0b121abb3b7b2a134ea6', 'Pò', null, '2020-03-16T12:32:24Z'); +insert into users (id, name, description, modified_at) values ('f6f4ed5c319acd75cd67813ae2bb089d', 'Yè', null, '2020-11-24T15:33:34Z'); +insert into users (id, name, description, modified_at) values ('54d001d0193bc59616bfc88658110eee', 'Lén', null, '2020-07-20T08:36:56Z'); +insert into users (id, name, description, modified_at) values ('c8423134d238a07ba68a35261b4b42a0', 'Méng', null, '2020-07-03T09:13:32Z'); +insert into users (id, name, description, modified_at) values ('21bc12b6e23027ee9a90ad34959813d1', 'André', null, '2020-10-24T10:43:07Z'); +insert into users (id, name, description, modified_at) values ('97f09f2f82e2ff1878aa1c4a9d2bdacc', 'Athéna', null, '2020-05-11T21:05:15Z'); +insert into users (id, name, description, modified_at) values ('201a326964c9878790bfc68d62b12101', 'Pélagie', null, '2020-09-26T18:31:53Z'); +insert into users (id, name, description, modified_at) values ('167f5fad4b1bbbe93be79204802bd808', 'Mén', null, '2020-10-21T00:21:17Z'); +insert into users (id, name, description, modified_at) values ('b907a180e4322c0a6ea87ccacf779312', 'Intéressant', null, '2020-02-05T17:31:16Z'); +insert into users (id, name, description, modified_at) values ('c3241cce34674f47334e8ddf3accd0d0', 'Valérie', 'Trade school as the place of occurrence of the external cause', '2020-04-17T04:51:07Z'); +insert into users (id, name, description, modified_at) values ('99362be9ecb62ec71a9b507520d8143c', 'Gisèle', null, '2020-04-12T19:15:51Z'); +insert into users (id, name, description, modified_at) values ('91f236c13f37370d84e642c0283973fc', 'Dorothée', null, '2020-07-15T19:46:25Z'); +insert into users (id, name, description, modified_at) values ('664c895e067d09c3bdb3848a3e536485', 'Naéva', null, '2020-01-12T06:49:37Z'); +insert into users (id, name, description, modified_at) values ('664778e88666b0d4671b58e5c7c7f318', 'Réjane', 'Nondisplaced fracture of proximal phalanx of right ring finger, subsequent encounter for fracture with routine healing', '2020-03-05T11:18:49Z'); +insert into users (id, name, description, modified_at) values ('1f17132dd36202a7c801614e00f3db9e', 'Néhémie', null, '2020-09-11T08:49:24Z'); +insert into users (id, name, description, modified_at) values ('621a6668fa5bcedc8846430ed2f54b86', 'Kuí', null, '2020-04-26T22:53:38Z'); +insert into users (id, name, description, modified_at) values ('daa0a6e67974a0ad2bf68b5553f4ebd7', 'Mélodie', null, '2020-10-10T13:12:47Z'); +insert into users (id, name, description, modified_at) values ('2e1bad6af4e7639d54d293e090418e64', 'Illustrée', null, '2020-01-13T19:01:46Z'); +insert into users (id, name, description, modified_at) values ('34103b54ccc28f548a01376f11bb2e51', 'Sélène', 'Driver of three-wheeled motor vehicle injured in collision with other nonmotor vehicle in traffic accident, sequela', '2020-11-01T06:05:38Z'); +insert into users (id, name, description, modified_at) values ('add5febc9d30f9df50da8b2cbcb39b82', 'Sòng', null, '2020-10-27T06:45:17Z'); +insert into users (id, name, description, modified_at) values ('1938537b1afff2d188a3e6b15d502698', 'Cloé', null, '2020-07-16T23:44:37Z'); +insert into users (id, name, description, modified_at) values ('198f2e605b0e7697fd916388a688f3d1', 'Maëline', 'Toxic effect of hydrogen sulfide, undetermined', '2020-06-06T21:21:36Z'); +insert into users (id, name, description, modified_at) values ('3d840f0d6e3db623ff99a790e2d7ffa9', 'Eloïse', null, '2020-04-18T05:49:22Z'); +insert into users (id, name, description, modified_at) values ('e7a1361fb4f2bdf5427df7c401e96966', 'Mélys', null, '2020-02-12T01:35:27Z'); +insert into users (id, name, description, modified_at) values ('20c74b5e0dd5d1a5d8d05211ecb12c0c', 'Adèle', null, '2020-10-07T06:33:57Z'); +insert into users (id, name, description, modified_at) values ('5e5f50d1f4864109f5080c54c715c43f', 'Gisèle', null, '2020-03-28T18:09:55Z'); +insert into users (id, name, description, modified_at) values ('57e595358822610b193b477dbf1b804a', 'Noémie', null, '2020-11-01T17:15:48Z'); +insert into users (id, name, description, modified_at) values ('4086b889083d9c4fa87de6f70ac40cf3', 'Pò', null, '2020-07-29T00:08:27Z'); +insert into users (id, name, description, modified_at) values ('4591887520d418abe1907a51a996ae22', 'Cléa', null, '2020-09-17T12:27:09Z'); +insert into users (id, name, description, modified_at) values ('dcaea6dd5eb34ea15c2b7869622da751', 'Mélia', 'Poisoning by unspecified primarily systemic and hematological agent, intentional self-harm', '2020-10-12T13:42:50Z'); +insert into users (id, name, description, modified_at) values ('d8f644243db48fa3f3a4bb794730da75', 'Marie-hélène', null, '2020-08-12T17:36:14Z'); +insert into users (id, name, description, modified_at) values ('6e6acf8855524394b07fb45bd7a79d00', 'Håkan', 'Incarcerated fracture (avulsion) of medial epicondyle of right humerus', '2020-04-08T17:59:59Z'); +insert into users (id, name, description, modified_at) values ('956873ce26d65cd1c339fed15e47ca87', 'Eloïse', 'Abrasion of unspecified upper arm, sequela', '2020-02-22T13:20:37Z'); +insert into users (id, name, description, modified_at) values ('dbbd969924fa51200dc9a2990ff2ba78', 'Maëly', null, '2020-05-28T03:55:23Z'); +insert into users (id, name, description, modified_at) values ('cd076bd0a5e965777f83bd67623c65dd', 'Estée', null, '2020-08-01T18:42:11Z'); +insert into users (id, name, description, modified_at) values ('fc83785d17262b161a849fa9d38cd0ad', 'Ophélie', 'Unspecified injury of unspecified pulmonary blood vessels', '2020-05-03T19:52:45Z'); +insert into users (id, name, description, modified_at) values ('80661dd370055995c433cd073b809275', 'Léa', null, '2020-01-19T08:57:18Z'); +insert into users (id, name, description, modified_at) values ('28011f196e49221531038466b3473172', 'Léana', null, '2020-10-12T21:34:51Z'); +insert into users (id, name, description, modified_at) values ('55a996fbb146a18bd685c22c47add5b0', 'Mà', null, '2020-07-12T11:47:45Z'); +insert into users (id, name, description, modified_at) values ('8da7dc9124c8893a2a85cffe4e968bff', 'Maï', 'Other specified injuries of vocal cord, sequela', '2020-10-19T06:39:56Z'); +insert into users (id, name, description, modified_at) values ('db40c29b390b9b34c1e58d669748296a', 'Wá', null, '2020-10-10T13:24:36Z'); +insert into users (id, name, description, modified_at) values ('4bd5b3cffce09fd38bde29f7d0b43dc4', 'Aí', null, '2020-06-25T16:54:40Z'); +insert into users (id, name, description, modified_at) values ('6c9ffe43395d0fc9aaad742b0752b627', 'Adélie', null, '2020-01-08T08:03:58Z'); +insert into users (id, name, description, modified_at) values ('8d632e19d52131d79bdaf18d5dd84d08', 'Anaïs', 'Melanocytic nevi of right lower limb, including hip', '2020-07-13T09:04:55Z'); +insert into users (id, name, description, modified_at) values ('588907f28c353da6aa56f950d1fbe4a4', 'Renée', null, '2020-11-23T02:48:50Z'); +insert into users (id, name, description, modified_at) values ('e6dd8813a2ea82ac557de3e7ae06c64e', 'Maïté', null, '2020-10-27T15:25:03Z'); +insert into users (id, name, description, modified_at) values ('b825ad9ffa0902148eba439490adc46e', 'Gérald', null, '2020-01-11T00:08:50Z'); +insert into users (id, name, description, modified_at) values ('735339c49b42d19d73b013d37279bf87', 'Léonie', null, '2020-10-01T16:31:06Z'); +insert into users (id, name, description, modified_at) values ('59b6e0bddfb1647f9b50b33ee63f329e', 'Adélie', 'Other contact with other marine mammals, subsequent encounter', '2020-01-08T09:53:51Z'); +insert into users (id, name, description, modified_at) values ('29acab66a617e36f249458647a8abe9e', 'Zhì', null, '2020-02-02T12:52:08Z'); +insert into users (id, name, description, modified_at) values ('9cb18fdc9ee738f91acdfa55f7f6b8e6', 'Gisèle', null, '2020-08-13T22:39:22Z'); +insert into users (id, name, description, modified_at) values ('e24bb854dadb7166cd5225e409d75015', 'Maïwenn', 'Other specified injury of ulnar artery at forearm level, unspecified arm, initial encounter', '2020-10-18T00:04:15Z'); +insert into users (id, name, description, modified_at) values ('dd68e06ed372461fb3c8b20499a3af41', 'Illustrée', null, '2020-07-25T01:47:58Z'); +insert into users (id, name, description, modified_at) values ('e23e908344a76494100bf6a5d06ff5cd', 'Stévina', null, '2020-10-19T05:03:03Z'); +insert into users (id, name, description, modified_at) values ('f7b914291dab0d24e77f6fafa633361b', 'Eloïse', null, '2020-06-21T03:28:40Z'); +insert into users (id, name, description, modified_at) values ('ea9ef4fac2ec7736276fa1f20ebaca22', 'Françoise', null, '2020-10-26T20:29:03Z'); +insert into users (id, name, description, modified_at) values ('a37595e39bcab66093c2b108776be3bc', 'Lài', 'Malignant neoplasm of base of tongue', '2020-05-27T08:52:35Z'); +insert into users (id, name, description, modified_at) values ('24c95cfd9b331fb041d08ebc8f428837', 'Estève', null, '2020-11-12T22:25:14Z'); +insert into users (id, name, description, modified_at) values ('ab0f48229dc1b285acc2b52599151c4b', 'Méryl', null, '2020-06-15T11:55:10Z'); +insert into users (id, name, description, modified_at) values ('e14440ce6bd4d398bfb9d585f89a3de9', 'Geneviève', null, '2020-01-23T21:28:07Z'); +insert into users (id, name, description, modified_at) values ('b472d3fd162a229a51283bea4bb9c421', 'Yú', null, '2020-09-23T13:28:02Z'); +insert into users (id, name, description, modified_at) values ('56f7d1f7248942443e609fde0c083fac', 'Athéna', 'Other fractures of lower end of left radius, subsequent encounter for open fracture type IIIA, IIIB, or IIIC with malunion', '2020-10-05T06:34:13Z'); +insert into users (id, name, description, modified_at) values ('2bf4a4ef3711ca48ef1e8905602f6db3', 'Mélodie', null, '2020-11-28T22:02:36Z'); +insert into users (id, name, description, modified_at) values ('202615991515ba9d254daead4d15ab24', 'Lén', null, '2020-05-26T10:41:18Z'); +insert into users (id, name, description, modified_at) values ('e207c19b054524593fcf88f4f8f7b094', 'Lèi', null, '2020-04-10T07:40:55Z'); +insert into users (id, name, description, modified_at) values ('e3f54e4be84d23dce74a13be6022ad4a', 'Märta', null, '2020-11-11T16:09:01Z'); +insert into users (id, name, description, modified_at) values ('34212f919888fd1a09774587b371fa59', 'Mahélie', null, '2020-10-31T17:43:53Z'); +insert into users (id, name, description, modified_at) values ('4a901a98801653637bdb8596d65f26fa', 'Marie-françoise', 'Contact with turkey', '2020-11-16T16:20:28Z'); +insert into users (id, name, description, modified_at) values ('88980d6d6e0df137e8dde36c05833555', 'Joséphine', 'Late congenital syphilitic osteochondropathy', '2020-05-07T20:28:53Z'); +insert into users (id, name, description, modified_at) values ('32205efe94b5b53278e8d542a342d933', 'Maïlys', null, '2020-05-30T22:59:20Z'); +insert into users (id, name, description, modified_at) values ('a5702903e70c10533d247ad7b1f1d468', 'Maëlann', null, '2020-03-09T10:31:51Z'); +insert into users (id, name, description, modified_at) values ('c8f06461d9c3af86a4dec9fee1dffd95', 'Françoise', null, '2020-06-28T01:06:04Z'); +insert into users (id, name, description, modified_at) values ('b62852263f8bd8f576498ad4d3efb310', 'Kù', null, '2020-10-27T04:42:11Z'); +insert into users (id, name, description, modified_at) values ('90b39132b612d3c0d7e33a25fbfb502c', 'Agnès', 'Person injured while boarding or alighting from streetcar, sequela', '2020-03-05T08:21:07Z'); +insert into users (id, name, description, modified_at) values ('77d74c60412a433b62b61b0faeae30e0', 'Esbjörn', null, '2020-04-03T04:33:53Z'); +insert into users (id, name, description, modified_at) values ('5113d8535ae8ebd4e99d9ad44e31f95a', 'Andréa', null, '2020-03-05T05:59:23Z'); +insert into users (id, name, description, modified_at) values ('b3a33e41e05f12aca11a01a8020809ac', 'Kévina', null, '2020-09-30T06:49:58Z'); +insert into users (id, name, description, modified_at) values ('659b5793a227e9d793b16448405c1a9d', 'Médiamass', null, '2020-04-19T10:21:19Z'); +insert into users (id, name, description, modified_at) values ('a3bdff249692ea2c32c5ed478cbf3dcf', 'Bérangère', null, '2020-04-04T14:48:43Z'); +insert into users (id, name, description, modified_at) values ('1d185eca3d675d504ac33ecf6164a2a9', 'Mélodie', null, '2020-08-26T14:18:02Z'); +insert into users (id, name, description, modified_at) values ('70506e6b6f95a3733bded0024988c61d', 'Kévina', null, '2020-10-14T05:39:03Z'); +insert into users (id, name, description, modified_at) values ('c40c2b5753bbc635325eeef5dc90c60f', 'Mélinda', null, '2020-03-06T09:47:59Z'); +insert into users (id, name, description, modified_at) values ('7906629663b64ec5764bd3078e095068', 'Lóng', null, '2020-04-24T00:04:22Z'); +insert into users (id, name, description, modified_at) values ('90d4ce704649cf3b9ed9d6c9bc90744d', 'Michèle', null, '2020-09-14T17:52:25Z'); +insert into users (id, name, description, modified_at) values ('f8e12d40930261f0193bd0c019db03ff', 'Lóng', null, '2020-04-20T02:57:47Z'); +insert into users (id, name, description, modified_at) values ('a474043bcf59162c1b2f43f5b6eb2446', 'Fèi', null, '2020-01-10T11:31:55Z'); +insert into users (id, name, description, modified_at) values ('67132126e22b28fd23d01dd763eca36c', 'Marie-josée', 'Juvenile arthritis, unspecified', '2020-07-21T09:10:43Z'); +insert into users (id, name, description, modified_at) values ('d7293d598521408060c83b06225bd839', 'Mélia', null, '2020-03-16T09:59:29Z'); +insert into users (id, name, description, modified_at) values ('09bd4f1725e242dfddf976153ab2b8bc', 'Mahélie', null, '2020-11-29T19:57:56Z'); +insert into users (id, name, description, modified_at) values ('837ec04bf71928a180215036a2516203', 'Mahélie', null, '2020-05-06T09:02:16Z'); +insert into users (id, name, description, modified_at) values ('d729da1b055e41d5f074c1f330113af7', 'Nélie', null, '2020-08-31T08:15:47Z'); +insert into users (id, name, description, modified_at) values ('7015d33cc815bf02b8c802ec4835e69c', 'Méthode', null, '2020-08-26T11:51:42Z'); +insert into users (id, name, description, modified_at) values ('370b39916abab7904dc30350e44a5696', 'Josée', null, '2020-04-02T16:10:00Z'); +insert into users (id, name, description, modified_at) values ('c331af2cb95141236105059f8241cec6', 'Mélia', null, '2020-06-17T06:54:55Z'); +insert into users (id, name, description, modified_at) values ('ce9d8ee6ee6b5acbdacc0705b9322ab2', 'Ruò', null, '2020-11-09T05:52:30Z'); +insert into users (id, name, description, modified_at) values ('b7ac9e4144267bda05f05c5c786939e4', 'Lauréna', 'Displaced Maisonneuve''s fracture of unspecified leg, subsequent encounter for open fracture type IIIA, IIIB, or IIIC with malunion', '2020-01-05T07:54:44Z'); +insert into users (id, name, description, modified_at) values ('6599078f6d6015d9bc6162135547ef9d', 'Pål', null, '2020-11-27T15:27:20Z'); +insert into users (id, name, description, modified_at) values ('07726dc1fa2aa749321df8f82bb2d141', 'Camélia', 'Other fracture of second thoracic vertebra, initial encounter for open fracture', '2020-11-03T15:18:06Z'); +insert into users (id, name, description, modified_at) values ('2ea8288fa38a9ce7cca438b1b096b689', 'Ruì', null, '2020-06-22T08:30:56Z'); +insert into users (id, name, description, modified_at) values ('11ea5c7044fde475d347b0dc3d5cddfc', 'Anaïs', 'Laceration without foreign body of abdominal wall, left upper quadrant with penetration into peritoneal cavity', '2020-06-13T05:13:48Z'); +insert into users (id, name, description, modified_at) values ('439bab96bf2c2ad1b026ad20012e355f', 'Séréna', null, '2020-01-24T02:40:17Z'); +insert into users (id, name, description, modified_at) values ('fe4d21fdef40fc387d01bb6f2077f9b7', 'Médiamass', null, '2020-05-26T06:18:39Z'); +insert into users (id, name, description, modified_at) values ('e4ed6a31b8d364f78569ba7ba2682536', 'Françoise', 'Complex regional pain syndrome I (CRPS I)', '2020-06-10T11:43:52Z'); +insert into users (id, name, description, modified_at) values ('9636e7c9718fdc0f6e6a97ed7ae9b127', 'Eliès', null, '2020-07-16T01:18:53Z'); +insert into users (id, name, description, modified_at) values ('5cdc49dcbef79c68d9cd7285a57d46e3', 'Cléa', 'Partial traumatic amputation at level between left shoulder and elbow, sequela', '2020-09-07T07:54:34Z'); +insert into users (id, name, description, modified_at) values ('3bfe85db19e8c40e89a580000de8b2a6', 'Kuí', 'Blindness, one eye, low vision other eye', '2020-03-07T20:48:24Z'); +insert into users (id, name, description, modified_at) values ('0066350c56a80fa4e95cab0a17ec7b38', 'Garçon', null, '2020-11-16T12:05:02Z'); +insert into users (id, name, description, modified_at) values ('1ba8669443c6de818924c04f20f04532', 'Salomé', 'Gynephobia', '2020-09-04T23:18:09Z'); +insert into users (id, name, description, modified_at) values ('28492ba445610208f221cfefdfe8e92a', 'Maï', null, '2020-05-08T18:00:55Z'); +insert into users (id, name, description, modified_at) values ('9c094393b2ec2dd208b9cec063a92f09', 'Maïlys', 'Unspecified occupant of three-wheeled motor vehicle injured in collision with heavy transport vehicle or bus in nontraffic accident', '2020-08-11T06:45:06Z'); +insert into users (id, name, description, modified_at) values ('311f34015c9343b1ec13c4dd7677c521', 'Intéressant', null, '2020-03-24T23:58:49Z'); +insert into users (id, name, description, modified_at) values ('fb3a36afbccd9ce65bf8581278049595', 'Maëline', 'Abscess of bursa', '2020-03-08T18:16:46Z'); +insert into users (id, name, description, modified_at) values ('f650a9bde040de5c2d7a642ff88d8172', 'Fèi', 'Ovarian pregnancy', '2020-05-11T18:21:17Z'); +insert into users (id, name, description, modified_at) values ('4178a0b916ae718d207615c3fd19354e', 'Gaïa', null, '2020-07-24T08:02:58Z'); +insert into users (id, name, description, modified_at) values ('6ab64ef101205ef72051b1361250ff25', 'Méthode', null, '2020-01-04T23:49:09Z'); +insert into users (id, name, description, modified_at) values ('8b0d90a30637ae5ca925820734cf8ea3', 'Marie-thérèse', null, '2020-06-18T13:36:48Z'); +insert into users (id, name, description, modified_at) values ('92bb151c447725f88db4d7349d930b58', 'Nuó', null, '2020-02-26T04:49:18Z'); +insert into users (id, name, description, modified_at) values ('43506659f2f67bca53167fe4b352098f', 'Réservés', null, '2020-04-07T11:35:47Z'); +insert into users (id, name, description, modified_at) values ('f22ce9bb3646643651532956504f1d92', 'Daphnée', null, '2020-10-20T03:08:07Z'); +insert into users (id, name, description, modified_at) values ('ec7adaf19e7fa9f6da5748b39f01a04f', 'Léonie', null, '2020-08-20T01:32:26Z'); +insert into users (id, name, description, modified_at) values ('af1cef5f2db70c071dc4c29dc4e7367a', 'Athéna', 'Nondisplaced fracture of neck of fifth metacarpal bone, right hand, sequela', '2020-08-08T21:45:40Z'); +insert into users (id, name, description, modified_at) values ('cbad39d68ed3cbde6917a9f1946a79ad', 'Intéressant', null, '2020-07-25T18:07:51Z'); +insert into users (id, name, description, modified_at) values ('0dd6b0d7e163e6660848701642a178bd', 'Séverine', null, '2020-03-04T19:51:02Z'); +insert into users (id, name, description, modified_at) values ('6722f1fca5215746bb72f739b26ee1ff', 'Loïs', null, '2020-11-03T23:38:31Z'); +insert into users (id, name, description, modified_at) values ('e145c5c69c764ce0eb467929e08bc51c', 'Ráo', null, '2020-05-15T17:26:48Z'); +insert into users (id, name, description, modified_at) values ('ddb3d6046dc0bfd7967f082374048a66', 'Dorothée', null, '2020-01-04T22:57:43Z'); +insert into users (id, name, description, modified_at) values ('a0fb08a6a5849643106e13fa55dbecfe', 'Annotés', null, '2020-11-12T18:57:38Z'); +insert into users (id, name, description, modified_at) values ('ebbea6d2b747443d6ba1545def69cb62', 'Eloïse', null, '2020-05-20T14:19:31Z'); +insert into users (id, name, description, modified_at) values ('81b5742f5148fadfe66bfc3971d9eb4a', 'Nuó', null, '2020-09-25T17:19:51Z'); +insert into users (id, name, description, modified_at) values ('221d12a99667ffd9fed8b3c2513da507', 'Thérèse', null, '2020-04-10T04:37:15Z'); +insert into users (id, name, description, modified_at) values ('2b866e7505df82cc585b791bd043e69f', 'Inès', null, '2020-04-06T07:27:31Z'); +insert into users (id, name, description, modified_at) values ('ba8dd133536fab1b1786618575316880', 'Irène', null, '2020-03-02T11:42:06Z'); +insert into users (id, name, description, modified_at) values ('2b1995bf7797c3b8127d64bff5886d7b', 'Adélaïde', null, '2020-01-23T11:14:37Z'); +insert into users (id, name, description, modified_at) values ('033ca5dd3ef5c3005cfccb068dc0d981', 'Märta', null, '2020-02-13T00:45:22Z'); +insert into users (id, name, description, modified_at) values ('a41c0f26ccdf2a3021420802a2b62be7', 'Naéva', null, '2020-02-08T03:07:26Z'); +insert into users (id, name, description, modified_at) values ('e0519abb504e16cc0ee17066345e77bd', 'Audréanne', null, '2020-04-16T08:24:38Z'); +insert into users (id, name, description, modified_at) values ('d7fbf19c59303f1c53b3efa765c6dd34', 'Lóng', null, '2020-08-27T04:21:01Z'); +insert into users (id, name, description, modified_at) values ('955d734bea53d2f376604b3cdbbcbc54', 'Maéna', 'Sedative, hypnotic or anxiolytic use, unspecified with withdrawal delirium', '2020-01-09T09:42:26Z'); +insert into users (id, name, description, modified_at) values ('3b3b10dd650dd81f5d630f4e92580d57', 'Björn', 'Unspecified fracture of upper end of right radius, subsequent encounter for open fracture type I or II with nonunion', '2020-04-05T07:28:13Z'); +insert into users (id, name, description, modified_at) values ('0d1ec5a8999c165aa260aedebd48945d', 'Marie-josée', null, '2020-04-02T15:40:39Z'); +insert into users (id, name, description, modified_at) values ('5a69f248e7c40ae9d4b3fa963a82c0fc', 'Maëly', 'Other physeal fracture of upper end of radius, right arm, subsequent encounter for fracture with routine healing', '2020-04-20T23:23:52Z'); +insert into users (id, name, description, modified_at) values ('f5c6796a813560e0972bdb7c144ecd34', 'Marylène', 'Other injury of stomach, sequela', '2020-09-10T03:57:06Z'); +insert into users (id, name, description, modified_at) values ('dfe2b3b145944684540175cb11dad959', 'Réjane', null, '2020-11-21T10:57:23Z'); +insert into users (id, name, description, modified_at) values ('891eeb5e7cc65c05484629978a7bc4ca', 'Hélène', 'Salter-Harris Type III physeal fracture of lower end of humerus, left arm', '2020-01-04T18:39:50Z'); +insert into users (id, name, description, modified_at) values ('bcb03c63706a016155b2705912489f8a', 'Håkan', null, '2020-02-01T05:22:50Z'); +insert into users (id, name, description, modified_at) values ('90dfb971fc73c9629ad18f3f2fa26ed6', 'Judicaël', 'Unspecified retinal break, left eye', '2020-03-19T09:39:43Z'); +insert into users (id, name, description, modified_at) values ('4822d4705ff6a60e8a3498b38a2393df', 'Noémie', 'Poisoning by mineralocorticoids and their antagonists, accidental (unintentional), sequela', '2020-01-14T22:49:49Z'); +insert into users (id, name, description, modified_at) values ('93c670540b3a6765e6374542ff4df568', 'Maéna', 'Fall from or off toilet with subsequent striking against object', '2020-06-04T12:51:50Z'); +insert into users (id, name, description, modified_at) values ('c291b88a9bcd098c3952ada5975a034a', 'Loïs', 'Unspecified fracture of shaft of right ulna, subsequent encounter for open fracture type I or II with nonunion', '2020-03-18T15:24:01Z'); +insert into users (id, name, description, modified_at) values ('9b59716eb65fdd35237a0458e656cd6f', 'Valérie', 'Other calcification of muscle, left lower leg', '2020-05-26T02:14:21Z'); +insert into users (id, name, description, modified_at) values ('321f6375fbf871a52efcfd7d65d85352', 'Nélie', null, '2020-04-12T14:41:38Z'); +insert into users (id, name, description, modified_at) values ('c09e344cc014708a38695f110ea32890', 'Börje', null, '2020-11-29T12:31:13Z'); +insert into users (id, name, description, modified_at) values ('4c5c472f02926ac74efda8bbe971bf6b', 'Lyséa', null, '2020-04-08T18:30:49Z'); +insert into users (id, name, description, modified_at) values ('37db1bbd00e9586dfbb2dd018e87d986', 'Lucrèce', null, '2020-10-09T04:09:16Z'); +insert into users (id, name, description, modified_at) values ('777f4c4a540a753813f90809c39dc7b3', 'Anaël', null, '2020-02-25T18:48:05Z'); +insert into users (id, name, description, modified_at) values ('8108129958ead6543a6fc888a55fe678', 'Adèle', 'Other ovarian cyst, left side', '2020-11-01T07:03:44Z'); +insert into users (id, name, description, modified_at) values ('7bc2a8c4d54088d16a6204699100dd73', 'Cléa', null, '2020-03-12T10:38:29Z'); +insert into users (id, name, description, modified_at) values ('b60728d4d15443089aed8d5ca66c47e6', 'Eliès', null, '2020-09-13T20:25:30Z'); +insert into users (id, name, description, modified_at) values ('112f5896bc2b00a22d0298c306903a7a', 'Salomé', null, '2020-07-10T20:49:33Z'); +insert into users (id, name, description, modified_at) values ('cb223f42c397ddcd4ae0027582f631b1', 'Lorène', null, '2020-11-10T02:22:21Z'); +insert into users (id, name, description, modified_at) values ('cdfdb0049ae874abdc3f6302b71542b3', 'Märta', null, '2020-07-02T05:08:20Z'); +insert into users (id, name, description, modified_at) values ('12204a7348e3579a20355db5039b15d0', 'Audréanne', 'Other psychoactive substance dependence with withdrawal with perceptual disturbance', '2020-05-12T10:08:37Z'); +insert into users (id, name, description, modified_at) values ('4105fba119a2ad05a07186020e711a78', 'Lén', null, '2020-05-29T22:39:31Z'); +insert into users (id, name, description, modified_at) values ('21326e8f65ed8bd0fe024bc7f9c14c6d', 'Gérald', 'Explosion and rupture of air tank, subsequent encounter', '2020-11-29T07:03:26Z'); +insert into users (id, name, description, modified_at) values ('1c6bee4664ec0af61fd80c97fe36e5b0', 'Joséphine', null, '2020-02-18T09:11:21Z'); +insert into users (id, name, description, modified_at) values ('a677b49058226d42c631bdc8ccc26471', 'Pénélope', null, '2020-05-03T07:10:35Z'); +insert into users (id, name, description, modified_at) values ('66c70eaf6fc5ef1c70295f948624df9b', 'Lèi', null, '2020-01-21T22:22:34Z'); +insert into users (id, name, description, modified_at) values ('d877dd9769dcd7266be6970eec4243ee', 'Lóng', null, '2020-01-06T15:35:41Z'); +insert into users (id, name, description, modified_at) values ('873314e11198d97eff984cb4db6f3800', 'Eliès', null, '2020-01-31T05:47:55Z'); +insert into users (id, name, description, modified_at) values ('2ef93e72db8edad7609cd11023bf494f', 'Åslög', 'Unspecified physeal fracture of unspecified metatarsal, subsequent encounter for fracture with routine healing', '2020-09-02T12:26:47Z'); +insert into users (id, name, description, modified_at) values ('54dee3b67b6fbfe45fe6af06100d97ef', 'Ruò', null, '2020-05-02T06:37:38Z'); +insert into users (id, name, description, modified_at) values ('0501eb8ea01a5c1bb003e6649e7c29db', 'Renée', null, '2020-09-09T23:33:37Z'); +insert into users (id, name, description, modified_at) values ('fa91c9df608fb3e4e244ba2ee5cab3e9', 'Dorothée', null, '2020-03-12T00:15:16Z'); +insert into users (id, name, description, modified_at) values ('f775747a940d3c9c67b7c349c5bad812', 'Vénus', null, '2020-02-03T22:53:58Z'); +insert into users (id, name, description, modified_at) values ('caf137a9e5f1ad5adf05ad3366f7439d', 'Sélène', null, '2020-11-29T20:52:40Z'); +insert into users (id, name, description, modified_at) values ('aeba3e7ff6a4536678f43dc16de6992f', 'Cécile', null, '2020-08-20T02:15:31Z'); +insert into users (id, name, description, modified_at) values ('6bfb925c301cb7c65e6d29198f8deeee', 'Eléonore', null, '2020-04-18T05:40:56Z'); +insert into users (id, name, description, modified_at) values ('26d7fd472f6f5ed14967baa1e98b544e', 'Mélys', 'Nondisplaced fracture of medial cuneiform of unspecified foot, initial encounter for closed fracture', '2020-05-20T23:18:23Z'); +insert into users (id, name, description, modified_at) values ('a4233d593503fd16b479adb8502bc248', 'Anaé', null, '2020-11-25T09:12:30Z'); +insert into users (id, name, description, modified_at) values ('5debb57ce748e3b83891faa867e97137', 'Alizée', null, '2020-07-22T21:59:59Z'); +insert into users (id, name, description, modified_at) values ('260d49e28af84a1e928bc6a8159c0b83', 'Françoise', 'Displaced fracture of head of right radius, subsequent encounter for open fracture type IIIA, IIIB, or IIIC with nonunion', '2020-02-14T09:16:12Z'); +insert into users (id, name, description, modified_at) values ('00242fc8271a43fb9dc96bae625c370e', 'Vénus', 'Choleperitonitis', '2020-05-16T14:36:39Z'); +insert into users (id, name, description, modified_at) values ('5ec7b178e1d4aca5c5638646a1422ba9', 'Östen', 'Granulomatous hepatitis, not elsewhere classified', '2020-08-22T23:20:00Z'); +insert into users (id, name, description, modified_at) values ('fc4b93becb92265d24e38a94ff75106a', 'Örjan', null, '2020-02-28T14:11:03Z'); +insert into users (id, name, description, modified_at) values ('048cb6ec82c085b93b598e7e4322b5d9', 'Fèi', null, '2020-09-02T10:30:58Z'); +insert into users (id, name, description, modified_at) values ('7a78a37ced23962b5671a176975c4246', 'Cléa', 'Other mechanical complication of internal fixation device of vertebrae, initial encounter', '2020-04-08T06:55:28Z'); +insert into users (id, name, description, modified_at) values ('2440faa3d5ff24a1abf7ebac57f7cea5', 'Laurène', null, '2020-08-12T07:27:39Z'); +insert into users (id, name, description, modified_at) values ('0e5beac1a56bd8bb57a56450143aa2cf', 'Maëlyss', null, '2020-10-26T16:16:43Z'); +insert into users (id, name, description, modified_at) values ('1828b127884df85b6921526bfa5b38ae', 'Björn', null, '2020-04-02T20:06:31Z'); +insert into users (id, name, description, modified_at) values ('ab2d525f045a69b8bba8c7f8e7b10786', 'Clémentine', null, '2020-08-22T14:52:35Z'); +insert into users (id, name, description, modified_at) values ('234d9875a136eab9176c625c6fd2d170', 'Léane', null, '2020-07-27T11:44:10Z'); +insert into users (id, name, description, modified_at) values ('a4982fb0833117219dcee8813f6cb42d', 'Yú', null, '2020-01-07T13:12:02Z'); +insert into users (id, name, description, modified_at) values ('3555d8c3e36f04064b86c39ed1e209a6', 'Lauréna', 'Primary blast injury of sigmoid colon', '2020-05-19T01:46:25Z'); +insert into users (id, name, description, modified_at) values ('1bdc1056b8c2b80d598312731cc125af', 'André', 'Other congenital deformities of skull, face and jaw', '2020-10-25T16:33:31Z'); +insert into users (id, name, description, modified_at) values ('2cefd94cb6f7890564ea17fef52ec8fa', 'Crééz', null, '2020-05-03T16:23:33Z'); +insert into users (id, name, description, modified_at) values ('8135e55fce2cceb64385cd03fffd84c6', 'Estève', 'Traumatic amputation at level between elbow and wrist', '2020-02-28T08:15:33Z'); +insert into users (id, name, description, modified_at) values ('a27f91745a301c6903b23475e5c7fe9c', 'Maëly', null, '2020-01-17T12:42:10Z'); +insert into users (id, name, description, modified_at) values ('b2b2150538c1aa66723889dbc8278d67', 'Gaëlle', null, '2020-09-09T18:33:23Z'); +insert into users (id, name, description, modified_at) values ('922b0740b3a8e608e1553aa07896b2d2', 'Bérangère', 'Deformity of unspecified orbit due to bone disease', '2020-02-28T06:45:05Z'); +insert into users (id, name, description, modified_at) values ('0f659a56668897e7adf98dde9575f986', 'Nélie', null, '2020-01-07T14:08:18Z'); +insert into users (id, name, description, modified_at) values ('2a8f32c3d5cd645b038320fb013f9938', 'Médiamass', null, '2020-10-31T04:38:38Z'); +insert into users (id, name, description, modified_at) values ('1318d6a5bfe080fa2ac0668de19b6e54', 'Cléa', 'Burn of first degree of right ear [any part, except ear drum], sequela', '2020-04-23T01:55:30Z'); +insert into users (id, name, description, modified_at) values ('f4e0edf309c4da8da2886f0c9125a183', 'Mårten', null, '2020-06-28T13:04:45Z'); +insert into users (id, name, description, modified_at) values ('7151a35f83d952df7ed791a9c35a2408', 'Stéphanie', null, '2020-08-21T00:03:12Z'); +insert into users (id, name, description, modified_at) values ('cf8e4429e3ba23cbb8181df604be0d90', 'Görel', null, '2020-09-10T07:56:55Z'); +insert into users (id, name, description, modified_at) values ('1b63572893266dffcfb10316a3df8996', 'Joséphine', null, '2020-04-21T05:42:41Z'); +insert into users (id, name, description, modified_at) values ('327df4bb11cf8d5510981b71740f526c', 'Sòng', 'Unspecified fracture of fourth thoracic vertebra, sequela', '2020-01-07T02:11:02Z'); +insert into users (id, name, description, modified_at) values ('909b3c9abfb8dd302d8bca1009d116f2', 'Léonie', 'Cannabis dependence with unspecified cannabis-induced disorder', '2020-09-20T14:02:12Z'); +insert into users (id, name, description, modified_at) values ('07107cccc504299e56f8921edcf1b11b', 'Loïca', 'Puncture wound with foreign body of abdominal wall, epigastric region with penetration into peritoneal cavity, initial encounter', '2020-07-18T05:42:54Z'); +insert into users (id, name, description, modified_at) values ('5c7a1980d30c50709d95a5915cada8a1', 'Frédérique', null, '2020-08-06T09:08:57Z'); +insert into users (id, name, description, modified_at) values ('2537a76ae7601a4713743a04329193ac', 'Maï', null, '2020-07-23T12:15:29Z'); +insert into users (id, name, description, modified_at) values ('22e2230af321069ca7bda2fc37297f65', 'Ráo', null, '2020-06-03T22:43:28Z'); +insert into users (id, name, description, modified_at) values ('4b6c15bdb93919990f83448d8c3f36f7', 'Mélodie', 'Hypertensive heart and chronic kidney disease without heart failure', '2020-11-01T01:01:29Z'); +insert into users (id, name, description, modified_at) values ('442eb8233b012184cb5d916351ab0913', 'Yáo', null, '2020-02-14T07:36:01Z'); +insert into users (id, name, description, modified_at) values ('94be91552eae4d568766206e5f731aed', 'Andrée', null, '2020-11-16T08:44:23Z'); +insert into users (id, name, description, modified_at) values ('a39302e5d02654e98f6c69d9b5a9a4b5', 'Maëlla', null, '2020-11-29T08:58:43Z'); +insert into users (id, name, description, modified_at) values ('2bc2574420bcc73514ba3d007ea3de4b', 'Maëlann', null, '2020-06-11T02:19:09Z'); +insert into users (id, name, description, modified_at) values ('8ff07cfe6e3356d5094dd9f058b4b50f', 'Valérie', 'Person on outside of bus injured in collision with railway train or railway vehicle in nontraffic accident, sequela', '2020-01-18T13:56:50Z'); +insert into users (id, name, description, modified_at) values ('95b65cec4d5105c9237d523cda84e642', 'Françoise', 'Toxic effect of venom of bees, accidental (unintentional)', '2020-07-13T23:19:19Z'); +insert into users (id, name, description, modified_at) values ('b09db40ae4c3d8b9b2cbad3a6528eaa4', 'Anaïs', null, '2020-03-19T01:03:00Z'); +insert into users (id, name, description, modified_at) values ('d4b3a4e9bdc8a325731b6abdefb4d67e', 'Cunégonde', null, '2020-02-04T13:29:17Z'); +insert into users (id, name, description, modified_at) values ('3409dd6844c6fe1950e4d5399fc51b47', 'Rachèle', 'Cysticercosis of other sites', '2020-03-15T12:19:58Z'); +insert into users (id, name, description, modified_at) values ('03936f9f866730004e24a99be493aaa8', 'Léandre', null, '2020-01-01T16:37:06Z'); +insert into users (id, name, description, modified_at) values ('8220a59df70146f9c95e681f54dd3b22', 'Eloïse', null, '2020-05-23T09:56:40Z'); +insert into users (id, name, description, modified_at) values ('5d46c0fea5bdc30730306231b65ec131', 'Mélys', null, '2020-01-22T22:41:26Z'); +insert into users (id, name, description, modified_at) values ('f450469cc983efcb3d3042342b3914ac', 'Cunégonde', 'Dislocation of interphalangeal joint of left great toe, sequela', '2020-07-25T06:36:10Z'); +insert into users (id, name, description, modified_at) values ('c2ac590b267a647e8e5e310bfeaa3d89', 'Måns', 'Bence Jones proteinuria', '2020-02-16T03:00:31Z'); +insert into users (id, name, description, modified_at) values ('af6e3a6e67219275867b1502fd8794cd', 'Almérinda', null, '2020-09-17T03:18:27Z'); +insert into users (id, name, description, modified_at) values ('0a3b68ed89753cdd5d846e6315a5320c', 'Estée', null, '2020-09-10T07:29:25Z'); +insert into users (id, name, description, modified_at) values ('61ba8f14af843809c68de12553b7cf63', 'Gwenaëlle', null, '2020-02-22T22:57:55Z'); +insert into users (id, name, description, modified_at) values ('be94d49f2635b8ca198886d402c681a3', 'Salomé', null, '2020-11-25T12:51:01Z'); +insert into users (id, name, description, modified_at) values ('4855fbd0f499e2a99451492ee5024da8', 'Béatrice', null, '2020-11-13T00:01:53Z'); +insert into users (id, name, description, modified_at) values ('b23559fb41a48a066da58073046dc908', 'Lài', null, '2020-03-08T03:37:34Z'); +insert into users (id, name, description, modified_at) values ('c0a531e2696c66362e48b722c1d669b5', 'Mélia', null, '2020-09-28T23:07:45Z'); +insert into users (id, name, description, modified_at) values ('73c1f1312ce681634b7d4485c9990b96', 'Hélène', null, '2020-03-12T19:34:08Z'); +insert into users (id, name, description, modified_at) values ('519147811c3056845e16d45ba6a6ceff', 'Björn', 'Dislocation of tarsal joint of unspecified foot, subsequent encounter', '2020-09-06T09:37:19Z'); +insert into users (id, name, description, modified_at) values ('85bfba067b2ab39a0f6d691c9e8bbcbf', 'Célia', 'Sprain of metatarsophalangeal joint of unspecified great toe, initial encounter', '2020-03-25T07:06:37Z'); +insert into users (id, name, description, modified_at) values ('e3e7b46bac77a6a7431c0cbe1ec0a9c9', 'Yóu', null, '2020-03-15T01:10:56Z'); +insert into users (id, name, description, modified_at) values ('b6ae4f073882f1ffe9b29206820111b2', 'Thérèse', null, '2020-05-24T03:00:26Z'); +insert into users (id, name, description, modified_at) values ('d8f2571b322b5b984ad74e0f405656a8', 'Anaé', null, '2020-03-16T07:43:23Z'); +insert into users (id, name, description, modified_at) values ('808878695ad0eefb280d5292c88f69f6', 'Méryl', null, '2020-06-10T03:42:57Z'); +insert into users (id, name, description, modified_at) values ('c1ab0827d2a7384c06e592fa04a49a18', 'Börje', null, '2020-07-29T14:30:23Z'); +insert into users (id, name, description, modified_at) values ('7c6455cc9e07d884cc9334f83cb65c2f', 'Marie-françoise', null, '2020-04-25T10:03:05Z'); +insert into users (id, name, description, modified_at) values ('58ba4d029375e0e6d9cb40035e6c6b1b', 'Laïla', 'Laceration without foreign body of unspecified breast', '2020-01-09T22:26:23Z'); +insert into users (id, name, description, modified_at) values ('3ec0a94d32c1a3cff1e9f8ea89d7f9b8', 'Simplifiés', 'Dislocation of acromioclavicular joint, greater than 200% displacement', '2020-09-16T09:28:23Z'); +insert into users (id, name, description, modified_at) values ('7c211d32fb0601b13ccf4ffb0334a58e', 'Cunégonde', null, '2020-03-10T05:57:53Z'); +insert into users (id, name, description, modified_at) values ('7b8a7b4e8fc2add6ac7d8f9f928f3574', 'Anaël', null, '2020-06-28T23:27:37Z'); +insert into users (id, name, description, modified_at) values ('81829cfe48b485faec32b165411e6738', 'Laurélie', null, '2020-01-11T10:51:36Z'); +insert into users (id, name, description, modified_at) values ('d62342791a9924825dd70eb0c8634ec6', 'Mélia', null, '2020-05-02T04:31:42Z'); +insert into users (id, name, description, modified_at) values ('0f041e293ce8fc6dda47f3a2377975b8', 'Zhì', null, '2020-01-09T15:41:56Z'); +insert into users (id, name, description, modified_at) values ('01b2899d469f6f175eebb9add3e5f29b', 'Naéva', null, '2020-08-22T14:27:08Z'); +insert into users (id, name, description, modified_at) values ('079736fa7d8cd451f7ad52ca1f7780b7', 'Maïwenn', null, '2020-11-07T18:54:37Z'); +insert into users (id, name, description, modified_at) values ('3ca4ca47aa17f2d931006eb82324e67c', 'Eléonore', 'Displaced fracture of olecranon process without intraarticular extension of unspecified ulna, subsequent encounter for open fracture type IIIA, IIIB, or IIIC with delayed healing', '2020-05-13T14:52:41Z'); +insert into users (id, name, description, modified_at) values ('ef2092fc6c70bc7a85a8d6772ca9b3dd', 'Andrée', 'Toxic effect of venom of other African and Asian snake, undetermined, subsequent encounter', '2020-11-26T11:32:00Z'); +insert into users (id, name, description, modified_at) values ('ac33f40b7e404099ba821270b8fbe848', 'Illustrée', null, '2020-07-19T01:03:10Z'); +insert into users (id, name, description, modified_at) values ('655243e47f159b2ff17ab6b25cc840b7', 'Annotée', null, '2020-06-28T06:07:13Z'); +insert into users (id, name, description, modified_at) values ('011317a92c88cecd1574d285832fdb2c', 'Ruì', null, '2020-11-17T16:53:29Z'); +insert into users (id, name, description, modified_at) values ('15695ab5613b86237f1a4df499d7ccde', 'Marie-thérèse', 'Pathological fracture, left ulna, subsequent encounter for fracture with malunion', '2020-10-09T00:51:35Z'); +insert into users (id, name, description, modified_at) values ('f1e6c0ba672df2683d2ef4c266be8fd1', 'Adélaïde', null, '2020-05-03T16:16:04Z'); +insert into users (id, name, description, modified_at) values ('45692c7874d31ad1930ed76a75efdac3', 'Geneviève', null, '2020-01-19T05:37:49Z'); +insert into users (id, name, description, modified_at) values ('2157352b4b36dcea55b06531cab8f8b2', 'Maïly', null, '2020-11-02T00:54:02Z'); +insert into users (id, name, description, modified_at) values ('d1db5317127715f4bf6760950216c3d5', 'Méng', 'Laceration of flexor muscle, fascia and tendon of left ring finger at wrist and hand level, initial encounter', '2020-04-16T15:15:19Z'); +insert into users (id, name, description, modified_at) values ('3c7a7d5f7646646fc608d698d6f7407c', 'Maïté', null, '2020-11-26T04:24:48Z'); +insert into users (id, name, description, modified_at) values ('754085db92de7679498c5787e4d155be', 'Aimée', 'Dislocation of unspecified thoracic vertebra, subsequent encounter', '2020-09-15T03:48:56Z'); +insert into users (id, name, description, modified_at) values ('83aba97b7b3b55b6a2df6139368eb9d7', 'Zhì', null, '2020-06-14T16:59:22Z'); +insert into users (id, name, description, modified_at) values ('67eca3c2dcd41058f85c0fbbdfad5c93', 'Maïlys', null, '2020-10-22T08:09:50Z'); +insert into users (id, name, description, modified_at) values ('85a76ec9d55c732502392c4de3d6ce60', 'Maïlis', null, '2020-06-02T15:47:42Z'); +insert into users (id, name, description, modified_at) values ('d4b3554ab18061a3e8608349d3fe7ae4', 'Lài', null, '2020-07-11T22:45:14Z'); +insert into users (id, name, description, modified_at) values ('de37bed134a06bafe64bd95cc2bcb5dc', 'Chloé', 'Toxic effect of chlorofluorocarbons, assault, initial encounter', '2020-06-16T08:03:58Z'); +insert into users (id, name, description, modified_at) values ('82720dea5b8bc72485f08a6cd5c30f1a', 'Yáo', null, '2020-03-10T22:36:03Z'); +insert into users (id, name, description, modified_at) values ('befaeb9bf00d52c7fec1a7a0686f10ab', 'Amélie', 'Labor and delivery complicated by short cord, fetus 3', '2020-11-13T18:58:08Z'); +insert into users (id, name, description, modified_at) values ('ec871f49f46dae0b3b78bd7e39411efd', 'Desirée', null, '2020-08-03T11:21:17Z'); +insert into users (id, name, description, modified_at) values ('0d8514caec3b8ca13851e913fca3ee67', 'Nadège', null, '2020-09-24T14:00:29Z'); +insert into users (id, name, description, modified_at) values ('6caa6363e5d2718fd0b8fe2acb085549', 'Léandre', null, '2020-09-02T17:42:52Z'); +insert into users (id, name, description, modified_at) values ('fd87701f2297934998185e47d90a649e', 'Bénédicte', null, '2020-05-17T14:03:55Z'); +insert into users (id, name, description, modified_at) values ('57be3cc77cd3a3ea1bcdd3c4d3f2cca9', 'Aimée', null, '2020-06-20T15:58:23Z'); +insert into users (id, name, description, modified_at) values ('00a7f37e370c34f4f27917ccc4d2a587', 'Séverine', 'Unspecified physeal fracture of left metatarsal', '2020-01-25T20:29:02Z'); +insert into users (id, name, description, modified_at) values ('094167d298634e063d612095b78092ed', 'Ráo', null, '2020-10-08T03:38:40Z'); +insert into users (id, name, description, modified_at) values ('55693c50b65e1f2477d1e4964fd1d460', 'Zoé', null, '2020-02-09T17:28:52Z'); +insert into users (id, name, description, modified_at) values ('adbf2f32e518071a7c0aa36665e19e53', 'Josée', null, '2020-07-17T00:56:42Z'); +insert into users (id, name, description, modified_at) values ('7c41ab585640e40791e9ffc7fa0ed865', 'Yú', null, '2020-07-17T13:49:34Z'); +insert into users (id, name, description, modified_at) values ('6f9e5ceccd6dac5f68c188cd57c93981', 'Cinéma', 'Contusion, laceration, and hemorrhage of cerebellum with loss of consciousness greater than 24 hours without return to pre-existing conscious level with patient surviving, initial encounter', '2020-07-31T22:23:34Z'); +insert into users (id, name, description, modified_at) values ('22977f49bebfbd2eaf09b89e2e14a53e', 'Maëlyss', null, '2020-11-16T09:04:03Z'); +insert into users (id, name, description, modified_at) values ('875d16ae7b9efc639b14cdd6da5f00a6', 'Léone', null, '2020-09-13T13:38:23Z'); +insert into users (id, name, description, modified_at) values ('b58e363de82457146fc0814ef923f5f9', 'Andréa', null, '2020-01-24T11:09:31Z'); +insert into users (id, name, description, modified_at) values ('99181a91bee3d4bab90c12a0c8b6d6ee', 'Mén', 'Displaced fracture of proximal phalanx of left ring finger, sequela', '2020-07-07T02:28:21Z'); +insert into users (id, name, description, modified_at) values ('41cd3134514eef05c47b3231faada0d6', 'Åsa', 'Allergic dermatitis of left lower eyelid', '2020-04-11T12:48:04Z'); +insert into users (id, name, description, modified_at) values ('693f8c90ef32f98478d75749945e55cc', 'Dorothée', null, '2020-06-28T15:01:33Z'); +insert into users (id, name, description, modified_at) values ('505f74be2c28f0a95a5603752b801e35', 'Torbjörn', 'Displaced fracture of fifth metatarsal bone, left foot, subsequent encounter for fracture with malunion', '2020-10-29T16:43:35Z'); +insert into users (id, name, description, modified_at) values ('7a950760cdbd53c35467b3e65699bfcf', 'Mégane', null, '2020-09-07T01:39:03Z'); +insert into users (id, name, description, modified_at) values ('448327c82838710c409871bac7059836', 'Adèle', null, '2020-10-27T11:25:27Z'); +insert into users (id, name, description, modified_at) values ('96af6af981b3c83357cbb7197dd444c9', 'André', null, '2020-11-09T14:03:36Z'); +insert into users (id, name, description, modified_at) values ('1495fa8a235676c83a9599e7ce1f19cc', 'Hélène', null, '2020-07-27T17:05:32Z'); +insert into users (id, name, description, modified_at) values ('ee74f36d6d5e172fb3a6c1a65cf42f34', 'Mà', null, '2020-07-17T01:18:47Z'); +insert into users (id, name, description, modified_at) values ('f744caaf59f3a0ff41390b91d404980d', 'Clémence', null, '2020-01-21T19:03:23Z'); +insert into users (id, name, description, modified_at) values ('5bbd1894fb89ac286545ef329767eb6e', 'Jú', 'Unspecified Zone I fracture of sacrum, subsequent encounter for fracture with nonunion', '2020-05-27T22:15:12Z'); +insert into users (id, name, description, modified_at) values ('dd0348b6ef45605b5bc659e2bb0b56f7', 'Stévina', 'Laceration with foreign body of unspecified great toe with damage to nail, initial encounter', '2020-05-05T07:26:22Z'); +insert into users (id, name, description, modified_at) values ('c3ac663b142c1ff817b0cfd90925d733', 'Bénédicte', null, '2020-02-22T23:21:46Z'); +insert into users (id, name, description, modified_at) values ('b47fe88cddc22bdbefafaf85807b4a2e', 'Naéva', null, '2020-08-02T08:34:57Z'); +insert into users (id, name, description, modified_at) values ('36986568b95093e15f242164f2ad1880', 'Mahélie', null, '2020-08-11T09:33:27Z'); +insert into users (id, name, description, modified_at) values ('cca81712e6ae2d742d89aaa42f70c0a3', 'Zoé', 'Pedal cyclist (driver) (passenger) injured in other specified transport accidents', '2020-08-04T22:49:30Z'); +insert into users (id, name, description, modified_at) values ('983eb2874aefc29763381a1b573d2cbb', 'Gaïa', null, '2020-03-19T15:15:31Z'); +insert into users (id, name, description, modified_at) values ('adcac12f01ddbc3f3e9f24f963c0ceb9', 'Almérinda', null, '2020-04-13T04:00:04Z'); +insert into users (id, name, description, modified_at) values ('6b9f5574afe768701baa0de6c9c3f65f', 'Maëlys', null, '2020-11-01T15:02:34Z'); +insert into users (id, name, description, modified_at) values ('3d072c44d7c30abf87b13ebdb9505704', 'Régine', null, '2020-03-19T02:39:52Z'); +insert into users (id, name, description, modified_at) values ('c2b4361fcadbb680f56716f2e48a50e4', 'Marie-françoise', null, '2020-01-31T04:02:07Z'); +insert into users (id, name, description, modified_at) values ('87d2238f36fb237b0c9fe1a0371e87e0', 'Hélène', null, '2020-09-17T22:49:40Z'); +insert into users (id, name, description, modified_at) values ('3566973c83bf99f2fa4a21ee88f6e8ad', 'Adélie', 'Major laceration of left carotid artery', '2020-03-14T21:47:12Z'); +insert into users (id, name, description, modified_at) values ('1e62ba4b2dcf4ce5739f967d1fd2d93b', 'Mélys', 'Other specified bursopathies, hand', '2020-01-03T18:35:52Z'); +insert into users (id, name, description, modified_at) values ('c3fd66fdee897a54dcc7398e3579ad24', 'Sòng', 'Laceration of muscle, fascia and tendon of long head of biceps, left arm, initial encounter', '2020-02-21T13:56:22Z'); +insert into users (id, name, description, modified_at) values ('9d3a8f13f9406980f0980c4db4bcf045', 'Åsa', 'Gonococcal heart infection', '2020-08-14T13:43:27Z'); +insert into users (id, name, description, modified_at) values ('e32078e07ecbd7bbacb2be61d4ef26b9', 'Anaël', null, '2020-02-23T15:42:56Z'); +insert into users (id, name, description, modified_at) values ('9bcfbcdab2b7db7cae01517450a1abe8', 'Méthode', 'Nondisplaced fracture of right tibial tuberosity', '2020-04-26T20:36:09Z'); +insert into users (id, name, description, modified_at) values ('a07f5faa2fe81d7b3d553adc85b7b1fc', 'Wá', null, '2020-09-15T20:35:56Z'); +insert into users (id, name, description, modified_at) values ('4fccd9d3217584a749e11fd9b81364d1', 'Angélique', null, '2020-01-02T21:07:10Z'); +insert into users (id, name, description, modified_at) values ('c0fe63a6379d88620340e13afc2fcb91', 'Mélinda', null, '2020-02-07T02:00:23Z'); +insert into users (id, name, description, modified_at) values ('743e8381309f9d42b2085e67520e4d7d', 'Angélique', null, '2020-11-28T04:33:25Z'); +insert into users (id, name, description, modified_at) values ('d3fc5906afb77953f7bb0f74679b6a04', 'Maïlis', null, '2020-08-02T08:14:36Z'); +insert into users (id, name, description, modified_at) values ('fc0435bb50422b373c64274f9d707b69', 'Gösta', null, '2020-07-24T14:21:20Z'); +insert into users (id, name, description, modified_at) values ('ba001e11e36a6e49566c173216d54c04', 'Illustrée', 'Sprain of metatarsophalangeal joint of right lesser toe(s), sequela', '2020-09-01T00:53:53Z'); +insert into users (id, name, description, modified_at) values ('d10e2cf945768eaa3a8ab00fe4f46aae', 'Joséphine', null, '2020-11-03T15:44:10Z'); +insert into users (id, name, description, modified_at) values ('fc7c14b42772cb02f70817ca3abe7389', 'Réservés', null, '2020-09-28T22:33:21Z'); +insert into users (id, name, description, modified_at) values ('36ceb41f984378e5b2a78b8dae56df06', 'Nélie', null, '2020-04-14T18:28:21Z'); +insert into users (id, name, description, modified_at) values ('5c92df5be4f7090f6984cc05f66aa326', 'Miléna', 'Idiopathic chronic gout, left knee, with tophus (tophi)', '2020-01-21T10:40:57Z'); +insert into users (id, name, description, modified_at) values ('c3072906304a718469f9510c2f55c49f', 'Gösta', null, '2020-04-28T16:29:22Z'); +insert into users (id, name, description, modified_at) values ('600edec067f9f236dc827566300078d6', 'Desirée', null, '2020-05-14T19:07:56Z'); +insert into users (id, name, description, modified_at) values ('ff4e4553e49add450dc10848cbfcbf07', 'Mégane', 'Malignant neoplasm of uvula', '2020-04-30T12:51:32Z'); +insert into users (id, name, description, modified_at) values ('f2403588b2a62b969159b17a6729ea24', 'Loïca', null, '2020-02-27T23:30:08Z'); +insert into users (id, name, description, modified_at) values ('7dd9a569c988ae786083bbff290f01a3', 'Célia', null, '2020-05-25T02:53:45Z'); +insert into users (id, name, description, modified_at) values ('d67fa8fb727d97d9d2eb4869377614e3', 'Clélia', null, '2020-10-21T21:05:05Z'); +insert into users (id, name, description, modified_at) values ('b112cc7e18f70a386acb2cdc10720028', 'Marie-josée', null, '2020-04-03T23:14:43Z'); +insert into users (id, name, description, modified_at) values ('a84bd6ad657d3d7ce5a826eae0653039', 'Lài', null, '2020-05-29T17:20:39Z'); +insert into users (id, name, description, modified_at) values ('576c776eb578f09096433054581c46ca', 'Anaé', null, '2020-04-21T12:56:14Z'); +insert into users (id, name, description, modified_at) values ('9120ffab7340ae74a43b899bac943a57', 'Mélinda', null, '2020-08-23T05:50:17Z'); +insert into users (id, name, description, modified_at) values ('2b8453e9444645d1732e308f19e8af5c', 'Maï', null, '2020-09-03T01:57:49Z'); +insert into users (id, name, description, modified_at) values ('621892ea712f44e219f536ff98ca7ddd', 'Marylène', null, '2020-06-06T10:00:05Z'); +insert into users (id, name, description, modified_at) values ('61abb14c36f16a00ddfc82abad37835a', 'Hélèna', null, '2020-06-10T14:48:25Z'); +insert into users (id, name, description, modified_at) values ('9d9dff668a4b59b2bd601b49d3630ffa', 'Adélaïde', null, '2020-10-10T04:22:42Z'); +insert into users (id, name, description, modified_at) values ('5bcbcc9dbdb377e1769e1579fceb590d', 'Zoé', null, '2020-10-17T06:03:00Z'); +insert into users (id, name, description, modified_at) values ('b9a811324b1eeeeed3b2f258f3d6476e', 'Björn', null, '2020-07-06T18:58:18Z'); +insert into users (id, name, description, modified_at) values ('5ca71ceccb44db48f1e2cfffc66f48e3', 'Daphnée', null, '2020-09-05T19:30:44Z'); +insert into users (id, name, description, modified_at) values ('2b4e031152e079918be901e5367fe22c', 'Maëline', null, '2020-10-13T20:30:42Z'); +insert into users (id, name, description, modified_at) values ('f6db6a6bbcf8a7f04dc6e2a96629e32e', 'Wá', null, '2020-04-26T16:36:29Z'); +insert into users (id, name, description, modified_at) values ('c15aff26ced7c8807d1b8211a328440f', 'Léonie', null, '2020-08-06T19:04:46Z'); +insert into users (id, name, description, modified_at) values ('ed184b7296356b6447c6084349bfc793', 'Lucrèce', null, '2020-04-06T10:38:27Z'); +insert into users (id, name, description, modified_at) values ('4b1ad8c70a2f6350623e3efd259dafca', 'Agnès', null, '2020-05-23T01:41:13Z'); +insert into users (id, name, description, modified_at) values ('ab3cdf8e63bd0fdea83108c2a35c0483', 'Médiamass', null, '2020-09-08T03:58:13Z'); +insert into users (id, name, description, modified_at) values ('40ad2885aa627d506a9fc971c7138300', 'Lài', null, '2020-10-24T07:00:40Z'); +insert into users (id, name, description, modified_at) values ('095779b21ab317065324e27089610912', 'Mélys', 'Laceration without foreign body of left front wall of thorax without penetration into thoracic cavity, subsequent encounter', '2020-10-27T18:00:11Z'); +insert into users (id, name, description, modified_at) values ('2270ad36911cef7ffbd471d2c75635e6', 'Almérinda', null, '2020-11-15T20:47:46Z'); +insert into users (id, name, description, modified_at) values ('c91d3f2a0ea9180c83abd7f7a7c238ac', 'Séréna', null, '2020-03-12T23:25:10Z'); +insert into users (id, name, description, modified_at) values ('d494086fb9786c605a98640facc99b40', 'Åslög', null, '2020-04-19T16:20:32Z'); +insert into users (id, name, description, modified_at) values ('2fd124605c26820ff69ec6239b6071fc', 'Simplifiés', 'Subluxation of unspecified parts of unspecified shoulder girdle, subsequent encounter', '2020-11-29T20:11:32Z'); +insert into users (id, name, description, modified_at) values ('8b98c4f2757df6fbac7bd598e678ba88', 'Adèle', null, '2020-09-07T11:54:22Z'); +insert into users (id, name, description, modified_at) values ('c45debc696a6147183012a7b149205ba', 'Eliès', 'Open bite of left front wall of thorax with penetration into thoracic cavity, initial encounter', '2020-06-27T15:12:47Z'); +insert into users (id, name, description, modified_at) values ('7d69b71b9ed86e4a80989d6e2775b218', 'Cléa', null, '2020-11-21T09:57:16Z'); +insert into users (id, name, description, modified_at) values ('df7223a742b2cb02abbad8a87e1a0bc5', 'Pål', null, '2020-06-08T22:13:34Z'); +insert into users (id, name, description, modified_at) values ('2dff52f9d1c1e7ee6d469b8a96fea44a', 'Mélinda', 'Other chondrocalcinosis, multiple sites', '2020-11-29T18:40:45Z'); +insert into users (id, name, description, modified_at) values ('5c716366341d906adbce53b7b03500e2', 'Noémie', 'Salter-Harris Type III physeal fracture of lower end of humerus, left arm, subsequent encounter for fracture with routine healing', '2020-03-09T10:17:02Z'); +insert into users (id, name, description, modified_at) values ('c40b246ebf0d7d702b85bb23c84ae596', 'Laurélie', null, '2020-06-03T21:54:06Z'); +insert into users (id, name, description, modified_at) values ('b393bf64d997a9f77de588b21de9a0c8', 'Kuí', 'Primary dysmenorrhea', '2020-06-25T23:40:57Z'); +insert into users (id, name, description, modified_at) values ('bd4eba0c8ff82c864f76e100be162231', 'Ruì', null, '2020-06-02T06:13:22Z'); +insert into users (id, name, description, modified_at) values ('4dff7bf5e99485152dd989c405a40288', 'Maïté', 'Displaced fracture of body of unspecified calcaneus, subsequent encounter for fracture with delayed healing', '2020-10-03T05:22:21Z'); +insert into users (id, name, description, modified_at) values ('d75d19f0e405f2485b16169d8214a7e6', 'Sòng', 'Greenstick fracture of shaft of radius, right arm', '2020-09-16T13:49:17Z'); +insert into users (id, name, description, modified_at) values ('d2eca9cf31ef3b36073e10f8c909ad26', 'Sélène', null, '2020-08-13T11:40:44Z'); +insert into users (id, name, description, modified_at) values ('626014c5dd7f4ff53ba8a11ca08b797f', 'Méthode', 'Laceration of extensor or abductor muscles, fascia and tendons of left thumb at forearm level, sequela', '2020-03-13T03:58:04Z'); +insert into users (id, name, description, modified_at) values ('f0f3fb4190987a8428fcd3f91a6169e6', 'Annotés', null, '2020-04-17T17:43:00Z'); +insert into users (id, name, description, modified_at) values ('4ade3f3bb519a16e75a9c8d1a03309ea', 'André', 'Follicular lymphoma grade IIIb, intra-abdominal lymph nodes', '2020-04-13T17:50:40Z'); +insert into users (id, name, description, modified_at) values ('2b2b28c40bef3710dad7861a0d64f231', 'Lèi', null, '2020-03-09T08:57:36Z'); +insert into users (id, name, description, modified_at) values ('a355a0b4b8ef93fcfbda579014139890', 'Bérangère', null, '2020-03-16T06:19:23Z'); +insert into users (id, name, description, modified_at) values ('65a3f4c80bfbedc9e5fb67149d226ed0', 'Josée', null, '2020-05-22T11:28:35Z'); +insert into users (id, name, description, modified_at) values ('f7441e880386575366a0b555bc83ed20', 'Aimée', 'Displaced fracture of shaft of first metacarpal bone, left hand, initial encounter for open fracture', '2020-04-16T20:34:26Z'); +insert into users (id, name, description, modified_at) values ('f31f27b7175d348156fb726e893575f5', 'Maëlle', 'Driver of bus injured in collision with fixed or stationary object in traffic accident, sequela', '2020-11-12T08:50:55Z'); +insert into users (id, name, description, modified_at) values ('f48e6d0feb33c922da83370f33cce6de', 'Ráo', null, '2020-09-20T00:25:27Z'); +insert into users (id, name, description, modified_at) values ('a3f527a4ecb018ae98ef2ebee6178611', 'Gisèle', null, '2020-10-21T20:01:11Z'); +insert into users (id, name, description, modified_at) values ('63846ab1df0dd8a0e3c0091465c92ee7', 'Valérie', null, '2020-07-18T19:18:39Z'); +insert into users (id, name, description, modified_at) values ('49abe4a61822f4b8c22898d5c6c7888a', 'Magdalène', 'Rheumatoid heart disease with rheumatoid arthritis of unspecified hand', '2020-03-27T16:16:22Z'); +insert into users (id, name, description, modified_at) values ('59f989ffd804ce1f069691c7dbe5414e', 'Léane', null, '2020-04-01T23:46:20Z'); +insert into users (id, name, description, modified_at) values ('add7a4f5d80ab63e0c1bac7b7d949775', 'Kallisté', null, '2020-04-24T09:08:12Z'); +insert into users (id, name, description, modified_at) values ('aa427377bdbe37455921788110da2d38', 'Gösta', null, '2020-10-02T01:20:24Z'); +insert into users (id, name, description, modified_at) values ('28564ededf09d0fd424a0c5d1581bcb1', 'Pò', null, '2020-11-06T22:10:40Z'); +insert into users (id, name, description, modified_at) values ('e29e0bf3b2c86c06b3506877537e2d84', 'Cinéma', 'Pathological fracture in neoplastic disease, left ulna, subsequent encounter for fracture with delayed healing', '2020-09-11T10:49:53Z'); +insert into users (id, name, description, modified_at) values ('cfcb1180a1d2f9eea758c0a8a77f62c9', 'Cléopatre', null, '2020-03-30T07:26:26Z'); +insert into users (id, name, description, modified_at) values ('38fafacded84ba422ed10861e5bab666', 'Lóng', 'Exfoliation due to erythematous condition involving 30-39 percent of body surface', '2020-03-18T11:53:35Z'); +insert into users (id, name, description, modified_at) values ('d55ab5844273eff700bfcf6263a8e377', 'Rachèle', null, '2020-04-11T05:07:03Z'); +insert into users (id, name, description, modified_at) values ('b69b9606276d928673b43c1e72a50aa8', 'Uò', 'Other injury due to other accident on board merchant ship, sequela', '2020-05-27T02:23:45Z'); +insert into users (id, name, description, modified_at) values ('0b52ac404155aeb498c81a877afe3490', 'Tán', null, '2020-04-23T11:53:05Z'); +insert into users (id, name, description, modified_at) values ('8b0f3baa4e66ac65a215a7dfa984a08c', 'Néhémie', 'Complete physeal arrest, right proximal humerus', '2020-08-20T09:39:40Z'); +insert into users (id, name, description, modified_at) values ('6f7b6c26f4c23e5ee6aec0b49159a558', 'Estée', null, '2020-01-02T01:15:04Z'); +insert into users (id, name, description, modified_at) values ('3ae8332760e319bc0b06f05ac3dfff88', 'Laurélie', null, '2020-07-26T11:06:48Z'); +insert into users (id, name, description, modified_at) values ('3a583bd2539fbb0b666035f31ac6ebae', 'Océane', null, '2020-08-12T20:21:39Z'); +insert into users (id, name, description, modified_at) values ('ba9dd867cab82620d206daca7b0ce6bf', 'Estée', null, '2020-01-21T00:50:29Z'); +insert into users (id, name, description, modified_at) values ('1319b2d5be7f60593f786e66dc3285e3', 'Sélène', null, '2020-04-14T13:03:59Z'); +insert into users (id, name, description, modified_at) values ('6c1984d096c9471fbe8f4a57b121c074', 'Anaëlle', 'Drug-induced chronic gout, multiple sites, without tophus (tophi)', '2020-03-14T14:50:01Z'); +insert into users (id, name, description, modified_at) values ('8130ad48772cc08ca6d5aa7521b5b6bd', 'Kuí', null, '2020-09-07T08:30:23Z'); +insert into users (id, name, description, modified_at) values ('1cff8236351687dad1bc1d240831d628', 'Alizée', null, '2020-02-03T02:05:39Z'); +insert into users (id, name, description, modified_at) values ('cece51ce8194358adca70d347737ba6b', 'Laurène', 'Abnormality of albumin', '2020-05-01T09:54:48Z'); +insert into users (id, name, description, modified_at) values ('1d81bacd5267000b18aec2b437d31d5c', 'Åsa', 'Unspecified injury of other blood vessels at wrist and hand level of unspecified arm', '2020-03-16T14:30:21Z'); +insert into users (id, name, description, modified_at) values ('835dc7afc220951aba2ea9c5d049fc48', 'Marylène', null, '2020-09-14T09:19:29Z'); +insert into users (id, name, description, modified_at) values ('666fda1ede9d8010b0ff99f788101013', 'Lén', null, '2020-09-23T03:11:33Z'); +insert into users (id, name, description, modified_at) values ('5e830442a3b7fdb71de35f9a05e4c6df', 'Kuí', null, '2020-10-29T18:52:49Z'); +insert into users (id, name, description, modified_at) values ('6425d9177845449b9cd1f0100712a647', 'Estée', 'Corrosion of third degree of multiple right fingers (nail), not including thumb, sequela', '2020-06-22T10:56:17Z'); +insert into users (id, name, description, modified_at) values ('bf38370a3be8d23841d13d1e3d576bfd', 'Yè', null, '2020-08-27T12:44:00Z'); +insert into users (id, name, description, modified_at) values ('1261279e8744a4729708f42d5aed9e8d', 'Eliès', null, '2020-08-23T09:05:22Z'); +insert into users (id, name, description, modified_at) values ('3a072c59049ad074d9c133f2882e12c8', 'Garçon', null, '2020-08-28T02:22:30Z'); +insert into users (id, name, description, modified_at) values ('645f1aa9c8ea2edcd51924452e4a7f24', 'Andréanne', 'Toxic effect of carbon dioxide, undetermined, subsequent encounter', '2020-02-10T07:44:00Z'); +insert into users (id, name, description, modified_at) values ('71cd8f745f4fb4379727353c96a38ad1', 'Adèle', null, '2020-02-15T02:49:26Z'); +insert into users (id, name, description, modified_at) values ('0db54b3d492d9b57134ab0505e7d3f99', 'Intéressant', null, '2020-01-03T10:19:58Z'); +insert into users (id, name, description, modified_at) values ('71359bcae4f50dd1013e0f31486aca6c', 'Ruò', 'Corrosion of cornea and conjunctival sac, left eye, subsequent encounter', '2020-04-27T14:09:56Z'); +insert into users (id, name, description, modified_at) values ('89f18291e7353a531d48a26f7cbcbff9', 'Marie-françoise', null, '2020-05-02T12:14:11Z'); +insert into users (id, name, description, modified_at) values ('ae38b5da86514514a3b7086b61f1290e', 'Aí', null, '2020-08-03T06:31:10Z'); +insert into users (id, name, description, modified_at) values ('512b5659c35747632afb0c8a74595ab9', 'Edmée', null, '2020-06-09T00:05:24Z'); +insert into users (id, name, description, modified_at) values ('ce396958e7314b9451084d1b3d7eab19', 'Léandre', null, '2020-05-02T17:55:39Z'); +insert into users (id, name, description, modified_at) values ('fa05a4c1fcb90e340b1bf4f40ee0d6f7', 'Méthode', 'Chronic tubotympanic suppurative otitis media, bilateral', '2020-10-18T22:44:30Z'); +insert into users (id, name, description, modified_at) values ('1361dc101b576ff9fdfab28c75477bb2', 'Ráo', null, '2020-11-16T03:37:51Z'); +insert into users (id, name, description, modified_at) values ('404d0b5968bacb27d2c1a01b9b20d0ef', 'Naëlle', null, '2020-09-27T13:03:49Z'); +insert into users (id, name, description, modified_at) values ('e4568ef9106f041269150c0839f77c4a', 'Mélissandre', null, '2020-04-27T05:36:35Z'); +insert into users (id, name, description, modified_at) values ('b43eb61217ad64d5f78ba5fb211e3760', 'Torbjörn', null, '2020-08-29T04:04:36Z'); +insert into users (id, name, description, modified_at) values ('2cc9145f243fa7f071726e58f4412541', 'Gisèle', null, '2020-06-07T00:49:19Z'); +insert into users (id, name, description, modified_at) values ('b981807dd66957833ac9ae040e27d867', 'Gaëlle', 'Poisoning by immunoglobulin, accidental (unintentional), sequela', '2020-01-21T10:01:37Z'); +insert into users (id, name, description, modified_at) values ('385bca0230c05cd05e0e8738b754df0d', 'Pò', null, '2020-11-24T15:13:10Z'); +insert into users (id, name, description, modified_at) values ('3f44b88ee1415092284cc66458abd7a3', 'Noëlla', null, '2020-10-02T01:06:56Z'); +insert into users (id, name, description, modified_at) values ('f069d6ee6eddb5bbacc5aab992812ec3', 'Salomé', null, '2020-06-11T08:28:10Z'); +insert into users (id, name, description, modified_at) values ('72c47b731383f759715f59390f851521', 'Léane', null, '2020-04-11T12:24:09Z'); +insert into users (id, name, description, modified_at) values ('5ee481414aa941b8f4dbde5fbda2458b', 'Magdalène', 'Nondisplaced fracture of medial wall of right acetabulum, sequela', '2020-05-01T16:28:42Z'); +insert into users (id, name, description, modified_at) values ('9aadcd8f344f084fce11278cb9abfef0', 'Lauréna', null, '2020-11-29T22:37:02Z'); +insert into users (id, name, description, modified_at) values ('6730f3471f8d394ed7cfd1df82aab88f', 'Aurélie', null, '2020-07-21T18:59:30Z'); +insert into users (id, name, description, modified_at) values ('fb3aabb67dbc6e84bdacae097914479e', 'Léandre', null, '2020-07-10T12:58:03Z'); +insert into users (id, name, description, modified_at) values ('820b80e63c714fd5317c41b2067fad6b', 'Célestine', null, '2020-06-10T20:37:21Z'); +insert into users (id, name, description, modified_at) values ('5509d51d80fd216b733e4322fa590961', 'Mélina', 'Puncture wound without foreign body of right front wall of thorax with penetration into thoracic cavity, subsequent encounter', '2020-08-17T18:31:18Z'); +insert into users (id, name, description, modified_at) values ('ddd035770d0cb28954614dc18b98754b', 'Naéva', null, '2020-01-06T22:05:36Z'); +insert into users (id, name, description, modified_at) values ('6a809a4db3d7cd91be63ef5cc4dd6150', 'Cinéma', null, '2020-06-10T04:29:38Z'); +insert into users (id, name, description, modified_at) values ('be6a88f5e0e8a47403f521c335fac8d8', 'Kallisté', 'Other otitis externa', '2020-04-19T21:02:32Z'); +insert into users (id, name, description, modified_at) values ('7e20f7b6e00a03eee5ddadefaa11ba9d', 'Géraldine', null, '2020-06-27T20:13:37Z'); +insert into users (id, name, description, modified_at) values ('49592a9351f9bbe90510a378bd1d8d2e', 'Liè', 'Multiple cranial nerve palsies in sarcoidosis', '2020-03-18T01:38:27Z'); +insert into users (id, name, description, modified_at) values ('f241b792f64b99c170d3ea3e3344f907', 'Océane', null, '2020-04-22T06:42:12Z'); +insert into users (id, name, description, modified_at) values ('376ce28f7de5c7678fbc61d6b2fd07c8', 'Tán', null, '2020-02-09T23:27:19Z'); +insert into users (id, name, description, modified_at) values ('6bdf403f7b3143d0692197481a91d847', 'Chloé', null, '2020-09-22T04:42:52Z'); +insert into users (id, name, description, modified_at) values ('f0d64900a0032efadadd7c1dc00e8d45', 'Lóng', 'Burn of second degree of multiple left fingers (nail), including thumb, subsequent encounter', '2020-02-07T02:05:04Z'); +insert into users (id, name, description, modified_at) values ('b51691136e606e2dd8672c73ba7f1e04', 'Dafnée', null, '2020-02-10T23:32:54Z'); +insert into users (id, name, description, modified_at) values ('84d509e1a558b5426a6eac6dd3232c85', 'Cléa', null, '2020-07-14T09:20:19Z'); +insert into users (id, name, description, modified_at) values ('bee1d451d6103c6cfdf9a15aa4d3a408', 'Miléna', null, '2020-11-27T02:20:28Z'); +insert into users (id, name, description, modified_at) values ('248c9bfc3f55cfcd47788dd8802cc35e', 'Kuí', null, '2020-01-30T16:07:00Z'); +insert into users (id, name, description, modified_at) values ('7a40844fb9062dbafd1c6fe3abf18d8b', 'Crééz', 'Displaced supracondylar fracture with intracondylar extension of lower end of left femur, subsequent encounter for open fracture type I or II with nonunion', '2020-05-20T14:20:13Z'); +insert into users (id, name, description, modified_at) values ('dcf103a9fccfa74c77429f8ca72b0a25', 'Valérie', 'Unspecified nondisplaced fracture of sixth cervical vertebra, subsequent encounter for fracture with routine healing', '2020-09-14T22:24:28Z'); +insert into users (id, name, description, modified_at) values ('345e36eba718d40f79d17c8d42ec05b3', 'Cléopatre', 'Other injury of other flexor muscle, fascia and tendon at forearm level, right arm', '2020-04-10T04:48:21Z'); +insert into users (id, name, description, modified_at) values ('476f793bc605a318552896d964444ef2', 'Mélanie', null, '2020-06-24T14:03:53Z'); +insert into users (id, name, description, modified_at) values ('6c8509a4ac1dc475dd9a29cc95230cb4', 'Pò', null, '2020-02-21T06:12:58Z'); +insert into users (id, name, description, modified_at) values ('ebf8a37f7d6962ef678902632ebd5fd6', 'Méthode', null, '2020-02-11T04:15:17Z'); +insert into users (id, name, description, modified_at) values ('db732bb31e445bdb956c253b047eae8a', 'Athéna', null, '2020-10-19T16:15:06Z'); +insert into users (id, name, description, modified_at) values ('138b8fc4e5d9e1a9650f96d341a63c80', 'Cléopatre', 'Displaced articular fracture of head of right femur, subsequent encounter for open fracture type I or II with delayed healing', '2020-06-12T13:43:18Z'); +insert into users (id, name, description, modified_at) values ('1dc4476865fd0918bda1cd03e699f0f5', 'Adélaïde', null, '2020-07-17T16:12:44Z'); +insert into users (id, name, description, modified_at) values ('7d07012b8c8f6a42010688722e7536bc', 'Méghane', null, '2020-09-14T13:06:18Z'); +insert into users (id, name, description, modified_at) values ('2e2f81110e9ea10c828196e98caf5d05', 'Cléopatre', 'Other specified osteochondropathies, unspecified upper arm', '2020-03-14T17:26:58Z'); +insert into users (id, name, description, modified_at) values ('297f00640900f78e230370bf1f5dbece', 'Intéressant', null, '2020-07-01T06:02:32Z'); +insert into users (id, name, description, modified_at) values ('72f442303d8279513343fc535a8cbb59', 'Måns', null, '2020-04-02T22:31:21Z'); +insert into users (id, name, description, modified_at) values ('9683e1461af0367646ee9276c2317aaa', 'Mélina', 'Sudden visual loss, left eye', '2020-03-14T09:07:20Z'); +insert into users (id, name, description, modified_at) values ('350d36444cca4533ea65d570cb5b1949', 'Daphnée', null, '2020-08-12T17:31:42Z'); +insert into users (id, name, description, modified_at) values ('01f1e5a751e19be504ce15eb6190c7e3', 'Mylène', null, '2020-05-03T08:08:56Z'); +insert into users (id, name, description, modified_at) values ('64df8a9bb92e63984d7b455836a3ad1b', 'Intéressant', null, '2020-09-14T10:43:45Z'); +insert into users (id, name, description, modified_at) values ('2729d5f8422ca7e5f31d737ca7dcd54d', 'Mylène', null, '2020-08-06T12:14:01Z'); +insert into users (id, name, description, modified_at) values ('03a8f9a48ceba5110cc1a550df3d3bf0', 'Dù', null, '2020-06-20T19:53:31Z'); +insert into users (id, name, description, modified_at) values ('a38308dd99cb6e63dc7b05c6d1361a1d', 'Erwéi', 'Other contact with other fish, initial encounter', '2020-11-18T08:46:40Z'); +insert into users (id, name, description, modified_at) values ('4d2c30aa127db23ed05e567124cdccf5', 'Marlène', 'Discharge of firework, subsequent encounter', '2020-06-25T10:45:44Z'); +insert into users (id, name, description, modified_at) values ('20a5919dba781e1f5e23fe685c794649', 'Desirée', null, '2020-05-18T08:51:21Z'); +insert into users (id, name, description, modified_at) values ('8cc897337a224749f58067cc4d4f7f11', 'Rébecca', null, '2020-09-10T19:46:54Z'); +insert into users (id, name, description, modified_at) values ('e7c5f9536c04d6c89b95a2e14e1711c9', 'Garçon', null, '2020-11-21T12:19:38Z'); +insert into users (id, name, description, modified_at) values ('e5e72c60274fccb36151fd733938f8fa', 'Laïla', null, '2020-07-26T18:21:35Z'); +insert into users (id, name, description, modified_at) values ('5696d5c46976be1d9840906c03f93582', 'Gérald', 'Other spotted fevers', '2020-05-03T10:12:02Z'); +insert into users (id, name, description, modified_at) values ('9e55d72fa79013274ea77fe08315cb32', 'Bénédicte', null, '2020-04-24T18:36:17Z'); +insert into users (id, name, description, modified_at) values ('cfa6370645f61c77d08b04458da93807', 'Tán', 'Burn of right eyelid and periocular area, initial encounter', '2020-10-30T18:15:22Z'); +insert into users (id, name, description, modified_at) values ('f25c8d49c807797bde4f3ff0f056ebd3', 'Naëlle', null, '2020-10-18T09:33:28Z'); +insert into users (id, name, description, modified_at) values ('b854188a9d2556fc67fddcd284fb4ee8', 'Valérie', null, '2020-05-30T13:35:43Z'); +insert into users (id, name, description, modified_at) values ('ca6fbb0e9d7140edb73d5905b4e33230', 'Jú', null, '2020-04-15T20:03:35Z'); +insert into users (id, name, description, modified_at) values ('5382d16101dd7891649c59da61b0ba75', 'Géraldine', 'Other unilateral secondary osteoarthritis of first carpometacarpal joint, unspecified hand', '2020-09-01T17:43:20Z'); +insert into users (id, name, description, modified_at) values ('993c18d8e7c228384c6dab032fb51a12', 'Jú', null, '2020-05-09T12:45:29Z'); +insert into users (id, name, description, modified_at) values ('157a79e0942e3bbe7dd77920b98ba5c9', 'Adèle', null, '2020-01-30T07:20:24Z'); +insert into users (id, name, description, modified_at) values ('82a5b8edecd61f187139c0fdf69856ee', 'Desirée', null, '2020-01-04T16:25:59Z'); +insert into users (id, name, description, modified_at) values ('e5278153e2512f9f5b53c774c5c5ec95', 'Léa', null, '2020-02-27T15:04:36Z'); +insert into users (id, name, description, modified_at) values ('61f3db039db0d17dd0fc85b82f2e11a8', 'Athéna', 'Acute perichondritis of left external ear', '2020-03-25T09:08:02Z'); +insert into users (id, name, description, modified_at) values ('7e81fe0c5e4f11496a960ba6809052e8', 'Esbjörn', null, '2020-04-27T01:37:26Z'); +insert into users (id, name, description, modified_at) values ('5e60cfdcc7806d3596a4a963b723fd0d', 'Cloé', null, '2020-07-16T13:33:37Z'); +insert into users (id, name, description, modified_at) values ('dede3dac4fb41ae96d6da733d2dc1fcb', 'Björn', null, '2020-03-06T06:24:24Z'); +insert into users (id, name, description, modified_at) values ('98a4c724d1a88d5daa7ef936e9882dd5', 'Publicité', 'Contact with and (suspected) exposure to soil pollution', '2020-01-21T12:01:15Z'); +insert into users (id, name, description, modified_at) values ('a44b83f1cf4a368461767f990e122312', 'Mélinda', null, '2020-01-04T19:40:02Z'); +insert into users (id, name, description, modified_at) values ('492576af4de338e82c47646700e3bbc0', 'Håkan', null, '2020-08-20T13:09:51Z'); +insert into users (id, name, description, modified_at) values ('5bd8a79917928c46cd7ffbc4c118003e', 'Naëlle', null, '2020-04-08T13:00:58Z'); +insert into users (id, name, description, modified_at) values ('044eea4ac60213c667b2dcabdfc0086d', 'Dafnée', null, '2020-09-01T15:39:02Z'); +insert into users (id, name, description, modified_at) values ('efb35642c93887d5d90d8e8d5919f324', 'Yú', 'Salter-Harris Type I physeal fracture of lower end of ulna', '2020-04-14T20:26:14Z'); +insert into users (id, name, description, modified_at) values ('7410d9e2a48b96f302fcfd628ca25b6a', 'Yáo', null, '2020-01-11T20:59:53Z'); +insert into users (id, name, description, modified_at) values ('39969ca1705f330c82a025012b56121d', 'Léana', null, '2020-04-17T22:22:30Z'); +insert into users (id, name, description, modified_at) values ('60b788ad169b07d80e71b98523c0fd25', 'Méryl', 'Therapeutic (nonsurgical) and rehabilitative radiological devices associated with adverse incidents', '2020-09-06T20:15:02Z'); +insert into users (id, name, description, modified_at) values ('fe2d505ee035081fea6b813954308fc6', 'Cécile', null, '2020-08-30T10:40:23Z'); +insert into users (id, name, description, modified_at) values ('18a70aa46a57d9356ecf92d4db406d93', 'Hélèna', null, '2020-07-29T13:30:10Z'); +insert into users (id, name, description, modified_at) values ('70ef04fcb5aaba9ba6734750f5c8fea8', 'Ráo', 'Unspecified benign mammary dysplasia of right breast', '2020-09-26T17:10:22Z'); +insert into users (id, name, description, modified_at) values ('39c0567b7e9b4d21ee4e9f6dbee07c9c', 'Solène', 'Other specified injury of extensor muscle, fascia and tendon of left thumb at wrist and hand level, sequela', '2020-03-22T13:55:31Z'); +insert into users (id, name, description, modified_at) values ('62317289942823b5dc24163330dae495', 'Maï', 'Displacement of unspecified cardiac and vascular devices and implants, subsequent encounter', '2020-07-28T01:59:39Z'); +insert into users (id, name, description, modified_at) values ('3b3be0806e7c475d5ab7209cb8fdb947', 'Aurélie', null, '2020-03-02T22:11:43Z'); +insert into users (id, name, description, modified_at) values ('82ce4a297490a6cbeba93c87b392220c', 'Yú', null, '2020-01-10T17:46:37Z'); +insert into users (id, name, description, modified_at) values ('19920099e4c390053269bf30f51c96b1', 'Cécile', null, '2020-05-21T16:59:57Z'); +insert into users (id, name, description, modified_at) values ('3f1ab38f13506a7b54a559d3097640b4', 'Laurélie', null, '2020-03-07T00:35:44Z'); +insert into users (id, name, description, modified_at) values ('24fd8e43aad01f56e525ee9f748e3689', 'Marie-josée', null, '2020-11-23T11:11:06Z'); +insert into users (id, name, description, modified_at) values ('427e0b347c15cb3fff85785044e059cc', 'Zhì', 'Unspecified occupant of bus injured in collision with fixed or stationary object in nontraffic accident, initial encounter', '2020-01-01T04:25:52Z'); +insert into users (id, name, description, modified_at) values ('8b52530fc9f868ab76adac31948db6fc', 'Maëlann', null, '2020-07-16T05:34:40Z'); +insert into users (id, name, description, modified_at) values ('2e0603e4663f96aac79b1caa4ebe32b1', 'Bérengère', null, '2020-05-14T02:03:15Z'); +insert into users (id, name, description, modified_at) values ('fd356255b170b48ba27d8252781260a0', 'Clélia', null, '2020-04-01T20:10:25Z'); +insert into users (id, name, description, modified_at) values ('a0436b706204ddafc40716d4407080c7', 'Håkan', 'Gestational diabetes mellitus in the puerperium, diet controlled', '2020-05-04T23:01:02Z'); +insert into users (id, name, description, modified_at) values ('30e614fa9d739ea3268b222de5a4d52b', 'Magdalène', null, '2020-04-17T14:51:06Z'); +insert into users (id, name, description, modified_at) values ('93b315b9307989a29af9eabf179d0944', 'Clélia', null, '2020-07-25T14:14:42Z'); +insert into users (id, name, description, modified_at) values ('4de8d336ca622b50f4f04460c5a8ff6e', 'Görel', null, '2020-01-11T23:30:44Z'); +insert into users (id, name, description, modified_at) values ('c43c40b7d54d5390a211d10ffc52c98f', 'Renée', null, '2020-11-02T20:51:58Z'); diff --git a/local/docker-compose.yml b/local/docker-compose.yml new file mode 100644 index 00000000..362b2117 --- /dev/null +++ b/local/docker-compose.yml @@ -0,0 +1,157 @@ +# Use postgres/example user/password credentials +version: '3.1' + +services: + + db: + image: postgres + restart: always + environment: + POSTGRES_PASSWORD: example + volumes: + - ./data:/docker-entrypoint-initdb.d + ports: + - 5432:5432 + + adminer: + image: adminer + restart: always + ports: + - 8080:8080 + + zookeeper: + container_name: local-zookeeper + image: confluentinc/cp-zookeeper:6.0.0 + ports: + - 2181:2181 + hostname: zookeeper + networks: + - local_kafka_network + environment: + - ZOOKEEPER_CLIENT_PORT=2181 + + kafka: + container_name: local-kafka + image: confluentinc/cp-kafka:6.0.0 + depends_on: + - zookeeper + ports: + - 9092:9092 + - 9101:9101 + - 29092:29092 + hostname: kafka + networks: + - local_kafka_network + environment: + - KAFKA_ZOOKEEPER_CONNECT=zookeeper:2181 + - KAFKA_LISTENER_SECURITY_PROTOCOL_MAP=PLAINTEXT:PLAINTEXT,PLAINTEXT_HOST:PLAINTEXT + - KAFKA_ADVERTISED_LISTENERS=PLAINTEXT://kafka:29092,PLAINTEXT_HOST://localhost:9092 + - KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR=1 + - KAFKA_JMX_PORT=9101 + + kafka-rest: + container_name: local-kafka-rest + image: confluentinc/cp-kafka-rest:6.0.0 + depends_on: + - kafka + - schema-registry + ports: + - 8082:8082 + hostname: kafka-rest + networks: + - local_kafka_network + environment: + - KAFKA_REST_HOST_NAME=kafka-rest + - KAFKA_REST_BOOTSTRAP_SERVERS=kafka:29092 + - KAFKA_REST_LISTENERS=http://kafka-rest:8082 + - KAFKA_REST_SCHEMA_REGISTRY_URL=http://schema-registry:8081 + + kafka-ui: + container_name: local-kafka-ui + image: landoop/kafka-topics-ui:latest + depends_on: + - kafka-rest + ports: + - 8000:8000 + hostname: kafka-ui + networks: + - local_kafka_network + environment: + - KAFKA_REST_PROXY_URL=http://kafka-rest:8082 + - PROXY=true + + schema-registry: + container_name: local-schema-registry + image: confluentinc/cp-schema-registry:6.0.0 + depends_on: + - kafka + ports: + - 8081:8081 + hostname: schema-registry + networks: + - local_kafka_network + environment: + - SCHEMA_REGISTRY_HOST_NAME=schema-registry + - SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS=kafka:29092 + volumes: + - ./data/:/data + - ./schema/:/schema + command: + - /bin/bash + - -c + - | + # install httpie + pip3 install --user httpie + # install jq + curl -sL https://github.com/stedolan/jq/releases/download/jq-1.6/jq-linux64 -o .local/bin/jq && chmod u+x .local/bin/jq + # start + /etc/confluent/docker/run + + schema-registry-ui: + container_name: local-schema-registry-ui + image: landoop/schema-registry-ui:latest + depends_on: + - schema-registry + ports: + - 8001:8000 + hostname: schema-registry-ui + networks: + - local_kafka_network + environment: + - SCHEMAREGISTRY_URL=http://schema-registry:8081 + - PROXY=true + +# # TODO replace kafka-rest+kafka-ui and schema-registry-ui +# kowl: +# container_name: local-kowl +# image: quay.io/cloudhut/kowl:v1.2.1 +# depends_on: +# - kafka +# - schema-registry +# # waits for connections to be available +# restart: on-failure +# ports: +# - 8002:8080 +# hostname: kowl +# networks: +# - local_kafka_network +# entrypoint: +# - /bin/ash +# - -c +# - | +# mkdir -p /etc/kowl +# cat > /etc/kowl/config.yaml < Date: Tue, 29 Dec 2020 23:11:00 +0000 Subject: [PATCH 04/15] feature parity --- doobie/src/main/scala/tamer/db/Db.scala | 5 +- doobie/src/main/scala/tamer/db/Setup.scala | 16 ++++-- doobie/src/main/scala/tamer/db/package.scala | 48 ++++++++++++------ .../src/main/scala/tamer/example/Main.scala | 49 ++++++++++++++++--- 4 files changed, 89 insertions(+), 29 deletions(-) diff --git a/doobie/src/main/scala/tamer/db/Db.scala b/doobie/src/main/scala/tamer/db/Db.scala index 14359937..b2d3b21d 100644 --- a/doobie/src/main/scala/tamer/db/Db.scala +++ b/doobie/src/main/scala/tamer/db/Db.scala @@ -4,6 +4,7 @@ package db import fs2.Chunk import java.time.Instant +import scala.util.hashing.byteswap64 object Db { abstract class Datable(val instant: Instant) @@ -11,7 +12,7 @@ object Db { case class ChunkWithMetadata[V](chunk: Chunk[V], pulledAt: Instant = Instant.now()) case class ValueWithMetadata[V](value: V, pulledAt: Instant = Instant.now()) - case class TimeSegment(from: Instant, to: Instant) extends State { - override lazy val stateId: Int = this.hashCode() + case class TimeSegment(from: Instant, to: Instant) extends HashableState { + override lazy val stateHash: Int = (byteswap64(from.getEpochSecond) + byteswap64(to.getEpochSecond)).intValue } } diff --git a/doobie/src/main/scala/tamer/db/Setup.scala b/doobie/src/main/scala/tamer/db/Setup.scala index 2b95af56..739f8f2f 100644 --- a/doobie/src/main/scala/tamer/db/Setup.scala +++ b/doobie/src/main/scala/tamer/db/Setup.scala @@ -13,11 +13,17 @@ trait QueryBuilder[V, -S] { def query(state: S): Query0[V] } -trait State { +trait HashableState { + // TODO: Evaluate if this is less invasive as a typeclass, the main cons + // TODO: is loss of expressivity, and since state is probably manually + // TODO: provided by the user (as opposed to automatically generated + // TODO: code) it should be easy to implement this. - /** Used for hashing purposes1 + /** It is required for this hash to be consistent even across executions + * for the same semantic state. This is in contrast with the built-in + * `hashCode` method. */ - val stateId: Int + val stateHash: Int } final case class ResultMetadata(queryExecutionTime: Long) @@ -26,7 +32,7 @@ final case class QueryResult[V](metadata: ResultMetadata, results: List[V]) case class Setup[ K <: Product: Encoder: Decoder: SchemaFor, V <: Product: Encoder: Decoder: SchemaFor, - S <: Product with State: Encoder: Decoder: SchemaFor + S <: Product with HashableState: Encoder: Decoder: SchemaFor ]( queryBuilder: QueryBuilder[V, S], override val defaultState: S, @@ -37,5 +43,5 @@ case class Setup[ Serde[V]().serializer, Serde[S]().serde, defaultState, - queryBuilder.queryId + defaultState.stateId + queryBuilder.queryId + defaultState.stateHash ) diff --git a/doobie/src/main/scala/tamer/db/package.scala b/doobie/src/main/scala/tamer/db/package.scala index b4058583..3dca6dbd 100644 --- a/doobie/src/main/scala/tamer/db/package.scala +++ b/doobie/src/main/scala/tamer/db/package.scala @@ -26,7 +26,7 @@ package object db { type DbTransactor = Has[Transactor[Task]] type TamerDBConfig = DbTransactor with QueryConfig - private[this] implicit final class InstantOps(private val instant: Instant) extends AnyVal { + implicit final class InstantOps(private val instant: Instant) extends AnyVal { def orNow: UIO[Instant] = UIO(Instant.now).map { case now if instant.isAfter(now) => now @@ -34,17 +34,12 @@ package object db { } } - final def mkSetup[K <: Product: Encoder: Decoder: SchemaFor, V <: Product with Datable: Ordering: Encoder: Decoder: SchemaFor]( + final def mkSetupWithTimeSegment[K <: Product: Encoder: Decoder: SchemaFor, V <: Product with Datable: Ordering: Encoder: Decoder: SchemaFor]( queryBuilder: TimeSegment => Query0[V] )(earliest: Instant, tumblingStep: Duration, keyExtract: V => K): Setup[K, V, TimeSegment] = { val timeSegment = TimeSegment(earliest, earliest.plus(tumblingStep)) - val qBuilder = new QueryBuilder[V, TimeSegment] { - override val queryId: Int = queryBuilder(timeSegment).sql.hashCode - override def query(state: TimeSegment): Query0[V] = queryBuilder(state) - } - def stateFold(timeSegment: TimeSegment)(queryResult: QueryResult[V]): UIO[TimeSegment] = if (queryResult.results.isEmpty) timeSegment.to.plus(tumblingStep).orNow.map(TimeSegment(timeSegment.from, _)) else { @@ -52,20 +47,34 @@ package object db { mostRecent.plus(tumblingStep).orNow.map(TimeSegment(mostRecent, _)) } - Setup[K, V, TimeSegment](qBuilder, timeSegment, keyExtract, stateFold) + mkSetup(queryBuilder)(timeSegment, keyExtract, stateFold) + } + + final def mkSetup[ + K <: Product: Encoder: Decoder: SchemaFor, + V <: Product: Encoder: Decoder: SchemaFor, + S <: Product with HashableState: Encoder: Decoder: SchemaFor + ]( + queryBuilder: S => Query0[V] + )(defaultState: S, keyExtract: V => K, stateFoldM: S => QueryResult[V] => UIO[S]): Setup[K, V, S] = { + val qBuilder = new QueryBuilder[V, S] { + override val queryId: Int = queryBuilder(defaultState).sql.hashCode + override def query(state: S): Query0[V] = queryBuilder(state) + } + Setup[K, V, S](queryBuilder = qBuilder, defaultState = defaultState, keyExtract = keyExtract, stateFoldM = stateFoldM) } private[this] val logTask: Task[LogWriter[Task]] = log4sFromName.provide("tamer.db") - final def iteration[K <: Product, V <: Product]( - setup: Setup[K, V, TimeSegment] - )(state: TimeSegment, q: Queue[(K, V)]): ZIO[TamerDBConfig, TamerError, TimeSegment] = + final def iteration[K <: Product, V <: Product, S <: Product with HashableState]( + setup: Setup[K, V, S] + )(state: S, q: Queue[(K, V)]): ZIO[TamerDBConfig, TamerError, S] = (for { log <- logTask cfg <- ConfigDb.queryConfig tnx <- ZIO.service[Transactor[Task]] query <- UIO(setup.queryBuilder.query(state)) - _ <- log.info(s"running ${query.sql} with params derived from $state") // TODO: make this debug + _ <- log.debug(s"running ${query.sql} with params derived from $state") start <- UIO(Instant.now()) values <- query @@ -88,8 +97,19 @@ package object db { ) } yield newState).mapError(e => TamerError(e.getLocalizedMessage, e)) - final def fetchWithTimeSegment[K <: Product, V <: Product]( - setup: Setup[K, V, TimeSegment] + final def fetchWithTimeSegment[K <: Product: Encoder: Decoder: SchemaFor, V <: Product with Datable: Ordering: Encoder: Decoder: SchemaFor]( + queryBuilder: TimeSegment => Query0[V] + )(earliest: Instant, tumblingStep: Duration, keyExtract: V => K): ZIO[Kafka with TamerDBConfig with ZEnv, TamerError, Unit] = { + val setup = mkSetupWithTimeSegment[K, V](queryBuilder)(earliest, tumblingStep, keyExtract) + fetch(setup) + } + + final def fetch[ + K <: Product: Encoder: Decoder: SchemaFor, + V <: Product: Encoder: Decoder: SchemaFor, + S <: Product with HashableState: Encoder: Decoder: SchemaFor + ]( + setup: Setup[K, V, S] ): ZIO[Kafka with TamerDBConfig with ZEnv, TamerError, Unit] = tamer.kafka.runLoop(setup)(iteration(setup)) diff --git a/example/src/main/scala/tamer/example/Main.scala b/example/src/main/scala/tamer/example/Main.scala index 6c29ac0c..e1dee928 100644 --- a/example/src/main/scala/tamer/example/Main.scala +++ b/example/src/main/scala/tamer/example/Main.scala @@ -3,19 +3,27 @@ package example import tamer.config.Config import tamer.db.Db.Datable -import tamer.db.{ConfigDb, DbTransactor, TamerDBConfig} +import tamer.db.{ConfigDb, DbTransactor, HashableState, InstantOps, QueryResult, TamerDBConfig} import tamer.kafka.Kafka import zio.blocking.Blocking import zio._ import doobie.implicits.legacy.instant._ import doobie.syntax.string._ -import log.effect.LogWriter import log.effect.zio.ZioLogWriter.log4sFromName import tamer.db.ConfigDb.{DbConfig, QueryConfig} import java.time.temporal.ChronoUnit._ import java.time.{Duration, Instant} +import scala.util.hashing.byteswap64 +final case class MyState(from: Instant, to: Instant) extends HashableState { + + /** It is required for this hash to be consistent even across executions + * for the same semantic state. This is in contrast with the built-in + * `hashCode` method. + */ + override val stateHash: Int = (byteswap64(from.getEpochSecond) + byteswap64(to.getEpochSecond)).intValue +} final case class Key(id: String) final case class Value(id: String, name: String, description: Option[String], modifiedAt: Instant) extends Datable(modifiedAt) object Value { @@ -27,21 +35,46 @@ object Main extends zio.App { val kafkaLayer: Layer[TamerError, Kafka] = Config.live >>> Kafka.live val queryConfigLayer: Layer[TamerError, DbConfig with QueryConfig] = ConfigDb.live val layer: Layer[TamerError, DbTransactor with Kafka with QueryConfig] = transactorLayer ++ kafkaLayer ++ queryConfigLayer - def keyExtract(value: Value): Key = Key(value.id) - private[this] val logTask: Task[LogWriter[Task]] = log4sFromName.provide("tamer.example") val program: ZIO[Kafka with TamerDBConfig with ZEnv, TamerError, Unit] = (for { - log <- logTask + log <- log4sFromName.provide("tamer.example") _ <- log.info("Starting tamer...") boot <- UIO(Instant.now()) - setup = tamer.db.mkSetup(ts => + _ <- tamer.db.fetchWithTimeSegment(ts => sql"""SELECT id, name, description, modified_at FROM users WHERE modified_at > ${ts.from} AND modified_at <= ${ts.to}""".query[Value] )( earliest = boot.minus(60, DAYS), tumblingStep = Duration.of(5, MINUTES), - keyExtract = keyExtract + keyExtract = (value: Value) => Key(value.id) + ) + } yield ()).mapError(e => TamerError("Could not run tamer example", e)) + + override final def run(args: List[String]): URIO[ZEnv, ExitCode] = program.provideCustomLayer(layer).exitCode +} + +object Main2 extends zio.App { + val transactorLayer: Layer[TamerError, DbTransactor] = (Blocking.live ++ ConfigDb.live) >>> db.hikariLayer + val kafkaLayer: Layer[TamerError, Kafka] = Config.live >>> Kafka.live + val queryConfigLayer: Layer[TamerError, DbConfig with QueryConfig] = ConfigDb.live + val layer: Layer[TamerError, DbTransactor with Kafka with QueryConfig] = transactorLayer ++ kafkaLayer ++ queryConfigLayer + val program: ZIO[Kafka with TamerDBConfig with ZEnv, TamerError, Unit] = (for { + log <- log4sFromName.provide("tamer.example") + _ <- log.info("Starting tamer...") + boot <- UIO(Instant.now()) + earliest = boot.minus(60, DAYS) + setup = tamer.db.mkSetup((s: MyState) => + sql"""SELECT id, name, description, modified_at FROM users WHERE modified_at > ${s.from} AND modified_at <= ${s.to}""".query[Value] + )( + defaultState = MyState(earliest, earliest.plus(5, MINUTES)), + keyExtract = (value: Value) => Key(value.id), + stateFoldM = (s: MyState) => { + case QueryResult(_, results) if results.isEmpty => s.to.plus(5, MINUTES).orNow.map(MyState(s.from, _)) + case QueryResult(_, results) => + val mostRecent = results.sortBy(_.modifiedAt).max.instant + mostRecent.plus(5, MINUTES).orNow.map(MyState(mostRecent, _)) + } ) _ <- log.info(s"Tamer initialized with setup $setup") - _ <- tamer.db.fetchWithTimeSegment(setup) + _ <- tamer.db.fetch(setup) } yield ()).mapError(e => TamerError("Could not run tamer example", e)) override final def run(args: List[String]): URIO[ZEnv, ExitCode] = program.provideCustomLayer(layer).exitCode From 71bf64a5f25045ddfb518387397fab3becc04e03 Mon Sep 17 00:00:00 2001 From: Andrea Passaglia Date: Tue, 29 Dec 2020 23:13:27 +0000 Subject: [PATCH 05/15] address review --- core/src/main/scala/tamer/kafka/Kafka.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/tamer/kafka/Kafka.scala b/core/src/main/scala/tamer/kafka/Kafka.scala index f0e3cab2..733ee9a8 100644 --- a/core/src/main/scala/tamer/kafka/Kafka.scala +++ b/core/src/main/scala/tamer/kafka/Kafka.scala @@ -18,7 +18,7 @@ import zio.kafka.consumer.{CommittableRecord, Consumer, ConsumerSettings, Subscr import zio.kafka.producer.{Producer, ProducerSettings} import zio.stream.ZStream -final case class StateKey(queryHash: String, groupId: String) +final case class StateKey(stateKey: String, groupId: String) object Kafka { trait Service { From 20952535b8d4539cb605b308fa03167977f8843f Mon Sep 17 00:00:00 2001 From: Andrea Passaglia Date: Wed, 30 Dec 2020 00:58:00 +0000 Subject: [PATCH 06/15] just shove the layers under the carpet --- doobie/src/main/scala/tamer/db/package.scala | 12 ++++++++--- .../src/main/scala/tamer/example/Main.scala | 20 ++++++++----------- 2 files changed, 17 insertions(+), 15 deletions(-) diff --git a/doobie/src/main/scala/tamer/db/package.scala b/doobie/src/main/scala/tamer/db/package.scala index 3dca6dbd..d1c2c021 100644 --- a/doobie/src/main/scala/tamer/db/package.scala +++ b/doobie/src/main/scala/tamer/db/package.scala @@ -10,6 +10,7 @@ import eu.timepit.refined.auto._ import fs2.Stream import log.effect.LogWriter import log.effect.zio.ZioLogWriter.log4sFromName +import tamer.config.Config import tamer.db.Compat.toIterable import tamer.db.ConfigDb.{DbConfig, QueryConfig} import tamer.db.Db.{Datable, TimeSegment, _} @@ -99,7 +100,7 @@ package object db { final def fetchWithTimeSegment[K <: Product: Encoder: Decoder: SchemaFor, V <: Product with Datable: Ordering: Encoder: Decoder: SchemaFor]( queryBuilder: TimeSegment => Query0[V] - )(earliest: Instant, tumblingStep: Duration, keyExtract: V => K): ZIO[Kafka with TamerDBConfig with ZEnv, TamerError, Unit] = { + )(earliest: Instant, tumblingStep: Duration, keyExtract: V => K): ZIO[ZEnv, TamerError, Unit] = { val setup = mkSetupWithTimeSegment[K, V](queryBuilder)(earliest, tumblingStep, keyExtract) fetch(setup) } @@ -110,8 +111,13 @@ package object db { S <: Product with HashableState: Encoder: Decoder: SchemaFor ]( setup: Setup[K, V, S] - ): ZIO[Kafka with TamerDBConfig with ZEnv, TamerError, Unit] = - tamer.kafka.runLoop(setup)(iteration(setup)) + ): ZIO[ZEnv, TamerError, Unit] = { + val kafkaLayer: Layer[TamerError, Kafka] = Config.live >>> Kafka.live + val transactorLayer: Layer[TamerError, DbTransactor] = (Blocking.live ++ ConfigDb.live) >>> db.hikariLayer + val queryConfigLayer: Layer[TamerError, DbConfig with QueryConfig] = ConfigDb.live + val defaultLayer: Layer[TamerError, DbTransactor with Kafka with QueryConfig] = transactorLayer ++ queryConfigLayer ++ kafkaLayer + tamer.kafka.runLoop(setup)(iteration(setup)).provideCustomLayer(defaultLayer) + } val hikariLayer: ZLayer[Blocking with DbConfig, TamerError, DbTransactor] = ZLayer.fromManaged { for { diff --git a/example/src/main/scala/tamer/example/Main.scala b/example/src/main/scala/tamer/example/Main.scala index e1dee928..bf9adadc 100644 --- a/example/src/main/scala/tamer/example/Main.scala +++ b/example/src/main/scala/tamer/example/Main.scala @@ -31,11 +31,7 @@ object Value { } object Main extends zio.App { - val transactorLayer: Layer[TamerError, DbTransactor] = (Blocking.live ++ ConfigDb.live) >>> db.hikariLayer - val kafkaLayer: Layer[TamerError, Kafka] = Config.live >>> Kafka.live - val queryConfigLayer: Layer[TamerError, DbConfig with QueryConfig] = ConfigDb.live - val layer: Layer[TamerError, DbTransactor with Kafka with QueryConfig] = transactorLayer ++ kafkaLayer ++ queryConfigLayer - val program: ZIO[Kafka with TamerDBConfig with ZEnv, TamerError, Unit] = (for { + val program: ZIO[ZEnv, TamerError, Unit] = (for { log <- log4sFromName.provide("tamer.example") _ <- log.info("Starting tamer...") boot <- UIO(Instant.now()) @@ -48,14 +44,14 @@ object Main extends zio.App { ) } yield ()).mapError(e => TamerError("Could not run tamer example", e)) - override final def run(args: List[String]): URIO[ZEnv, ExitCode] = program.provideCustomLayer(layer).exitCode + override final def run(args: List[String]): URIO[ZEnv, ExitCode] = program.exitCode } -object Main2 extends zio.App { - val transactorLayer: Layer[TamerError, DbTransactor] = (Blocking.live ++ ConfigDb.live) >>> db.hikariLayer - val kafkaLayer: Layer[TamerError, Kafka] = Config.live >>> Kafka.live - val queryConfigLayer: Layer[TamerError, DbConfig with QueryConfig] = ConfigDb.live - val layer: Layer[TamerError, DbTransactor with Kafka with QueryConfig] = transactorLayer ++ kafkaLayer ++ queryConfigLayer +object MainGeneralized extends zio.App { + val transactorLayer: Layer[TamerError, DbTransactor] = (Blocking.live ++ ConfigDb.live) >>> db.hikariLayer + val kafkaLayer: Layer[TamerError, Kafka] = Config.live >>> Kafka.live + val queryConfigLayer: Layer[TamerError, DbConfig with QueryConfig] = ConfigDb.live + val myLayer: Layer[TamerError, DbTransactor with Kafka with QueryConfig] = transactorLayer ++ kafkaLayer ++ queryConfigLayer val program: ZIO[Kafka with TamerDBConfig with ZEnv, TamerError, Unit] = (for { log <- log4sFromName.provide("tamer.example") _ <- log.info("Starting tamer...") @@ -77,5 +73,5 @@ object Main2 extends zio.App { _ <- tamer.db.fetch(setup) } yield ()).mapError(e => TamerError("Could not run tamer example", e)) - override final def run(args: List[String]): URIO[ZEnv, ExitCode] = program.provideCustomLayer(layer).exitCode + override final def run(args: List[String]): URIO[ZEnv, ExitCode] = program.provideCustomLayer(myLayer).exitCode } From c6e4d6d91d93649655742b83e1bd2fee8a3ae14d Mon Sep 17 00:00:00 2001 From: Andrea Passaglia Date: Wed, 30 Dec 2020 00:59:02 +0000 Subject: [PATCH 07/15] reorder main --- example/src/main/scala/tamer/example/Main.scala | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/example/src/main/scala/tamer/example/Main.scala b/example/src/main/scala/tamer/example/Main.scala index bf9adadc..bf07b378 100644 --- a/example/src/main/scala/tamer/example/Main.scala +++ b/example/src/main/scala/tamer/example/Main.scala @@ -16,14 +16,6 @@ import java.time.temporal.ChronoUnit._ import java.time.{Duration, Instant} import scala.util.hashing.byteswap64 -final case class MyState(from: Instant, to: Instant) extends HashableState { - - /** It is required for this hash to be consistent even across executions - * for the same semantic state. This is in contrast with the built-in - * `hashCode` method. - */ - override val stateHash: Int = (byteswap64(from.getEpochSecond) + byteswap64(to.getEpochSecond)).intValue -} final case class Key(id: String) final case class Value(id: String, name: String, description: Option[String], modifiedAt: Instant) extends Datable(modifiedAt) object Value { @@ -47,6 +39,15 @@ object Main extends zio.App { override final def run(args: List[String]): URIO[ZEnv, ExitCode] = program.exitCode } +final case class MyState(from: Instant, to: Instant) extends HashableState { + + /** It is required for this hash to be consistent even across executions + * for the same semantic state. This is in contrast with the built-in + * `hashCode` method. + */ + override val stateHash: Int = (byteswap64(from.getEpochSecond) + byteswap64(to.getEpochSecond)).intValue +} + object MainGeneralized extends zio.App { val transactorLayer: Layer[TamerError, DbTransactor] = (Blocking.live ++ ConfigDb.live) >>> db.hikariLayer val kafkaLayer: Layer[TamerError, Kafka] = Config.live >>> Kafka.live From d3e9874ad12e0ceaa4f9f0daeb5e1ba77bd96c1e Mon Sep 17 00:00:00 2001 From: Andrea Passaglia Date: Wed, 30 Dec 2020 01:13:26 +0000 Subject: [PATCH 08/15] Datables don't need the user to define ordering anymore --- doobie/src/main/scala/tamer/db/Db.scala | 4 ++++ example/src/main/scala/tamer/example/Main.scala | 3 --- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/doobie/src/main/scala/tamer/db/Db.scala b/doobie/src/main/scala/tamer/db/Db.scala index b2d3b21d..f89fbc07 100644 --- a/doobie/src/main/scala/tamer/db/Db.scala +++ b/doobie/src/main/scala/tamer/db/Db.scala @@ -8,6 +8,10 @@ import scala.util.hashing.byteswap64 object Db { abstract class Datable(val instant: Instant) + object Datable { + val underlyingOrdering: Ordering[Instant] = implicitly[Ordering[Instant]] + implicit def ordering[Subtype <: Datable]: Ordering[Subtype] = (x: Datable, y: Datable) => underlyingOrdering.compare(x.instant, y.instant) + } case class ChunkWithMetadata[V](chunk: Chunk[V], pulledAt: Instant = Instant.now()) case class ValueWithMetadata[V](value: V, pulledAt: Instant = Instant.now()) diff --git a/example/src/main/scala/tamer/example/Main.scala b/example/src/main/scala/tamer/example/Main.scala index bf07b378..695293f5 100644 --- a/example/src/main/scala/tamer/example/Main.scala +++ b/example/src/main/scala/tamer/example/Main.scala @@ -18,9 +18,6 @@ import scala.util.hashing.byteswap64 final case class Key(id: String) final case class Value(id: String, name: String, description: Option[String], modifiedAt: Instant) extends Datable(modifiedAt) -object Value { - implicit val ordering: Ordering[Value] = (x: Value, y: Value) => x.modifiedAt.compareTo(y.modifiedAt) -} object Main extends zio.App { val program: ZIO[ZEnv, TamerError, Unit] = (for { From e8f6b917cb967a7119a91f1e3734f0b08ee7ad49 Mon Sep 17 00:00:00 2001 From: Andrea Passaglia Date: Wed, 30 Dec 2020 01:17:45 +0000 Subject: [PATCH 09/15] dateable => timestamped and some scaladoc --- doobie/src/main/scala/tamer/db/Db.scala | 14 ++++++++++---- doobie/src/main/scala/tamer/db/package.scala | 8 ++++---- example/src/main/scala/tamer/example/Main.scala | 6 +++--- 3 files changed, 17 insertions(+), 11 deletions(-) diff --git a/doobie/src/main/scala/tamer/db/Db.scala b/doobie/src/main/scala/tamer/db/Db.scala index f89fbc07..34b74808 100644 --- a/doobie/src/main/scala/tamer/db/Db.scala +++ b/doobie/src/main/scala/tamer/db/Db.scala @@ -7,10 +7,16 @@ import java.time.Instant import scala.util.hashing.byteswap64 object Db { - abstract class Datable(val instant: Instant) - object Datable { - val underlyingOrdering: Ordering[Instant] = implicitly[Ordering[Instant]] - implicit def ordering[Subtype <: Datable]: Ordering[Subtype] = (x: Datable, y: Datable) => underlyingOrdering.compare(x.instant, y.instant) + + /** By specifying a field here, tamer will order database records according + * to this date. Usually you want your latest update timestamp here. + * @param timestamp the value tamer will use to order the record by. + */ + abstract class Timestamped(val timestamp: Instant) + object Timestamped { + val underlyingOrdering: Ordering[Instant] = implicitly[Ordering[Instant]] + implicit def ordering[Subtype <: Timestamped]: Ordering[Subtype] = + (x: Timestamped, y: Timestamped) => underlyingOrdering.compare(x.timestamp, y.timestamp) } case class ChunkWithMetadata[V](chunk: Chunk[V], pulledAt: Instant = Instant.now()) diff --git a/doobie/src/main/scala/tamer/db/package.scala b/doobie/src/main/scala/tamer/db/package.scala index d1c2c021..636a10e3 100644 --- a/doobie/src/main/scala/tamer/db/package.scala +++ b/doobie/src/main/scala/tamer/db/package.scala @@ -13,7 +13,7 @@ import log.effect.zio.ZioLogWriter.log4sFromName import tamer.config.Config import tamer.db.Compat.toIterable import tamer.db.ConfigDb.{DbConfig, QueryConfig} -import tamer.db.Db.{Datable, TimeSegment, _} +import tamer.db.Db.{Timestamped, TimeSegment, _} import tamer.kafka.Kafka import zio.blocking.Blocking import zio.interop.catz._ @@ -35,7 +35,7 @@ package object db { } } - final def mkSetupWithTimeSegment[K <: Product: Encoder: Decoder: SchemaFor, V <: Product with Datable: Ordering: Encoder: Decoder: SchemaFor]( + final def mkSetupWithTimeSegment[K <: Product: Encoder: Decoder: SchemaFor, V <: Product with Timestamped: Ordering: Encoder: Decoder: SchemaFor]( queryBuilder: TimeSegment => Query0[V] )(earliest: Instant, tumblingStep: Duration, keyExtract: V => K): Setup[K, V, TimeSegment] = { @@ -44,7 +44,7 @@ package object db { def stateFold(timeSegment: TimeSegment)(queryResult: QueryResult[V]): UIO[TimeSegment] = if (queryResult.results.isEmpty) timeSegment.to.plus(tumblingStep).orNow.map(TimeSegment(timeSegment.from, _)) else { - val mostRecent = queryResult.results.max.instant + val mostRecent = queryResult.results.max.timestamp mostRecent.plus(tumblingStep).orNow.map(TimeSegment(mostRecent, _)) } @@ -98,7 +98,7 @@ package object db { ) } yield newState).mapError(e => TamerError(e.getLocalizedMessage, e)) - final def fetchWithTimeSegment[K <: Product: Encoder: Decoder: SchemaFor, V <: Product with Datable: Ordering: Encoder: Decoder: SchemaFor]( + final def fetchWithTimeSegment[K <: Product: Encoder: Decoder: SchemaFor, V <: Product with Timestamped: Ordering: Encoder: Decoder: SchemaFor]( queryBuilder: TimeSegment => Query0[V] )(earliest: Instant, tumblingStep: Duration, keyExtract: V => K): ZIO[ZEnv, TamerError, Unit] = { val setup = mkSetupWithTimeSegment[K, V](queryBuilder)(earliest, tumblingStep, keyExtract) diff --git a/example/src/main/scala/tamer/example/Main.scala b/example/src/main/scala/tamer/example/Main.scala index 695293f5..a10179b5 100644 --- a/example/src/main/scala/tamer/example/Main.scala +++ b/example/src/main/scala/tamer/example/Main.scala @@ -2,7 +2,7 @@ package tamer package example import tamer.config.Config -import tamer.db.Db.Datable +import tamer.db.Db.Timestamped import tamer.db.{ConfigDb, DbTransactor, HashableState, InstantOps, QueryResult, TamerDBConfig} import tamer.kafka.Kafka import zio.blocking.Blocking @@ -17,7 +17,7 @@ import java.time.{Duration, Instant} import scala.util.hashing.byteswap64 final case class Key(id: String) -final case class Value(id: String, name: String, description: Option[String], modifiedAt: Instant) extends Datable(modifiedAt) +final case class Value(id: String, name: String, description: Option[String], modifiedAt: Instant) extends Timestamped(modifiedAt) object Main extends zio.App { val program: ZIO[ZEnv, TamerError, Unit] = (for { @@ -63,7 +63,7 @@ object MainGeneralized extends zio.App { stateFoldM = (s: MyState) => { case QueryResult(_, results) if results.isEmpty => s.to.plus(5, MINUTES).orNow.map(MyState(s.from, _)) case QueryResult(_, results) => - val mostRecent = results.sortBy(_.modifiedAt).max.instant + val mostRecent = results.sortBy(_.modifiedAt).max.timestamp mostRecent.plus(5, MINUTES).orNow.map(MyState(mostRecent, _)) } ) From 035745e8095caef15488fe3b9b69b99adf7bfebb Mon Sep 17 00:00:00 2001 From: Andrea Passaglia Date: Wed, 30 Dec 2020 10:41:59 +0000 Subject: [PATCH 10/15] Update core/src/main/scala/tamer/Setup.scala Co-authored-by: Julien Jean Paul Sirocchi --- core/src/main/scala/tamer/Setup.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/tamer/Setup.scala b/core/src/main/scala/tamer/Setup.scala index 05b77f04..3abc2a1a 100644 --- a/core/src/main/scala/tamer/Setup.scala +++ b/core/src/main/scala/tamer/Setup.scala @@ -8,5 +8,5 @@ abstract class Setup[-K, -V, S]( val valueSerializer: Serializer[Registry with Topic, V], val stateSerde: ZSerde[Registry with Topic, S], val defaultState: S, - val queryHash: Int + val stateKey: String ) From 467461792662f83a7d460e886ad9a04bdc1d0a13 Mon Sep 17 00:00:00 2001 From: Andrea Passaglia Date: Wed, 30 Dec 2020 10:44:08 +0000 Subject: [PATCH 11/15] address pr review --- core/src/main/scala/tamer/Setup.scala | 2 +- core/src/main/scala/tamer/kafka/Kafka.scala | 2 +- doobie/src/main/scala/tamer/db/Db.scala | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/tamer/Setup.scala b/core/src/main/scala/tamer/Setup.scala index 05b77f04..7594cc94 100644 --- a/core/src/main/scala/tamer/Setup.scala +++ b/core/src/main/scala/tamer/Setup.scala @@ -8,5 +8,5 @@ abstract class Setup[-K, -V, S]( val valueSerializer: Serializer[Registry with Topic, V], val stateSerde: ZSerde[Registry with Topic, S], val defaultState: S, - val queryHash: Int + val stateKey: Int ) diff --git a/core/src/main/scala/tamer/kafka/Kafka.scala b/core/src/main/scala/tamer/kafka/Kafka.scala index 733ee9a8..e50b9358 100644 --- a/core/src/main/scala/tamer/kafka/Kafka.scala +++ b/core/src/main/scala/tamer/kafka/Kafka.scala @@ -77,7 +77,7 @@ object Kafka { sp: Producer.Service[Registry with Topic, StateKey, State], layer: ULayer[Registry with Topic] ) = - ZStream.fromEffect(logTask <*> UIO(StateKey(setup.queryHash.toHexString, cfg.state.groupId))).flatMap { // TODO: no need for UIO, it's pure + ZStream.fromEffect(logTask <*> UIO(StateKey(setup.stateKey.toHexString, cfg.state.groupId))).flatMap { // TODO: no need for UIO, it's pure case (log, stateKey) => ZStream .fromEffect(subscribe(sc)) diff --git a/doobie/src/main/scala/tamer/db/Db.scala b/doobie/src/main/scala/tamer/db/Db.scala index 34b74808..5d2b90c1 100644 --- a/doobie/src/main/scala/tamer/db/Db.scala +++ b/doobie/src/main/scala/tamer/db/Db.scala @@ -10,7 +10,7 @@ object Db { /** By specifying a field here, tamer will order database records according * to this date. Usually you want your latest update timestamp here. - * @param timestamp the value tamer will use to order the record by. + * @param timestamp the value tamer will use to order the records by. */ abstract class Timestamped(val timestamp: Instant) object Timestamped { From 6e6026cd64a5d1439f6a5174ddd7135e0da1007a Mon Sep 17 00:00:00 2001 From: Andrea Passaglia Date: Fri, 1 Jan 2021 20:50:13 +0000 Subject: [PATCH 12/15] cleaned, update docs --- README.md | 16 +++++ build.sbt | 2 - core/src/main/scala/tamer/Setup.scala | 4 +- core/src/main/scala/tamer/kafka/Kafka.scala | 67 ++++++++++--------- doobie/src/main/scala/tamer/db/ConfigDb.scala | 4 +- doobie/src/main/scala/tamer/db/Setup.scala | 14 ++-- .../src/main/scala/tamer/example/Main.scala | 6 -- local/docker-compose.yml | 32 --------- local/runDb.sh | 20 ++++++ 9 files changed, 84 insertions(+), 81 deletions(-) create mode 100644 local/runDb.sh diff --git a/README.md b/README.md index 803bad9d..77eb0ca3 100644 --- a/README.md +++ b/README.md @@ -18,6 +18,22 @@ libraryDependencies += "io.laserdisc" %% "tamer" % version See [here](example/src/main/scala/tamer/example/Main.scala) for a sample application that makes use of Tamer. +## End to end testing + +### Database module + +Basic manual testing is available for the code in the example module `tamer.example.Main`. +Thi code covers getting data from a synthetic Postgres database. + +Make sure you have docker installed before proceeding. + +From the `local` folder launch `docker-compose up` (you can enter `docker-compose down` +if you want to start from scratch). After that you should be able to access the kafka +gui from http://localhost:8000. + +Start the `runDb.sh` program which contains some example environment variables. +If tamer works you should see messages appearing in the kafka gui. + ## License Tamer is licensed under the **[MIT License](LICENSE)** (the "License"); you may not use this software except in diff --git a/build.sbt b/build.sbt index 18238051..6997928a 100644 --- a/build.sbt +++ b/build.sbt @@ -19,7 +19,6 @@ lazy val V = new { val scalatest = "3.2.3" val silencer = "1.7.1" val zio = "1.0.3" - val `zio-s3` = "latest.integration" val `zio-interop` = "2.2.0.1" val `zio-kafka` = "0.13.0" } @@ -79,7 +78,6 @@ lazy val D = new { val zio = Seq( "dev.zio" %% "zio-interop-cats" % V.`zio-interop`, "dev.zio" %% "zio-kafka" % V.`zio-kafka`, - "dev.zio" %% "zio-s3" % V.`zio-s3`, "dev.zio" %% "zio-streams" % V.zio, "dev.zio" %% "zio-test" % V.zio ) diff --git a/core/src/main/scala/tamer/Setup.scala b/core/src/main/scala/tamer/Setup.scala index 7594cc94..0578dc93 100644 --- a/core/src/main/scala/tamer/Setup.scala +++ b/core/src/main/scala/tamer/Setup.scala @@ -9,4 +9,6 @@ abstract class Setup[-K, -V, S]( val stateSerde: ZSerde[Registry with Topic, S], val defaultState: S, val stateKey: Int -) +) { + def show: String = "not available, please implement the show method to display setup" +} diff --git a/core/src/main/scala/tamer/kafka/Kafka.scala b/core/src/main/scala/tamer/kafka/Kafka.scala index e50b9358..3aacd4de 100644 --- a/core/src/main/scala/tamer/kafka/Kafka.scala +++ b/core/src/main/scala/tamer/kafka/Kafka.scala @@ -52,9 +52,12 @@ object Kafka { val stateKeySerde = Serde[StateKey](isKey = true) val stateConsumer = Consumer.make(cSettings) val stateProducer = Producer.make(pSettings, stateKeySerde.serializer, setup.stateSerde) + val stateKey = StateKey(setup.stateKey.toHexString, cfg.state.groupId) val producer = Producer.make(pSettings, setup.keySerializer, setup.valueSerializer) val queue = Managed.make(Queue.bounded[(K, V)](cfg.bufferSize))(_.shutdown) + def printSetup(logWriter: LogWriter[Task]) = logWriter.info(s"initializing kafka loop with setup: \n${setup.show}") + def mkRegistry(src: SchemaRegistryClient, topic: String) = (ZLayer.succeed(src) >>> Registry.live) ++ (ZLayer.succeed(topic) >>> Topic.live) def mkRecordChunk(kvs: List[(K, V)]) = Chunk.fromIterable(kvs.map { case (k, v) => new ProducerRecord(cfg.sink.topic, k, v) }) @@ -67,7 +70,6 @@ object Kafka { log.info(s"pushed ${kvs.size} messages to ${cfg.sink.topic}") } } - def mkRecord(k: StateKey, v: State) = new ProducerRecord(cfg.state.topic, k, v) def waitAssignment(sc: Consumer.Service) = sc.assignment.withFilter(_.nonEmpty).retry(tenTimes) def subscribe(sc: Consumer.Service) = sc.subscribe(stateTopicSub) *> waitAssignment(sc).flatMap(sc.endOffsets(_)).map(_.values.exists(_ > 0L)) @@ -77,41 +79,40 @@ object Kafka { sp: Producer.Service[Registry with Topic, StateKey, State], layer: ULayer[Registry with Topic] ) = - ZStream.fromEffect(logTask <*> UIO(StateKey(setup.stateKey.toHexString, cfg.state.groupId))).flatMap { // TODO: no need for UIO, it's pure - case (log, stateKey) => - ZStream - .fromEffect(subscribe(sc)) - .flatMap { - case true => ZStream.fromEffect(log.info(s"consumer group ${cfg.state.groupId} resuming consumption from ${cfg.state.topic}")) - case false => - ZStream.fromEffect { - log.info(s"consumer group ${cfg.state.groupId} never consumed from ${cfg.state.topic}, setting offset to earliest") *> - sp.produceAsync(mkRecord(stateKey, setup.defaultState)) + ZStream.fromEffect(logTask).tap(printSetup).flatMap { log => + ZStream + .fromEffect(subscribe(sc)) + .flatMap { + case true => ZStream.fromEffect(log.info(s"consumer group ${cfg.state.groupId} resuming consumption from ${cfg.state.topic}")) + case false => + ZStream.fromEffect { + log.info(s"consumer group ${cfg.state.groupId} never consumed from ${cfg.state.topic}, setting offset to earliest") *> + sp.produceAsync(mkRecord(stateKey, setup.defaultState)) + .provideSomeLayer[Blocking](layer) + .flatten + .flatMap(rm => log.info(s"pushed initial state ${setup.defaultState} to $rm")) + } + } + .drain ++ + sc.plainStream(stateKeySerde.deserializer, setup.stateSerde) + .provideSomeLayer[Blocking with Clock](layer) + .mapM { + case CommittableRecord(record, offset) if record.key == stateKey => + log.debug( + s"consumer group ${cfg.state.groupId} consumed state ${record.value} from ${offset.topicPartition}@${offset.offset}" + ) *> + f(record.value, q).flatMap { newState => + sp.produceAsync(mkRecord(stateKey, newState)) .provideSomeLayer[Blocking](layer) .flatten - .flatMap(rm => log.info(s"pushed initial state ${setup.defaultState} to $rm")) - } + .flatMap(rmd => log.debug(s"pushed state $newState to $rmd")) + .as(offset) + } + case CommittableRecord(_, offset) => + log.debug( + s"consumer group ${cfg.state.groupId} ignored state (wrong key) from ${offset.topicPartition}@${offset.offset}" + ) *> UIO(offset) } - .drain ++ - sc.plainStream(stateKeySerde.deserializer, setup.stateSerde) - .provideSomeLayer[Blocking with Clock](layer) - .mapM { - case CommittableRecord(record, offset) if record.key == stateKey => - log.debug( - s"consumer group ${cfg.state.groupId} consumed state ${record.value} from ${offset.topicPartition}@${offset.offset}" - ) *> - f(record.value, q).flatMap { newState => - sp.produceAsync(mkRecord(stateKey, newState)) - .provideSomeLayer[Blocking](layer) - .flatten - .flatMap(rmd => log.debug(s"pushed state $newState to $rmd")) - .as(offset) - } - case CommittableRecord(_, offset) => - log.debug( - s"consumer group ${cfg.state.groupId} ignored state (wrong key) from ${offset.topicPartition}@${offset.offset}" - ) *> UIO(offset) - } } ZStream diff --git a/doobie/src/main/scala/tamer/db/ConfigDb.scala b/doobie/src/main/scala/tamer/db/ConfigDb.scala index 3f53d0b8..ca7fd451 100644 --- a/doobie/src/main/scala/tamer/db/ConfigDb.scala +++ b/doobie/src/main/scala/tamer/db/ConfigDb.scala @@ -29,7 +29,7 @@ object ConfigDb { ).parMapN(Db) private[this] val queryConfigValue = env("QUERY_FETCH_CHUNK_SIZE").as[PosInt].map(Query) - private[this] val deleteMeConfigValue = (dbConfigValue, queryConfigValue).parMapN(DatabaseConfig.apply) + private[this] val configValue = (dbConfigValue, queryConfigValue).parMapN(DatabaseConfig.apply) trait Service { val dbConfig: URIO[DbConfig, Db] @@ -37,7 +37,7 @@ object ConfigDb { } val live: Layer[TamerError, Has[Db] with Has[Query]] = ZLayer.fromEffectMany { - deleteMeConfigValue.load[Task].refineToOrDie[ConfigException].mapError(ce => TamerError(ce.error.redacted.show, ce)).map { + configValue.load[Task].refineToOrDie[ConfigException].mapError(ce => TamerError(ce.error.redacted.show, ce)).map { case DatabaseConfig(db, query) => Has(db) ++ Has(query) } } diff --git a/doobie/src/main/scala/tamer/db/Setup.scala b/doobie/src/main/scala/tamer/db/Setup.scala index 739f8f2f..6da80543 100644 --- a/doobie/src/main/scala/tamer/db/Setup.scala +++ b/doobie/src/main/scala/tamer/db/Setup.scala @@ -14,10 +14,6 @@ trait QueryBuilder[V, -S] { } trait HashableState { - // TODO: Evaluate if this is less invasive as a typeclass, the main cons - // TODO: is loss of expressivity, and since state is probably manually - // TODO: provided by the user (as opposed to automatically generated - // TODO: code) it should be easy to implement this. /** It is required for this hash to be consistent even across executions * for the same semantic state. This is in contrast with the built-in @@ -44,4 +40,12 @@ case class Setup[ Serde[S]().serde, defaultState, queryBuilder.queryId + defaultState.stateHash - ) + ) { + override def show: String = s""" + |query: ${queryBuilder.query(defaultState).sql} + |query id: ${queryBuilder.queryId} + |default state: $defaultState + |default state id: ${defaultState.stateHash} + |default state key: $stateKey + |""".stripMargin.stripLeading() +} diff --git a/example/src/main/scala/tamer/example/Main.scala b/example/src/main/scala/tamer/example/Main.scala index a10179b5..db7e0d6e 100644 --- a/example/src/main/scala/tamer/example/Main.scala +++ b/example/src/main/scala/tamer/example/Main.scala @@ -9,7 +9,6 @@ import zio.blocking.Blocking import zio._ import doobie.implicits.legacy.instant._ import doobie.syntax.string._ -import log.effect.zio.ZioLogWriter.log4sFromName import tamer.db.ConfigDb.{DbConfig, QueryConfig} import java.time.temporal.ChronoUnit._ @@ -21,8 +20,6 @@ final case class Value(id: String, name: String, description: Option[String], mo object Main extends zio.App { val program: ZIO[ZEnv, TamerError, Unit] = (for { - log <- log4sFromName.provide("tamer.example") - _ <- log.info("Starting tamer...") boot <- UIO(Instant.now()) _ <- tamer.db.fetchWithTimeSegment(ts => sql"""SELECT id, name, description, modified_at FROM users WHERE modified_at > ${ts.from} AND modified_at <= ${ts.to}""".query[Value] @@ -51,8 +48,6 @@ object MainGeneralized extends zio.App { val queryConfigLayer: Layer[TamerError, DbConfig with QueryConfig] = ConfigDb.live val myLayer: Layer[TamerError, DbTransactor with Kafka with QueryConfig] = transactorLayer ++ kafkaLayer ++ queryConfigLayer val program: ZIO[Kafka with TamerDBConfig with ZEnv, TamerError, Unit] = (for { - log <- log4sFromName.provide("tamer.example") - _ <- log.info("Starting tamer...") boot <- UIO(Instant.now()) earliest = boot.minus(60, DAYS) setup = tamer.db.mkSetup((s: MyState) => @@ -67,7 +62,6 @@ object MainGeneralized extends zio.App { mostRecent.plus(5, MINUTES).orNow.map(MyState(mostRecent, _)) } ) - _ <- log.info(s"Tamer initialized with setup $setup") _ <- tamer.db.fetch(setup) } yield ()).mapError(e => TamerError("Could not run tamer example", e)) diff --git a/local/docker-compose.yml b/local/docker-compose.yml index 362b2117..0ed5a88a 100644 --- a/local/docker-compose.yml +++ b/local/docker-compose.yml @@ -121,37 +121,5 @@ services: - SCHEMAREGISTRY_URL=http://schema-registry:8081 - PROXY=true -# # TODO replace kafka-rest+kafka-ui and schema-registry-ui -# kowl: -# container_name: local-kowl -# image: quay.io/cloudhut/kowl:v1.2.1 -# depends_on: -# - kafka -# - schema-registry -# # waits for connections to be available -# restart: on-failure -# ports: -# - 8002:8080 -# hostname: kowl -# networks: -# - local_kafka_network -# entrypoint: -# - /bin/ash -# - -c -# - | -# mkdir -p /etc/kowl -# cat > /etc/kowl/config.yaml < Date: Sat, 2 Jan 2021 17:49:44 +0000 Subject: [PATCH 13/15] Update README.md Co-authored-by: Julien Jean Paul Sirocchi --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 77eb0ca3..372ec30a 100644 --- a/README.md +++ b/README.md @@ -23,7 +23,7 @@ See [here](example/src/main/scala/tamer/example/Main.scala) for a sample applica ### Database module Basic manual testing is available for the code in the example module `tamer.example.Main`. -Thi code covers getting data from a synthetic Postgres database. +This code covers getting data from a synthetic Postgres database. Make sure you have docker installed before proceeding. From 6578e7df176408dd395c29f808cb876f4af47040 Mon Sep 17 00:00:00 2001 From: Andrea Passaglia Date: Sat, 2 Jan 2021 22:27:16 +0000 Subject: [PATCH 14/15] final draft after review --- build.sbt | 6 +- core/src/main/scala/tamer/config/Config.scala | 4 -- core/src/main/scala/tamer/kafka/Kafka.scala | 2 +- doobie/src/main/scala/tamer/db/Db.scala | 31 +++++----- doobie/src/main/scala/tamer/db/Setup.scala | 33 ++++++++++ doobie/src/main/scala/tamer/db/package.scala | 61 ++++++------------- .../{Main.scala => DatabaseGeneralized.scala} | 39 +++--------- .../scala/tamer/example/DatabaseSimple.scala | 28 +++++++++ 8 files changed, 106 insertions(+), 98 deletions(-) rename example/src/main/scala/tamer/example/{Main.scala => DatabaseGeneralized.scala} (68%) create mode 100644 example/src/main/scala/tamer/example/DatabaseSimple.scala diff --git a/build.sbt b/build.sbt index 6997928a..af4f7bc1 100644 --- a/build.sbt +++ b/build.sbt @@ -136,14 +136,14 @@ lazy val commonSettings = Seq( licenses += "MIT" -> url("http://opensource.org/licenses/MIT"), developers += Developer("sirocchj", "Julien Sirocchi", "julien.sirocchi@gmail.com", url("https://github.com/sirocchj")), scalacOptions ++= versionDependent(scalaVersion.value), - resolvers ++= Seq("confluent" at "https://packages.confluent.io/maven/", "snapshots" at "https://oss.sonatype.org/content/repositories/snapshots") + resolvers ++= Seq("confluent" at "https://packages.confluent.io/maven/") ) lazy val tamer = project .in(file("core")) .settings(commonSettings) .settings( - name := "tamer", + name := "tamer-core", libraryDependencies ++= (D.cats ++ D.config ++ D.kafka ++ D.logs ++ D.refined ++ D.serialization ++ D.silencer ++ D.tests ++ D.zio) .map(_.withSources) .map(_.withJavadoc), @@ -158,7 +158,7 @@ lazy val doobie = project .dependsOn(tamer) .settings(commonSettings) .settings( - name := "doobie", + name := "tamer-doobie", libraryDependencies ++= D.doobie ) diff --git a/core/src/main/scala/tamer/config/Config.scala b/core/src/main/scala/tamer/config/Config.scala index f28eb44f..954bc512 100644 --- a/core/src/main/scala/tamer/config/Config.scala +++ b/core/src/main/scala/tamer/config/Config.scala @@ -42,10 +42,6 @@ object Config { kafkaStateConfigValue ).parMapN(Kafka) - trait Service { - val kafkaConfig: URIO[KafkaConfig, Kafka] - } - val live: Layer[TamerError, KafkaConfig] = ZLayer.fromEffect { kafkaConfigValue.load[Task].refineToOrDie[ConfigException].mapError(ce => TamerError(ce.error.redacted.show, ce)) } diff --git a/core/src/main/scala/tamer/kafka/Kafka.scala b/core/src/main/scala/tamer/kafka/Kafka.scala index 3aacd4de..d2e2bbd5 100644 --- a/core/src/main/scala/tamer/kafka/Kafka.scala +++ b/core/src/main/scala/tamer/kafka/Kafka.scala @@ -21,7 +21,7 @@ import zio.stream.ZStream final case class StateKey(stateKey: String, groupId: String) object Kafka { - trait Service { + sealed trait Service { def runLoop[K, V, State, R](setup: Setup[K, V, State])( f: (State, Queue[(K, V)]) => ZIO[R, TamerError, State] ): ZIO[R with Blocking with Clock, TamerError, Unit] diff --git a/doobie/src/main/scala/tamer/db/Db.scala b/doobie/src/main/scala/tamer/db/Db.scala index 5d2b90c1..e4848576 100644 --- a/doobie/src/main/scala/tamer/db/Db.scala +++ b/doobie/src/main/scala/tamer/db/Db.scala @@ -6,23 +6,20 @@ import fs2.Chunk import java.time.Instant import scala.util.hashing.byteswap64 -object Db { - - /** By specifying a field here, tamer will order database records according - * to this date. Usually you want your latest update timestamp here. - * @param timestamp the value tamer will use to order the records by. - */ - abstract class Timestamped(val timestamp: Instant) - object Timestamped { - val underlyingOrdering: Ordering[Instant] = implicitly[Ordering[Instant]] - implicit def ordering[Subtype <: Timestamped]: Ordering[Subtype] = - (x: Timestamped, y: Timestamped) => underlyingOrdering.compare(x.timestamp, y.timestamp) - } +/** By specifying a field here, tamer will order database records according + * to this date. Usually you want your latest update timestamp here. + * @param timestamp the value tamer will use to order the records by. + */ +abstract class Timestamped(val timestamp: Instant) +object Timestamped { + val underlyingOrdering: Ordering[Instant] = implicitly[Ordering[Instant]] + implicit def ordering[Subtype <: Timestamped]: Ordering[Subtype] = + (x: Timestamped, y: Timestamped) => underlyingOrdering.compare(x.timestamp, y.timestamp) +} - case class ChunkWithMetadata[V](chunk: Chunk[V], pulledAt: Instant = Instant.now()) - case class ValueWithMetadata[V](value: V, pulledAt: Instant = Instant.now()) +case class ChunkWithMetadata[V](chunk: Chunk[V], pulledAt: Instant = Instant.now()) +case class ValueWithMetadata[V](value: V, pulledAt: Instant = Instant.now()) - case class TimeSegment(from: Instant, to: Instant) extends HashableState { - override lazy val stateHash: Int = (byteswap64(from.getEpochSecond) + byteswap64(to.getEpochSecond)).intValue - } +case class TimeSegment(from: Instant, to: Instant) extends HashableState { + override lazy val stateHash: Int = (byteswap64(from.getEpochSecond) + byteswap64(to.getEpochSecond)).intValue } diff --git a/doobie/src/main/scala/tamer/db/Setup.scala b/doobie/src/main/scala/tamer/db/Setup.scala index 6da80543..84582068 100644 --- a/doobie/src/main/scala/tamer/db/Setup.scala +++ b/doobie/src/main/scala/tamer/db/Setup.scala @@ -5,6 +5,8 @@ import doobie.Query0 import tamer.Serde import zio.UIO +import java.time.{Duration, Instant} + trait QueryBuilder[V, -S] { /** Used for hashing purposes @@ -49,3 +51,34 @@ case class Setup[ |default state key: $stateKey |""".stripMargin.stripLeading() } +object Setup { + final def apply[ + K <: Product: Encoder: Decoder: SchemaFor, + V <: Product: Encoder: Decoder: SchemaFor, + S <: Product with HashableState: Encoder: Decoder: SchemaFor + ]( + queryBuilder: S => Query0[V] + )(defaultState: S, keyExtract: V => K, stateFoldM: S => QueryResult[V] => UIO[S]): Setup[K, V, S] = { + val qBuilder = new QueryBuilder[V, S] { + override val queryId: Int = queryBuilder(defaultState).sql.hashCode + override def query(state: S): Query0[V] = queryBuilder(state) + } + new Setup[K, V, S](queryBuilder = qBuilder, defaultState = defaultState, keyExtract = keyExtract, stateFoldM = stateFoldM) + } + + final def fromTimeSegment[K <: Product: Encoder: Decoder: SchemaFor, V <: Product with Timestamped: Ordering: Encoder: Decoder: SchemaFor]( + queryBuilder: TimeSegment => Query0[V] + )(earliest: Instant, tumblingStep: Duration, keyExtract: V => K): Setup[K, V, TimeSegment] = { + + val timeSegment = TimeSegment(earliest, earliest.plus(tumblingStep)) + + def stateFold(timeSegment: TimeSegment)(queryResult: QueryResult[V]): UIO[TimeSegment] = + if (queryResult.results.isEmpty) timeSegment.to.plus(tumblingStep).orNow.map(TimeSegment(timeSegment.from, _)) + else { + val mostRecent = queryResult.results.max.timestamp + mostRecent.plus(tumblingStep).orNow.map(TimeSegment(mostRecent, _)) + } + + Setup(queryBuilder)(timeSegment, keyExtract, stateFold) + } +} diff --git a/doobie/src/main/scala/tamer/db/package.scala b/doobie/src/main/scala/tamer/db/package.scala index 636a10e3..6512c688 100644 --- a/doobie/src/main/scala/tamer/db/package.scala +++ b/doobie/src/main/scala/tamer/db/package.scala @@ -13,7 +13,6 @@ import log.effect.zio.ZioLogWriter.log4sFromName import tamer.config.Config import tamer.db.Compat.toIterable import tamer.db.ConfigDb.{DbConfig, QueryConfig} -import tamer.db.Db.{Timestamped, TimeSegment, _} import tamer.kafka.Kafka import zio.blocking.Blocking import zio.interop.catz._ @@ -24,8 +23,13 @@ import java.time.{Duration, Instant} import scala.concurrent.ExecutionContext package object db { - type DbTransactor = Has[Transactor[Task]] - type TamerDBConfig = DbTransactor with QueryConfig + final type DbTransactor = Has[Transactor[Task]] + final type TamerDBConfig = DbTransactor with QueryConfig + + private final val kafkaLayer: Layer[TamerError, Kafka] = Config.live >>> Kafka.live + private final val transactorLayer: Layer[TamerError, DbTransactor] = (Blocking.live ++ ConfigDb.live) >>> db.hikariLayer + private final val queryConfigLayer: Layer[TamerError, DbConfig with QueryConfig] = ConfigDb.live + private final val defaultLayer: Layer[TamerError, DbTransactor with Kafka with QueryConfig] = transactorLayer ++ queryConfigLayer ++ kafkaLayer implicit final class InstantOps(private val instant: Instant) extends AnyVal { def orNow: UIO[Instant] = @@ -35,39 +39,9 @@ package object db { } } - final def mkSetupWithTimeSegment[K <: Product: Encoder: Decoder: SchemaFor, V <: Product with Timestamped: Ordering: Encoder: Decoder: SchemaFor]( - queryBuilder: TimeSegment => Query0[V] - )(earliest: Instant, tumblingStep: Duration, keyExtract: V => K): Setup[K, V, TimeSegment] = { - - val timeSegment = TimeSegment(earliest, earliest.plus(tumblingStep)) - - def stateFold(timeSegment: TimeSegment)(queryResult: QueryResult[V]): UIO[TimeSegment] = - if (queryResult.results.isEmpty) timeSegment.to.plus(tumblingStep).orNow.map(TimeSegment(timeSegment.from, _)) - else { - val mostRecent = queryResult.results.max.timestamp - mostRecent.plus(tumblingStep).orNow.map(TimeSegment(mostRecent, _)) - } - - mkSetup(queryBuilder)(timeSegment, keyExtract, stateFold) - } + private[this] final val logTask: Task[LogWriter[Task]] = log4sFromName.provide("tamer.db") - final def mkSetup[ - K <: Product: Encoder: Decoder: SchemaFor, - V <: Product: Encoder: Decoder: SchemaFor, - S <: Product with HashableState: Encoder: Decoder: SchemaFor - ]( - queryBuilder: S => Query0[V] - )(defaultState: S, keyExtract: V => K, stateFoldM: S => QueryResult[V] => UIO[S]): Setup[K, V, S] = { - val qBuilder = new QueryBuilder[V, S] { - override val queryId: Int = queryBuilder(defaultState).sql.hashCode - override def query(state: S): Query0[V] = queryBuilder(state) - } - Setup[K, V, S](queryBuilder = qBuilder, defaultState = defaultState, keyExtract = keyExtract, stateFoldM = stateFoldM) - } - - private[this] val logTask: Task[LogWriter[Task]] = log4sFromName.provide("tamer.db") - - final def iteration[K <: Product, V <: Product, S <: Product with HashableState]( + private final def iteration[K <: Product, V <: Product, S <: Product with HashableState]( setup: Setup[K, V, S] )(state: S, q: Queue[(K, V)]): ZIO[TamerDBConfig, TamerError, S] = (for { @@ -101,7 +75,7 @@ package object db { final def fetchWithTimeSegment[K <: Product: Encoder: Decoder: SchemaFor, V <: Product with Timestamped: Ordering: Encoder: Decoder: SchemaFor]( queryBuilder: TimeSegment => Query0[V] )(earliest: Instant, tumblingStep: Duration, keyExtract: V => K): ZIO[ZEnv, TamerError, Unit] = { - val setup = mkSetupWithTimeSegment[K, V](queryBuilder)(earliest, tumblingStep, keyExtract) + val setup = Setup.fromTimeSegment[K, V](queryBuilder)(earliest, tumblingStep, keyExtract) fetch(setup) } @@ -111,15 +85,10 @@ package object db { S <: Product with HashableState: Encoder: Decoder: SchemaFor ]( setup: Setup[K, V, S] - ): ZIO[ZEnv, TamerError, Unit] = { - val kafkaLayer: Layer[TamerError, Kafka] = Config.live >>> Kafka.live - val transactorLayer: Layer[TamerError, DbTransactor] = (Blocking.live ++ ConfigDb.live) >>> db.hikariLayer - val queryConfigLayer: Layer[TamerError, DbConfig with QueryConfig] = ConfigDb.live - val defaultLayer: Layer[TamerError, DbTransactor with Kafka with QueryConfig] = transactorLayer ++ queryConfigLayer ++ kafkaLayer + ): ZIO[ZEnv, TamerError, Unit] = tamer.kafka.runLoop(setup)(iteration(setup)).provideCustomLayer(defaultLayer) - } - val hikariLayer: ZLayer[Blocking with DbConfig, TamerError, DbTransactor] = ZLayer.fromManaged { + final val hikariLayer: ZLayer[Blocking with DbConfig, TamerError, DbTransactor] = ZLayer.fromManaged { for { cfg <- ConfigDb.dbConfig.toManaged_ connectEC <- ZIO.descriptor.map(_.executor.asEC).toManaged_ @@ -128,7 +97,11 @@ package object db { } yield managedTransactor } - def mkTransactor(db: ConfigDb.Db, connectEC: ExecutionContext, transactEC: ExecutionContext): Managed[TamerError, HikariTransactor[Task]] = + private final def mkTransactor( + db: ConfigDb.Db, + connectEC: ExecutionContext, + transactEC: ExecutionContext + ): Managed[TamerError, HikariTransactor[Task]] = HikariTransactor .newHikariTransactor[Task](db.driver, db.uri, db.username, db.password, connectEC, Blocker.liftExecutionContext(transactEC)) .toManagedZIO diff --git a/example/src/main/scala/tamer/example/Main.scala b/example/src/main/scala/tamer/example/DatabaseGeneralized.scala similarity index 68% rename from example/src/main/scala/tamer/example/Main.scala rename to example/src/main/scala/tamer/example/DatabaseGeneralized.scala index db7e0d6e..91b54d94 100644 --- a/example/src/main/scala/tamer/example/Main.scala +++ b/example/src/main/scala/tamer/example/DatabaseGeneralized.scala @@ -1,38 +1,19 @@ -package tamer -package example +package tamer.example -import tamer.config.Config -import tamer.db.Db.Timestamped -import tamer.db.{ConfigDb, DbTransactor, HashableState, InstantOps, QueryResult, TamerDBConfig} -import tamer.kafka.Kafka -import zio.blocking.Blocking -import zio._ import doobie.implicits.legacy.instant._ import doobie.syntax.string._ +import tamer.{TamerError, db} +import tamer.config.Config import tamer.db.ConfigDb.{DbConfig, QueryConfig} +import tamer.db.{ConfigDb, DbTransactor, HashableState, InstantOps, QueryResult, Setup, TamerDBConfig} +import tamer.kafka.Kafka +import zio._ +import zio.blocking.Blocking +import java.time.Instant import java.time.temporal.ChronoUnit._ -import java.time.{Duration, Instant} import scala.util.hashing.byteswap64 -final case class Key(id: String) -final case class Value(id: String, name: String, description: Option[String], modifiedAt: Instant) extends Timestamped(modifiedAt) - -object Main extends zio.App { - val program: ZIO[ZEnv, TamerError, Unit] = (for { - boot <- UIO(Instant.now()) - _ <- tamer.db.fetchWithTimeSegment(ts => - sql"""SELECT id, name, description, modified_at FROM users WHERE modified_at > ${ts.from} AND modified_at <= ${ts.to}""".query[Value] - )( - earliest = boot.minus(60, DAYS), - tumblingStep = Duration.of(5, MINUTES), - keyExtract = (value: Value) => Key(value.id) - ) - } yield ()).mapError(e => TamerError("Could not run tamer example", e)) - - override final def run(args: List[String]): URIO[ZEnv, ExitCode] = program.exitCode -} - final case class MyState(from: Instant, to: Instant) extends HashableState { /** It is required for this hash to be consistent even across executions @@ -42,7 +23,7 @@ final case class MyState(from: Instant, to: Instant) extends HashableState { override val stateHash: Int = (byteswap64(from.getEpochSecond) + byteswap64(to.getEpochSecond)).intValue } -object MainGeneralized extends zio.App { +object DatabaseGeneralized extends zio.App { val transactorLayer: Layer[TamerError, DbTransactor] = (Blocking.live ++ ConfigDb.live) >>> db.hikariLayer val kafkaLayer: Layer[TamerError, Kafka] = Config.live >>> Kafka.live val queryConfigLayer: Layer[TamerError, DbConfig with QueryConfig] = ConfigDb.live @@ -50,7 +31,7 @@ object MainGeneralized extends zio.App { val program: ZIO[Kafka with TamerDBConfig with ZEnv, TamerError, Unit] = (for { boot <- UIO(Instant.now()) earliest = boot.minus(60, DAYS) - setup = tamer.db.mkSetup((s: MyState) => + setup = Setup((s: MyState) => sql"""SELECT id, name, description, modified_at FROM users WHERE modified_at > ${s.from} AND modified_at <= ${s.to}""".query[Value] )( defaultState = MyState(earliest, earliest.plus(5, MINUTES)), diff --git a/example/src/main/scala/tamer/example/DatabaseSimple.scala b/example/src/main/scala/tamer/example/DatabaseSimple.scala new file mode 100644 index 00000000..20f15276 --- /dev/null +++ b/example/src/main/scala/tamer/example/DatabaseSimple.scala @@ -0,0 +1,28 @@ +package tamer +package example + +import doobie.implicits.legacy.instant._ +import doobie.syntax.string._ +import tamer.db.Timestamped +import zio._ + +import java.time.temporal.ChronoUnit._ +import java.time.{Duration, Instant} + +final case class Key(id: String) +final case class Value(id: String, name: String, description: Option[String], modifiedAt: Instant) extends Timestamped(modifiedAt) + +object DatabaseSimple extends zio.App { + val program: ZIO[ZEnv, TamerError, Unit] = (for { + boot <- UIO(Instant.now()) + _ <- tamer.db.fetchWithTimeSegment(ts => + sql"""SELECT id, name, description, modified_at FROM users WHERE modified_at > ${ts.from} AND modified_at <= ${ts.to}""".query[Value] + )( + earliest = boot.minus(60, DAYS), + tumblingStep = Duration.of(5, MINUTES), + keyExtract = (value: Value) => Key(value.id) + ) + } yield ()).mapError(e => TamerError("Could not run tamer example", e)) + + override final def run(args: List[String]): URIO[ZEnv, ExitCode] = program.exitCode +} From c1097e0e317dd519ba81e46b4eaf240bcd5ab38e Mon Sep 17 00:00:00 2001 From: Andrea Passaglia Date: Sun, 3 Jan 2021 16:56:28 +0000 Subject: [PATCH 15/15] final final draft reviewed --- README.md | 7 ++++--- core/src/main/scala/tamer/config/package.scala | 6 +----- core/src/main/scala/tamer/kafka/Kafka.scala | 6 +++--- doobie/src/main/scala/tamer/db/ConfigDb.scala | 16 +++++++--------- doobie/src/main/scala/tamer/db/Setup.scala | 8 ++++---- local/runDb.sh | 2 +- 6 files changed, 20 insertions(+), 25 deletions(-) diff --git a/README.md b/README.md index 372ec30a..b630dd47 100644 --- a/README.md +++ b/README.md @@ -16,20 +16,21 @@ Add Tamer as a dependency to your project: libraryDependencies += "io.laserdisc" %% "tamer" % version ``` -See [here](example/src/main/scala/tamer/example/Main.scala) for a sample application that makes use of Tamer. +See [here](example/src/main/scala/tamer/example/DatabaseSimple.scala) for a sample application that makes use of Tamer. ## End to end testing ### Database module -Basic manual testing is available for the code in the example module `tamer.example.Main`. +Basic manual testing is available for the code in the example module `tamer.example.DatabaseSimple` +(and/or `tamer.example.DatabaseGeneralized`). This code covers getting data from a synthetic Postgres database. Make sure you have docker installed before proceeding. From the `local` folder launch `docker-compose up` (you can enter `docker-compose down` if you want to start from scratch). After that you should be able to access the kafka -gui from http://localhost:8000. +gui from [http://localhost:8000](http://localhost:8000). Start the `runDb.sh` program which contains some example environment variables. If tamer works you should see messages appearing in the kafka gui. diff --git a/core/src/main/scala/tamer/config/package.scala b/core/src/main/scala/tamer/config/package.scala index 0e8abd55..b4b479c6 100644 --- a/core/src/main/scala/tamer/config/package.scala +++ b/core/src/main/scala/tamer/config/package.scala @@ -4,15 +4,11 @@ import eu.timepit.refined.api.Refined import eu.timepit.refined.boolean.{And, Or} import eu.timepit.refined.collection.{Forall, NonEmpty} import eu.timepit.refined.string.{IPv4, Uri, Url} -import zio.{Has, URIO, ZIO} +import zio.Has package object config { type HostList = List[String] Refined (NonEmpty And Forall[IPv4 Or Uri]) - type Password = String - type UriString = String Refined Uri type UrlString = String Refined Url type KafkaConfig = Has[Config.Kafka] - - val kafkaConfig: URIO[KafkaConfig, Config.Kafka] = ZIO.access(_.get) } diff --git a/core/src/main/scala/tamer/kafka/Kafka.scala b/core/src/main/scala/tamer/kafka/Kafka.scala index d2e2bbd5..a3d7ee5c 100644 --- a/core/src/main/scala/tamer/kafka/Kafka.scala +++ b/core/src/main/scala/tamer/kafka/Kafka.scala @@ -14,14 +14,14 @@ import zio.blocking.Blocking import zio.clock.Clock import zio.duration._ import zio.kafka.consumer.Consumer.{AutoOffsetStrategy, OffsetRetrieval} -import zio.kafka.consumer.{CommittableRecord, Consumer, ConsumerSettings, Subscription} +import zio.kafka.consumer.{CommittableRecord, Consumer, ConsumerSettings, Offset, Subscription} import zio.kafka.producer.{Producer, ProducerSettings} import zio.stream.ZStream final case class StateKey(stateKey: String, groupId: String) object Kafka { - sealed trait Service { + trait Service { def runLoop[K, V, State, R](setup: Setup[K, V, State])( f: (State, Queue[(K, V)]) => ZIO[R, TamerError, State] ): ZIO[R with Blocking with Clock, TamerError, Unit] @@ -78,7 +78,7 @@ object Kafka { q: Queue[(K, V)], sp: Producer.Service[Registry with Topic, StateKey, State], layer: ULayer[Registry with Topic] - ) = + ): ZStream[R with Blocking with Clock, Throwable, Offset] = ZStream.fromEffect(logTask).tap(printSetup).flatMap { log => ZStream .fromEffect(subscribe(sc)) diff --git a/doobie/src/main/scala/tamer/db/ConfigDb.scala b/doobie/src/main/scala/tamer/db/ConfigDb.scala index ca7fd451..7056669e 100644 --- a/doobie/src/main/scala/tamer/db/ConfigDb.scala +++ b/doobie/src/main/scala/tamer/db/ConfigDb.scala @@ -1,16 +1,19 @@ -package tamer.db +package tamer +package db import ciris.{ConfigException, env} import ciris.refined.refTypeConfigDecoder import cats.implicits._ +import eu.timepit.refined.api.Refined +import eu.timepit.refined.string.Uri import eu.timepit.refined.types.numeric.PosInt import eu.timepit.refined.types.string.NonEmptyString -import tamer.TamerError -import tamer.config.{Password, UriString} import zio.interop.catz.{taskConcurrentInstance, zioContextShift} import zio.{Has, Layer, Task, URIO, ZIO, ZLayer} object ConfigDb { + type Password = String + type UriString = String Refined Uri type DbConfig = Has[Db] type QueryConfig = Has[Query] @@ -31,12 +34,7 @@ object ConfigDb { private[this] val configValue = (dbConfigValue, queryConfigValue).parMapN(DatabaseConfig.apply) - trait Service { - val dbConfig: URIO[DbConfig, Db] - val queryConfig: URIO[QueryConfig, Query] - } - - val live: Layer[TamerError, Has[Db] with Has[Query]] = ZLayer.fromEffectMany { + val live: Layer[TamerError, DbConfig with QueryConfig] = ZLayer.fromEffectMany { configValue.load[Task].refineToOrDie[ConfigException].mapError(ce => TamerError(ce.error.redacted.show, ce)).map { case DatabaseConfig(db, query) => Has(db) ++ Has(query) } diff --git a/doobie/src/main/scala/tamer/db/Setup.scala b/doobie/src/main/scala/tamer/db/Setup.scala index 84582068..f7941441 100644 --- a/doobie/src/main/scala/tamer/db/Setup.scala +++ b/doobie/src/main/scala/tamer/db/Setup.scala @@ -1,8 +1,8 @@ -package tamer.db +package tamer +package db import com.sksamuel.avro4s.{Decoder, Encoder, SchemaFor} import doobie.Query0 -import tamer.Serde import zio.UIO import java.time.{Duration, Instant} @@ -27,7 +27,7 @@ trait HashableState { final case class ResultMetadata(queryExecutionTime: Long) final case class QueryResult[V](metadata: ResultMetadata, results: List[V]) -case class Setup[ +final case class Setup[ K <: Product: Encoder: Decoder: SchemaFor, V <: Product: Encoder: Decoder: SchemaFor, S <: Product with HashableState: Encoder: Decoder: SchemaFor @@ -36,7 +36,7 @@ case class Setup[ override val defaultState: S, keyExtract: V => K, stateFoldM: S => QueryResult[V] => UIO[S] -) extends tamer.Setup[K, V, S]( +) extends _root_.tamer.Setup[K, V, S]( Serde[K](isKey = true).serializer, Serde[V]().serializer, Serde[S]().serde, diff --git a/local/runDb.sh b/local/runDb.sh index a6d908cd..7cfde510 100644 --- a/local/runDb.sh +++ b/local/runDb.sh @@ -17,4 +17,4 @@ export KAFKA_STATE_CLIENT_ID=state-client SCRIPT_PATH=$(cd "$(dirname "${BASH_SOURCE[0]}")" || exit; pwd -P) cd "$SCRIPT_PATH"/.. || exit -sbt "example/runMain tamer.example.Main" -jvm-debug 5005 \ No newline at end of file +sbt "example/runMain tamer.example.DatabaseSimple" -jvm-debug 5005 \ No newline at end of file