diff --git a/core/src/main/scala/swaydb/core/Core.scala b/core/src/main/scala/swaydb/core/Core.scala
index a510d6be7..6571ddbd8 100644
--- a/core/src/main/scala/swaydb/core/Core.scala
+++ b/core/src/main/scala/swaydb/core/Core.scala
@@ -38,9 +38,11 @@ import swaydb.data.compaction.LevelMeter
import swaydb.data.config._
import swaydb.data.order.{KeyOrder, TimeOrder}
import swaydb.data.slice.{Slice, SliceOption}
+import swaydb.data.util.Futures.FutureImplicits
import swaydb.data.util.TupleOrNone
import swaydb.{Bag, IO, OK, Prepare}
+import scala.concurrent.{ExecutionContext, Future}
import scala.concurrent.duration._
/**
@@ -147,7 +149,8 @@ private[swaydb] object Core {
private[swaydb] class Core[BAG[_]](val zero: LevelZero,
threadStateCache: ThreadStateCache,
- onClose: => IO.Defer[swaydb.Error.Close, Unit])(implicit bag: Bag[BAG]) {
+ onClose: FiniteDuration => Future[Unit])(implicit bag: Bag[BAG],
+ shutdownExecutionContext: ExecutionContext) {
private val serial = bag.createSerial()
@@ -308,11 +311,15 @@ private[swaydb] class Core[BAG[_]](val zero: LevelZero,
def levelMeter(levelNumber: Int): Option[LevelMeter] =
zero.meterFor(levelNumber)
- def close(): BAG[Unit] =
- onClose.run(0)
+ def close(retryInterval: FiniteDuration): BAG[Unit] =
+ IO.fromFuture(onClose(retryInterval)).run(0)
- def delete(): BAG[Unit] =
- onClose.flatMapIO(_ => zero.delete).run(0)
+ def deleteFuture(retryInterval: FiniteDuration): Future[Unit] =
+ onClose(retryInterval)
+ .and(zero.delete(retryInterval))
+
+ def delete(retryInterval: FiniteDuration): BAG[Unit] =
+ IO.fromFuture(deleteFuture(retryInterval)).run(0)
def clear(readState: ThreadReadState): BAG[OK] =
zero.run(_.clear(readState))
@@ -322,5 +329,5 @@ private[swaydb] class Core[BAG[_]](val zero: LevelZero,
zero = zero,
threadStateCache = threadStateCache,
onClose = onClose
- )(bag)
+ )(bag, shutdownExecutionContext)
}
diff --git a/core/src/main/scala/swaydb/core/CoreInitializer.scala b/core/src/main/scala/swaydb/core/CoreInitializer.scala
index b19733713..0a45571c6 100644
--- a/core/src/main/scala/swaydb/core/CoreInitializer.scala
+++ b/core/src/main/scala/swaydb/core/CoreInitializer.scala
@@ -28,7 +28,7 @@ import java.nio.file.Paths
import com.typesafe.scalalogging.LazyLogging
import swaydb.Error.Level.ExceptionHandler
-import swaydb.core.CoreShutdown.shutdown
+import swaydb.core.CoreShutdown.close
import swaydb.core.actor.{ByteBufferSweeper, FileSweeper, MemorySweeper}
import swaydb.core.function.FunctionStore
import swaydb.core.actor.ByteBufferSweeper.ByteBufferSweeperActor
@@ -50,7 +50,7 @@ import swaydb.data.slice.Slice
import swaydb.data.storage.{AppendixStorage, LevelStorage}
import swaydb.{ActorRef, ActorWire, Bag, Error, IO, Scheduler}
-import scala.concurrent.Await
+import scala.concurrent.{Await, Future}
import scala.concurrent.duration._
/**
@@ -240,17 +240,17 @@ private[core] object CoreInitializer extends LazyLogging {
sys.addShutdownHook {
implicit val scheduler = Scheduler()
- Await.result(shutdown(zero, shutdownTimeout), shutdownTimeout)
+ implicit val bag = Bag.future
+ Await.result(IO.Defer(close(zero, shutdownTimeout)).run(0), shutdownTimeout)
}
//trigger initial wakeUp.
sendInitialWakeUp(compactor)
- def onClose =
- IO.fromFuture[swaydb.Error.Close, Unit] {
- implicit val scheduler = Scheduler()
- CoreShutdown.shutdown(zero, shutdownTimeout)
- }
+ def onClose(retryInterval: FiniteDuration): Future[Unit] = {
+ implicit val scheduler = Scheduler()
+ CoreShutdown.close(zero, retryInterval)
+ }
new Core[Bag.Less](
zero = zero,
diff --git a/core/src/main/scala/swaydb/core/CoreShutdown.scala b/core/src/main/scala/swaydb/core/CoreShutdown.scala
index 01ff15d43..aa1a6f82b 100644
--- a/core/src/main/scala/swaydb/core/CoreShutdown.scala
+++ b/core/src/main/scala/swaydb/core/CoreShutdown.scala
@@ -25,14 +25,10 @@
package swaydb.core
import com.typesafe.scalalogging.LazyLogging
-import swaydb.core.actor.ByteBufferSweeper.ByteBufferSweeperActor
-import swaydb.core.actor.FileSweeper.FileSweeperActor
-import swaydb.core.actor.{ByteBufferSweeper, FileSweeper, MemorySweeper}
-import swaydb.core.io.file.BlockCache
import swaydb.core.level.compaction.Compactor
import swaydb.core.level.compaction.throttle.ThrottleState
import swaydb.core.level.zero.LevelZero
-import swaydb.data.util.Futures
+import swaydb.data.util.Futures.FutureImplicits
import swaydb.{ActorWire, Bag, Scheduler}
import scala.concurrent.Future
@@ -47,117 +43,26 @@ private[core] object CoreShutdown extends LazyLogging {
* - Flushes all files and persists them to disk for persistent databases.
*/
- def shutdown(zero: LevelZero,
- retryOnBusyDelay: FiniteDuration)(implicit compactor: ActorWire[Compactor[ThrottleState], ThrottleState],
- fileSweeper: Option[FileSweeperActor],
- blockCache: Option[BlockCache.State],
- keyValueMemorySweeper: Option[MemorySweeper.KeyValue],
- scheduler: Scheduler,
- cleaner: ByteBufferSweeperActor): Future[Unit] = {
+ def close(zero: LevelZero,
+ retryInterval: FiniteDuration)(implicit compactor: ActorWire[Compactor[ThrottleState], ThrottleState],
+ scheduler: Scheduler) = {
implicit val ec = scheduler.ec
implicit val futureBag = Bag.future(scheduler.ec)
logger.info("****** Shutting down ******")
logger.info("Stopping compaction!")
- val compactionShutdown =
- compactor
- .ask
- .flatMap {
- (impl, state, self) =>
- impl.terminate(state, self)
- }
- .recoverWith {
- case exception =>
- logger.error("Failed compaction shutdown.", exception)
- Future.failed(exception)
- }
-
- val levelsShutdown =
- compactionShutdown flatMap {
- _ =>
- logger.info("Closing files!")
- zero.close onLeftSideEffect {
- error =>
- logger.error("Failed to close files.", error.exception)
- } toFuture
- }
-
- val blockCacheShutdown =
- levelsShutdown flatMap {
- _ =>
- logger.info("Clearing blockCache.")
- blockCache match {
- case Some(value) =>
- Future.successful(value.clear())
-
- case None =>
- Futures.unit
- }
- }
-
- val keyValueSweeperShutdown =
- blockCacheShutdown flatMap {
- _ =>
- keyValueMemorySweeper.foreach {
- sweeper =>
- logger.info("Clearing cached key-values")
- sweeper.terminateAndClear()
- }
- Futures.unit
- }
-
- val fileSweeperShutdown =
- keyValueSweeperShutdown flatMap {
- _ =>
- fileSweeper match {
- case Some(fileSweeper) =>
- logger.info(s"Terminating FileSweeperActor.")
- fileSweeper.terminateAndRecover(retryOnBusyDelay)
-
- case None =>
- Futures.unit
- }
+ compactor
+ .ask
+ .flatMap {
+ (impl, state, self) =>
+ impl.terminate(state, self)
}
-
- val bufferCleanerResult =
- fileSweeperShutdown flatMap {
- _ =>
- logger.info(s"Terminating ByteBufferCleanerActor.")
- cleaner.get() match {
- case Some(actor) =>
- actor.terminateAndRecover(retryOnBusyDelay) flatMap {
- _ =>
- logger.info(s"Terminated ByteBufferCleanerActor. Awaiting shutdown response.")
- actor ask ByteBufferSweeper.Command.IsTerminatedAndCleaned[Unit]
- } flatMap {
- isShut =>
- if (isShut)
- Futures.`true`
- else
- Futures.`false`
- }
-
- case None =>
- Futures.unit
- }
+ .recoverWith {
+ case exception =>
+ logger.error("Failed compaction shutdown.", exception)
+ Future.failed(exception)
}
-
- val releaseLocks =
- bufferCleanerResult flatMap {
- _ =>
- logger.info("Releasing locks.")
- zero.releaseLocks onLeftSideEffect {
- error =>
- logger.error("Failed to release locks.", error.exception)
- } toFuture
- }
-
- releaseLocks flatMap {
- _ =>
- logger.info("Terminating Scheduler.")
- scheduler.terminate()
- Futures.unit
- }
+ .and(zero.close(retryInterval))
}
}
diff --git a/core/src/main/scala/swaydb/core/actor/ByteBufferCleaner.scala b/core/src/main/scala/swaydb/core/actor/ByteBufferCleaner.scala
index bdf379ab2..2bf3f60a7 100644
--- a/core/src/main/scala/swaydb/core/actor/ByteBufferCleaner.scala
+++ b/core/src/main/scala/swaydb/core/actor/ByteBufferCleaner.scala
@@ -88,6 +88,5 @@ private[core] object ByteBufferCleaner extends LazyLogging {
val errorMessage = s"ByteBuffer cleaner not initialised. Failed to clean MMAP file: ${bufferPath.toString}."
val exception = error.exception
logger.error(errorMessage, exception)
- throw new Exception(errorMessage, exception) //also throw to output to stdout in-case logging is not enabled since this is critical.
}
}
diff --git a/core/src/main/scala/swaydb/core/actor/ByteBufferSweeper.scala b/core/src/main/scala/swaydb/core/actor/ByteBufferSweeper.scala
index 287d309f8..3118da262 100644
--- a/core/src/main/scala/swaydb/core/actor/ByteBufferSweeper.scala
+++ b/core/src/main/scala/swaydb/core/actor/ByteBufferSweeper.scala
@@ -36,9 +36,11 @@ import swaydb.core.io.file.Effect
import swaydb.core.util.Counter
import swaydb.core.util.FiniteDurations._
import swaydb.data.config.ActorConfig.QueueOrder
+import swaydb.data.util.Futures
import scala.annotation.tailrec
import scala.collection.mutable
+import scala.concurrent.{Await, ExecutionContext}
import scala.concurrent.duration._
@@ -125,6 +127,59 @@ private[core] object ByteBufferSweeper extends LazyLogging {
case class State(var cleaner: Option[Cleaner],
pendingClean: mutable.HashMap[Path, Counter.Request[Command.Clean]])
+ def closeAsync[BAG[_]](retryOnBusyDelay: FiniteDuration)(implicit sweeper: ByteBufferSweeperActor,
+ bag: Bag.Async[BAG],
+ scheduler: Scheduler): BAG[Unit] =
+ sweeper.get() match {
+ case Some(actor) =>
+ bag.flatMap(actor.terminateAndRecoverAsync(retryOnBusyDelay)) {
+ _ =>
+ val ask = actor ask Command.IsTerminatedAndCleaned(resubmitted = false)
+ bag.transform(ask) {
+ isCleaned =>
+ if (isCleaned)
+ logger.info(s"${ByteBufferSweeper.getClass.getSimpleName.split("\\$").last} terminated!")
+ else
+ logger.error(s"Failed to terminate ${ByteBufferSweeper.getClass.getSimpleName.split("\\$").last}")
+ }
+ }
+
+ case None =>
+ bag.unit
+ }
+
+ /**
+ * Closes the [[ByteBufferSweeperActor]] synchronously. This simply calls the Actor function [[Actor.terminateAndRecoverSync]]
+ * and dispatches [[Command.IsTerminatedAndCleaned]] to assert that all files are closed and flushed.
+ *
+ * @param retryBlock
+ * @param timeout
+ * @param sweeper
+ * @param bag
+ * @param ec
+ * @tparam BAG
+ * @return
+ */
+ def closeSync[BAG[_]](retryBlock: FiniteDuration,
+ timeout: FiniteDuration)(implicit sweeper: ByteBufferSweeperActor,
+ bag: Bag.Sync[BAG],
+ ec: ExecutionContext): BAG[Unit] =
+ sweeper.get() match {
+ case Some(actor) =>
+ bag.map(actor.terminateAndRecoverSync(retryBlock)) {
+ _ =>
+ val future = actor ask Command.IsTerminatedAndCleaned(resubmitted = false)
+ val isCleaned = Await.result(future, timeout)
+ if (isCleaned)
+ logger.info(s"${ByteBufferSweeper.getClass.getSimpleName.split("\\$").last} terminated!")
+ else
+ logger.error(s"Failed to terminate ${ByteBufferSweeper.getClass.getSimpleName.split("\\$").last}")
+ }
+
+ case None =>
+ bag.unit
+ }
+
/**
* Maintains the count of all delete request for each memory-mapped file.
*/
@@ -182,7 +237,6 @@ private[core] object ByteBufferSweeper extends LazyLogging {
val errorMessage = s"Failed to clean MappedByteBuffer at path '${path.toString}'."
val exception = error.exception
logger.error(errorMessage, exception)
- throw IO.throwable(errorMessage, exception) //also throw to output to stdout in-case logging is not enabled since this is critical.
}
case None =>
diff --git a/core/src/main/scala/swaydb/core/actor/FileSweeper.scala b/core/src/main/scala/swaydb/core/actor/FileSweeper.scala
index 8b70ed93d..b742d06a8 100644
--- a/core/src/main/scala/swaydb/core/actor/FileSweeper.scala
+++ b/core/src/main/scala/swaydb/core/actor/FileSweeper.scala
@@ -26,11 +26,11 @@ package swaydb.core.actor
import java.nio.file.Path
import com.typesafe.scalalogging.LazyLogging
-import swaydb.core.actor.ByteBufferSweeper.{ByteBufferSweeperActor, State}
import swaydb.core.cache.{Cache, CacheNoIO}
import swaydb.data.config.{ActorConfig, FileCache}
-import swaydb.{Actor, ActorRef, IO}
+import swaydb.{Actor, ActorRef, Bag, IO, Scheduler}
+import scala.concurrent.duration.FiniteDuration
import scala.ref.WeakReference
private[core] trait FileSweeperItem {
@@ -103,6 +103,21 @@ private[swaydb] object FileSweeper extends LazyLogging {
)
}
+ def closeAsync[BAG[_]](retryOnBusyDelay: FiniteDuration)(implicit fileSweeper: FileSweeperActor,
+ bag: Bag.Async[BAG],
+ scheduler: Scheduler): BAG[Unit] =
+ bag.transform(fileSweeper.terminateAndRecoverAsync(retryOnBusyDelay)) {
+ _ =>
+ logger.info(this.getClass.getSimpleName + " terminated!")
+ }
+
+ def closeSync[BAG[_]](retryOnBusyDelay: FiniteDuration)(implicit fileSweeper: FileSweeperActor,
+ bag: Bag.Sync[BAG]): BAG[Unit] =
+ bag.transform(fileSweeper.terminateAndRecoverSync(retryOnBusyDelay)) {
+ _ =>
+ logger.info(this.getClass.getSimpleName + " terminated!")
+ }
+
private def processCommand(command: Command): Unit =
command match {
case Command.Delete(file) =>
diff --git a/core/src/main/scala/swaydb/core/actor/MemorySweeper.scala b/core/src/main/scala/swaydb/core/actor/MemorySweeper.scala
index 764086ba7..7b4b3e48f 100644
--- a/core/src/main/scala/swaydb/core/actor/MemorySweeper.scala
+++ b/core/src/main/scala/swaydb/core/actor/MemorySweeper.scala
@@ -24,6 +24,7 @@
package swaydb.core.actor
+import com.typesafe.scalalogging.LazyLogging
import swaydb.core.data.{KeyValue, Persistent}
import swaydb.core.io.file.BlockCache
import swaydb.core.util.HashedMap
@@ -61,7 +62,7 @@ private[core] sealed trait MemorySweeper
* Cleared all cached data. [[MemorySweeper]] is not required for Memory only databases
* and zero databases.
*/
-private[core] object MemorySweeper {
+private[core] object MemorySweeper extends LazyLogging {
def apply(memoryCache: MemoryCache): Option[MemorySweeper.Enabled] =
memoryCache match {
@@ -100,6 +101,16 @@ private[core] object MemorySweeper {
)
}
+ def close(keyValueMemorySweeper: Option[MemorySweeper.KeyValue]): Unit =
+ keyValueMemorySweeper.foreach(close)
+
+ def close(sweeper: MemorySweeper.KeyValue): Unit =
+ sweeper.actor foreach {
+ actor =>
+ logger.info("Clearing cached key-values")
+ actor.terminateAndClear()
+ }
+
def weigher(entry: Command): Int =
entry match {
case command: Command.BlockCache =>
diff --git a/core/src/main/scala/swaydb/core/io/file/BlockCache.scala b/core/src/main/scala/swaydb/core/io/file/BlockCache.scala
index 2b5076af0..9c63d608e 100644
--- a/core/src/main/scala/swaydb/core/io/file/BlockCache.scala
+++ b/core/src/main/scala/swaydb/core/io/file/BlockCache.scala
@@ -24,6 +24,7 @@
package swaydb.core.io.file
+import com.typesafe.scalalogging.LazyLogging
import swaydb.core.actor.MemorySweeper
import swaydb.core.util.HashedMap
import swaydb.data.slice.{Slice, SliceOption}
@@ -33,7 +34,7 @@ import scala.annotation.tailrec
/**
* Stores all the read bytes given the configured disk blockSize.
*/
-private[core] object BlockCache {
+private[core] object BlockCache extends LazyLogging {
// var diskSeeks = 0
// var memorySeeks = 0
@@ -88,6 +89,15 @@ private[core] object BlockCache {
map remove key
}
+ def close(blockCache: Option[BlockCache.State]): Unit =
+ blockCache.foreach(close)
+
+ def close(blockCache: BlockCache.State): Unit = {
+ logger.info("Cleared BlockCache")
+ blockCache.clear()
+ blockCache.sweeper.terminateAndClear()
+ }
+
def seekSize(keyPosition: Int,
size: Int,
file: DBFileType,
diff --git a/core/src/main/scala/swaydb/core/level/Level.scala b/core/src/main/scala/swaydb/core/level/Level.scala
index 0df87a976..180f8fce6 100644
--- a/core/src/main/scala/swaydb/core/level/Level.scala
+++ b/core/src/main/scala/swaydb/core/level/Level.scala
@@ -30,7 +30,7 @@ import java.nio.file.{Path, StandardOpenOption}
import com.typesafe.scalalogging.LazyLogging
import swaydb.Error.Level.ExceptionHandler
import swaydb.IO._
-import swaydb.core.actor.{FileSweeper, MemorySweeper}
+import swaydb.core.actor.{ByteBufferSweeper, FileSweeper, MemorySweeper}
import swaydb.core.data.{KeyValue, _}
import swaydb.core.function.FunctionStore
import swaydb.core.actor.ByteBufferSweeper.ByteBufferSweeperActor
@@ -56,11 +56,12 @@ import swaydb.data.order.{KeyOrder, TimeOrder}
import swaydb.data.slice.Slice._
import swaydb.data.slice.{Slice, SliceOption}
import swaydb.data.storage.{AppendixStorage, LevelStorage}
-import swaydb.{Error, IO}
+import swaydb.data.util.Futures.FutureImplicits
+import swaydb.{Bag, Error, IO, Scheduler}
import scala.collection.mutable
import scala.collection.mutable.ListBuffer
-import scala.concurrent.Promise
+import scala.concurrent.{Await, ExecutionContext, Future, Promise}
import scala.concurrent.duration._
import scala.jdk.CollectionConverters._
@@ -315,29 +316,6 @@ private[core] object Level extends LazyLogging {
segmentsToCollapse
}
-
- def delete(level: NextLevel): IO[swaydb.Error.Delete, Unit] =
- level
- .close
- .flatMap {
- _ =>
- import swaydb.Error.Delete.ExceptionHandler
-
- level
- .nextLevel
- .map(_.delete)
- .getOrElse(IO.unit)
- .and {
- level
- .pathDistributor
- .dirs
- .foreachIO {
- path =>
- IO(Effect.walkDelete(path.path))
- }
- .getOrElse(IO.unit)
- }
- }
}
private[core] case class Level(dirs: Seq[Dir],
@@ -358,7 +336,7 @@ private[core] case class Level(dirs: Seq[Dir],
functionStore: FunctionStore,
removeWriter: MapEntryWriter[MapEntry.Remove[Slice[Byte]]],
addWriter: MapEntryWriter[MapEntry.Put[Slice[Byte], Segment]],
- keyValueMemorySweeper: Option[MemorySweeper.KeyValue],
+ val keyValueMemorySweeper: Option[MemorySweeper.KeyValue],
val fileSweeper: FileSweeperActor,
val bufferCleaner: ByteBufferSweeperActor,
val blockCache: Option[BlockCache.State],
@@ -440,12 +418,6 @@ private[core] case class Level(dirs: Seq[Dir],
def appendixPath: Path =
rootPath.resolve("appendix")
- def releaseLocks: IO[swaydb.Error.Close, Unit] =
- IO[swaydb.Error.Close, Unit](Effect.release(lock)) flatMap {
- _ =>
- nextLevel.map(_.releaseLocks) getOrElse IO.unit
- }
-
def nextPushDelay: FiniteDuration =
throttle(meter).pushDelay
@@ -1556,31 +1528,76 @@ private[core] case class Level(dirs: Seq[Dir],
.getNearestDeadlineSegment(segmentsInLevel())
.isSomeS
- def close: IO[swaydb.Error.Close, Unit] =
- nextLevel
- .map(_.close)
- .getOrElse(IO.unit)
+ override val isTrash: Boolean =
+ false
+
+ override def isZero: Boolean =
+ false
+
+ def lastSegmentId: Option[Long] =
+ appendix
+ .last()
+ .mapS(_.segmentId)
+
+ override def stateId: Long =
+ segmentIDGenerator.currentId
+
+ override def nextCompactionDelay: FiniteDuration =
+ throttle(meter).pushDelay
+
+ override def nextThrottlePushCount: Int =
+ throttle(meter).segmentsToPush
+
+
+ override def minSegmentSize: Int =
+ segmentConfig.minSize
+
+ /**
+ * Closing and delete functions
+ */
+
+ def releaseLocks: IO[swaydb.Error.Close, Unit] =
+ IO[swaydb.Error.Close, Unit](Effect.release(lock))
.flatMap {
_ =>
- IO(appendix.close())
- .onLeftSideEffect {
- failure =>
- logger.error("{}: Failed to close appendix", pathDistributor.head, failure)
- }
-
- closeSegments()
- .onLeftSideEffect {
- failure =>
- logger.error("{}: Failed to close segments", pathDistributor.head, failure)
- }
+ nextLevel.map(_.releaseLocks) getOrElse IO.unit
+ }
+ .onLeftSideEffect {
+ failure =>
+ logger.error("{}: Failed to release locks", pathDistributor.head, failure)
+ }
- releaseLocks
- .onLeftSideEffect {
- failure =>
- logger.error("{}: Failed to release locks", pathDistributor.head, failure)
- }
+ def closeAppendix(): IO[Error.Level, Unit] =
+ IO(appendix.close())
+ .onLeftSideEffect {
+ failure =>
+ logger.error("{}: Failed to close appendix", pathDistributor.head, failure)
}
+ private def closeSweepers(retryInterval: FiniteDuration)(implicit executionContext: ExecutionContext): Future[Unit] = {
+ implicit val scheduler = Scheduler()
+ val result = LevelCloser.closeAsync[Future](retryInterval)
+
+ result.onComplete {
+ _ =>
+ scheduler.terminate()
+ }
+
+ result
+ }
+
+ def close(retryInterval: FiniteDuration)(implicit executionContext: ExecutionContext): Future[Unit] =
+ closeSweepers(retryInterval)
+ .and(closeNoSweep())
+
+ def closeNoSweep(): IO[swaydb.Error.Level, Unit] =
+ nextLevel
+ .map(_.closeNoSweep)
+ .getOrElse(IO.unit)
+ .and(closeAppendix())
+ .and(closeSegments())
+ .and(releaseLocks)
+
def closeSegments(): IO[swaydb.Error.Level, Unit] = {
segmentsInLevel()
.foreachIO(segment => IO(segment.close), failFast = false)
@@ -1598,29 +1615,28 @@ private[core] case class Level(dirs: Seq[Dir],
}
}
- override val isTrash: Boolean =
- false
-
- override def isZero: Boolean =
- false
-
- def lastSegmentId: Option[Long] =
- appendix
- .last()
- .mapS(_.segmentId)
-
- override def stateId: Long =
- segmentIDGenerator.currentId
-
- override def nextCompactionDelay: FiniteDuration =
- throttle(meter).pushDelay
+ private def deleteNextLevelNoSweep() =
+ nextLevel
+ .map(_.deleteNoSweep)
+ .getOrElse(IO.unit)
- override def nextThrottlePushCount: Int =
- throttle(meter).segmentsToPush
+ private def deleteFiles() =
+ pathDistributor
+ .dirs
+ .foreachIO {
+ path =>
+ IO(Effect.walkDelete(path.path))
+ }
+ .getOrElse(IO.unit)
- override def delete: IO[swaydb.Error.Delete, Unit] =
- Level.delete(self)
+ override def deleteNoSweep: IO[swaydb.Error.Level, Unit] =
+ closeNoSweep()
+ .and(deleteNextLevelNoSweep())
+ .and(deleteFiles())
- override def minSegmentSize: Int =
- segmentConfig.minSize
+ override def delete(retryInterval: FiniteDuration)(implicit executionContext: ExecutionContext): Future[Unit] =
+ close(retryInterval)
+ .and(closeNoSweep())
+ .and(deleteNextLevelNoSweep())
+ .and(deleteFiles())
}
diff --git a/core/src/main/scala/swaydb/core/level/LevelCloser.scala b/core/src/main/scala/swaydb/core/level/LevelCloser.scala
new file mode 100644
index 000000000..f18342ef8
--- /dev/null
+++ b/core/src/main/scala/swaydb/core/level/LevelCloser.scala
@@ -0,0 +1,72 @@
+/*
+ * Copyright (c) 2020 Simer JS Plaha (simer.j@gmail.com - @simerplaha)
+ *
+ * This file is a part of SwayDB.
+ *
+ * SwayDB is free software: you can redistribute it and/or modify
+ * it under the terms of the GNU Affero General Public License as
+ * published by the Free Software Foundation, either version 3 of the
+ * License, or (at your option) any later version.
+ *
+ * SwayDB is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ * GNU Affero General Public License for more details.
+ *
+ * You should have received a copy of the GNU Affero General Public License
+ * along with SwayDB. If not, see .
+ *
+ * Additional permission under the GNU Affero GPL version 3 section 7:
+ * If you modify this Program, or any covered work, by linking or combining
+ * it with other code, such other code is not for that reason alone subject
+ * to any of the requirements of the GNU Affero GPL version 3.
+ */
+
+package swaydb.core.level
+
+import com.typesafe.scalalogging.LazyLogging
+import swaydb.core.actor.ByteBufferSweeper.ByteBufferSweeperActor
+import swaydb.core.actor.FileSweeper.FileSweeperActor
+import swaydb.core.actor.{ByteBufferSweeper, FileSweeper, MemorySweeper}
+import swaydb.core.io.file.BlockCache
+import swaydb.{Bag, Scheduler}
+
+import scala.concurrent.ExecutionContext
+import scala.concurrent.duration.FiniteDuration
+
+object LevelCloser extends LazyLogging {
+
+ def closeAsync[BAG[_]](retryOnBusyDelay: FiniteDuration)(implicit keyValueMemorySweeper: Option[MemorySweeper.KeyValue],
+ blockCache: Option[BlockCache.State],
+ fileSweeper: FileSweeperActor,
+ bufferCleaner: ByteBufferSweeperActor,
+ bag: Bag.Async[BAG],
+ scheduler: Scheduler) = {
+
+ MemorySweeper.close(keyValueMemorySweeper)
+ BlockCache.close(blockCache)
+
+ bag.flatMap(FileSweeper.closeAsync(retryOnBusyDelay)) {
+ _ =>
+ ByteBufferSweeper.closeAsync(retryOnBusyDelay)
+ }
+ }
+
+ def closeSync[BAG[_]](retryDelays: FiniteDuration,
+ timeout: FiniteDuration)(implicit keyValueMemorySweeper: Option[MemorySweeper.KeyValue],
+ blockCache: Option[BlockCache.State],
+ fileSweeper: FileSweeperActor,
+ bufferCleaner: ByteBufferSweeperActor,
+ ec: ExecutionContext,
+ bag: Bag.Sync[BAG]) = {
+
+ MemorySweeper.close(keyValueMemorySweeper)
+ BlockCache.close(blockCache)
+
+ bag.flatMap(FileSweeper.closeSync(retryDelays)) {
+ _ =>
+ ByteBufferSweeper.closeSync(retryDelays, timeout)
+ }
+ }
+
+}
diff --git a/core/src/main/scala/swaydb/core/level/LevelRef.scala b/core/src/main/scala/swaydb/core/level/LevelRef.scala
index 686bfda9f..2f00a28a7 100644
--- a/core/src/main/scala/swaydb/core/level/LevelRef.scala
+++ b/core/src/main/scala/swaydb/core/level/LevelRef.scala
@@ -36,6 +36,7 @@ import swaydb.data.slice.{Slice, SliceOption}
import scala.annotation.tailrec
import scala.collection.mutable.ListBuffer
+import scala.concurrent.{ExecutionContext, Future}
import scala.concurrent.duration.FiniteDuration
object LevelRef {
@@ -247,7 +248,9 @@ private[core] trait LevelRef {
def sizeOfSegments: Long
- def close: IO[swaydb.Error.Close, Unit]
+ def close(retryInterval: FiniteDuration)(implicit executionContext: ExecutionContext): Future[Unit]
+
+ def closeNoSweep: IO[swaydb.Error.Level, Unit]
def closeSegments(): IO[swaydb.Error.Level, Unit]
@@ -263,5 +266,7 @@ private[core] trait LevelRef {
def nextCompactionDelay: FiniteDuration
- def delete: IO[swaydb.Error.Delete, Unit]
+ def delete(retryInterval: FiniteDuration)(implicit executionContext: ExecutionContext): Future[Unit]
+
+ def deleteNoSweep: IO[swaydb.Error.Level, Unit]
}
\ No newline at end of file
diff --git a/core/src/main/scala/swaydb/core/level/NextLevel.scala b/core/src/main/scala/swaydb/core/level/NextLevel.scala
index 8e1ba4019..0b976c7bd 100644
--- a/core/src/main/scala/swaydb/core/level/NextLevel.scala
+++ b/core/src/main/scala/swaydb/core/level/NextLevel.scala
@@ -123,5 +123,5 @@ trait NextLevel extends LevelRef {
def nextThrottlePushCount: Int
- def delete: IO[swaydb.Error.Delete, Unit]
+ def deleteNoSweep: IO[swaydb.Error.Level, Unit]
}
diff --git a/core/src/main/scala/swaydb/core/level/TrashLevel.scala b/core/src/main/scala/swaydb/core/level/TrashLevel.scala
index b530e6a3f..c8ad7a7d7 100644
--- a/core/src/main/scala/swaydb/core/level/TrashLevel.scala
+++ b/core/src/main/scala/swaydb/core/level/TrashLevel.scala
@@ -28,12 +28,13 @@ import java.nio.file.{Path, Paths}
import swaydb.Error.Segment.ExceptionHandler
import swaydb.core.data.{KeyValue, Memory, MemoryOption}
-import swaydb.core.segment.{ThreadReadState, Segment, SegmentOption}
+import swaydb.core.segment.{Segment, SegmentOption, ThreadReadState}
import swaydb.data.compaction.{LevelMeter, Throttle}
import swaydb.data.slice.{Slice, SliceOption}
+import swaydb.data.util.Futures
import swaydb.{Error, IO}
-import scala.concurrent.Promise
+import scala.concurrent.{ExecutionContext, Future, Promise}
import scala.concurrent.duration._
private[core] object TrashLevel extends NextLevel {
@@ -116,7 +117,7 @@ private[core] object TrashLevel extends NextLevel {
override def releaseLocks: IO[swaydb.Error.Close, Unit] =
IO.unit
- override val close: IO[swaydb.Error.Close, Unit] =
+ override val closeNoSweep: IO[swaydb.Error.Close, Unit] =
IO.unit
override def meterFor(levelNumber: Int): Option[LevelMeter] =
@@ -216,5 +217,11 @@ private[core] object TrashLevel extends NextLevel {
override def isCopyable(minKey: Slice[Byte], maxKey: Slice[Byte], maxKeyInclusive: Boolean): Boolean =
true
- override def delete: IO[swaydb.Error.Delete, Unit] = IO.unit
+ override def deleteNoSweep: IO[swaydb.Error.Delete, Unit] = IO.unit
+
+ override def close(retryInterval: FiniteDuration)(implicit executionContext: ExecutionContext): Future[Unit] =
+ Futures.unit
+
+ override def delete(retryInterval: FiniteDuration)(implicit executionContext: ExecutionContext): Future[Unit] =
+ Futures.unit
}
diff --git a/core/src/main/scala/swaydb/core/level/zero/LevelZero.scala b/core/src/main/scala/swaydb/core/level/zero/LevelZero.scala
index df07f0a91..d356c4dca 100644
--- a/core/src/main/scala/swaydb/core/level/zero/LevelZero.scala
+++ b/core/src/main/scala/swaydb/core/level/zero/LevelZero.scala
@@ -51,9 +51,12 @@ import swaydb.data.compaction.LevelMeter
import swaydb.data.order.{KeyOrder, TimeOrder}
import swaydb.data.slice.{Slice, SliceOption}
import swaydb.data.storage.Level0Storage
+import swaydb.data.util.Futures
+import swaydb.data.util.Futures.FutureImplicits
import swaydb.data.util.StorageUnits._
-import swaydb.{Actor, Bag, IO, OK}
+import swaydb.{Actor, Bag, Error, IO, OK}
+import scala.concurrent.{ExecutionContext, Future}
import scala.concurrent.duration.{Deadline, _}
import scala.jdk.CollectionConverters._
@@ -77,9 +80,6 @@ private[core] object LevelZero extends LazyLogging {
else
logger.info("cacheKeyValueIds is false. Key-value IDs cache disabled!")
- //LevelZero does not required FileSweeper since they are all Map files.
- implicit val fileSweeper: FileSweeperActor = Actor.deadActor()
-
implicit val skipListMerger: SkipListMerger[SliceOption[Byte], MemoryOption, Slice[Byte], Memory] = LevelZeroSkipListMerger
val mapsAndPathAndLock =
storage match {
@@ -112,6 +112,8 @@ private[core] object LevelZero extends LazyLogging {
Effect createFileIfAbsent lockFile
IO(FileChannel.open(lockFile, StandardOpenOption.WRITE).tryLock()) flatMap {
lock =>
+ //LevelZero does not required FileSweeper since they are all Map files.
+ implicit val fileSweeper: FileSweeperActor = Actor.deadActor()
logger.info("{}: Recovering Maps.", path)
Maps.persistent[SliceOption[Byte], MemoryOption, Slice[Byte], Memory](
path = path,
@@ -155,6 +157,9 @@ private[core] object LevelZero extends LazyLogging {
timer map {
implicit timer =>
+ //LevelZero does not required FileSweeper since they are all Map files.
+ implicit val fileSweeper: FileSweeperActor = Actor.deadActor()
+
val map =
Maps.memory[SliceOption[Byte], MemoryOption, Slice[Byte], Memory](
fileSize = mapSize,
@@ -180,18 +185,6 @@ private[core] object LevelZero extends LazyLogging {
)
}
}
-
- def delete(zero: LevelZero): IO[swaydb.Error.Delete, Unit] =
- zero
- .close
- .flatMap {
- _ =>
- zero
- .nextLevel
- .map(_.delete)
- .getOrElse(IO.unit)
- .and(IO[swaydb.Error.Delete, Unit](Effect.walkDelete(zero.path.getParent)))
- }
}
private[swaydb] case class LevelZero(path: Path,
@@ -832,27 +825,6 @@ private[swaydb] case class LevelZero(path: Path,
def existsOnDisk: Boolean =
Effect.exists(path)
- def close: IO[swaydb.Error.Close, Unit] = {
- // Delay.cancelTimer()
- maps
- .close
- .onLeftSideEffect {
- exception =>
- logger.error(s"$path: Failed to close maps", exception)
- }
-
- releaseLocks
-
- nextLevel
- .map(_.close)
- .getOrElse(IO.unit)
- }
-
- def closeSegments: IO[swaydb.Error.Level, Unit] =
- nextLevel
- .map(_.closeSegments())
- .getOrElse(IO.unit)
-
def mightContainKey(key: Slice[Byte]): Boolean =
maps.contains(key) ||
nextLevel.exists(_.mightContainKey(key))
@@ -953,9 +925,6 @@ private[swaydb] case class LevelZero(path: Path,
override def nextCompactionDelay: FiniteDuration =
throttle(levelZeroMeter)
- override def delete: IO[swaydb.Error.Delete, Unit] =
- LevelZero.delete(this)
-
def iterator(state: ThreadReadState): Iterator[PutOption] =
new Iterator[PutOption] {
var nextKeyValue: PutOption = _
@@ -990,6 +959,55 @@ private[swaydb] case class LevelZero(path: Path,
nextKeyValue
}
+ private def closeMaps: IO[Error.Map, Unit] =
+ maps
+ .close
+ .onLeftSideEffect {
+ exception =>
+ logger.error(s"$path: Failed to close maps", exception)
+ }
+
+ private def closeNextLevelNoSweep: IO[Error.Level, Unit] =
+ nextLevel
+ .map(_.closeNoSweep)
+ .getOrElse(IO.unit)
+
+ def closeNoSweep: IO[swaydb.Error.Level, Unit] =
+ closeMaps
+ .and(closeNextLevelNoSweep)
+ .and(releaseLocks)
+
+ private def closeNextLevel(retryInterval: FiniteDuration)(implicit executionContext: ExecutionContext): Future[Unit] =
+ nextLevel
+ .map(_.close(retryInterval))
+ .getOrElse(Futures.unit)
+
+ override def close(retryInterval: FiniteDuration)(implicit executionContext: ExecutionContext): Future[Unit] =
+ closeMaps
+ .toFuture
+ .and(closeNextLevel(retryInterval))
+ .and(releaseLocks)
+
+ def closeSegments: IO[swaydb.Error.Level, Unit] =
+ nextLevel
+ .map(_.closeSegments())
+ .getOrElse(IO.unit)
+
+ private def deleteNextLevelNoSweep: IO[Error.Level, Unit] =
+ nextLevel
+ .map(_.deleteNoSweep)
+ .getOrElse(IO.unit)
+
+ override def deleteNoSweep: IO[swaydb.Error.Level, Unit] =
+ closeNoSweep
+ .and(deleteNextLevelNoSweep)
+ .and(IO(Effect.walkDelete(path.getParent)))
+
+ override def delete(retryInterval: FiniteDuration)(implicit executionContext: ExecutionContext): Future[Unit] =
+ close(retryInterval)
+ .and(deleteNextLevelNoSweep)
+ .and(IO(Effect.walkDelete(path.getParent)))
+
final def run[R, BAG[_]](apply: LevelZero => R)(implicit bag: Bag[BAG]): BAG[R] =
bag.suspend {
try
diff --git a/core/src/test/scala/swaydb/core/CommonAssertions.scala b/core/src/test/scala/swaydb/core/CommonAssertions.scala
index 57affd29b..7384783db 100644
--- a/core/src/test/scala/swaydb/core/CommonAssertions.scala
+++ b/core/src/test/scala/swaydb/core/CommonAssertions.scala
@@ -1774,7 +1774,7 @@ object CommonAssertions {
*/
def ensureClose(): Unit = {
maps.close.value
- maps.bufferCleaner.actor.receiveAllBlocking(Int.MaxValue).get
+ maps.bufferCleaner.actor.receiveAllBlocking(Int.MaxValue, 1.second).get
(maps.bufferCleaner.actor ask ByteBufferSweeper.Command.IsTerminatedAndCleaned[Unit]).await(10.seconds)
}
}
diff --git a/core/src/test/scala/swaydb/core/TestBase.scala b/core/src/test/scala/swaydb/core/TestBase.scala
index e7ae66ea9..3ad52cd34 100644
--- a/core/src/test/scala/swaydb/core/TestBase.scala
+++ b/core/src/test/scala/swaydb/core/TestBase.scala
@@ -66,7 +66,9 @@ import swaydb.data.slice.{Slice, SliceOption}
import swaydb.data.storage.{AppendixStorage, Level0Storage, LevelStorage}
import swaydb.data.util.{Futures, OperatingSystem}
import swaydb.data.util.StorageUnits._
+import TestCaseSweeper._
+import scala.concurrent.ExecutionContext
import scala.concurrent.duration._
import scala.util.Random
@@ -216,7 +218,8 @@ trait TestBase extends AnyWordSpec with Matchers with BeforeAndAfterAll with Eve
keyValueMemorySweeper: Option[MemorySweeper.KeyValue] = TestSweeper.memorySweeperMax,
fileSweeper: FileSweeperActor = TestSweeper.fileSweeper,
cleaner: ByteBufferSweeperActor = TestSweeper.bufferCleaner,
- timeOrder: TimeOrder[Slice[Byte]] = TimeOrder.long): map.Map[SliceOption[Byte], MemoryOption, Slice[Byte], Memory] = {
+ timeOrder: TimeOrder[Slice[Byte]] = TimeOrder.long,
+ sweeper: TestCaseSweeper = null): map.Map[SliceOption[Byte], MemoryOption, Slice[Byte], Memory] = {
import swaydb.core.map.serializer.LevelZeroMapEntryReader._
import swaydb.core.map.serializer.LevelZeroMapEntryWriter._
implicit val merger = swaydb.core.level.zero.LevelZeroSkipListMerger
@@ -243,6 +246,10 @@ trait TestBase extends AnyWordSpec with Matchers with BeforeAndAfterAll with Eve
keyValue =>
testMap.writeSync(MapEntry.Put(keyValue.key, keyValue))
}
+
+ if(sweeper != null)
+ testMap.clean()
+
testMap
}
}
@@ -261,7 +268,8 @@ trait TestBase extends AnyWordSpec with Matchers with BeforeAndAfterAll with Eve
fileSweeper: FileSweeperActor = TestSweeper.fileSweeper,
cleaner: ByteBufferSweeperActor = TestSweeper.bufferCleaner,
timeOrder: TimeOrder[Slice[Byte]] = TimeOrder.long,
- blockCache: Option[BlockCache.State] = TestSweeper.randomBlockCache): Segment = {
+ blockCache: Option[BlockCache.State] = TestSweeper.randomBlockCache,
+ sweeper: TestCaseSweeper = null): Segment = {
val segmentId = Effect.fileId(path)._1 - 1
@@ -300,7 +308,8 @@ trait TestBase extends AnyWordSpec with Matchers with BeforeAndAfterAll with Eve
timeOrder: TimeOrder[Slice[Byte]] = TimeOrder.long,
pathsDistributor: PathsDistributor,
idGenerator: IDGenerator,
- blockCache: Option[BlockCache.State] = TestSweeper.randomBlockCache): Slice[Segment] = {
+ blockCache: Option[BlockCache.State] = TestSweeper.randomBlockCache,
+ sweeper: TestCaseSweeper = null): Slice[Segment] = {
implicit val segmentIO: SegmentIO =
SegmentIO(
@@ -312,26 +321,32 @@ trait TestBase extends AnyWordSpec with Matchers with BeforeAndAfterAll with Eve
segmentConfig = segmentConfig
)
- if (levelStorage.memory)
- Segment.memory(
- minSegmentSize = segmentConfig.minSize,
- maxKeyValueCountPerSegment = segmentConfig.maxCount,
- pathsDistributor = pathsDistributor,
- createdInLevel = createdInLevel,
- keyValues = MergeStats.memoryBuilder(keyValues).close
- )
- else
- Segment.persistent(
- pathsDistributor = pathsDistributor,
- createdInLevel = createdInLevel,
- bloomFilterConfig = bloomFilterConfig,
- hashIndexConfig = hashIndexConfig,
- binarySearchIndexConfig = binarySearchIndexConfig,
- sortedIndexConfig = sortedIndexConfig,
- valuesConfig = valuesConfig,
- segmentConfig = segmentConfig,
- mergeStats = MergeStats.persistentBuilder(keyValues).close(sortedIndexConfig.enableAccessPositionIndex)
- )
+ val segment =
+ if (levelStorage.memory)
+ Segment.memory(
+ minSegmentSize = segmentConfig.minSize,
+ maxKeyValueCountPerSegment = segmentConfig.maxCount,
+ pathsDistributor = pathsDistributor,
+ createdInLevel = createdInLevel,
+ keyValues = MergeStats.memoryBuilder(keyValues).close
+ )
+ else
+ Segment.persistent(
+ pathsDistributor = pathsDistributor,
+ createdInLevel = createdInLevel,
+ bloomFilterConfig = bloomFilterConfig,
+ hashIndexConfig = hashIndexConfig,
+ binarySearchIndexConfig = binarySearchIndexConfig,
+ sortedIndexConfig = sortedIndexConfig,
+ valuesConfig = valuesConfig,
+ segmentConfig = segmentConfig,
+ mergeStats = MergeStats.persistentBuilder(keyValues).close(sortedIndexConfig.enableAccessPositionIndex)
+ )
+
+ if (sweeper != null)
+ segment.foreach(_.clean())
+
+ segment
}
}
@@ -361,11 +376,13 @@ trait TestBase extends AnyWordSpec with Matchers with BeforeAndAfterAll with Eve
bloomFilterConfig: BloomFilterBlock.Config = BloomFilterBlock.Config.random,
segmentConfig: SegmentBlock.Config = SegmentBlock.Config.random2(pushForward = false, deleteEventually = false, mmap = mmapSegments),
keyValues: Slice[Memory] = Slice.empty)(implicit keyOrder: KeyOrder[Slice[Byte]] = KeyOrder.default,
- keyValueMemorySweeper: Option[MemorySweeper.KeyValue] = TestSweeper.memorySweeperMax,
- fileSweeper: FileSweeperActor = TestSweeper.fileSweeper,
- cleaner: ByteBufferSweeperActor = TestSweeper.bufferCleaner,
- blockCache: Option[BlockCache.State] = TestSweeper.randomBlockCache,
- timeOrder: TimeOrder[Slice[Byte]] = TimeOrder.long): Level =
+ timeOrder: TimeOrder[Slice[Byte]] = TimeOrder.long,
+ levelSweeper: TestCaseSweeper): Level = {
+ implicit val keyValueMemorySweeper: Option[MemorySweeper.KeyValue] = levelSweeper.keyValueMemorySweeper
+ implicit val fileSweeper: FileSweeperActor = levelSweeper.fileSweeper
+ implicit val cleaner: ByteBufferSweeperActor = levelSweeper.cleaner
+ implicit val blockCache: Option[BlockCache.State] = levelSweeper.blockCache
+
Level(
levelStorage = levelStorage,
appendixStorage = appendixStorage,
@@ -383,7 +400,8 @@ trait TestBase extends AnyWordSpec with Matchers with BeforeAndAfterAll with Eve
_ =>
level
}
- }.right.value
+ }.right.value.clean()
+ }
}
object TestLevelZero {
@@ -392,10 +410,10 @@ trait TestBase extends AnyWordSpec with Matchers with BeforeAndAfterAll with Eve
mapSize: Long = randomIntMax(10.mb),
brake: LevelZeroMeter => Accelerator = Accelerator.brake(),
throttle: LevelZeroMeter => FiniteDuration = _ => Duration.Zero)(implicit keyOrder: KeyOrder[Slice[Byte]] = KeyOrder.default,
- keyValueMemorySweeper: Option[MemorySweeper.KeyValue] = TestSweeper.memorySweeperMax,
timeOrder: TimeOrder[Slice[Byte]] = TimeOrder.long,
- fileSweeper: FileSweeperActor = TestSweeper.fileSweeper,
- cleaner: ByteBufferSweeperActor = TestSweeper.bufferCleaner): LevelZero =
+ sweeper: TestCaseSweeper): LevelZero = {
+ implicit val cleaner: ByteBufferSweeperActor = sweeper.cleaner
+
LevelZero(
mapSize = mapSize,
storage = level0Storage,
@@ -404,7 +422,8 @@ trait TestBase extends AnyWordSpec with Matchers with BeforeAndAfterAll with Eve
cacheKeyValueIds = randomBoolean(),
throttle = throttle,
acceleration = brake
- ).value
+ ).value.clean()
+ }
}
def createFile(bytes: Slice[Byte]): Path =
@@ -532,7 +551,8 @@ trait TestBase extends AnyWordSpec with Matchers with BeforeAndAfterAll with Eve
level2KeyValues: TestTimer => Slice[Memory] = _ => Slice.empty,
assertLevel2: (Slice[Memory], LevelRef) => Unit = (_, _) => (),
assertAllLevels: (Slice[Memory], Slice[Memory], Slice[Memory], LevelRef) => Unit = (_, _, _, _) => (),
- throttleOn: Boolean = false)(implicit keyOrder: KeyOrder[Slice[Byte]] = KeyOrder.default): Unit = {
+ throttleOn: Boolean = false)(implicit keyOrder: KeyOrder[Slice[Byte]] = KeyOrder.default,
+ ec: ExecutionContext = TestExecutionContext.executionContext): Unit = {
def iterationMessage =
s"Thread: ${Thread.currentThread().getId} - throttleOn: $throttleOn"
@@ -556,6 +576,8 @@ trait TestBase extends AnyWordSpec with Matchers with BeforeAndAfterAll with Eve
println("Starting levels")
+ implicit val levelSweeper: TestCaseSweeper = TestCaseSweeper()
+
val level4 = TestLevel(throttle = levelThrottle)
val level3 = TestLevel(nextLevel = Some(level4), throttle = levelThrottle)
val level2 = TestLevel(nextLevel = Some(level3), throttle = levelThrottle)
@@ -681,7 +703,7 @@ trait TestBase extends AnyWordSpec with Matchers with BeforeAndAfterAll with Eve
runAsserts(asserts)
- level0.delete.runRandomIO.right.value
+ level0.deleteNoSweep.runRandomIO.right.value
val terminate =
compaction map {
@@ -721,7 +743,8 @@ trait TestBase extends AnyWordSpec with Matchers with BeforeAndAfterAll with Eve
assertLevel3: LevelRef => Unit,
level3: Level,
assertAllLevels: LevelRef => Unit,
- assertLevel3ForAllLevels: Boolean)(implicit keyOrder: KeyOrder[Slice[Byte]] = KeyOrder.default): Unit = {
+ assertLevel3ForAllLevels: Boolean)(implicit keyOrder: KeyOrder[Slice[Byte]] = KeyOrder.default,
+ levelSweeper: TestCaseSweeper): Unit = {
println("level3.putKeyValues")
if (level3KeyValues.nonEmpty) level3.putKeyValuesTest(level3KeyValues).runRandomIO.right.value
println("level2.putKeyValues")
@@ -784,6 +807,7 @@ trait TestBase extends AnyWordSpec with Matchers with BeforeAndAfterAll with Eve
hashIndexConfig: HashIndexBlock.Config = HashIndexBlock.Config.random,
bloomFilterConfig: BloomFilterBlock.Config = BloomFilterBlock.Config.random)(implicit keyOrder: KeyOrder[Slice[Byte]] = KeyOrder.default,
keyValueMemorySweeper: Option[MemorySweeper.KeyValue] = TestSweeper.memorySweeperMax,
+ ec: ExecutionContext = TestExecutionContext.executionContext,
segmentIO: SegmentIO = SegmentIO.random) = {
println(s"assertSegment - keyValues: ${keyValues.size}")
diff --git a/core/src/test/scala/swaydb/core/TestCaseSweeper.scala b/core/src/test/scala/swaydb/core/TestCaseSweeper.scala
new file mode 100644
index 000000000..8bd9cf14d
--- /dev/null
+++ b/core/src/test/scala/swaydb/core/TestCaseSweeper.scala
@@ -0,0 +1,186 @@
+/*
+ * Copyright (c) 2020 Simer JS Plaha (simer.j@gmail.com - @simerplaha)
+ *
+ * This file is a part of SwayDB.
+ *
+ * SwayDB is free software: you can redistribute it and/or modify
+ * it under the terms of the GNU Affero General Public License as
+ * published by the Free Software Foundation, either version 3 of the
+ * License, or (at your option) any later version.
+ *
+ * SwayDB is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ * GNU Affero General Public License for more details.
+ *
+ * You should have received a copy of the GNU Affero General Public License
+ * along with SwayDB. If not, see .
+ *
+ * Additional permission under the GNU Affero GPL version 3 section 7:
+ * If you modify this Program, or any covered work, by linking or combining
+ * it with other code, such other code is not for that reason alone subject
+ * to any of the requirements of the GNU Affero GPL version 3.
+ */
+
+package swaydb.core
+
+import java.nio.file.Path
+
+import com.typesafe.scalalogging.LazyLogging
+import swaydb.{Bag, IO, OK}
+import swaydb.core.RunThis._
+import swaydb.core.TestSweeper._
+import swaydb.core.actor.ByteBufferSweeper.ByteBufferSweeperActor
+import swaydb.core.actor.FileSweeper.FileSweeperActor
+import swaydb.core.actor.{ByteBufferSweeper, FileSweeper, MemorySweeper}
+import swaydb.core.cache.{Cache, CacheNoIO}
+import swaydb.core.io.file.{BlockCache, Effect}
+import swaydb.core.level.LevelRef
+import swaydb.core.segment.Segment
+
+import scala.collection.mutable.ListBuffer
+import scala.concurrent.Future
+import scala.concurrent.duration.DurationInt
+import scala.util.Try
+
+/**
+ * Manages cleaning levels, segments, maps etc for Levels. Initialises Actors lazily as required and
+ * destroys them after the test is complete.
+ *
+ * {{{
+ * TestLevelSweeper {
+ * implicit sweeper =>
+ * // test code here amd all levels, segments creates
+ * // here will get deleted after the test is complete.
+ * }
+ * }}}
+ */
+
+object TestCaseSweeper extends LazyLogging {
+
+ def apply(): TestCaseSweeper = {
+ new TestCaseSweeper(
+ keyValueMemorySweepers = ListBuffer(Cache.noIO[Unit, Option[MemorySweeper.KeyValue]](true, true, None)((_, _) => createMemorySweeperRandom())),
+ fileSweepers = ListBuffer(Cache.noIO[Unit, FileSweeperActor](true, true, None)((_, _) => createFileSweeper())),
+ cleaners = ListBuffer(Cache.noIO[Unit, ByteBufferSweeperActor](true, true, None)((_, _) => createBufferCleaner())),
+ blockCaches = ListBuffer(Cache.noIO[Unit, Option[BlockCache.State]](true, true, None)((_, _) => createBlockCacheRandom())),
+ levels = ListBuffer.empty,
+ segments = ListBuffer.empty,
+ maps = ListBuffer.empty,
+ paths = ListBuffer.empty
+ )
+ }
+
+ def deleteParentPath(path: Path) = {
+ val parentPath = path.getParent
+ //also delete parent folder of Segment. TestSegments are created with a parent folder.
+ if (Effect.exists(parentPath) && Try(parentPath.getFileName.toString.toInt).isSuccess)
+ Effect.walkDelete(parentPath)
+ }
+
+ def apply[T](code: TestCaseSweeper => T): T = {
+ val sweeper = TestCaseSweeper()
+ try
+ code(sweeper)
+ finally {
+ implicit val bag = Bag.future(TestExecutionContext.executionContext)
+
+ def future = Future.sequence(sweeper.levels.map(_.delete(5.second)))
+
+ IO.fromFuture(future).run(0).await(10.seconds)
+
+ sweeper.segments.foreach {
+ segment =>
+ if (segment.existsOnDisk)
+ segment.delete
+ deleteParentPath(segment.path)
+ }
+
+ sweeper.maps.foreach {
+ map =>
+ if (map.pathOption.exists(Effect.exists))
+ map.delete
+ map.pathOption.foreach(deleteParentPath)
+ }
+
+ //calling this after since delete would've already invoked these.
+ sweeper.keyValueMemorySweepers.foreach(_.get().foreach(MemorySweeper.close))
+ sweeper.fileSweepers.foreach(_.get().foreach(sweeper => FileSweeper.closeSync(1.second)(sweeper, Bag.less)))
+ sweeper.cleaners.foreach(_.get().foreach(cleaner => ByteBufferSweeper.closeSync(1.second, 10.seconds)(cleaner, Bag.less, TestExecutionContext.executionContext)))
+ sweeper.blockCaches.foreach(_.get().foreach(BlockCache.close))
+
+ sweeper.paths.foreach(Effect.deleteIfExists)
+ }
+ }
+
+
+ implicit class TestLevelLevelSweeperImplicits[L <: LevelRef](level: L) {
+ def clean()(implicit sweeper: TestCaseSweeper): L =
+ sweeper cleanLevel level
+ }
+
+ implicit class TestLevelSegmentSweeperImplicits[L <: Segment](segment: L) {
+ def clean()(implicit sweeper: TestCaseSweeper): L =
+ sweeper cleanSegment segment
+ }
+
+ implicit class TestMapsSweeperImplicits[OK, OV, K <: OK, V <: OV](map: swaydb.core.map.Map[OK, OV, K, V]) {
+ def clean()(implicit sweeper: TestCaseSweeper): swaydb.core.map.Map[OK, OV, K, V] =
+ sweeper cleanMap map
+ }
+
+ implicit class TestLevelPathSweeperImplicits(path: Path) {
+ def clean()(implicit sweeper: TestCaseSweeper): Path =
+ sweeper cleanPath path
+ }
+}
+
+
+/**
+ * Manages cleaning levels, segments, maps etc for Levels. Initialises Actors lazily as required and
+ * destroys them after the test is complete.
+ *
+ * {{{
+ * TestLevelSweeper {
+ * implicit sweeper =>
+ * // test code here and all levels, segments creates
+ * // here will get deleted after the test is complete.
+ * }
+ * }}}
+ */
+
+class TestCaseSweeper private(private val keyValueMemorySweepers: ListBuffer[CacheNoIO[Unit, Option[MemorySweeper.KeyValue]]],
+ private val fileSweepers: ListBuffer[CacheNoIO[Unit, FileSweeperActor]],
+ private val cleaners: ListBuffer[CacheNoIO[Unit, ByteBufferSweeperActor]],
+ private val blockCaches: ListBuffer[CacheNoIO[Unit, Option[BlockCache.State]]],
+ private val levels: ListBuffer[LevelRef],
+ private val segments: ListBuffer[Segment],
+ private val maps: ListBuffer[map.Map[_, _, _, _]],
+ private val paths: ListBuffer[Path]) {
+
+
+ lazy val keyValueMemorySweeper = keyValueMemorySweepers.head.value(())
+ lazy val fileSweeper = fileSweepers.head.value(())
+ lazy val cleaner = cleaners.head.value(())
+ lazy val blockCache = blockCaches.head.value(())
+
+ def cleanLevel[T <: LevelRef](levelRef: T): T = {
+ levels += levelRef
+ levelRef
+ }
+
+ def cleanSegment[S <: Segment](segment: S): S = {
+ segments += segment
+ segment
+ }
+
+ def cleanMap[OK, OV, K <: OK, V <: OV](map: swaydb.core.map.Map[OK, OV, K, V]): swaydb.core.map.Map[OK, OV, K, V] = {
+ maps += map
+ map
+ }
+
+ def cleanPath(path: Path): Path = {
+ paths += path
+ path
+ }
+}
\ No newline at end of file
diff --git a/core/src/test/scala/swaydb/core/TestData.scala b/core/src/test/scala/swaydb/core/TestData.scala
index 42c138ed0..a8021aec7 100644
--- a/core/src/test/scala/swaydb/core/TestData.scala
+++ b/core/src/test/scala/swaydb/core/TestData.scala
@@ -71,6 +71,7 @@ import swaydb.data.util.StorageUnits._
import swaydb.serializers.Default._
import swaydb.serializers._
import swaydb.{IO, Scheduler}
+import TestCaseSweeper._
import scala.concurrent.ExecutionContext
import scala.concurrent.duration._
@@ -155,16 +156,14 @@ object TestData {
}
implicit class ReopenLevel(level: Level)(implicit keyOrder: KeyOrder[Slice[Byte]] = KeyOrder.default,
- ec: ExecutionContext,
- timeOrder: TimeOrder[Slice[Byte]] = TimeOrder.long,
- keyValueMemorySweeper: Option[MemorySweeper.KeyValue] = TestSweeper.memorySweeperMax) {
+ timeOrder: TimeOrder[Slice[Byte]] = TimeOrder.long) {
import swaydb.Error.Level.ExceptionHandler
import swaydb.IO._
//This test function is doing too much. This shouldn't be the case! There needs to be an easier way to write
//key-values in a Level without that level copying it forward to lower Levels.
- def putKeyValuesTest(keyValues: Slice[Memory]): IO[swaydb.Error.Level, Unit] = {
+ def putKeyValuesTest(keyValues: Slice[Memory])(implicit sweeper: TestCaseSweeper): IO[swaydb.Error.Level, Unit] = {
implicit val idGenerator = level.segmentIDGenerator
@@ -178,6 +177,7 @@ object TestData {
implicit val fileSweeper = level.fileSweeper
implicit val blockCache = level.blockCache
implicit val bufferCleaner = level.bufferCleaner
+ implicit val keyValueSweeper = level.keyValueMemorySweeper
if (keyValues.isEmpty)
IO.unit
@@ -222,7 +222,7 @@ object TestData {
hashIndexConfig = level.hashIndexConfig,
bloomFilterConfig = level.bloomFilterConfig,
segmentConfig = level.segmentConfig.copy(minSize = Int.MaxValue, maxCount = Int.MaxValue)
- )
+ ).map(_.clean())
} flatMap {
segments =>
segments should have size 1
@@ -240,16 +240,15 @@ object TestData {
}
}
- def reopen: Level =
+ def reopen(implicit sweeper: TestCaseSweeper): Level =
reopen()
- def tryReopen: IO[swaydb.Error.Level, Level] =
+ def tryReopen(implicit sweeper: TestCaseSweeper): IO[swaydb.Error.Level, Level] =
tryReopen()
def reopen(segmentSize: Int = level.minSegmentSize,
throttle: LevelMeter => Throttle = level.throttle,
- nextLevel: Option[NextLevel] = level.nextLevel)(implicit keyValueMemorySweeper: Option[MemorySweeper.KeyValue] = TestSweeper.memorySweeperMax,
- fileSweeper: FileSweeperActor = fileSweeper): Level =
+ nextLevel: Option[NextLevel] = level.nextLevel)(implicit sweeper: TestCaseSweeper): Level =
tryReopen(
segmentSize = segmentSize,
throttle = throttle,
@@ -258,14 +257,16 @@ object TestData {
def tryReopen(segmentSize: Int = level.minSegmentSize,
throttle: LevelMeter => Throttle = level.throttle,
- nextLevel: Option[NextLevel] = level.nextLevel)(implicit keyValueMemorySweeper: Option[MemorySweeper.KeyValue] = TestSweeper.memorySweeperMax,
- fileSweeper: FileSweeperActor = fileSweeper,
- bufferCleaner: ByteBufferSweeperActor = bufferCleaner,
- blockCache: Option[BlockCache.State] = TestSweeper.randomBlockCache): IO[swaydb.Error.Level, Level] =
+ nextLevel: Option[NextLevel] = level.nextLevel)(implicit sweeper: TestCaseSweeper): IO[swaydb.Error.Level, Level] =
level.releaseLocks flatMap {
_ =>
level.closeSegments flatMap {
_ =>
+ implicit val fileSweeper = level.fileSweeper
+ implicit val blockCache = level.blockCache
+ implicit val bufferCleaner = level.bufferCleaner
+ implicit val keyValueSweeper = level.keyValueMemorySweeper
+
Level(
bloomFilterConfig = level.bloomFilterConfig,
hashIndexConfig = level.hashIndexConfig,
@@ -281,28 +282,27 @@ object TestData {
appendixStorage = AppendixStorage.Persistent(mmap = MMAP.randomForMap(), 4.mb),
nextLevel = nextLevel,
throttle = throttle
- )
+ ).map(_.clean())
}
}
}
- implicit class ReopenLevelZero(level: LevelZero)(implicit keyOrder: KeyOrder[Slice[Byte]] = KeyOrder.default,
- ec: ExecutionContext) {
+ implicit class ReopenLevelZero(level: LevelZero)(implicit keyOrder: KeyOrder[Slice[Byte]] = KeyOrder.default) {
import swaydb.core.map.serializer.LevelZeroMapEntryWriter._
- def reopen: LevelZero =
+ def reopen(implicit sweeper: TestCaseSweeper): LevelZero =
reopen()
- def reopen(mapSize: Long = level.maps.map.size)(implicit keyValueMemorySweeper: Option[MemorySweeper.KeyValue] = TestSweeper.memorySweeperMax,
- timeOrder: TimeOrder[Slice[Byte]] = TimeOrder.long,
- fileSweeper: FileSweeperActor = fileSweeper,
- bufferCleaner: ByteBufferSweeperActor = bufferCleaner): LevelZero = {
+ def reopen(mapSize: Long = level.maps.map.size)(implicit timeOrder: TimeOrder[Slice[Byte]] = TimeOrder.long,
+ sweeper: TestCaseSweeper): LevelZero = {
val reopened =
level.releaseLocks flatMap {
_ =>
level.closeSegments flatMap {
_ =>
+ implicit val actor = sweeper.cleaner
+
LevelZero(
mapSize = mapSize,
enableTimer = true,
@@ -316,7 +316,7 @@ object TestData {
nextLevel = level.nextLevel,
acceleration = Accelerator.brake(),
throttle = level.throttle
- )
+ ).map(_.clean())
}
}
reopened.runRandomIO.right.value
diff --git a/core/src/test/scala/swaydb/core/TestSweeper.scala b/core/src/test/scala/swaydb/core/TestSweeper.scala
index f75288ae3..1afc04f6f 100644
--- a/core/src/test/scala/swaydb/core/TestSweeper.scala
+++ b/core/src/test/scala/swaydb/core/TestSweeper.scala
@@ -24,6 +24,7 @@
package swaydb.core
+import swaydb.Scheduler
import swaydb.core.CommonAssertions._
import swaydb.core.actor.{ByteBufferSweeper, FileSweeper, MemorySweeper}
import swaydb.core.actor.ByteBufferSweeper.ByteBufferSweeperActor
@@ -36,35 +37,66 @@ import scala.concurrent.duration._
private[swaydb] object TestSweeper {
- implicit val level0PushDownPool = TestExecutionContext.executionContext
-
- val memorySweeperMax: Option[MemorySweeper.All] =
- MemorySweeper(MemoryCache.All(4098, 1.mb / 2, 600.mb, None, false, ActorConfig.TimeLoop("TimeLoop test", 10.seconds, level0PushDownPool)))
+ def createMemorySweeperMax(): Option[MemorySweeper.All] =
+ MemorySweeper(MemoryCache.All(4098, 1.mb / 2, 600.mb, None, false, ActorConfig.TimeLoop("TimeLoop test", 10.seconds, TestExecutionContext.executionContext)))
.map(_.asInstanceOf[MemorySweeper.All])
- val memorySweeper10: Option[MemorySweeper.All] =
- MemorySweeper(MemoryCache.All(4098, 1.mb / 2, 600.mb, Some(1), false, ActorConfig.TimeLoop("TimeLoop test 2", 10.seconds, level0PushDownPool)))
+ lazy val memorySweeperMax: Option[MemorySweeper.All] = createMemorySweeperMax()
+
+ def createMemorySweeper10(): Option[MemorySweeper.All] =
+ MemorySweeper(MemoryCache.All(4098, 1.mb / 2, 600.mb, Some(1), false, ActorConfig.TimeLoop("TimeLoop test 2", 10.seconds, TestExecutionContext.executionContext)))
.map(_.asInstanceOf[MemorySweeper.All])
- val memorySweeperBlock: Option[MemorySweeper.BlockSweeper] =
- MemorySweeper(MemoryCache.ByteCacheOnly(4098, 1.mb / 2, 600.mb, ActorConfig.Basic("Basic Actor", level0PushDownPool)))
+ lazy val memorySweeper10: Option[MemorySweeper.All] = createMemorySweeper10()
+
+ def createMemoryBlockSweeper(): Option[MemorySweeper.BlockSweeper] =
+ MemorySweeper(MemoryCache.ByteCacheOnly(4098, 1.mb / 2, 600.mb, ActorConfig.Basic("Basic Actor", TestExecutionContext.executionContext)))
.map(_.asInstanceOf[MemorySweeper.BlockSweeper])
- val keyValueSweeperBlock: Option[MemorySweeper.KeyValueSweeper] =
- MemorySweeper(MemoryCache.KeyValueCacheOnly(600.mb, Some(100), Some(ActorConfig.Basic("Basic Actor 2", level0PushDownPool))))
+ lazy val memorySweeperBlock: Option[MemorySweeper.BlockSweeper] = createMemoryBlockSweeper()
+
+ def createKeyValueSweeperBlock(): Option[MemorySweeper.KeyValueSweeper] =
+ MemorySweeper(MemoryCache.KeyValueCacheOnly(600.mb, Some(100), Some(ActorConfig.Basic("Basic Actor 2", TestExecutionContext.executionContext))))
.map(_.asInstanceOf[MemorySweeper.KeyValueSweeper])
- val someMemorySweeperMax = memorySweeperMax
- val someMemorySweeper10 = memorySweeper10
+ lazy val keyValueSweeperBlock: Option[MemorySweeper.KeyValueSweeper] =
+ createKeyValueSweeperBlock()
+
+ lazy val someMemorySweeperMax = memorySweeperMax
+ lazy val someMemorySweeper10 = memorySweeper10
+
+ def createMemorySweeperRandom() =
+ eitherOne(
+ createMemorySweeper10(),
+ createMemorySweeperMax(),
+ None
+ )
- val blockCache: Option[BlockCache.State] =
- memorySweeperMax.map(BlockCache.init)
+ def createBlockCache(memorySweeper: Option[MemorySweeper.All]): Option[BlockCache.State] =
+ memorySweeper.map(BlockCache.init)
+
+ def createBlockCacheBlockSweeper(blockSweeper: Option[MemorySweeper.BlockSweeper]): Option[BlockCache.State] =
+ blockSweeper.map(BlockCache.init)
+
+ def createBlockCacheRandom(): Option[BlockCache.State] =
+ eitherOne(
+ createBlockCache(orNone(createMemorySweeperRandom())),
+ createBlockCacheBlockSweeper(orNone(createMemoryBlockSweeper()))
+ )
+
+ lazy val blockCache: Option[BlockCache.State] = createBlockCache(memorySweeperMax)
def randomBlockCache: Option[BlockCache.State] =
orNone(blockCache)
- val fileSweeper: FileSweeperActor =
- FileSweeper(50, ActorConfig.Basic("Basic test 3", level0PushDownPool)).value(())
+ def createFileSweeper(): FileSweeperActor =
+ FileSweeper(50, ActorConfig.Basic("Basic test 3", TestExecutionContext.executionContext)).value(())
+
+ lazy val fileSweeper: FileSweeperActor = createFileSweeper()
+
+ def createBufferCleaner(): ByteBufferSweeperActor =
+ ByteBufferSweeper()(Scheduler()(TestExecutionContext.executionContext))
- val bufferCleaner: ByteBufferSweeperActor = ByteBufferSweeper()(TestData.scheduler)
+ lazy val bufferCleaner: ByteBufferSweeperActor =
+ createBufferCleaner()
}
diff --git a/core/src/test/scala/swaydb/core/actor/ActorSpec.scala b/core/src/test/scala/swaydb/core/actor/ActorSpec.scala
index 759d5c731..4c192321b 100644
--- a/core/src/test/scala/swaydb/core/actor/ActorSpec.scala
+++ b/core/src/test/scala/swaydb/core/actor/ActorSpec.scala
@@ -508,7 +508,7 @@ class ActorSpec extends AnyWordSpec with Matchers {
actor send i
}
- val result = actor.terminateAndRecover[Future](0.seconds)
+ val result = actor.terminateAndRecoverAsync[Future](0.seconds)
(101 to 200) foreach {
i =>
@@ -573,7 +573,7 @@ class ActorSpec extends AnyWordSpec with Matchers {
//terminate here does not work because it's in the future
//and the second send messages will get returned as success
//so actor.terminate() is invoked before terminateAndRecover.
- val future = actor.terminateAndRecover[Future](0.seconds)
+ val future = actor.terminateAndRecoverAsync[Future](0.seconds)
eventual(20.seconds) {
success.asScala.toList shouldBe (1 to 100).toList
@@ -674,7 +674,7 @@ class ActorSpec extends AnyWordSpec with Matchers {
//random enough to randomly balance messages for both success and recovered queue.
//see logged output to see how messages are distributed.
if (Random.nextDouble() < 0.001) {
- val future = actor.terminateAndRecover[Future](0.seconds)
+ val future = actor.terminateAndRecoverAsync[Future](0.seconds)
terminated = true
future
} else {
@@ -688,7 +688,7 @@ class ActorSpec extends AnyWordSpec with Matchers {
//if it's not already terminates then terminate it so that cache Actors also drop their
//stashed messages.
if (!terminated)
- futures += actor.terminateAndRecover[Future](0.seconds)
+ futures += actor.terminateAndRecoverAsync[Future](0.seconds)
println(s"Messages sent. Success: ${success.size()}. Recovered messages: ${recovered.size()}")
//does not throw exception
diff --git a/core/src/test/scala/swaydb/core/io/file/ByteBufferSweeperSpec.scala b/core/src/test/scala/swaydb/core/io/file/ByteBufferSweeperSpec.scala
index ef7108c11..0d53d7c1f 100644
--- a/core/src/test/scala/swaydb/core/io/file/ByteBufferSweeperSpec.scala
+++ b/core/src/test/scala/swaydb/core/io/file/ByteBufferSweeperSpec.scala
@@ -76,8 +76,8 @@ class ByteBufferSweeperSpec extends TestBase {
innerFile.isBufferEmpty shouldBe true
}
- fileSweeper.terminateAndRecover(terminateTimeout).await(terminateTimeout)
- cleaner.actor.terminateAndRecover(terminateTimeout).await(terminateTimeout)
+ fileSweeper.terminateAndRecoverAsync(terminateTimeout).await(terminateTimeout)
+ cleaner.actor.terminateAndRecoverAsync(terminateTimeout).await(terminateTimeout)
cleaner.actor.messageCount shouldBe 0
cleaner.actor.isTerminated shouldBe true
}
@@ -123,10 +123,10 @@ class ByteBufferSweeperSpec extends TestBase {
//keep this test running for a few seconds.
sleep(timeout)
- fileSweeper.terminateAndRecover(terminateTimeout).await(terminateTimeout)
+ fileSweeper.terminateAndRecoverAsync(terminateTimeout).await(terminateTimeout)
fileSweeper.messageCount shouldBe 0
- cleaner.actor.terminateAndRecover(terminateTimeout).await(terminateTimeout)
+ cleaner.actor.terminateAndRecoverAsync(terminateTimeout).await(terminateTimeout)
cleaner.actor.messageCount shouldBe 0
cleaner.actor.isTerminated shouldBe true
}
@@ -148,7 +148,7 @@ class ByteBufferSweeperSpec extends TestBase {
map shouldBe empty
map.get(path) shouldBe empty
- cleaner.actor.terminateAndRecover(terminateTimeout).await(terminateTimeout)
+ cleaner.actor.terminateAndRecoverAsync(terminateTimeout).await(terminateTimeout)
cleaner.actor.messageCount shouldBe 0
cleaner.actor.isTerminated shouldBe true
}
@@ -183,7 +183,7 @@ class ByteBufferSweeperSpec extends TestBase {
map.get(path).value.counter.get() shouldBe (i - 1)
}
- cleaner.actor.terminateAndRecover(terminateTimeout).await(terminateTimeout)
+ cleaner.actor.terminateAndRecoverAsync(terminateTimeout).await(terminateTimeout)
cleaner.actor.messageCount shouldBe 0
cleaner.actor.isTerminated shouldBe true
}
@@ -206,7 +206,7 @@ class ByteBufferSweeperSpec extends TestBase {
Effect.delete(path)
Effect.exists(path) shouldBe false
- cleaner.actor.terminateAndRecover(terminateTimeout).await(terminateTimeout)
+ cleaner.actor.terminateAndRecoverAsync(terminateTimeout).await(terminateTimeout)
cleaner.actor.messageCount shouldBe 0
cleaner.actor.isTerminated shouldBe true
}
@@ -242,7 +242,7 @@ class ByteBufferSweeperSpec extends TestBase {
//state should be cleared
cleaner.actor.ask(Command.IsAllClean[Unit]).await(1.minute)
- cleaner.actor.terminateAndRecover(terminateTimeout).await(terminateTimeout)
+ cleaner.actor.terminateAndRecoverAsync(terminateTimeout).await(terminateTimeout)
cleaner.actor.messageCount shouldBe 0
cleaner.actor.isTerminated shouldBe true
}
@@ -269,7 +269,7 @@ class ByteBufferSweeperSpec extends TestBase {
//state should be cleared
cleaner.actor.ask(Command.IsAllClean[Unit]).await(1.minute)
- cleaner.actor.terminateAndRecover(terminateTimeout).await(terminateTimeout)
+ cleaner.actor.terminateAndRecoverAsync(terminateTimeout).await(terminateTimeout)
cleaner.actor.messageCount shouldBe 0
cleaner.actor.isTerminated shouldBe true
}
@@ -298,7 +298,7 @@ class ByteBufferSweeperSpec extends TestBase {
//state should be cleared
cleaner.actor.ask(Command.IsAllClean[Unit]).await(1.minute)
- cleaner.actor.terminateAndRecover(terminateTimeout).await(terminateTimeout)
+ cleaner.actor.terminateAndRecoverAsync(terminateTimeout).await(terminateTimeout)
cleaner.actor.messageCount shouldBe 0
cleaner.actor.isTerminated shouldBe true
}
@@ -324,7 +324,7 @@ class ByteBufferSweeperSpec extends TestBase {
//state should be cleared
cleaner.actor.ask(Command.IsAllClean[Unit]).await(1.minute)
- cleaner.actor.terminateAndRecover(terminateTimeout).await(terminateTimeout)
+ cleaner.actor.terminateAndRecoverAsync(terminateTimeout).await(terminateTimeout)
cleaner.actor.messageCount shouldBe 0
cleaner.actor.isTerminated shouldBe true
}
@@ -393,7 +393,7 @@ class ByteBufferSweeperSpec extends TestBase {
}
//execute all pending Delete commands.
- cleaner.actor.receiveAllBlocking(Int.MaxValue).get
+ cleaner.actor.receiveAllBlocking(Int.MaxValue, 1.second).get
//there might me some delete messages waiting to be scheduled.
eventual(1.minute) {
@@ -439,7 +439,7 @@ class ByteBufferSweeperSpec extends TestBase {
val paths = (1 to 100) map (_ => sendRandomRequests())
//execute all pending Delete commands.
- cleaner.actor.terminateAndRecover[Future](1.second).await(1.minute)
+ cleaner.actor.terminateAndRecoverAsync[Future](1.second).await(1.minute)
eventual(1.minute) {
(cleaner.actor ask Command.IsTerminatedAndCleaned[Unit]).await(2.seconds) shouldBe true
diff --git a/core/src/test/scala/swaydb/core/level/LevelCollpaseSpec.scala b/core/src/test/scala/swaydb/core/level/LevelCollpaseSpec.scala
index dcf1cbeb4..f2872d0ce 100644
--- a/core/src/test/scala/swaydb/core/level/LevelCollpaseSpec.scala
+++ b/core/src/test/scala/swaydb/core/level/LevelCollpaseSpec.scala
@@ -29,13 +29,14 @@ import swaydb.IOValues._
import swaydb.core.CommonAssertions._
import swaydb.core.RunThis._
import swaydb.core.TestData._
+import swaydb.core.TestCaseSweeper._
import swaydb.core.actor.FileSweeper.FileSweeperActor
-import swaydb.core.actor.{FileSweeper, MemorySweeper}
+import swaydb.core.actor.MemorySweeper
import swaydb.core.data._
import swaydb.core.level.zero.LevelZeroSkipListMerger
import swaydb.core.segment.ThreadReadState
import swaydb.core.segment.format.a.block.segment.SegmentBlock
-import swaydb.core.{TestBase, TestSweeper, TestTimer}
+import swaydb.core.{TestBase, TestCaseSweeper, TestSweeper, TestTimer}
import swaydb.data.config.MMAP
import swaydb.data.order.{KeyOrder, TimeOrder}
import swaydb.data.slice.Slice
@@ -81,35 +82,37 @@ sealed trait LevelCollapseSpec extends TestBase {
"collapse" should {
"collapse small Segments to 50% of the size when the Segment's size was reduced by deleting 50% of it's key-values" in {
- //disable throttling so that it does not automatically collapse small Segments
- val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb))
- val keyValues = randomPutKeyValues(1000, addPutDeadlines = false)(TestTimer.Empty)
- level.putKeyValuesTest(keyValues).runRandomIO.right.value
-
- val segmentCountBeforeDelete = level.segmentsCount()
- segmentCountBeforeDelete > 1 shouldBe true
-
- assertAllSegmentsCreatedInLevel(level)
-
- val keyValuesNoDeleted = ListBuffer.empty[KeyValue]
- val deleteEverySecond =
- keyValues.zipWithIndex flatMap {
- case (keyValue, index) =>
- if (index % 2 == 0)
- Some(Memory.Remove(keyValue.key, None, Time.empty))
- else {
- keyValuesNoDeleted += keyValue
- None
+ TestCaseSweeper {
+ implicit sweeper =>
+ //disable throttling so that it does not automatically collapse small Segments
+ val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb))
+ val keyValues = randomPutKeyValues(1000, addPutDeadlines = false)(TestTimer.Empty)
+ level.putKeyValuesTest(keyValues).runRandomIO.right.value
+
+ val segmentCountBeforeDelete = level.segmentsCount()
+ segmentCountBeforeDelete > 1 shouldBe true
+
+ assertAllSegmentsCreatedInLevel(level)
+
+ val keyValuesNoDeleted = ListBuffer.empty[KeyValue]
+ val deleteEverySecond =
+ keyValues.zipWithIndex flatMap {
+ case (keyValue, index) =>
+ if (index % 2 == 0)
+ Some(Memory.Remove(keyValue.key, None, Time.empty))
+ else {
+ keyValuesNoDeleted += keyValue
+ None
+ }
}
- }
- //delete half of the key values which will create small Segments
- level.putKeyValuesTest(Slice(deleteEverySecond.toArray)).runRandomIO.right.value
- level.collapse(level.segmentsInLevel()).right.right.value.right.value
- //since every second key-value was delete, the number of Segments is reduced to half
- level.segmentFilesInAppendix shouldBe <=((segmentCountBeforeDelete / 2) + 1) //+1 for odd number of key-values
- assertReads(Slice(keyValuesNoDeleted.toArray), level)
-
- level.delete.runRandomIO.right.value
+ //delete half of the key values which will create small Segments
+ level.putKeyValuesTest(Slice(deleteEverySecond.toArray)).runRandomIO.right.value
+ level.collapse(level.segmentsInLevel()).right.right.value.right.value
+ //since every second key-value was delete, the number of Segments is reduced to half
+ level.segmentFilesInAppendix shouldBe <=((segmentCountBeforeDelete / 2) + 1) //+1 for odd number of key-values
+ assertReads(Slice(keyValuesNoDeleted.toArray), level)
+
+ }
}
"collapse all small Segments into one of the existing small Segments, if the Segment was reopened with a larger segment size" in {
@@ -117,34 +120,36 @@ sealed trait LevelCollapseSpec extends TestBase {
//memory Level cannot be reopened.
} else {
runThis(1.times) {
- // implicit val compressionType: Option[KeyValueCompressionType] = randomCompressionTypeOption(keyValuesCount)
- //disable throttling so that it does not automatically collapse small Segments
- val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb, pushForward = false, deleteEventually = false))
-
- assertAllSegmentsCreatedInLevel(level)
+ TestCaseSweeper {
+ implicit sweeper =>
+ // implicit val compressionType: Option[KeyValueCompressionType] = randomCompressionTypeOption(keyValuesCount)
+ //disable throttling so that it does not automatically collapse small Segments
+ val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb, pushForward = false, deleteEventually = false))
+
+ assertAllSegmentsCreatedInLevel(level)
+
+ val keyValues = randomPutKeyValues(1000, addPutDeadlines = false)(TestTimer.Empty)
+ level.putKeyValuesTest(keyValues).runRandomIO.right.value
+ //dispatch another push to trigger split
+ level.putKeyValuesTest(Slice(keyValues.head)).runRandomIO.right.value
+
+ level.segmentsCount() > 1 shouldBe true
+ level.closeNoSweep().runRandomIO.right.value
+
+ //reopen the Level with larger min segment size
+ val reopenLevel = level.reopen(segmentSize = 20.mb)
+ reopenLevel.collapse(level.segmentsInLevel()).right.right.value.right.value
+
+ //resulting segments is 1
+ eventually {
+ level.segmentFilesOnDisk should have size 1
+ }
+ //can still read Segments
+ assertReads(keyValues, reopenLevel)
+ val reopen2 = reopenLevel.reopen
+ eventual(assertReads(keyValues, reopen2))
- val keyValues = randomPutKeyValues(1000, addPutDeadlines = false)(TestTimer.Empty)
- level.putKeyValuesTest(keyValues).runRandomIO.right.value
- //dispatch another push to trigger split
- level.putKeyValuesTest(Slice(keyValues.head)).runRandomIO.right.value
-
- level.segmentsCount() > 1 shouldBe true
- level.close.runRandomIO.right.value
-
- //reopen the Level with larger min segment size
- val reopenLevel = level.reopen(segmentSize = 20.mb)
- reopenLevel.collapse(level.segmentsInLevel()).right.right.value.right.value
-
- //resulting segments is 1
- eventually {
- level.segmentFilesOnDisk should have size 1
}
- //can still read Segments
- assertReads(keyValues, reopenLevel)
- val reopen2 = reopenLevel.reopen
- eventual(assertReads(keyValues, reopen2))
-
- level.delete.runRandomIO.right.value
}
}
}
@@ -152,56 +157,60 @@ sealed trait LevelCollapseSpec extends TestBase {
"clear expired key-values" in {
//this test is similar as the above collapsing small Segment test.
//Remove or expiring key-values should have the same result
- val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb))
- val expiryAt = 5.seconds.fromNow
- val keyValues = randomPutKeyValues(1000, valueSize = 0, startId = Some(0), addPutDeadlines = false)(TestTimer.Empty)
- level.putKeyValuesTest(keyValues).runRandomIO.right.value
- val segmentCountBeforeDelete = level.segmentsCount()
- segmentCountBeforeDelete > 1 shouldBe true
-
- val keyValuesNotExpired = ListBuffer.empty[KeyValue]
- val expireEverySecond =
- keyValues.zipWithIndex flatMap {
- case (keyValue, index) =>
- if (index % 2 == 0)
- Some(Memory.Remove(keyValue.key, Some(expiryAt + index.millisecond), Time.empty))
- else {
- keyValuesNotExpired += keyValue
- None
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb))
+ val expiryAt = 5.seconds.fromNow
+ val keyValues = randomPutKeyValues(1000, valueSize = 0, startId = Some(0), addPutDeadlines = false)(TestTimer.Empty)
+ level.putKeyValuesTest(keyValues).runRandomIO.right.value
+ val segmentCountBeforeDelete = level.segmentsCount()
+ segmentCountBeforeDelete > 1 shouldBe true
+
+ val keyValuesNotExpired = ListBuffer.empty[KeyValue]
+ val expireEverySecond =
+ keyValues.zipWithIndex flatMap {
+ case (keyValue, index) =>
+ if (index % 2 == 0)
+ Some(Memory.Remove(keyValue.key, Some(expiryAt + index.millisecond), Time.empty))
+ else {
+ keyValuesNotExpired += keyValue
+ None
+ }
}
- }
-
- //delete half of the key values which will create small Segments
- level.putKeyValuesTest(Slice(expireEverySecond.toArray)).runRandomIO.right.value
- keyValues.zipWithIndex foreach {
- case (keyValue, index) =>
- if (index % 2 == 0)
- level.get(keyValue.key, ThreadReadState.random).runRandomIO.right.value.toOptionPut.value.deadline should contain(expiryAt + index.millisecond)
- }
+ //delete half of the key values which will create small Segments
+ level.putKeyValuesTest(Slice(expireEverySecond.toArray)).runRandomIO.right.value
+ keyValues.zipWithIndex foreach {
+ case (keyValue, index) =>
- sleep(20.seconds)
- level.collapse(level.segmentsInLevel()).right.right.value.right.value
- level.segmentFilesInAppendix should be <= ((segmentCountBeforeDelete / 2) + 1)
+ if (index % 2 == 0)
+ level.get(keyValue.key, ThreadReadState.random).runRandomIO.right.value.toOptionPut.value.deadline should contain(expiryAt + index.millisecond)
+ }
- assertReads(Slice(keyValuesNotExpired.toArray), level)
+ sleep(20.seconds)
+ level.collapse(level.segmentsInLevel()).right.right.value.right.value
+ level.segmentFilesInAppendix should be <= ((segmentCountBeforeDelete / 2) + 1)
- level.delete.runRandomIO.right.value
+ assertReads(Slice(keyValuesNotExpired.toArray), level)
+ }
}
}
"update createdInLevel" in {
- val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb))
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb))
- val keyValues = randomPutKeyValues(keyValuesCount, addExpiredPutDeadlines = false)
- val maps = TestMap(keyValues)
- level.put(maps).right.right.value.right.value
+ val keyValues = randomPutKeyValues(keyValuesCount, addExpiredPutDeadlines = false)
+ val maps = TestMap(keyValues)
+ level.put(maps).right.right.value.right.value
- val nextLevel = TestLevel()
- nextLevel.put(level.segmentsInLevel()).right.right.value.right.value
+ val nextLevel = TestLevel()
+ nextLevel.put(level.segmentsInLevel()).right.right.value.right.value
- if (persistent) nextLevel.segmentsInLevel() foreach (_.createdInLevel shouldBe level.levelNumber)
- nextLevel.collapse(nextLevel.segmentsInLevel()).right.right.value.right.value
- nextLevel.segmentsInLevel() foreach (_.createdInLevel shouldBe nextLevel.levelNumber)
+ if (persistent) nextLevel.segmentsInLevel() foreach (_.createdInLevel shouldBe level.levelNumber)
+ nextLevel.collapse(nextLevel.segmentsInLevel()).right.right.value.right.value
+ nextLevel.segmentsInLevel() foreach (_.createdInLevel shouldBe nextLevel.levelNumber)
+ }
}
}
diff --git a/core/src/test/scala/swaydb/core/level/LevelCopySpec.scala b/core/src/test/scala/swaydb/core/level/LevelCopySpec.scala
index bbf11c70e..2e639e61c 100644
--- a/core/src/test/scala/swaydb/core/level/LevelCopySpec.scala
+++ b/core/src/test/scala/swaydb/core/level/LevelCopySpec.scala
@@ -38,7 +38,9 @@ import swaydb.core.io.file.BlockCache
import swaydb.core.level.zero.LevelZeroSkipListMerger
import swaydb.core.segment.Segment
import swaydb.core.segment.format.a.block.segment.SegmentBlock
-import swaydb.core.{TestBase, TestSweeper, TestTimer}
+import swaydb.core.{TestBase, TestCaseSweeper, TestSweeper, TestTimer}
+import TestCaseSweeper._
+import TestCaseSweeper._
import swaydb.data.config.MMAP
import swaydb.data.order.{KeyOrder, TimeOrder}
import swaydb.data.slice.Slice
@@ -82,67 +84,79 @@ sealed trait LevelCopySpec extends TestBase with MockFactory with PrivateMethodT
"copy" should {
"copy segments" in {
- val level = TestLevel()
- level.isEmpty shouldBe true
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel()
+ level.isEmpty shouldBe true
- val keyValues1 = randomIntKeyStringValues()
- val keyValues2 = randomIntKeyStringValues()
- val segments = Iterable(TestSegment(keyValues1), TestSegment(keyValues2))
- val copiedSegments = level.copyLocal(segments).value
+ val keyValues1 = randomIntKeyStringValues()
+ val keyValues2 = randomIntKeyStringValues()
+ val segments = Iterable(TestSegment(keyValues1), TestSegment(keyValues2))
+ val copiedSegments = level.copyLocal(segments).value
- val allKeyValues = Slice((keyValues1 ++ keyValues2).toArray)
+ val allKeyValues = Slice((keyValues1 ++ keyValues2).toArray)
- level.isEmpty shouldBe true //copy function does not write to appendix.
+ level.isEmpty shouldBe true //copy function does not write to appendix.
- if (persistent) level.segmentFilesOnDisk should not be empty
+ if (persistent) level.segmentFilesOnDisk should not be empty
- Segment.getAllKeyValues(copiedSegments) shouldBe allKeyValues
+ Segment.getAllKeyValues(copiedSegments) shouldBe allKeyValues
+ }
}
"fail copying Segments if it failed to copy one of the Segments" in {
- val level = TestLevel()
- level.isEmpty shouldBe true
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel()
+ level.isEmpty shouldBe true
- val segment1 = TestSegment()
- val segment2 = TestSegment()
+ val segment1 = TestSegment()
+ val segment2 = TestSegment()
- segment2.delete // delete segment2 so there is a failure in copying Segments
+ segment2.delete // delete segment2 so there is a failure in copying Segments
- val segments = Iterable(segment1, segment2)
- level.copyLocal(segments).left.value.exception shouldBe a[NoSuchFileException]
+ val segments = Iterable(segment1, segment2)
+ level.copyLocal(segments).left.value.exception shouldBe a[NoSuchFileException]
- level.isEmpty shouldBe true
- if (persistent) level.reopen.isEmpty shouldBe true
+ level.isEmpty shouldBe true
+ if (persistent) level.reopen.isEmpty shouldBe true
+ }
}
"copy Map" in {
- val level = TestLevel()
- level.isEmpty shouldBe true
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel()
+ level.isEmpty shouldBe true
- val keyValues = randomPutKeyValues(keyValuesCount)
- val copiedSegments = level.copy(TestMap(keyValues)).value
- level.isEmpty shouldBe true //copy function does not write to appendix.
+ val keyValues = randomPutKeyValues(keyValuesCount)
+ val copiedSegments = level.copy(TestMap(keyValues)).value
+ level.isEmpty shouldBe true //copy function does not write to appendix.
- if (persistent) level.segmentFilesOnDisk should not be empty
+ if (persistent) level.segmentFilesOnDisk should not be empty
- Segment.getAllKeyValues(copiedSegments) shouldBe keyValues
+ Segment.getAllKeyValues(copiedSegments) shouldBe keyValues
+ }
}
}
"copy map directly into lower level" in {
- val level2 = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb))
- val level1 = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb, pushForward = true), nextLevel = Some(level2))
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level2 = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb))
+ val level1 = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb, pushForward = true), nextLevel = Some(level2))
- val keyValues = randomPutKeyValues(keyValuesCount)
- val maps = TestMap(keyValues)
+ val keyValues = randomPutKeyValues(keyValuesCount)
+ val maps = TestMap(keyValues)
- level1.put(maps).right.right.value.right.value should contain only level2.levelNumber
+ level1.put(maps).right.right.value.right.value should contain only level2.levelNumber
- level1.isEmpty shouldBe true
- level2.isEmpty shouldBe false
+ level1.isEmpty shouldBe true
+ level2.isEmpty shouldBe false
- assertReads(keyValues, level1)
+ assertReads(keyValues, level1)
- level1.segmentsInLevel() foreach (_.createdInLevel shouldBe level2.levelNumber)
+ level1.segmentsInLevel() foreach (_.createdInLevel shouldBe level2.levelNumber)
+ }
}
}
diff --git a/core/src/test/scala/swaydb/core/level/LevelKeyValuesSpec.scala b/core/src/test/scala/swaydb/core/level/LevelKeyValuesSpec.scala
index fd9eefb20..4bb9b41d5 100644
--- a/core/src/test/scala/swaydb/core/level/LevelKeyValuesSpec.scala
+++ b/core/src/test/scala/swaydb/core/level/LevelKeyValuesSpec.scala
@@ -31,13 +31,14 @@ import swaydb.IOValues._
import swaydb.core.CommonAssertions._
import swaydb.core.RunThis._
import swaydb.core.TestData._
+import swaydb.core.TestCaseSweeper._
import swaydb.core.actor.FileSweeper.FileSweeperActor
-import swaydb.core.actor.{FileSweeper, MemorySweeper}
+import swaydb.core.actor.MemorySweeper
import swaydb.core.data._
import swaydb.core.level.zero.LevelZeroSkipListMerger
import swaydb.core.segment.ThreadReadState
import swaydb.core.segment.format.a.block.segment.SegmentBlock
-import swaydb.core.{TestBase, TestSweeper, TestTimer}
+import swaydb.core.{TestBase, TestCaseSweeper, TestSweeper, TestTimer}
import swaydb.data.config.MMAP
import swaydb.data.order.{KeyOrder, TimeOrder}
import swaydb.data.slice.Slice
@@ -84,291 +85,324 @@ sealed trait LevelKeyValuesSpec extends TestBase with MockFactory with PrivateMe
"put KeyValues" should {
"write a key-values to the Level" in {
- val level = TestLevel()
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel()
- val keyValues = randomPutKeyValues(startId = Some(1))
- level.putKeyValuesTest(keyValues).runRandomIO.right.value
- level.putKeyValuesTest(Slice(keyValues.head)).runRandomIO.right.value
+ val keyValues = randomPutKeyValues(startId = Some(1))
+ level.putKeyValuesTest(keyValues).runRandomIO.right.value
+ level.putKeyValuesTest(Slice(keyValues.head)).runRandomIO.right.value
- level.segmentsInLevel() foreach {
- segment =>
- segment.createdInLevel shouldBe level.levelNumber
- }
+ level.segmentsInLevel() foreach {
+ segment =>
+ segment.createdInLevel shouldBe level.levelNumber
+ }
- assertReads(keyValues, level)
+ assertReads(keyValues, level)
- if (persistent) {
- val reopen = level.reopen
- assertReads(keyValues, reopen)
+ if (persistent) {
+ val reopen = level.reopen
+ assertReads(keyValues, reopen)
+ }
}
}
"return an empty level if all the key values in the Level were REMOVED and if Level is the only Level" in {
- val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb))
-
- val keyValues = randomPutKeyValues(keyValuesCount)
- level.putKeyValuesTest(keyValues).runRandomIO.right.value
-
- val deleteKeyValues = Slice.create[Memory](keyValues.size * 2)
- keyValues foreach {
- keyValue =>
- deleteKeyValues add Memory.remove(keyValue.key)
- }
- //also add another set of Delete key-values where the keys do not belong to the Level but since there is no lower level
- //these delete keys should also be removed
- val lastKeyValuesId = keyValues.last.key.read[Int] + 10000
- (lastKeyValuesId until keyValues.size + lastKeyValuesId) foreach {
- id =>
- deleteKeyValues add Memory.remove(id, randomly(expiredDeadline()))
- }
-
- level.putKeyValuesTest(deleteKeyValues).runRandomIO.right.value
- level.segmentFilesInAppendix shouldBe 0
-
- level.isEmpty shouldBe true
- if (persistent) {
- level.reopen.isEmpty shouldBe true
- level.segmentFilesOnDisk shouldBe empty
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb))
+
+ val keyValues = randomPutKeyValues(keyValuesCount)
+ level.putKeyValuesTest(keyValues).runRandomIO.right.value
+
+ val deleteKeyValues = Slice.create[Memory](keyValues.size * 2)
+ keyValues foreach {
+ keyValue =>
+ deleteKeyValues add Memory.remove(keyValue.key)
+ }
+ //also add another set of Delete key-values where the keys do not belong to the Level but since there is no lower level
+ //these delete keys should also be removed
+ val lastKeyValuesId = keyValues.last.key.read[Int] + 10000
+ (lastKeyValuesId until keyValues.size + lastKeyValuesId) foreach {
+ id =>
+ deleteKeyValues add Memory.remove(id, randomly(expiredDeadline()))
+ }
+
+ level.putKeyValuesTest(deleteKeyValues).runRandomIO.right.value
+ level.segmentFilesInAppendix shouldBe 0
+
+ level.isEmpty shouldBe true
+ if (persistent) {
+ level.reopen.isEmpty shouldBe true
+ level.segmentFilesOnDisk shouldBe empty
+ }
}
}
"not return an empty level if all the key values in the Level were REMOVED but it has lower level" in {
- val level = TestLevel(nextLevel = Some(TestLevel()))
-
- val keyValues = randomPutKeyValues()
- level.putKeyValuesTest(keyValues).runRandomIO.right.value
-
- val deleteKeyValues = Slice.create[Memory](keyValues.size)
- keyValues foreach {
- keyValue =>
- deleteKeyValues add Memory.remove(keyValue.key)
- }
-
- level.putKeyValuesTest(deleteKeyValues).runRandomIO.right.value
- level.isEmpty shouldBe false
- keyValues foreach {
- keyValue =>
- level.get(keyValue.key, ThreadReadState.random).runRandomIO.right.value.toOptionPut shouldBe empty
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel(nextLevel = Some(TestLevel()))
+
+ val keyValues = randomPutKeyValues()
+ level.putKeyValuesTest(keyValues).runRandomIO.right.value
+
+ val deleteKeyValues = Slice.create[Memory](keyValues.size)
+ keyValues foreach {
+ keyValue =>
+ deleteKeyValues add Memory.remove(keyValue.key)
+ }
+
+ level.putKeyValuesTest(deleteKeyValues).runRandomIO.right.value
+ level.isEmpty shouldBe false
+ keyValues foreach {
+ keyValue =>
+ level.get(keyValue.key, ThreadReadState.random).runRandomIO.right.value.toOptionPut shouldBe empty
+ }
}
}
"return an empty level if all the key values in the Level were REMOVED by RANGE and if Level is the only Level" in {
- val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb))
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb))
- val keyValues = randomPutKeyValues(keyValuesCount)
- level.putKeyValuesTest(keyValues).runRandomIO.right.value
+ val keyValues = randomPutKeyValues(keyValuesCount)
+ level.putKeyValuesTest(keyValues).runRandomIO.right.value
- level.putKeyValuesTest(Slice(Memory.Range(keyValues.head.key, keyValues.last.key.readInt() + 1, Value.FromValue.Null, Value.remove(None)))).runRandomIO.right.value
- level.segmentFilesInAppendix shouldBe 0
+ level.putKeyValuesTest(Slice(Memory.Range(keyValues.head.key, keyValues.last.key.readInt() + 1, Value.FromValue.Null, Value.remove(None)))).runRandomIO.right.value
+ level.segmentFilesInAppendix shouldBe 0
- level.isEmpty shouldBe true
- if (persistent) {
- level.reopen.isEmpty shouldBe true
- level.segmentFilesOnDisk shouldBe empty
+ level.isEmpty shouldBe true
+ if (persistent) {
+ level.reopen.isEmpty shouldBe true
+ level.segmentFilesOnDisk shouldBe empty
+ }
}
}
"not return an empty level if all the key values in the Level were REMOVED by RANGE but it has a lower Level" in {
- val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb), nextLevel = Some(TestLevel()))
-
- val keyValues = randomPutKeyValues(keyValuesCount)
- level.putKeyValuesTest(keyValues).runRandomIO.right.value
- val segmentsCountBeforeRemove = level.segmentFilesInAppendix
-
- level.putKeyValuesTest(Slice(Memory.Range(keyValues.head.key, keyValues.last.key.readInt() + 1, Value.FromValue.Null, Value.remove(None)))).runRandomIO.right.value
- level.segmentFilesInAppendix shouldBe segmentsCountBeforeRemove
-
- level.isEmpty shouldBe false
- if (persistent) {
- level.reopen.isEmpty shouldBe false
- level.segmentFilesOnDisk should have size segmentsCountBeforeRemove
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb), nextLevel = Some(TestLevel()))
+
+ val keyValues = randomPutKeyValues(keyValuesCount)
+ level.putKeyValuesTest(keyValues).runRandomIO.right.value
+ val segmentsCountBeforeRemove = level.segmentFilesInAppendix
+
+ level.putKeyValuesTest(Slice(Memory.Range(keyValues.head.key, keyValues.last.key.readInt() + 1, Value.FromValue.Null, Value.remove(None)))).runRandomIO.right.value
+ level.segmentFilesInAppendix shouldBe segmentsCountBeforeRemove
+
+ level.isEmpty shouldBe false
+ if (persistent) {
+ level.reopen.isEmpty shouldBe false
+ level.segmentFilesOnDisk should have size segmentsCountBeforeRemove
+ }
}
}
"return an empty level if all the key values in the Level were EXPIRED and if Level is the only Level" in {
- val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb))
-
- val keyValues = randomPutKeyValues(keyValuesCount)
- level.putKeyValuesTest(keyValues).runRandomIO.right.value
-
- val deleteKeyValues = Slice.create[Memory](keyValues.size * 2)
- keyValues foreach {
- keyValue =>
- deleteKeyValues add Memory.remove(keyValue.key, 1.seconds)
- }
- //also add another set of Delete key-values where the keys do not belong to the Level but since there is no lower level
- //these delete keys should also be removed
- val lastKeyValuesId = keyValues.last.key.read[Int] + 1
- (lastKeyValuesId until keyValues.size + lastKeyValuesId) foreach {
- id =>
- deleteKeyValues add Memory.remove(id, randomly(expiredDeadline()))
- }
-
- level.nextLevel shouldBe empty
-
- level.putKeyValuesTest(deleteKeyValues).runRandomIO.right.value
-
- sleep(2.seconds)
-
- level.segmentsInLevel() foreach {
- segment =>
- level.refresh(segment).right.right.value.right.value
- }
-
- //expired key-values return empty after 2.seconds
- keyValues foreach {
- keyValue =>
- level.get(keyValue.key, ThreadReadState.random).runRandomIO.right.value.toOptionPut shouldBe empty
- }
-
- level.segmentFilesInAppendix shouldBe 0
-
- level.isEmpty shouldBe true
-
- if (persistent) {
- level.reopen.isEmpty shouldBe true
- level.segmentFilesOnDisk shouldBe empty
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb))
+
+ val keyValues = randomPutKeyValues(keyValuesCount)
+ level.putKeyValuesTest(keyValues).runRandomIO.right.value
+
+ val deleteKeyValues = Slice.create[Memory](keyValues.size * 2)
+ keyValues foreach {
+ keyValue =>
+ deleteKeyValues add Memory.remove(keyValue.key, 1.seconds)
+ }
+ //also add another set of Delete key-values where the keys do not belong to the Level but since there is no lower level
+ //these delete keys should also be removed
+ val lastKeyValuesId = keyValues.last.key.read[Int] + 1
+ (lastKeyValuesId until keyValues.size + lastKeyValuesId) foreach {
+ id =>
+ deleteKeyValues add Memory.remove(id, randomly(expiredDeadline()))
+ }
+
+ level.nextLevel shouldBe empty
+
+ level.putKeyValuesTest(deleteKeyValues).runRandomIO.right.value
+
+ sleep(2.seconds)
+
+ level.segmentsInLevel() foreach {
+ segment =>
+ level.refresh(segment).right.right.value.right.value
+ }
+
+ //expired key-values return empty after 2.seconds
+ keyValues foreach {
+ keyValue =>
+ level.get(keyValue.key, ThreadReadState.random).runRandomIO.right.value.toOptionPut shouldBe empty
+ }
+
+ level.segmentFilesInAppendix shouldBe 0
+
+ level.isEmpty shouldBe true
+
+ if (persistent) {
+ level.reopen.isEmpty shouldBe true
+ level.segmentFilesOnDisk shouldBe empty
+ }
}
}
"not return an empty level if all the key values in the Level were EXPIRED and if Level has a lower Level" in {
- val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb), nextLevel = Some(TestLevel()))
-
- val keyValues = randomPutKeyValues(keyValuesCount)
- level.putKeyValuesTest(keyValues).runRandomIO.right.value
-
- val deleteKeyValues = Slice.create[Memory](keyValues.size * 2)
- keyValues foreach {
- keyValue =>
- deleteKeyValues add Memory.remove(keyValue.key, 0.seconds)
- }
- //also add another set of Delete key-values where the keys do not belong to the Level but since there is no lower level
- //these delete keys should also be removed
- val lastKeyValuesId = keyValues.last.key.read[Int] + 1
- (lastKeyValuesId until keyValues.size + lastKeyValuesId) foreach {
- id =>
- deleteKeyValues add Memory.remove(id, randomly(expiredDeadline()))
- }
-
- level.putKeyValuesTest(deleteKeyValues).runRandomIO.right.value
-
- //expired key-values return empty.
- keyValues foreach {
- keyValue =>
- level.get(keyValue.key, ThreadReadState.random).runRandomIO.right.value.toOptionPut shouldBe empty
- }
-
- //sleep for 2.seconds and Segments should still exists.
- sleep(2.seconds)
- level.isEmpty shouldBe false
- level.segmentFilesInAppendix should be >= 1
-
- if (persistent) {
- level.reopen.isEmpty shouldBe false
- level.segmentFilesOnDisk.size should be >= 1
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb), nextLevel = Some(TestLevel()))
+
+ val keyValues = randomPutKeyValues(keyValuesCount)
+ level.putKeyValuesTest(keyValues).runRandomIO.right.value
+
+ val deleteKeyValues = Slice.create[Memory](keyValues.size * 2)
+ keyValues foreach {
+ keyValue =>
+ deleteKeyValues add Memory.remove(keyValue.key, 0.seconds)
+ }
+ //also add another set of Delete key-values where the keys do not belong to the Level but since there is no lower level
+ //these delete keys should also be removed
+ val lastKeyValuesId = keyValues.last.key.read[Int] + 1
+ (lastKeyValuesId until keyValues.size + lastKeyValuesId) foreach {
+ id =>
+ deleteKeyValues add Memory.remove(id, randomly(expiredDeadline()))
+ }
+
+ level.putKeyValuesTest(deleteKeyValues).runRandomIO.right.value
+
+ //expired key-values return empty.
+ keyValues foreach {
+ keyValue =>
+ level.get(keyValue.key, ThreadReadState.random).runRandomIO.right.value.toOptionPut shouldBe empty
+ }
+
+ //sleep for 2.seconds and Segments should still exists.
+ sleep(2.seconds)
+ level.isEmpty shouldBe false
+ level.segmentFilesInAppendix should be >= 1
+
+ if (persistent) {
+ level.reopen.isEmpty shouldBe false
+ level.segmentFilesOnDisk.size should be >= 1
+ }
}
}
"return an empty level if all the key values in the Level were EXPIRED by RANGE and if Level is the only Level" in {
- val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb))
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb))
- val keyValues = randomPutKeyValues(keyValuesCount)
- level.putKeyValuesTest(keyValues).runRandomIO.right.value
+ val keyValues = randomPutKeyValues(keyValuesCount)
+ level.putKeyValuesTest(keyValues).runRandomIO.right.value
- level.putKeyValuesTest(Slice(Memory.Range(keyValues.head.key, keyValues.last.key.readInt() + 1, Value.FromValue.Null, Value.remove(2.seconds.fromNow)))).runRandomIO.right.value
+ level.putKeyValuesTest(Slice(Memory.Range(keyValues.head.key, keyValues.last.key.readInt() + 1, Value.FromValue.Null, Value.remove(2.seconds.fromNow)))).runRandomIO.right.value
- //expired key-values return empty after 2.seconds
- eventual(5.seconds) {
- keyValues foreach {
- keyValue =>
- level.get(keyValue.key, ThreadReadState.random).runRandomIO.right.value.toOptionPut shouldBe empty
- }
- }
+ //expired key-values return empty after 2.seconds
+ eventual(5.seconds) {
+ keyValues foreach {
+ keyValue =>
+ level.get(keyValue.key, ThreadReadState.random).runRandomIO.right.value.toOptionPut shouldBe empty
+ }
+ }
- level.segmentsInLevel() foreach {
- segment =>
- level.refresh(segment).right.right.value.right.value
- }
+ level.segmentsInLevel() foreach {
+ segment =>
+ level.refresh(segment).right.right.value.right.value
+ }
- level.segmentFilesInAppendix shouldBe 0
+ level.segmentFilesInAppendix shouldBe 0
- level.isEmpty shouldBe true
+ level.isEmpty shouldBe true
- if (persistent) {
- level.reopen.isEmpty shouldBe true
- level.segmentFilesOnDisk shouldBe empty
+ if (persistent) {
+ level.reopen.isEmpty shouldBe true
+ level.segmentFilesOnDisk shouldBe empty
+ }
}
}
"not return an empty level if all the key values in the Level were EXPIRED by RANGE and if Level has a last Level" in {
- val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb), nextLevel = Some(TestLevel()))
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb), nextLevel = Some(TestLevel()))
- val keyValues = randomPutKeyValues(keyValuesCount)
- level.putKeyValuesTest(keyValues).runRandomIO.right.value
+ val keyValues = randomPutKeyValues(keyValuesCount)
+ level.putKeyValuesTest(keyValues).runRandomIO.right.value
- level.putKeyValuesTest(Slice(Memory.Range(keyValues.head.key, keyValues.last.key.readInt() + 1, Value.FromValue.Null, Value.remove(2.seconds.fromNow)))).runRandomIO.right.value
+ level.putKeyValuesTest(Slice(Memory.Range(keyValues.head.key, keyValues.last.key.readInt() + 1, Value.FromValue.Null, Value.remove(2.seconds.fromNow)))).runRandomIO.right.value
- //expired key-values return empty after 2.seconds
- eventual(5.seconds) {
- keyValues foreach {
- keyValue =>
- level.get(keyValue.key, ThreadReadState.random).runRandomIO.right.value.toOptionPut shouldBe empty
- }
- }
+ //expired key-values return empty after 2.seconds
+ eventual(5.seconds) {
+ keyValues foreach {
+ keyValue =>
+ level.get(keyValue.key, ThreadReadState.random).runRandomIO.right.value.toOptionPut shouldBe empty
+ }
+ }
- level.segmentFilesInAppendix should be >= 1
+ level.segmentFilesInAppendix should be >= 1
- level.isEmpty shouldBe false
+ level.isEmpty shouldBe false
- if (persistent) {
- level.reopen.isEmpty shouldBe false
- level.segmentFilesOnDisk.size should be >= 1
+ if (persistent) {
+ level.reopen.isEmpty shouldBe false
+ level.segmentFilesOnDisk.size should be >= 1
+ }
}
}
}
"putKeyValues" should {
"write key values to target segments and update appendix" in {
- val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 10.mb, deleteEventually = false))
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 10.mb, deleteEventually = false))
- val targetSegmentKeyValues = randomIntKeyStringValues()
- val targetSegment = TestSegment(keyValues = targetSegmentKeyValues, path = testSegmentFile.resolveSibling("10.seg")).runRandomIO.right.value
+ val targetSegmentKeyValues = randomIntKeyStringValues()
+ val targetSegment = TestSegment(keyValues = targetSegmentKeyValues, path = testSegmentFile.resolveSibling("10.seg")).runRandomIO.right.value
- val keyValues = randomPutKeyValues()
- val function = PrivateMethod[IO[swaydb.Error.Segment, Unit]]('putKeyValues)
- (level invokePrivate function(keyValues.size, keyValues, Seq(targetSegment), None)).runRandomIO.right.value
+ val keyValues = randomPutKeyValues()
+ val function = PrivateMethod[IO[swaydb.Error.Segment, Unit]]('putKeyValues)
+ (level invokePrivate function(keyValues.size, keyValues, Seq(targetSegment), None)).runRandomIO.right.value
- targetSegment.existsOnDisk shouldBe false //target Segment should be deleted
+ targetSegment.existsOnDisk shouldBe false //target Segment should be deleted
- assertGet(keyValues, level)
- assertGet(targetSegmentKeyValues, level)
- level.takeSmallSegments(10) should not be empty //min segment size is 10.mb
+ assertGet(keyValues, level)
+ assertGet(targetSegmentKeyValues, level)
+ level.takeSmallSegments(10) should not be empty //min segment size is 10.mb
- if (persistent) {
- val reopen = level.reopen
- assertGet(keyValues, reopen)
- assertGet(targetSegmentKeyValues, reopen)
+ if (persistent) {
+ val reopen = level.reopen
+ assertGet(keyValues, reopen)
+ assertGet(targetSegmentKeyValues, reopen)
+ }
}
}
"fail put if writing one KeyValue fails" in {
- val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 10.mb))
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 10.mb))
- val targetSegmentKeyValues = randomIntKeyStringValues()
- val targetSegment = TestSegment(keyValues = targetSegmentKeyValues).runRandomIO.right.value
+ val targetSegmentKeyValues = randomIntKeyStringValues()
+ val targetSegment = TestSegment(keyValues = targetSegmentKeyValues).runRandomIO.right.value
- val keyValues: Slice[KeyValue] = Slice.create[KeyValue](3) //null KeyValue will throw an exception and the put should be reverted
- keyValues.add(Memory.put(123))
- keyValues.add(Memory.put(1234, 12345))
- keyValues.add(Persistent.Put(_key = 1235, None, null, Time.empty, 10, 10, 10, 10, 10, 0)) //give it a null Reader so that it fails reading the value.
+ val keyValues: Slice[KeyValue] = Slice.create[KeyValue](3) //null KeyValue will throw an exception and the put should be reverted
+ keyValues.add(Memory.put(123))
+ keyValues.add(Memory.put(1234, 12345))
+ keyValues.add(Persistent.Put(_key = 1235, None, null, Time.empty, 10, 10, 10, 10, 10, 0)) //give it a null Reader so that it fails reading the value.
- val function = PrivateMethod[IO[swaydb.Error.Segment, Unit]]('putKeyValues)
- val failed = level invokePrivate function(keyValues.size, keyValues, Iterable(targetSegment), None)
- failed.isLeft shouldBe true
- failed.left.get.exception shouldBe a[NullPointerException]
+ val function = PrivateMethod[IO[swaydb.Error.Segment, Unit]]('putKeyValues)
+ val failed = level invokePrivate function(keyValues.size, keyValues, Iterable(targetSegment), None)
+ failed.isLeft shouldBe true
+ failed.left.get.exception shouldBe a[NullPointerException]
- level.get(123, ThreadReadState.random).runRandomIO.right.value.toOptionPut shouldBe empty
- level.get(1234, ThreadReadState.random).runRandomIO.right.value.toOptionPut shouldBe empty
+ level.get(123, ThreadReadState.random).runRandomIO.right.value.toOptionPut shouldBe empty
+ level.get(1234, ThreadReadState.random).runRandomIO.right.value.toOptionPut shouldBe empty
+ }
}
}
}
diff --git a/core/src/test/scala/swaydb/core/level/LevelMapSpec.scala b/core/src/test/scala/swaydb/core/level/LevelMapSpec.scala
index 6e177d7bc..b7a885f2d 100644
--- a/core/src/test/scala/swaydb/core/level/LevelMapSpec.scala
+++ b/core/src/test/scala/swaydb/core/level/LevelMapSpec.scala
@@ -29,18 +29,16 @@ import org.scalatest.PrivateMethodTester
import swaydb.IO
import swaydb.IOValues._
import swaydb.core.CommonAssertions._
-import swaydb.core.RunThis._
import swaydb.core.TestData._
-import swaydb.core.actor.{FileSweeper, MemorySweeper}
-import swaydb.core.data._
-import swaydb.core.actor.ByteBufferSweeper
import swaydb.core.actor.ByteBufferSweeper.ByteBufferSweeperActor
import swaydb.core.actor.FileSweeper.FileSweeperActor
+import swaydb.core.actor.MemorySweeper
+import swaydb.core.data._
import swaydb.core.level.zero.LevelZeroSkipListMerger
import swaydb.core.map.{Map, MapEntry, SkipListMerger}
import swaydb.core.segment.ThreadReadState
import swaydb.core.segment.format.a.block.segment.SegmentBlock
-import swaydb.core.{TestBase, TestSweeper, TestTimer}
+import swaydb.core.{TestBase, TestCaseSweeper, TestSweeper, TestTimer}
import swaydb.data.config.MMAP
import swaydb.data.order.{KeyOrder, TimeOrder}
import swaydb.data.slice.{Slice, SliceOption}
@@ -48,6 +46,7 @@ import swaydb.data.util.OperatingSystem
import swaydb.data.util.StorageUnits._
import swaydb.serializers.Default._
import swaydb.serializers._
+import TestCaseSweeper._
class LevelMapSpec0 extends LevelMapSpec
@@ -89,68 +88,82 @@ sealed trait LevelMapSpec extends TestBase with MockFactory with PrivateMethodTe
import swaydb.core.map.serializer.LevelZeroMapEntryWriter._
implicit val merged: SkipListMerger[SliceOption[Byte], MemoryOption, Slice[Byte], Memory] = LevelZeroSkipListMerger
- val map =
- if (persistent)
- Map.persistent[SliceOption[Byte], MemoryOption, Slice[Byte], Memory](
- nullKey = Slice.Null,
- nullValue = Memory.Null,
- folder = randomIntDirectory,
- mmap = MMAP.Enabled(OperatingSystem.isWindows),
- flushOnOverflow = true,
- fileSize = 1.mb,
- dropCorruptedTailEntries = false
- ).runRandomIO.right.value.item
- else
- Map.memory[SliceOption[Byte], MemoryOption, Slice[Byte], Memory](
- nullKey = Slice.Null,
- nullValue = Memory.Null
- )
-
- val keyValues = randomPutKeyValues(keyValuesCount, addRemoves = true, addPutDeadlines = false)
- keyValues foreach {
- keyValue =>
- map.writeSync(MapEntry.Put(keyValue.key, keyValue))
+ def createTestMap()(implicit sweeper: TestCaseSweeper) = {
+ val map =
+ if (persistent)
+ Map.persistent[SliceOption[Byte], MemoryOption, Slice[Byte], Memory](
+ nullKey = Slice.Null,
+ nullValue = Memory.Null,
+ folder = randomIntDirectory,
+ mmap = MMAP.Enabled(OperatingSystem.isWindows),
+ flushOnOverflow = true,
+ fileSize = 1.mb,
+ dropCorruptedTailEntries = false
+ ).runRandomIO.right.value.item.clean()
+ else
+ Map.memory[SliceOption[Byte], MemoryOption, Slice[Byte], Memory](
+ nullKey = Slice.Null,
+ nullValue = Memory.Null
+ )
+
+ val keyValues = randomPutKeyValues(keyValuesCount, addRemoves = true, addPutDeadlines = false)
+ keyValues foreach {
+ keyValue =>
+ map.writeSync(MapEntry.Put(keyValue.key, keyValue))
+ }
+
+ (map, keyValues)
}
"succeed" when {
"writing to an empty Level" in {
- val level = TestLevel()
- level.put(map).right.right.value.right.value should contain only level.levelNumber
- //since this is a new Segment and Level has no sub-level, all the deleted key-values will value removed.
- val (deletedKeyValues, otherKeyValues) = keyValues.partition(_.isInstanceOf[Memory.Remove])
-
- assertReads(otherKeyValues, level)
-
- //deleted key-values do not exist.
- deletedKeyValues foreach {
- deleted =>
- level.get(deleted.key, ThreadReadState.random).runRandomIO.right.value.toOptionPut shouldBe empty
+ TestCaseSweeper {
+ implicit sweeper =>
+ val (map, keyValues) = createTestMap()
+
+ val level = TestLevel()
+ level.put(map).right.right.value.right.value should contain only level.levelNumber
+ //since this is a new Segment and Level has no sub-level, all the deleted key-values will value removed.
+ val (deletedKeyValues, otherKeyValues) = keyValues.partition(_.isInstanceOf[Memory.Remove])
+
+ assertReads(otherKeyValues, level)
+
+ //deleted key-values do not exist.
+ deletedKeyValues foreach {
+ deleted =>
+ level.get(deleted.key, ThreadReadState.random).runRandomIO.right.value.toOptionPut shouldBe empty
+ }
}
}
"writing to a non empty Level" in {
- val level = TestLevel()
+ TestCaseSweeper {
+ implicit sweeper =>
+ val (map, keyValues) = createTestMap()
- //creating a Segment with existing string key-values
- val existingKeyValues = Array(Memory.put("one", "one"), Memory.put("two", "two"), Memory.put("three", "three"))
+ val level = TestLevel()
- val sortedExistingKeyValues =
- Slice(
- Array(
- //also randomly set expired deadline for Remove.
- Memory.put("one", "one"), Memory.put("two", "two"), Memory.put("three", "three"), Memory.remove("four", randomly(expiredDeadline()))
- ).sorted(keyOrder.on[KeyValue](_.key)))
+ //creating a Segment with existing string key-values
+ val existingKeyValues = Array(Memory.put("one", "one"), Memory.put("two", "two"), Memory.put("three", "three"))
- level.putKeyValuesTest(sortedExistingKeyValues).runRandomIO.right.value
+ val sortedExistingKeyValues =
+ Slice(
+ Array(
+ //also randomly set expired deadline for Remove.
+ Memory.put("one", "one"), Memory.put("two", "two"), Memory.put("three", "three"), Memory.remove("four", randomly(expiredDeadline()))
+ ).sorted(keyOrder.on[KeyValue](_.key)))
- //put a new map
- level.put(map).right.right.value.right.value should contain only level.levelNumber
- assertGet(keyValues.filterNot(_.isInstanceOf[Memory.Remove]), level)
+ level.putKeyValuesTest(sortedExistingKeyValues).runRandomIO.right.value
- level.get("one", ThreadReadState.random).runRandomIO.right.value.getPut shouldBe existingKeyValues(0)
- level.get("two", ThreadReadState.random).runRandomIO.right.value.getPut shouldBe existingKeyValues(1)
- level.get("three", ThreadReadState.random).runRandomIO.right.value.getPut shouldBe existingKeyValues(2)
- level.get("four", ThreadReadState.random).runRandomIO.right.value.toOptionPut shouldBe empty
+ //put a new map
+ level.put(map).right.right.value.right.value should contain only level.levelNumber
+ assertGet(keyValues.filterNot(_.isInstanceOf[Memory.Remove]), level)
+
+ level.get("one", ThreadReadState.random).runRandomIO.right.value.getPut shouldBe existingKeyValues(0)
+ level.get("two", ThreadReadState.random).runRandomIO.right.value.getPut shouldBe existingKeyValues(1)
+ level.get("three", ThreadReadState.random).runRandomIO.right.value.getPut shouldBe existingKeyValues(2)
+ level.get("four", ThreadReadState.random).runRandomIO.right.value.toOptionPut shouldBe empty
+ }
}
}
}
@@ -160,80 +173,102 @@ sealed trait LevelMapSpec extends TestBase with MockFactory with PrivateMethodTe
import swaydb.core.map.serializer.LevelZeroMapEntryWriter._
implicit val merged: SkipListMerger[SliceOption[Byte], MemoryOption, Slice[Byte], Memory] = LevelZeroSkipListMerger
- val map =
- if (persistent)
- Map.persistent[SliceOption[Byte], MemoryOption, Slice[Byte], Memory](
- nullKey = Slice.Null,
- nullValue = Memory.Null,
- folder = randomIntDirectory,
- mmap = MMAP.Enabled(OperatingSystem.isWindows),
- flushOnOverflow = true,
- fileSize = 1.mb,
- dropCorruptedTailEntries = false).runRandomIO.right.value.item
- else
- Map.memory[SliceOption[Byte], MemoryOption, Slice[Byte], Memory](
- nullKey = Slice.Null,
- nullValue = Memory.Null
- )
-
- val keyValues = randomPutKeyValues(keyValuesCount, addRemoves = true, addPutDeadlines = false)
- keyValues foreach {
- keyValue =>
- map.writeSync(MapEntry.Put(keyValue.key, keyValue))
+ def createTestMap()(implicit sweeper: TestCaseSweeper) = {
+ val map =
+ if (persistent)
+ Map.persistent[SliceOption[Byte], MemoryOption, Slice[Byte], Memory](
+ nullKey = Slice.Null,
+ nullValue = Memory.Null,
+ folder = randomIntDirectory,
+ mmap = MMAP.Enabled(OperatingSystem.isWindows),
+ flushOnOverflow = true,
+ fileSize = 1.mb,
+ dropCorruptedTailEntries = false).runRandomIO.right.value.item.clean()
+ else
+ Map.memory[SliceOption[Byte], MemoryOption, Slice[Byte], Memory](
+ nullKey = Slice.Null,
+ nullValue = Memory.Null
+ )
+
+ val keyValues = randomPutKeyValues(keyValuesCount, addRemoves = true, addPutDeadlines = false)
+ keyValues foreach {
+ keyValue =>
+ map.writeSync(MapEntry.Put(keyValue.key, keyValue))
+ }
+
+ (map, keyValues)
}
"succeed" when {
"writing to an empty Level by copying to last Level" in {
- val nextLevel = mock[NextLevel]
-
- (nextLevel.isTrash _).expects() returning false
-
- (nextLevel.isCopyable(_: Map[SliceOption[Byte], MemoryOption, Slice[Byte], Memory])) expects * onCall {
- putMap: Map[SliceOption[Byte], MemoryOption, Slice[Byte], Memory] =>
- putMap.pathOption shouldBe map.pathOption
- true
- }
-
- (nextLevel.put(_: Map[SliceOption[Byte], MemoryOption, Slice[Byte], Memory])) expects * onCall {
- putMap: Map[SliceOption[Byte], MemoryOption, Slice[Byte], Memory] =>
- putMap.pathOption shouldBe map.pathOption
- implicit val nothingExceptionHandler = IO.ExceptionHandler.Nothing
- IO.Right[Nothing, IO[Nothing, Set[Int]]](IO.Right[Nothing, Set[Int]](Set(Int.MaxValue)))
+ TestCaseSweeper {
+ implicit sweeper =>
+ val (map, keyValues) = createTestMap()
+
+ val nextLevel = mock[NextLevel]
+
+ (nextLevel.isTrash _).expects() returning false
+
+ (nextLevel.isCopyable(_: Map[SliceOption[Byte], MemoryOption, Slice[Byte], Memory])) expects * onCall {
+ putMap: Map[SliceOption[Byte], MemoryOption, Slice[Byte], Memory] =>
+ putMap.pathOption shouldBe map.pathOption
+ true
+ }
+
+ (nextLevel.put(_: Map[SliceOption[Byte], MemoryOption, Slice[Byte], Memory])) expects * onCall {
+ putMap: Map[SliceOption[Byte], MemoryOption, Slice[Byte], Memory] =>
+ putMap.pathOption shouldBe map.pathOption
+ implicit val nothingExceptionHandler = IO.ExceptionHandler.Nothing
+ IO.Right[Nothing, IO[Nothing, Set[Int]]](IO.Right[Nothing, Set[Int]](Set(Int.MaxValue)))
+ }
+
+ (nextLevel.closeNoSweep _).expects().returning(IO[swaydb.Error.Level, Unit](())).atLeastOnce()
+ (nextLevel.closeSegments _).expects().returning(IO[swaydb.Error.Level, Unit](())).atLeastOnce()
+ (nextLevel.releaseLocks _).expects().returning(IO[swaydb.Error.Close, Unit](())).atLeastOnce()
+ (nextLevel.deleteNoSweep _).expects().returning(IO[swaydb.Error.Level, Unit](())).atLeastOnce()
+
+ val level = TestLevel(nextLevel = Some(nextLevel), segmentConfig = SegmentBlock.Config.random2(pushForward = true))
+ level.put(map).right.right.value.right.value should contain only Int.MaxValue
+ assertGetNoneFromThisLevelOnly(keyValues, level) //because nextLevel is a mock.
}
-
- val level = TestLevel(nextLevel = Some(nextLevel), segmentConfig = SegmentBlock.Config.random2(pushForward = true))
- level.put(map).right.right.value.right.value should contain only Int.MaxValue
- assertGetNoneFromThisLevelOnly(keyValues, level) //because nextLevel is a mock.
}
"writing to non empty Levels by copying to last Level if key-values do not overlap upper Level" in {
- val nextLevel = mock[NextLevel]
-
- val lastLevelKeyValues = randomPutKeyValues(keyValuesCount, addRemoves = true, addPutDeadlines = false, startId = Some(1))
- val map = TestMap(lastLevelKeyValues)
-
- (nextLevel.isTrash _).expects() returning false
-
- (nextLevel.isCopyable(_: Map[SliceOption[Byte], MemoryOption, Slice[Byte], Memory])) expects * onCall {
- putMap: Map[SliceOption[Byte], MemoryOption, Slice[Byte], Memory] =>
- putMap.pathOption shouldBe map.pathOption
- true
- }
-
- (nextLevel.put(_: Map[SliceOption[Byte], MemoryOption, Slice[Byte], Memory])) expects * onCall {
- putMap: Map[SliceOption[Byte], MemoryOption, Slice[Byte], Memory] =>
- putMap.pathOption shouldBe map.pathOption
- implicit val nothingExceptionHandler = IO.ExceptionHandler.Nothing
- IO.Right[Nothing, IO[Nothing, Set[Int]]](IO.Right[Nothing, Set[Int]](Set(Int.MaxValue)))
+ TestCaseSweeper {
+ implicit sweeper =>
+ val nextLevel = mock[NextLevel]
+
+ val lastLevelKeyValues = randomPutKeyValues(keyValuesCount, addRemoves = true, addPutDeadlines = false, startId = Some(1))
+ val map = TestMap(lastLevelKeyValues)
+
+ (nextLevel.isTrash _).expects() returning false
+
+ (nextLevel.isCopyable(_: Map[SliceOption[Byte], MemoryOption, Slice[Byte], Memory])) expects * onCall {
+ putMap: Map[SliceOption[Byte], MemoryOption, Slice[Byte], Memory] =>
+ putMap.pathOption shouldBe map.pathOption
+ true
+ }
+
+ (nextLevel.put(_: Map[SliceOption[Byte], MemoryOption, Slice[Byte], Memory])) expects * onCall {
+ putMap: Map[SliceOption[Byte], MemoryOption, Slice[Byte], Memory] =>
+ putMap.pathOption shouldBe map.pathOption
+ implicit val nothingExceptionHandler = IO.ExceptionHandler.Nothing
+ IO.Right[Nothing, IO[Nothing, Set[Int]]](IO.Right[Nothing, Set[Int]](Set(Int.MaxValue)))
+ }
+
+ (nextLevel.closeNoSweep _).expects().returning(IO[swaydb.Error.Level, Unit](())).atLeastOnce()
+ (nextLevel.closeSegments _).expects().returning(IO[swaydb.Error.Level, Unit](())).atLeastOnce()
+ (nextLevel.releaseLocks _).expects().returning(IO[swaydb.Error.Close, Unit](())).atLeastOnce()
+ (nextLevel.deleteNoSweep _).expects().returning(IO[swaydb.Error.Level, Unit](())).atLeastOnce()
+
+ val level = TestLevel(nextLevel = Some(nextLevel), segmentConfig = SegmentBlock.Config.random2(pushForward = true))
+ val keyValues = randomPutKeyValues(keyValuesCount, addRemoves = true, addPutDeadlines = false, startId = Some(lastLevelKeyValues.last.key.readInt() + 1000))
+ level.putKeyValues(keyValues.size, keyValues, Seq(TestSegment(keyValues)), None).runRandomIO.right.value
+
+ level.put(map).right.right.value.right.value should contain only Int.MaxValue
+ assertGetNoneFromThisLevelOnly(lastLevelKeyValues, level) //because nextLevel is a mock.
+ assertGetFromThisLevelOnly(keyValues, level)
}
-
- val level = TestLevel(nextLevel = Some(nextLevel), segmentConfig = SegmentBlock.Config.random2(pushForward = true))
- val keyValues = randomPutKeyValues(keyValuesCount, addRemoves = true, addPutDeadlines = false, startId = Some(lastLevelKeyValues.last.key.readInt() + 1000))
- level.putKeyValues(keyValues.size, keyValues, Seq(TestSegment(keyValues)), None).runRandomIO.right.value
-
- level.put(map).right.right.value.right.value should contain only Int.MaxValue
- assertGetNoneFromThisLevelOnly(lastLevelKeyValues, level) //because nextLevel is a mock.
- assertGetFromThisLevelOnly(keyValues, level)
}
}
}
diff --git a/core/src/test/scala/swaydb/core/level/LevelReadNoneSpec.scala b/core/src/test/scala/swaydb/core/level/LevelReadNoneSpec.scala
index d0445d3e0..ba80d0146 100644
--- a/core/src/test/scala/swaydb/core/level/LevelReadNoneSpec.scala
+++ b/core/src/test/scala/swaydb/core/level/LevelReadNoneSpec.scala
@@ -24,7 +24,6 @@
package swaydb.core.level
-import org.scalamock.scalatest.MockFactory
import swaydb.IOValues._
import swaydb.core.CommonAssertions._
import swaydb.core.RunThis._
@@ -59,7 +58,7 @@ class LevelReadNoneSpec3 extends LevelReadNoneSpec {
override def inMemoryStorage = true
}
-sealed trait LevelReadNoneSpec extends TestBase with MockFactory {
+sealed trait LevelReadNoneSpec extends TestBase {
// override def deleteFiles = false
diff --git a/core/src/test/scala/swaydb/core/level/LevelReadSomeSpec.scala b/core/src/test/scala/swaydb/core/level/LevelReadSomeSpec.scala
index 005eb043d..3f19da9d2 100644
--- a/core/src/test/scala/swaydb/core/level/LevelReadSomeSpec.scala
+++ b/core/src/test/scala/swaydb/core/level/LevelReadSomeSpec.scala
@@ -30,7 +30,7 @@ import swaydb.IO
import swaydb.IOValues._
import swaydb.core.CommonAssertions._
import swaydb.core.RunThis._
-import swaydb.core.TestBase
+import swaydb.core.{TestBase, TestCaseSweeper}
import swaydb.core.TestData._
import swaydb.core.segment.ThreadReadState
import swaydb.data.config.MMAP
@@ -124,16 +124,19 @@ sealed trait LevelReadSomeSpec extends TestBase with MockFactory {
//if test failed check merging all key-values result in the key returning none.
implicit val keyOrder = KeyOrder.default
implicit val timeOrder = TimeOrder.long
- val level: Level = TestLevel()
- level.putKeyValuesTest(level2KeyValues).runRandomIO.right.value
- level.putKeyValuesTest(level1KeyValues).runRandomIO.right.value
- level.putKeyValuesTest(level0KeyValues).runRandomIO.right.value
-
- //if after merging into a single Level the result is not empty then print all the failed exceptions.
- Try(IO.Defer(level.get(update.key, ThreadReadState.random).toOptionPut).runIO.runRandomIO.right.value shouldBe empty).failed foreach {
- exception =>
- exception.printStackTrace()
- throw testException
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level: Level = TestLevel()
+ level.putKeyValuesTest(level2KeyValues).runRandomIO.right.value
+ level.putKeyValuesTest(level1KeyValues).runRandomIO.right.value
+ level.putKeyValuesTest(level0KeyValues).runRandomIO.right.value
+
+ //if after merging into a single Level the result is not empty then print all the failed exceptions.
+ Try(IO.Defer(level.get(update.key, ThreadReadState.random).toOptionPut).runIO.runRandomIO.right.value shouldBe empty).failed foreach {
+ exception =>
+ exception.printStackTrace()
+ throw testException
+ }
}
case Failure(exception) =>
diff --git a/core/src/test/scala/swaydb/core/level/LevelReadSpec.scala b/core/src/test/scala/swaydb/core/level/LevelReadSpec.scala
index e143f4dc6..248f24260 100644
--- a/core/src/test/scala/swaydb/core/level/LevelReadSpec.scala
+++ b/core/src/test/scala/swaydb/core/level/LevelReadSpec.scala
@@ -35,7 +35,7 @@ import swaydb.core.segment.format.a.block.hashindex.HashIndexBlock
import swaydb.core.segment.format.a.block.segment.SegmentBlock
import swaydb.core.segment.format.a.block.sortedindex.SortedIndexBlock
import swaydb.core.segment.format.a.block.values.ValuesBlock
-import swaydb.core.{TestBase, TestTimer}
+import swaydb.core.{TestBase, TestCaseSweeper, TestTimer}
import swaydb.data.compaction.Throttle
import swaydb.data.config.MMAP
import swaydb.data.order.KeyOrder
@@ -75,130 +75,145 @@ sealed trait LevelReadSpec extends TestBase with MockFactory {
"Level.mightContainKey" should {
"return true for key-values that exists or else false (bloom filter test on reboot)" in {
- val keyValues = randomPutKeyValues(keyValuesCount, addPutDeadlines = false)
+ TestCaseSweeper {
+ implicit sweeper =>
+ val keyValues = randomPutKeyValues(keyValuesCount, addPutDeadlines = false)
- def assert(level: Level) = {
- keyValues foreach {
- keyValue =>
- level.mightContainKey(keyValue.key).runRandomIO.right.value shouldBe true
- }
+ def assert(level: Level) = {
+ keyValues foreach {
+ keyValue =>
+ level.mightContainKey(keyValue.key).runRandomIO.right.value shouldBe true
+ }
- level.mightContainKey("THIS KEY DOES NOT EXISTS").runRandomIO.right.value shouldBe false
- }
+ level.mightContainKey("THIS KEY DOES NOT EXISTS").runRandomIO.right.value shouldBe false
+ }
- val level = TestLevel()
- level.putKeyValuesTest(keyValues).runRandomIO.right.value
+ val level = TestLevel()
+ level.putKeyValuesTest(keyValues).runRandomIO.right.value
- assert(level)
- if (persistent) assert(level.reopen)
+ assert(level)
+ if (persistent) assert(level.reopen)
+ }
}
}
"Level.takeSmallSegments" should {
"filter smaller segments from a Level" in {
- //disable throttling so small segment compaction does not occur
- val level = TestLevel(nextLevel = None, throttle = (_) => Throttle(Duration.Zero, 0), segmentConfig = SegmentBlock.Config.random2(minSegmentSize = 1.kb))
-
- val keyValues = randomPutKeyValues(1000, addPutDeadlines = false)
- level.putKeyValuesTest(keyValues).runRandomIO.right.value
- //do another put so split occurs.
- level.putKeyValuesTest(keyValues.headSlice).runRandomIO.right.value
- level.segmentsCount() > 1 shouldBe true //ensure there are Segments in this Level
-
- if (persistent) {
- val reopen = level.reopen(segmentSize = 10.mb)
-
- reopen.takeSmallSegments(10000) should not be empty
- //iterate again on the same Iterable.
- // This test is to ensure that returned List is not a java Iterable which is only iterable once.
- reopen.takeSmallSegments(10000) should not be empty
-
- reopen.reopen(segmentSize = 10.mb).takeLargeSegments(1) shouldBe empty
+ TestCaseSweeper {
+ implicit sweeper =>
+ //disable throttling so small segment compaction does not occur
+ val level = TestLevel(nextLevel = None, throttle = (_) => Throttle(Duration.Zero, 0), segmentConfig = SegmentBlock.Config.random2(minSegmentSize = 1.kb))
+
+ val keyValues = randomPutKeyValues(1000, addPutDeadlines = false)
+ level.putKeyValuesTest(keyValues).runRandomIO.right.value
+ //do another put so split occurs.
+ level.putKeyValuesTest(keyValues.headSlice).runRandomIO.right.value
+ level.segmentsCount() > 1 shouldBe true //ensure there are Segments in this Level
+
+ if (persistent) {
+ val reopen = level.reopen(segmentSize = 10.mb)
+
+ reopen.takeSmallSegments(10000) should not be empty
+ //iterate again on the same Iterable.
+ // This test is to ensure that returned List is not a java Iterable which is only iterable once.
+ reopen.takeSmallSegments(10000) should not be empty
+
+ reopen.reopen(segmentSize = 10.mb).takeLargeSegments(1) shouldBe empty
+ }
}
}
}
"Level.meter" should {
"return Level stats" in {
- val level = TestLevel()
-
- val putKeyValues = randomPutKeyValues(keyValuesCount)
- //refresh so that if there is a compression running, this Segment will compressed.
- val segments =
- TestSegment(putKeyValues, segmentConfig = SegmentBlock.Config.random(minSegmentSize = Int.MaxValue, maxKeyValuesPerSegment = Int.MaxValue))
- .runRandomIO
- .right.value
- .refresh(
- removeDeletes = false,
- createdInLevel = 0,
- valuesConfig = ValuesBlock.Config.random,
- sortedIndexConfig = SortedIndexBlock.Config.random,
- binarySearchIndexConfig = BinarySearchIndexBlock.Config.random,
- hashIndexConfig = HashIndexBlock.Config.random,
- bloomFilterConfig = BloomFilterBlock.Config.random,
- segmentConfig = SegmentBlock.Config.random2(minSegmentSize = 100.mb)
- ).runRandomIO.right.value
-
- segments should have size 1
- val segment = segments.head
-
- level.put(Seq(segment)).right.right.value.right.value
-
- level.meter.segmentsCount shouldBe 1
- level.meter.levelSize shouldBe segment.segmentSize
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel()
+
+ val putKeyValues = randomPutKeyValues(keyValuesCount)
+ //refresh so that if there is a compression running, this Segment will compressed.
+ val segments =
+ TestSegment(putKeyValues, segmentConfig = SegmentBlock.Config.random(minSegmentSize = Int.MaxValue, maxKeyValuesPerSegment = Int.MaxValue))
+ .runRandomIO
+ .right.value
+ .refresh(
+ removeDeletes = false,
+ createdInLevel = 0,
+ valuesConfig = ValuesBlock.Config.random,
+ sortedIndexConfig = SortedIndexBlock.Config.random,
+ binarySearchIndexConfig = BinarySearchIndexBlock.Config.random,
+ hashIndexConfig = HashIndexBlock.Config.random,
+ bloomFilterConfig = BloomFilterBlock.Config.random,
+ segmentConfig = SegmentBlock.Config.random2(minSegmentSize = 100.mb)
+ ).runRandomIO.right.value
+
+ segments should have size 1
+ val segment = segments.head
+
+ level.put(Seq(segment)).right.right.value.right.value
+
+ level.meter.segmentsCount shouldBe 1
+ level.meter.levelSize shouldBe segment.segmentSize
+ }
}
}
"Level.meterFor" should {
"forward request to the right level" in {
- val level2 = TestLevel()
- val level1 = TestLevel(nextLevel = Some(level2))
-
- val putKeyValues = randomPutKeyValues(keyValuesCount)
- //refresh so that if there is a compression running, this Segment will compressed.
- val segments =
- TestSegment(putKeyValues, segmentConfig = SegmentBlock.Config.random(minSegmentSize = Int.MaxValue, maxKeyValuesPerSegment = Int.MaxValue))
- .runRandomIO.right.value
- .refresh(
- removeDeletes = false,
- createdInLevel = 0,
- valuesConfig = ValuesBlock.Config.random,
- sortedIndexConfig = SortedIndexBlock.Config.random,
- binarySearchIndexConfig = BinarySearchIndexBlock.Config.random,
- hashIndexConfig = HashIndexBlock.Config.random,
- bloomFilterConfig = BloomFilterBlock.Config.random,
- segmentConfig = SegmentBlock.Config.random2(minSegmentSize = 100.mb)
- ).runRandomIO.right.value
-
- segments should have size 1
- val segment = segments.head
-
- level2.put(Seq(segment)).right.right.value.right.value
-
- level1.meter.levelSize shouldBe 0
- level1.meter.segmentsCount shouldBe 0
-
- val level1Meter = level1.meterFor(level1.pathDistributor.headPath.folderId.toInt).get
- level1Meter.levelSize shouldBe 0
- level1Meter.segmentsCount shouldBe 0
-
- level2.meter.segmentsCount shouldBe 1
- level2.meter.levelSize shouldBe segment.segmentSize
-
- val level2Meter = level1.meterFor(level2.pathDistributor.headPath.folderId.toInt).get
- level2Meter.segmentsCount shouldBe 1
- level2Meter.levelSize shouldBe segment.segmentSize
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level2 = TestLevel()
+ val level1 = TestLevel(nextLevel = Some(level2))
+
+ val putKeyValues = randomPutKeyValues(keyValuesCount)
+ //refresh so that if there is a compression running, this Segment will compressed.
+ val segments =
+ TestSegment(putKeyValues, segmentConfig = SegmentBlock.Config.random(minSegmentSize = Int.MaxValue, maxKeyValuesPerSegment = Int.MaxValue))
+ .runRandomIO.right.value
+ .refresh(
+ removeDeletes = false,
+ createdInLevel = 0,
+ valuesConfig = ValuesBlock.Config.random,
+ sortedIndexConfig = SortedIndexBlock.Config.random,
+ binarySearchIndexConfig = BinarySearchIndexBlock.Config.random,
+ hashIndexConfig = HashIndexBlock.Config.random,
+ bloomFilterConfig = BloomFilterBlock.Config.random,
+ segmentConfig = SegmentBlock.Config.random2(minSegmentSize = 100.mb)
+ ).runRandomIO.right.value
+
+ segments should have size 1
+ val segment = segments.head
+
+ level2.put(Seq(segment)).right.right.value.right.value
+
+ level1.meter.levelSize shouldBe 0
+ level1.meter.segmentsCount shouldBe 0
+
+ val level1Meter = level1.meterFor(level1.pathDistributor.headPath.folderId.toInt).get
+ level1Meter.levelSize shouldBe 0
+ level1Meter.segmentsCount shouldBe 0
+
+ level2.meter.segmentsCount shouldBe 1
+ level2.meter.levelSize shouldBe segment.segmentSize
+
+ val level2Meter = level1.meterFor(level2.pathDistributor.headPath.folderId.toInt).get
+ level2Meter.segmentsCount shouldBe 1
+ level2Meter.levelSize shouldBe segment.segmentSize
+ }
}
"return None is Level does not exist" in {
- val level2 = TestLevel()
- val level1 = TestLevel(nextLevel = Some(level2))
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level2 = TestLevel()
+ val level1 = TestLevel(nextLevel = Some(level2))
- val putKeyValues = randomPutKeyValues(keyValuesCount)
- val segment = TestSegment(putKeyValues).runRandomIO.right.value
- level2.put(Seq(segment)).right.right.value.right.value
+ val putKeyValues = randomPutKeyValues(keyValuesCount)
+ val segment = TestSegment(putKeyValues).runRandomIO.right.value
+ level2.put(Seq(segment)).right.right.value.right.value
- level1.meterFor(3) shouldBe empty
+ level1.meterFor(3) shouldBe empty
+ }
}
}
}
diff --git a/core/src/test/scala/swaydb/core/level/LevelRefSpec.scala b/core/src/test/scala/swaydb/core/level/LevelRefSpec.scala
index fa1af6bb9..43041ca6f 100644
--- a/core/src/test/scala/swaydb/core/level/LevelRefSpec.scala
+++ b/core/src/test/scala/swaydb/core/level/LevelRefSpec.scala
@@ -28,9 +28,10 @@ import java.nio.file.Path
import org.scalamock.scalatest.MockFactory
import swaydb.IOValues._
-import swaydb.core.TestBase
+import swaydb.core.{TestBase, TestCaseSweeper}
import scala.collection.mutable.ListBuffer
+import scala.concurrent.duration.DurationInt
class LevelRefSpec extends TestBase with MockFactory {
@@ -53,95 +54,102 @@ class LevelRefSpec extends TestBase with MockFactory {
"getLevels" should {
"return all levels" in {
- val level3 = TestLevel()
- val level2 = TestLevel(nextLevel = Some(level3))
- val level1 = TestLevel(nextLevel = Some(level2))
- val level0 = TestLevelZero(nextLevel = Some(level1))
-
- val allPaths = Seq(level0, level1, level2, level3).map(_.rootPath)
-
- LevelRef.getLevels(level0).map(_.rootPath) shouldBe allPaths
- LevelRef.getLevels(level1).map(_.rootPath) shouldBe allPaths.drop(1)
- LevelRef.getLevels(level2).map(_.rootPath) shouldBe allPaths.drop(2)
- LevelRef.getLevels(level3).map(_.rootPath) shouldBe allPaths.drop(3)
-
- level0.close.runRandomIO.right.value
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level3 = TestLevel()
+ val level2 = TestLevel(nextLevel = Some(level3))
+ val level1 = TestLevel(nextLevel = Some(level2))
+ val level0 = TestLevelZero(nextLevel = Some(level1))
+
+ val allPaths = Seq(level0, level1, level2, level3).map(_.rootPath)
+
+ LevelRef.getLevels(level0).map(_.rootPath) shouldBe allPaths
+ LevelRef.getLevels(level1).map(_.rootPath) shouldBe allPaths.drop(1)
+ LevelRef.getLevels(level2).map(_.rootPath) shouldBe allPaths.drop(2)
+ LevelRef.getLevels(level3).map(_.rootPath) shouldBe allPaths.drop(3)
+ }
}
}
"foldLeft" when {
"single level" in {
- val level = TestLevel()
- val paths =
- level.foldLeftLevels(ListBuffer.empty[Path]) {
- case (paths, level) =>
- paths += level.rootPath
- }
-
- paths should contain only level.rootPath
-
- level.close.runRandomIO.right.value
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel()
+ val paths =
+ level.foldLeftLevels(ListBuffer.empty[Path]) {
+ case (paths, level) =>
+ paths += level.rootPath
+ }
+
+ paths should contain only level.rootPath
+ }
}
"multi level" in {
- val level3 = TestLevel()
- val level2 = TestLevel(nextLevel = Some(level3))
- val level1 = TestLevel(nextLevel = Some(level2))
- val level0 = TestLevelZero(nextLevel = Some(level1))
-
- def paths(level: LevelRef): Iterable[Path] =
- level.foldLeftLevels(ListBuffer.empty[Path]) {
- case (paths, level) =>
- paths += level.rootPath
- }
-
- val allPaths = Seq(level0, level1, level2, level3).map(_.rootPath)
-
- paths(level0) shouldBe allPaths
- paths(level1) shouldBe allPaths.drop(1)
- paths(level2) shouldBe allPaths.drop(2)
- paths(level3) shouldBe allPaths.drop(3)
-
- level0.close.runRandomIO.right.value
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level3 = TestLevel()
+ val level2 = TestLevel(nextLevel = Some(level3))
+ val level1 = TestLevel(nextLevel = Some(level2))
+ val level0 = TestLevelZero(nextLevel = Some(level1))
+
+ def paths(level: LevelRef): Iterable[Path] =
+ level.foldLeftLevels(ListBuffer.empty[Path]) {
+ case (paths, level) =>
+ paths += level.rootPath
+ }
+
+ val allPaths = Seq(level0, level1, level2, level3).map(_.rootPath)
+
+ paths(level0) shouldBe allPaths
+ paths(level1) shouldBe allPaths.drop(1)
+ paths(level2) shouldBe allPaths.drop(2)
+ paths(level3) shouldBe allPaths.drop(3)
+ }
}
}
"map" when {
"single level" in {
- val level = TestLevel()
- val paths = level.mapLevels(_.rootPath)
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel()
+ val paths = level.mapLevels(_.rootPath)
- paths should contain only level.rootPath
- level.close.runRandomIO.right.value
+ paths should contain only level.rootPath
+ }
}
"multi level" in {
- val level3 = TestLevel()
- val level2 = TestLevel(nextLevel = Some(level3))
- val level1 = TestLevel(nextLevel = Some(level2))
- val level0 = TestLevelZero(nextLevel = Some(level1))
-
- def paths(level: LevelRef) = level.mapLevels(_.rootPath)
-
- val allPaths = Seq(level0, level1, level2, level3).map(_.rootPath)
-
- paths(level0) shouldBe allPaths
- paths(level1) shouldBe allPaths.drop(1)
- paths(level2) shouldBe allPaths.drop(2)
- paths(level3) shouldBe allPaths.drop(3)
-
- level0.close.runRandomIO.right.value
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level3 = TestLevel()
+ val level2 = TestLevel(nextLevel = Some(level3))
+ val level1 = TestLevel(nextLevel = Some(level2))
+ val level0 = TestLevelZero(nextLevel = Some(level1))
+
+ def paths(level: LevelRef) = level.mapLevels(_.rootPath)
+
+ val allPaths = Seq(level0, level1, level2, level3).map(_.rootPath)
+
+ paths(level0) shouldBe allPaths
+ paths(level1) shouldBe allPaths.drop(1)
+ paths(level2) shouldBe allPaths.drop(2)
+ paths(level3) shouldBe allPaths.drop(3)
+ }
}
}
"reversedLevels" in {
- val level3 = TestLevel()
- val level2 = TestLevel(nextLevel = Some(level3))
- val level1 = TestLevel(nextLevel = Some(level2))
- val level0 = TestLevelZero(nextLevel = Some(level1))
-
- level0.reverseLevels.map(_.rootPath) shouldBe Seq(level3.rootPath, level2.rootPath, level1.rootPath, level0.rootPath)
-
- level0.close.runRandomIO.right.value
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level3 = TestLevel()
+ val level2 = TestLevel(nextLevel = Some(level3))
+ val level1 = TestLevel(nextLevel = Some(level2))
+ val level0 = TestLevelZero(nextLevel = Some(level1))
+
+ level0.reverseLevels.map(_.rootPath) shouldBe Seq(level3.rootPath, level2.rootPath, level1.rootPath, level0.rootPath)
+ }
}
}
diff --git a/core/src/test/scala/swaydb/core/level/LevelRefreshSpec.scala b/core/src/test/scala/swaydb/core/level/LevelRefreshSpec.scala
index 0cbe45efe..02af281e6 100644
--- a/core/src/test/scala/swaydb/core/level/LevelRefreshSpec.scala
+++ b/core/src/test/scala/swaydb/core/level/LevelRefreshSpec.scala
@@ -34,7 +34,7 @@ import swaydb.core.actor.{FileSweeper, MemorySweeper}
import swaydb.core.data._
import swaydb.core.level.zero.LevelZeroSkipListMerger
import swaydb.core.segment.format.a.block.segment.SegmentBlock
-import swaydb.core.{TestBase, TestSweeper, TestTimer}
+import swaydb.core.{TestBase, TestCaseSweeper, TestSweeper, TestTimer}
import swaydb.data.config.MMAP
import swaydb.data.order.{KeyOrder, TimeOrder}
import swaydb.data.slice.Slice
@@ -81,40 +81,46 @@ sealed trait LevelRefreshSpec extends TestBase with MockFactory with PrivateMeth
"refresh" should {
"remove expired key-values" in {
- val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.byte))
- val keyValues = randomPutKeyValues(1000, valueSize = 0, startId = Some(0))(TestTimer.Empty)
- level.putKeyValuesTest(keyValues).runRandomIO.right.value
- //dispatch another put request so that existing Segment gets split
- level.putKeyValuesTest(Slice(keyValues.head)).runRandomIO.right.value
- level.segmentsCount() should be >= 1
-
- //expire all key-values
- level.putKeyValuesTest(Slice(Memory.Range(0, Int.MaxValue, Value.FromValue.Null, Value.Remove(Some(2.seconds.fromNow), Time.empty)))).runRandomIO.right.value
- level.segmentFilesInAppendix should be > 1
-
- sleep(3.seconds)
- level.segmentsInLevel() foreach {
- segment =>
- level.refresh(segment).right.right.value
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.byte))
+ val keyValues = randomPutKeyValues(1000, valueSize = 0, startId = Some(0))(TestTimer.Empty)
+ level.putKeyValuesTest(keyValues).runRandomIO.right.value
+ //dispatch another put request so that existing Segment gets split
+ level.putKeyValuesTest(Slice(keyValues.head)).runRandomIO.right.value
+ level.segmentsCount() should be >= 1
+
+ //expire all key-values
+ level.putKeyValuesTest(Slice(Memory.Range(0, Int.MaxValue, Value.FromValue.Null, Value.Remove(Some(2.seconds.fromNow), Time.empty)))).runRandomIO.right.value
+ level.segmentFilesInAppendix should be > 1
+
+ sleep(3.seconds)
+ level.segmentsInLevel() foreach {
+ segment =>
+ level.refresh(segment).right.right.value
+ }
+
+ level.segmentFilesInAppendix shouldBe 0
}
-
- level.segmentFilesInAppendix shouldBe 0
}
"update createdInLevel" in {
- val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb))
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb))
- val keyValues = randomPutKeyValues(keyValuesCount, addExpiredPutDeadlines = false)
- val maps = TestMap(keyValues)
- level.put(maps).right.right.value
+ val keyValues = randomPutKeyValues(keyValuesCount, addExpiredPutDeadlines = false)
+ val maps = TestMap(keyValues)
+ level.put(maps).right.right.value
- val nextLevel = TestLevel()
- nextLevel.put(level.segmentsInLevel()).right.right.value
+ val nextLevel = TestLevel()
+ nextLevel.put(level.segmentsInLevel()).right.right.value
- if (persistent)
- nextLevel.segmentsInLevel() foreach (_.createdInLevel shouldBe level.levelNumber)
- nextLevel.segmentsInLevel() foreach (segment => nextLevel.refresh(segment).right.right.value)
- nextLevel.segmentsInLevel() foreach (_.createdInLevel shouldBe nextLevel.levelNumber)
+ if (persistent)
+ nextLevel.segmentsInLevel() foreach (_.createdInLevel shouldBe level.levelNumber)
+ nextLevel.segmentsInLevel() foreach (segment => nextLevel.refresh(segment).right.right.value)
+ nextLevel.segmentsInLevel() foreach (_.createdInLevel shouldBe nextLevel.levelNumber)
+ }
}
}
}
diff --git a/core/src/test/scala/swaydb/core/level/LevelRemoveSegmentSpec.scala b/core/src/test/scala/swaydb/core/level/LevelRemoveSegmentSpec.scala
index 97c8ed0ea..e7fb414a3 100644
--- a/core/src/test/scala/swaydb/core/level/LevelRemoveSegmentSpec.scala
+++ b/core/src/test/scala/swaydb/core/level/LevelRemoveSegmentSpec.scala
@@ -33,7 +33,7 @@ import swaydb.core.actor.FileSweeper.FileSweeperActor
import swaydb.core.actor.{FileSweeper, MemorySweeper}
import swaydb.core.level.zero.LevelZeroSkipListMerger
import swaydb.core.segment.format.a.block.segment.SegmentBlock
-import swaydb.core.{TestBase, TestSweeper, TestTimer}
+import swaydb.core.{TestBase, TestCaseSweeper, TestSweeper, TestTimer}
import swaydb.data.config.MMAP
import swaydb.data.order.{KeyOrder, TimeOrder}
import swaydb.data.slice.Slice
@@ -76,16 +76,19 @@ sealed trait LevelRemoveSegmentSpec extends TestBase with MockFactory with Priva
"removeSegments" should {
"remove segments from disk and remove them from appendix" in {
- val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb, deleteEventually = false))
- level.putKeyValuesTest(randomPutKeyValues(keyValuesCount)).runRandomIO.right.value
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb, deleteEventually = false))
+ level.putKeyValuesTest(randomPutKeyValues(keyValuesCount)).runRandomIO.right.value
- level.removeSegments(level.segmentsInLevel()).runRandomIO.right.value
+ level.removeSegments(level.segmentsInLevel()).runRandomIO.right.value
- level.isEmpty shouldBe true
+ level.isEmpty shouldBe true
- if (persistent) {
- level.segmentFilesOnDisk shouldBe empty
- level.reopen.isEmpty shouldBe true
+ if (persistent) {
+ level.segmentFilesOnDisk shouldBe empty
+ level.reopen.isEmpty shouldBe true
+ }
}
}
}
diff --git a/core/src/test/scala/swaydb/core/level/LevelSegmentSpec.scala b/core/src/test/scala/swaydb/core/level/LevelSegmentSpec.scala
index b395b4dd8..abb089299 100644
--- a/core/src/test/scala/swaydb/core/level/LevelSegmentSpec.scala
+++ b/core/src/test/scala/swaydb/core/level/LevelSegmentSpec.scala
@@ -30,10 +30,10 @@ import org.scalamock.scalatest.MockFactory
import swaydb.IO
import swaydb.IOValues._
import swaydb.core.CommonAssertions._
-import swaydb.core.RunThis._
import swaydb.core.TestData._
+import swaydb.core.TestCaseSweeper.TestLevelPathSweeperImplicits
import swaydb.core.actor.FileSweeper.FileSweeperActor
-import swaydb.core.actor.{FileSweeper, MemorySweeper}
+import swaydb.core.actor.MemorySweeper
import swaydb.core.data._
import swaydb.core.io.file.Effect._
import swaydb.core.level.zero.LevelZeroSkipListMerger
@@ -41,7 +41,7 @@ import swaydb.core.segment.Segment
import swaydb.core.segment.format.a.block.segment.SegmentBlock
import swaydb.core.util.PipeOps._
import swaydb.core.util.{Extension, IDGenerator}
-import swaydb.core.{TestBase, TestSweeper, TestTimer}
+import swaydb.core.{TestBase, TestCaseSweeper, TestSweeper, TestTimer}
import swaydb.data.config.{Dir, MMAP}
import swaydb.data.order.{KeyOrder, TimeOrder}
import swaydb.data.slice.Slice
@@ -88,226 +88,259 @@ sealed trait LevelSegmentSpec extends TestBase with MockFactory {
"writing Segments to single level" should {
"succeed" when {
"level is empty" in {
- val level = TestLevel()
- val keyValues = randomIntKeyStringValues(keyValuesCount)
- val segment = TestSegment(keyValues)
- segment.close.runRandomIO.right.value
- level.put(segment).right.right.value.right.value should contain only level.levelNumber
- assertReads(keyValues, level)
- level.close.runRandomIO.right.value
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel()
+ val keyValues = randomIntKeyStringValues(keyValuesCount)
+ val segment = TestSegment(keyValues)
+ segment.close.runRandomIO.right.value
+ level.put(segment).right.right.value.right.value should contain only level.levelNumber
+ assertReads(keyValues, level)
+ }
}
"level is non-empty" in {
- //small Segment size so that small Segments do not collapse when running this test
- // as reads do not value retried on failure in Level, they only value retried in LevelZero.
- val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 100.bytes))
- val keyValues = randomIntKeyStringValues(keyValuesCount)
- val segment = TestSegment(keyValues)
- level.put(segment).right.right.value.right.value should contain only level.levelNumber
-
- val keyValues2 = randomIntKeyStringValues(keyValuesCount * 10)
- val segment2 = TestSegment(keyValues2).runRandomIO.right.value
- level.put(segment2).right.right.value.right.value should contain only level.levelNumber
-
- assertGet(keyValues, level)
- assertGet(keyValues2, level)
+ TestCaseSweeper {
+ implicit sweeper =>
+ //small Segment size so that small Segments do not collapse when running this test
+ // as reads do not value retried on failure in Level, they only value retried in LevelZero.
+ val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 100.bytes))
+ val keyValues = randomIntKeyStringValues(keyValuesCount)
+ val segment = TestSegment(keyValues)
+ level.put(segment).right.right.value.right.value should contain only level.levelNumber
+
+ val keyValues2 = randomIntKeyStringValues(keyValuesCount * 10)
+ val segment2 = TestSegment(keyValues2).runRandomIO.right.value
+ level.put(segment2).right.right.value.right.value should contain only level.levelNumber
+
+ assertGet(keyValues, level)
+ assertGet(keyValues2, level)
+ }
}
"writing multiple Segments to an empty Level" in {
- val level = TestLevel()
- val keyValues = randomIntKeyStringValues(keyValuesCount * 3, valueSize = 1000)
-
- val (keyValues1, keyValues2, keyValues3) =
- keyValues
- .splitAt(keyValues.size / 3)
- .==> {
- case (split1, split2) =>
- val (two, three) = split2.splitAt(split2.size / 2)
- (split1, two, three)
- }
-
- val segments = Seq(TestSegment(keyValues1).runRandomIO.right.value, TestSegment(keyValues2).runRandomIO.right.value, TestSegment(keyValues3).runRandomIO.right.value)
- level.put(segments).right.right.value.right.value should contain only level.levelNumber
-
- assertReads(keyValues, level)
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel()
+ val keyValues = randomIntKeyStringValues(keyValuesCount * 3, valueSize = 1000)
+
+ val (keyValues1, keyValues2, keyValues3) =
+ keyValues
+ .splitAt(keyValues.size / 3)
+ .==> {
+ case (split1, split2) =>
+ val (two, three) = split2.splitAt(split2.size / 2)
+ (split1, two, three)
+ }
+
+ val segments = Seq(TestSegment(keyValues1).runRandomIO.right.value, TestSegment(keyValues2).runRandomIO.right.value, TestSegment(keyValues3).runRandomIO.right.value)
+ level.put(segments).right.right.value.right.value should contain only level.levelNumber
+
+ assertReads(keyValues, level)
+ }
}
"writing multiple Segments to a non empty Level" in {
- val level = TestLevel()
- val allKeyValues = randomPutKeyValues(keyValuesCount * 3, valueSize = 1000, addPutDeadlines = false)(TestTimer.Empty)
- val slicedKeyValues = allKeyValues.groupedSlice(3)
- val keyValues1 = slicedKeyValues(0)
- val keyValues2 = slicedKeyValues(1)
- val keyValues3 = slicedKeyValues(2)
-
- //create a level with key-values
- level.putKeyValuesTest(keyValues2).runRandomIO.right.value
- level.isEmpty shouldBe false
-
- val segments =
- Seq(
- TestSegment(keyValues1, segmentConfig = SegmentBlock.Config.random(minSegmentSize = Int.MaxValue)),
- TestSegment(keyValues3, segmentConfig = SegmentBlock.Config.random(minSegmentSize = Int.MaxValue))
- )
-
- level.put(segments).right.right.value.right.value should contain only level.levelNumber
-
- assertReads(allKeyValues, level)
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel()
+ val allKeyValues = randomPutKeyValues(keyValuesCount * 3, valueSize = 1000, addPutDeadlines = false)(TestTimer.Empty)
+ val slicedKeyValues = allKeyValues.groupedSlice(3)
+ val keyValues1 = slicedKeyValues(0)
+ val keyValues2 = slicedKeyValues(1)
+ val keyValues3 = slicedKeyValues(2)
+
+ //create a level with key-values
+ level.putKeyValuesTest(keyValues2).runRandomIO.right.value
+ level.isEmpty shouldBe false
+
+ val segments =
+ Seq(
+ TestSegment(keyValues1, segmentConfig = SegmentBlock.Config.random(minSegmentSize = Int.MaxValue)),
+ TestSegment(keyValues3, segmentConfig = SegmentBlock.Config.random(minSegmentSize = Int.MaxValue))
+ )
+
+ level.put(segments).right.right.value.right.value should contain only level.levelNumber
+
+ assertReads(allKeyValues, level)
+ }
}
"distribute Segments to multiple directories based on the distribution ratio" in {
if (persistent) {
- val dir = testDir.resolve("distributeSegmentsTest")
-
- def assertDistribution() = {
- dir.resolve(1.toString).files(Extension.Seg) should have size 7
- dir.resolve(2.toString).files(Extension.Seg) should have size 14
- dir.resolve(3.toString).files(Extension.Seg) should have size 21
- dir.resolve(4.toString).files(Extension.Seg) should have size 28
- dir.resolve(5.toString).files(Extension.Seg) should have size 30
- }
-
- val storage =
- LevelStorage.Persistent(
- dir = dir.resolve(1.toString),
- otherDirs =
- Seq(
- Dir(dir.resolve(2.toString), 2),
- Dir(dir.resolve(3.toString), 3),
- Dir(dir.resolve(4.toString), 4),
- Dir(dir.resolve(5.toString), 5)
+ TestCaseSweeper {
+ implicit sweeper =>
+ val dir = testDir.resolve("distributeSegmentsTest").clean()
+
+ def assertDistribution() = {
+ dir.resolve(1.toString).files(Extension.Seg) should have size 7
+ dir.resolve(2.toString).files(Extension.Seg) should have size 14
+ dir.resolve(3.toString).files(Extension.Seg) should have size 21
+ dir.resolve(4.toString).files(Extension.Seg) should have size 28
+ dir.resolve(5.toString).files(Extension.Seg) should have size 30
+ }
+
+ val storage =
+ LevelStorage.Persistent(
+ dir = dir.resolve(1.toString),
+ otherDirs =
+ Seq(
+ Dir(dir.resolve(2.toString), 2),
+ Dir(dir.resolve(3.toString), 3),
+ Dir(dir.resolve(4.toString), 4),
+ Dir(dir.resolve(5.toString), 5)
+ )
)
- )
- val keyValues = randomPutKeyValues(100)
- val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.byte, deleteEventually = false), levelStorage = storage)
+ val keyValues = randomPutKeyValues(100)
- level.putKeyValuesTest(keyValues).runRandomIO.right.value
- level.segmentsCount() shouldBe keyValues.size
- assertDistribution()
+ val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.byte, deleteEventually = false), levelStorage = storage)
- //write the same key-values again so that all Segments are updated. This should still maintain the Segment distribution
- level.putKeyValuesTest(keyValues).runRandomIO.right.value
- assertDistribution()
+ level.putKeyValuesTest(keyValues).runRandomIO.right.value
+ level.segmentsCount() shouldBe keyValues.size
+ assertDistribution()
- //shuffle key-values should still maintain distribution order
- Random.shuffle(keyValues.grouped(10)) foreach {
- keyValues =>
+ //write the same key-values again so that all Segments are updated. This should still maintain the Segment distribution
level.putKeyValuesTest(keyValues).runRandomIO.right.value
- }
- assertDistribution()
+ assertDistribution()
+
+ //shuffle key-values should still maintain distribution order
+ Random.shuffle(keyValues.grouped(10)) foreach {
+ keyValues =>
+ level.putKeyValuesTest(keyValues).runRandomIO.right.value
+ }
+ assertDistribution()
+
+ //delete some key-values
+ Random.shuffle(keyValues.grouped(10)).take(2) foreach {
+ keyValues =>
+ val deleteKeyValues = keyValues.map(keyValue => Memory.remove(keyValue.key)).toSlice
+ level.putKeyValuesTest(deleteKeyValues).runRandomIO.right.value
+ }
- //delete some key-values
- Random.shuffle(keyValues.grouped(10)).take(2) foreach {
- keyValues =>
- val deleteKeyValues = keyValues.map(keyValue => Memory.remove(keyValue.key)).toSlice
- level.putKeyValuesTest(deleteKeyValues).runRandomIO.right.value
+ level.putKeyValuesTest(keyValues).runRandomIO.right.value
+ assertDistribution()
}
-
- level.putKeyValuesTest(keyValues).runRandomIO.right.value
- assertDistribution()
}
}
"copy Segments if segmentsToMerge is empty" in {
- val keyValues = randomKeyValues(keyValuesCount).groupedSlice(5)
- val segmentToCopy = keyValues map (keyValues => TestSegment(keyValues))
+ TestCaseSweeper {
+ implicit sweeper =>
+ val keyValues = randomKeyValues(keyValuesCount).groupedSlice(5)
+ val segmentToCopy = keyValues map (keyValues => TestSegment(keyValues))
- val level = TestLevel()
+ val level = TestLevel()
- level.put(Seq.empty, segmentToCopy, Seq.empty).runRandomIO.right.value should contain only level.levelNumber
+ level.put(Seq.empty, segmentToCopy, Seq.empty).runRandomIO.right.value should contain only level.levelNumber
- level.isEmpty shouldBe false
- assertReads(keyValues.flatten, level)
+ level.isEmpty shouldBe false
+ assertReads(keyValues.flatten, level)
+ }
}
"copy and merge Segments" in {
- val keyValues = randomKeyValues(100).groupedSlice(10).toArray
- val segmentToCopy = keyValues.take(5) map (keyValues => TestSegment(keyValues))
- val segmentToMerge = keyValues.drop(5).take(4) map (keyValues => TestSegment(keyValues))
- val targetSegment = TestSegment(keyValues.last).runRandomIO.right.value
+ TestCaseSweeper {
+ implicit sweeper =>
+ val keyValues = randomKeyValues(100).groupedSlice(10).toArray
+ val segmentToCopy = keyValues.take(5) map (keyValues => TestSegment(keyValues))
+ val segmentToMerge = keyValues.drop(5).take(4) map (keyValues => TestSegment(keyValues))
+ val targetSegment = TestSegment(keyValues.last).runRandomIO.right.value
- val level = TestLevel()
- level.put(targetSegment).right.right.value.right.value should contain only level.levelNumber
- level.put(segmentToMerge, segmentToCopy, Seq(targetSegment)).runRandomIO.right.value should contain only level.levelNumber
+ val level = TestLevel()
+ level.put(targetSegment).right.right.value.right.value should contain only level.levelNumber
+ level.put(segmentToMerge, segmentToCopy, Seq(targetSegment)).runRandomIO.right.value should contain only level.levelNumber
- level.isEmpty shouldBe false
+ level.isEmpty shouldBe false
- assertGet(keyValues.flatten, level)
+ assertGet(keyValues.flatten, level)
+ }
}
}
"fail" when {
"fail when writing a deleted segment" in {
- val level = TestLevel()
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel()
- val keyValues = randomIntKeyStringValues()
- val segment = TestSegment(keyValues)
- segment.delete
+ val keyValues = randomIntKeyStringValues()
+ val segment = TestSegment(keyValues)
+ segment.delete
- val result = level.put(segment).right.right.value.left.get
- if (persistent)
- result.exception shouldBe a[NoSuchFileException]
- else
- result.exception shouldBe a[Exception]
+ val result = level.put(segment).right.right.value.left.get
+ if (persistent)
+ result.exception shouldBe a[NoSuchFileException]
+ else
+ result.exception shouldBe a[Exception]
- level.isEmpty shouldBe true
+ level.isEmpty shouldBe true
- //if it's a persistent Level, reopen to ensure that Segment did not value committed.
- if (persistent) level.reopen.isEmpty shouldBe true
+ //if it's a persistent Level, reopen to ensure that Segment did not value committed.
+ if (persistent) level.reopen.isEmpty shouldBe true
+ }
}
"return failure if segmentToMerge has no target Segment" in {
- val keyValues = randomKeyValues(keyValuesCount)
- val segmentsToMerge = TestSegment(keyValues)
- val level = TestLevel()
- level.put(Seq(segmentsToMerge), Seq(), Seq()).left.get shouldBe swaydb.Error.MergeKeyValuesWithoutTargetSegment(keyValues.size)
+ TestCaseSweeper {
+ implicit sweeper =>
+ val keyValues = randomKeyValues(keyValuesCount)
+ val segmentsToMerge = TestSegment(keyValues)
+ val level = TestLevel()
+ level.put(Seq(segmentsToMerge), Seq(), Seq()).left.get shouldBe swaydb.Error.MergeKeyValuesWithoutTargetSegment(keyValues.size)
+ }
}
"revert copy if merge fails" in {
if (persistent) {
- val keyValues = randomKeyValues(100)(TestTimer.Empty).groupedSlice(10).toArray
- val segmentToCopy = keyValues.take(5) map (keyValues => TestSegment(keyValues))
- val segmentToMerge = keyValues.drop(5).take(4) map (keyValues => TestSegment(keyValues))
- val targetSegment = TestSegment(keyValues.last).runRandomIO.right.value
-
- val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 150.bytes, deleteEventually = false))
- level.put(targetSegment).right.right.value.right.value should contain only level.levelNumber
-
- //segment to copy
- val id = IDGenerator.segmentId(level.segmentIDGenerator.nextID + 9)
- level.pathDistributor.queuedPaths foreach { //create this file in all paths.
- _ =>
- Files.createFile(level.pathDistributor.next.resolve(id))
+ TestCaseSweeper {
+ implicit sweeper =>
+ val keyValues = randomKeyValues(100)(TestTimer.Empty).groupedSlice(10).toArray
+ val segmentToCopy = keyValues.take(5) map (keyValues => TestSegment(keyValues))
+ val segmentToMerge = keyValues.drop(5).take(4) map (keyValues => TestSegment(keyValues))
+ val targetSegment = TestSegment(keyValues.last).runRandomIO.right.value
+
+ val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 150.bytes, deleteEventually = false))
+ level.put(targetSegment).right.right.value.right.value should contain only level.levelNumber
+
+ //segment to copy
+ val id = IDGenerator.segmentId(level.segmentIDGenerator.nextID + 9)
+ level.pathDistributor.queuedPaths foreach { //create this file in all paths.
+ _ =>
+ Files.createFile(level.pathDistributor.next.resolve(id))
+ }
+
+ val appendixBeforePut = level.segmentsInLevel()
+ val levelFilesBeforePut = level.segmentFilesOnDisk
+ level.put(segmentToMerge, segmentToCopy, Seq(targetSegment)).left.get.exception shouldBe a[FileAlreadyExistsException]
+ level.segmentFilesOnDisk shouldBe levelFilesBeforePut
+ level.segmentsInLevel().map(_.path) shouldBe appendixBeforePut.map(_.path)
}
-
- val appendixBeforePut = level.segmentsInLevel()
- val levelFilesBeforePut = level.segmentFilesOnDisk
- level.put(segmentToMerge, segmentToCopy, Seq(targetSegment)).left.get.exception shouldBe a[FileAlreadyExistsException]
- level.segmentFilesOnDisk shouldBe levelFilesBeforePut
- level.segmentsInLevel().map(_.path) shouldBe appendixBeforePut.map(_.path)
}
}
"revert copy on failure" in {
if (persistent) {
- val keyValues = randomKeyValues(keyValuesCount).groupedSlice(5)
- val segmentToCopy = keyValues map (keyValues => TestSegment(keyValues))
+ TestCaseSweeper {
+ implicit sweeper =>
+ val keyValues = randomKeyValues(keyValuesCount).groupedSlice(5)
+ val segmentToCopy = keyValues map (keyValues => TestSegment(keyValues))
- val level = TestLevel()
+ val level = TestLevel()
- //create a file with the same Segment name as the 4th Segment file. This should result in failure.
- val id = IDGenerator.segmentId(level.segmentIDGenerator.nextID + 4)
- level.pathDistributor.queuedPaths foreach { //create this file in all paths.
- _ =>
- Files.createFile(level.pathDistributor.next.resolve(id))
- }
- val levelFilesBeforePut = level.segmentFilesOnDisk
+ //create a file with the same Segment name as the 4th Segment file. This should result in failure.
+ val id = IDGenerator.segmentId(level.segmentIDGenerator.nextID + 4)
+ level.pathDistributor.queuedPaths foreach { //create this file in all paths.
+ _ =>
+ Files.createFile(level.pathDistributor.next.resolve(id))
+ }
+ val levelFilesBeforePut = level.segmentFilesOnDisk
- level.put(Seq.empty, segmentToCopy, Seq.empty).left.get.exception shouldBe a[FileAlreadyExistsException]
+ level.put(Seq.empty, segmentToCopy, Seq.empty).left.get.exception shouldBe a[FileAlreadyExistsException]
- level.isEmpty shouldBe true
- level.segmentFilesOnDisk shouldBe levelFilesBeforePut
+ level.isEmpty shouldBe true
+ level.segmentFilesOnDisk shouldBe levelFilesBeforePut
+ }
}
}
}
@@ -316,159 +349,193 @@ sealed trait LevelSegmentSpec extends TestBase with MockFactory {
"writing Segments to two levels" should {
"succeed" when {
"upper level has overlapping Segments" in {
- val nextLevel = mock[NextLevel]
-
- //no key-values value forwarded to next Level
- (nextLevel.isTrash _).expects() returning false
- (nextLevel.close _).expects() returning IO.unit
- (nextLevel.releaseLocks _).expects() returning IO.unit
- (nextLevel.closeSegments _).expects() returning IO.unit
-
- val level = TestLevel(nextLevel = Some(nextLevel))
- val keyValues = randomIntKeyStringValues(keyValuesCount, startId = Some(1))
- level.putKeyValues(keyValues.size, keyValues, Seq(TestSegment(keyValues)), None).value //write first Segment to Level
- assertGetFromThisLevelOnly(keyValues, level)
-
- level.put(TestSegment(keyValues.take(1))).right.right.value.right.value should contain only level.levelNumber
- level.put(TestSegment(keyValues.takeRight(1))).right.right.value.right.value should contain only level.levelNumber
-
- level.close.value
+ TestCaseSweeper {
+ implicit sweeper =>
+ val nextLevel = mock[NextLevel]
+
+ //no key-values value forwarded to next Level
+ (nextLevel.isTrash _).expects() returning false
+ (nextLevel.closeNoSweep _).expects().returning(IO[swaydb.Error.Level, Unit](())).atLeastOnce()
+ (nextLevel.closeSegments _).expects().returning(IO[swaydb.Error.Level, Unit](())).atLeastOnce()
+ (nextLevel.releaseLocks _).expects().returning(IO[swaydb.Error.Close, Unit](())).atLeastOnce()
+ (nextLevel.deleteNoSweep _).expects().returning(IO[swaydb.Error.Level, Unit](())).atLeastOnce()
+
+ val level = TestLevel(nextLevel = Some(nextLevel))
+ val keyValues = randomIntKeyStringValues(keyValuesCount, startId = Some(1))
+ level.putKeyValues(keyValues.size, keyValues, Seq(TestSegment(keyValues)), None).value //write first Segment to Level
+ assertGetFromThisLevelOnly(keyValues, level)
+
+ level.put(TestSegment(keyValues.take(1))).right.right.value.right.value should contain only level.levelNumber
+ level.put(TestSegment(keyValues.takeRight(1))).right.right.value.right.value should contain only level.levelNumber
+ }
}
"upper level has no overlapping Segments and nextLevel allows Segment copying" in {
- val nextLevel = mock[NextLevel]
- (nextLevel.isTrash _).expects() returning false
-
- val level = TestLevel(nextLevel = Some(nextLevel), segmentConfig = SegmentBlock.Config.random(pushForward = true))
- val keyValues = randomIntKeyStringValues(keyValuesCount, startId = Some(1))
- level.putKeyValues(keyValues.size, keyValues, Seq(TestSegment(keyValues)), None).runRandomIO.right.value //write first Segment to Level
- assertGetFromThisLevelOnly(keyValues, level)
-
- //write non-overlapping key-values
- val nextMaxKey = keyValues.last.key.readInt() + 1000
- val keyValues2 = randomIntKeyStringValues(keyValuesCount, startId = Some(nextMaxKey))
- val segment = TestSegment(keyValues2).runRandomIO.right.value
-
- (nextLevel.partitionUnreservedCopyable _).expects(*) onCall { //check if it can copied into next Level
- segments: Iterable[Segment] =>
- segments should have size 1
- segments.head.path shouldBe segment.path
- (segments, Iterable.empty)
- }
+ TestCaseSweeper {
+ implicit sweeper =>
+ val nextLevel = mock[NextLevel]
+ (nextLevel.isTrash _).expects() returning false
+
+ val level = TestLevel(nextLevel = Some(nextLevel), segmentConfig = SegmentBlock.Config.random(pushForward = true))
+ val keyValues = randomIntKeyStringValues(keyValuesCount, startId = Some(1))
+ level.putKeyValues(keyValues.size, keyValues, Seq(TestSegment(keyValues)), None).runRandomIO.right.value //write first Segment to Level
+ assertGetFromThisLevelOnly(keyValues, level)
+
+ //write non-overlapping key-values
+ val nextMaxKey = keyValues.last.key.readInt() + 1000
+ val keyValues2 = randomIntKeyStringValues(keyValuesCount, startId = Some(nextMaxKey))
+ val segment = TestSegment(keyValues2).runRandomIO.right.value
+
+ (nextLevel.partitionUnreservedCopyable _).expects(*) onCall { //check if it can copied into next Level
+ segments: Iterable[Segment] =>
+ segments should have size 1
+ segments.head.path shouldBe segment.path
+ (segments, Iterable.empty)
+ }
- (nextLevel.put(_: Iterable[Segment])) expects * onCall { //copy into next Level
- segments: Iterable[Segment] =>
- segments should have size 1
- segments.head.path shouldBe segment.path
- implicit val nothingExceptionHandler = IO.ExceptionHandler.Nothing
- IO.Right[Nothing, IO[Nothing, Set[Int]]](IO.Right[Nothing, Set[Int]](Set(Int.MaxValue)))
- }
+ (nextLevel.put(_: Iterable[Segment])) expects * onCall { //copy into next Level
+ segments: Iterable[Segment] =>
+ segments should have size 1
+ segments.head.path shouldBe segment.path
+ implicit val nothingExceptionHandler = IO.ExceptionHandler.Nothing
+ IO.Right[Nothing, IO[Nothing, Set[Int]]](IO.Right[Nothing, Set[Int]](Set(Int.MaxValue)))
+ }
+
+ (nextLevel.closeNoSweep _).expects().returning(IO[swaydb.Error.Level, Unit](())).atLeastOnce()
+ (nextLevel.closeSegments _).expects().returning(IO[swaydb.Error.Level, Unit](())).atLeastOnce()
+ (nextLevel.releaseLocks _).expects().returning(IO[swaydb.Error.Close, Unit](())).atLeastOnce()
+ (nextLevel.deleteNoSweep _).expects().returning(IO[swaydb.Error.Level, Unit](())).atLeastOnce()
- level.put(segment).right.right.value.right.value should contain only Int.MaxValue
+ level.put(segment).right.right.value.right.value should contain only Int.MaxValue
- assertGet(keyValues, level) //previous existing key-values should still exist
- assertGetNoneFromThisLevelOnly(keyValues2, level) //newly added key-values do not exist because nextLevel is mocked.
+ assertGet(keyValues, level) //previous existing key-values should still exist
+ assertGetNoneFromThisLevelOnly(keyValues2, level) //newly added key-values do not exist because nextLevel is mocked.
+ }
}
"upper level has no overlapping Segments and nextLevel does not allows Segment copying due to reserved Segments" in {
- val nextLevel = mock[NextLevel]
- (nextLevel.isTrash _).expects() returning false
-
- val level = TestLevel(nextLevel = Some(nextLevel), segmentConfig = SegmentBlock.Config.random(pushForward = true))
- val keyValues = randomIntKeyStringValues(keyValuesCount, startId = Some(1))
- level.putKeyValues(keyValues.size, keyValues, Seq(TestSegment(keyValues)), None).runRandomIO.right.value //write first Segment to Level
- assertGetFromThisLevelOnly(keyValues, level)
-
- //write non-overlapping key-values
- val nextMaxKey = keyValues.last.key.readInt() + 1000
- val keyValues2 = randomIntKeyStringValues(keyValuesCount, startId = Some(nextMaxKey))
- val segment = TestSegment(keyValues2).runRandomIO.right.value
-
- (nextLevel.partitionUnreservedCopyable _).expects(*) onCall { //check if it can copied into next Level
- segments: Iterable[Segment] =>
- segments should have size 1
- segments.head.path shouldBe segment.path
- (Iterable.empty, segments)
- }
+ TestCaseSweeper {
+ implicit sweeper =>
+ val nextLevel = mock[NextLevel]
+ (nextLevel.isTrash _).expects() returning false
+
+ val level = TestLevel(nextLevel = Some(nextLevel), segmentConfig = SegmentBlock.Config.random(pushForward = true))
+ val keyValues = randomIntKeyStringValues(keyValuesCount, startId = Some(1))
+ level.putKeyValues(keyValues.size, keyValues, Seq(TestSegment(keyValues)), None).runRandomIO.right.value //write first Segment to Level
+ assertGetFromThisLevelOnly(keyValues, level)
+
+ //write non-overlapping key-values
+ val nextMaxKey = keyValues.last.key.readInt() + 1000
+ val keyValues2 = randomIntKeyStringValues(keyValuesCount, startId = Some(nextMaxKey))
+ val segment = TestSegment(keyValues2).runRandomIO.right.value
+
+ (nextLevel.partitionUnreservedCopyable _).expects(*) onCall { //check if it can copied into next Level
+ segments: Iterable[Segment] =>
+ segments should have size 1
+ segments.head.path shouldBe segment.path
+ (Iterable.empty, segments)
+ }
- level.put(segment).right.right.value.right.value should contain only level.levelNumber
+ (nextLevel.closeNoSweep _).expects().returning(IO[swaydb.Error.Level, Unit](())).atLeastOnce()
+ (nextLevel.closeSegments _).expects().returning(IO[swaydb.Error.Level, Unit](())).atLeastOnce()
+ (nextLevel.releaseLocks _).expects().returning(IO[swaydb.Error.Close, Unit](())).atLeastOnce()
+ (nextLevel.deleteNoSweep _).expects().returning(IO[swaydb.Error.Level, Unit](())).atLeastOnce()
- assertGet(keyValues, level) //previous existing key-values should still exist
- assertGetFromThisLevelOnly(keyValues2, level) //newly added key-values do not exist because nextLevel is mocked.
+ level.put(segment).right.right.value.right.value should contain only level.levelNumber
+
+ assertGet(keyValues, level) //previous existing key-values should still exist
+ assertGetFromThisLevelOnly(keyValues2, level) //newly added key-values do not exist because nextLevel is mocked.
+ }
}
"lower level can copy 1 of 2 Segments" in {
- val nextLevel = mock[NextLevel]
- (nextLevel.isTrash _).expects() returning false
-
- val level = TestLevel(nextLevel = Some(nextLevel), segmentConfig = SegmentBlock.Config.random(pushForward = true))
- val keyValues = randomIntKeyStringValues(keyValuesCount, startId = Some(1))
- level.putKeyValues(keyValues.size, keyValues, Seq(TestSegment(keyValues)), None).runRandomIO.right.value //write first Segment to Level
- assertGet(keyValues, level)
-
- //write non-overlapping key-values
- val nextMaxKey = keyValues.last.key.readInt() + 1000
- val keyValues2 = randomIntKeyStringValues(keyValuesCount, startId = Some(nextMaxKey)).groupedSlice(2)
- val segment2 = TestSegment(keyValues2.head).runRandomIO.right.value
- val segment3 = TestSegment(keyValues2.last).runRandomIO.right.value
-
- (nextLevel.partitionUnreservedCopyable _).expects(*) onCall {
- segments: Iterable[Segment] =>
- segments should have size 2
- segments.head.path shouldBe segment2.path
- segments.last.path shouldBe segment3.path
- (Seq(segments.last), Seq(segments.head)) //last Segment is copyable.
- }
+ TestCaseSweeper {
+ implicit sweeper =>
+ val nextLevel = mock[NextLevel]
+ (nextLevel.isTrash _).expects() returning false
+
+ val level = TestLevel(nextLevel = Some(nextLevel), segmentConfig = SegmentBlock.Config.random(pushForward = true))
+ val keyValues = randomIntKeyStringValues(keyValuesCount, startId = Some(1))
+ level.putKeyValues(keyValues.size, keyValues, Seq(TestSegment(keyValues)), None).runRandomIO.right.value //write first Segment to Level
+ assertGet(keyValues, level)
+
+ //write non-overlapping key-values
+ val nextMaxKey = keyValues.last.key.readInt() + 1000
+ val keyValues2 = randomIntKeyStringValues(keyValuesCount, startId = Some(nextMaxKey)).groupedSlice(2)
+ val segment2 = TestSegment(keyValues2.head).runRandomIO.right.value
+ val segment3 = TestSegment(keyValues2.last).runRandomIO.right.value
+
+ (nextLevel.partitionUnreservedCopyable _).expects(*) onCall {
+ segments: Iterable[Segment] =>
+ segments should have size 2
+ segments.head.path shouldBe segment2.path
+ segments.last.path shouldBe segment3.path
+ (Seq(segments.last), Seq(segments.head)) //last Segment is copyable.
+ }
- (nextLevel.put(_: Iterable[Segment])) expects * onCall { //successfully copied last Segment into next Level.
- segments: Iterable[Segment] =>
- segments should have size 1
- segments.head.path shouldBe segment3.path
- implicit val nothingExceptionHandler = IO.ExceptionHandler.Nothing
- IO.Right[Nothing, IO[Nothing, Set[Int]]](IO.Right[Nothing, Set[Int]](Set(Int.MaxValue)))
- }
+ (nextLevel.put(_: Iterable[Segment])) expects * onCall { //successfully copied last Segment into next Level.
+ segments: Iterable[Segment] =>
+ segments should have size 1
+ segments.head.path shouldBe segment3.path
+ implicit val nothingExceptionHandler = IO.ExceptionHandler.Nothing
+ IO.Right[Nothing, IO[Nothing, Set[Int]]](IO.Right[Nothing, Set[Int]](Set(Int.MaxValue)))
+ }
- level.put(Seq(segment2, segment3)).right.right.value.right.value should contain only(level.levelNumber, Int.MaxValue)
+ (nextLevel.closeNoSweep _).expects().returning(IO[swaydb.Error.Level, Unit](())).atLeastOnce()
+ (nextLevel.closeSegments _).expects().returning(IO[swaydb.Error.Level, Unit](())).atLeastOnce()
+ (nextLevel.releaseLocks _).expects().returning(IO[swaydb.Error.Close, Unit](())).atLeastOnce()
+ (nextLevel.deleteNoSweep _).expects().returning(IO[swaydb.Error.Level, Unit](())).atLeastOnce()
- assertGetFromThisLevelOnly(keyValues, level) //all key-values value persisted into upper level.
- //segment2's key-values still readable from upper Level since they were copied locally.
- assertGetFromThisLevelOnly(keyValues2.head, level) //all key-values value persisted into upper level.
- assertGetNoneFromThisLevelOnly(keyValues2.last, level) //they were copied to lower level.
+ level.put(Seq(segment2, segment3)).right.right.value.right.value should contain only(level.levelNumber, Int.MaxValue)
+
+ assertGetFromThisLevelOnly(keyValues, level) //all key-values value persisted into upper level.
+ //segment2's key-values still readable from upper Level since they were copied locally.
+ assertGetFromThisLevelOnly(keyValues2.head, level) //all key-values value persisted into upper level.
+ assertGetNoneFromThisLevelOnly(keyValues2.last, level) //they were copied to lower level.
+ }
}
"lower level can copy all Segments but fails to copy" in {
- val nextLevel = mock[NextLevel]
- (nextLevel.isTrash _).expects() returning false
-
- val level = TestLevel(nextLevel = Some(nextLevel), segmentConfig = SegmentBlock.Config.random(pushForward = true))
- val keyValues = randomIntKeyStringValues(keyValuesCount, startId = Some(1))
- level.putKeyValues(keyValues.size, keyValues, Seq(TestSegment(keyValues)), None).runRandomIO.right.value //write first Segment to Level
- assertGet(keyValues, level)
-
- //write non-overlapping key-values
- val nextMaxKey = keyValues.last.key.readInt() + 1000
- val keyValues2 = randomIntKeyStringValues(keyValuesCount, startId = Some(nextMaxKey))
- val segment = TestSegment(keyValues2).runRandomIO.right.value
-
- (nextLevel.partitionUnreservedCopyable _).expects(*) onCall { //check if it can copied into next Level
- segments: Iterable[Segment] =>
- segments should have size 1
- segments.head.path shouldBe segment.path //new segments gets requested to push forward.
- (segments, Iterable.empty)
- }
+ TestCaseSweeper {
+ implicit sweeper =>
+ val nextLevel = mock[NextLevel]
+ (nextLevel.isTrash _).expects() returning false
+
+ val level = TestLevel(nextLevel = Some(nextLevel), segmentConfig = SegmentBlock.Config.random(pushForward = true))
+ val keyValues = randomIntKeyStringValues(keyValuesCount, startId = Some(1))
+ level.putKeyValues(keyValues.size, keyValues, Seq(TestSegment(keyValues)), None).runRandomIO.right.value //write first Segment to Level
+ assertGet(keyValues, level)
+
+ //write non-overlapping key-values
+ val nextMaxKey = keyValues.last.key.readInt() + 1000
+ val keyValues2 = randomIntKeyStringValues(keyValuesCount, startId = Some(nextMaxKey))
+ val segment = TestSegment(keyValues2).runRandomIO.right.value
+
+ (nextLevel.partitionUnreservedCopyable _).expects(*) onCall { //check if it can copied into next Level
+ segments: Iterable[Segment] =>
+ segments should have size 1
+ segments.head.path shouldBe segment.path //new segments gets requested to push forward.
+ (segments, Iterable.empty)
+ }
- (nextLevel.put(_: Iterable[Segment])) expects * onCall { //copy into next Level
- segments: Iterable[Segment] =>
- segments should have size 1
- segments.head.path shouldBe segment.path
- implicit val nothingExceptionHandler = IO.ExceptionHandler.Nothing
- IO.Right[Nothing, IO[swaydb.Error.Level, Set[Int]]](IO[swaydb.Error.Level, Set[Int]](throw IO.throwable("Kaboom!!")))
- }
+ (nextLevel.put(_: Iterable[Segment])) expects * onCall { //copy into next Level
+ segments: Iterable[Segment] =>
+ segments should have size 1
+ segments.head.path shouldBe segment.path
+ implicit val nothingExceptionHandler = IO.ExceptionHandler.Nothing
+ IO.Right[Nothing, IO[swaydb.Error.Level, Set[Int]]](IO[swaydb.Error.Level, Set[Int]](throw IO.throwable("Kaboom!!")))
+ }
- level.put(segment).right.right.value.right.value should contain only level.levelNumber
+ (nextLevel.closeNoSweep _).expects().returning(IO[swaydb.Error.Level, Unit](())).atLeastOnce()
+ (nextLevel.closeSegments _).expects().returning(IO[swaydb.Error.Level, Unit](())).atLeastOnce()
+ (nextLevel.releaseLocks _).expects().returning(IO[swaydb.Error.Close, Unit](())).atLeastOnce()
+ (nextLevel.deleteNoSweep _).expects().returning(IO[swaydb.Error.Level, Unit](())).atLeastOnce()
- assertGetFromThisLevelOnly(keyValues, level) //all key-values value persisted into upper level.
- assertGetFromThisLevelOnly(keyValues2, level) //all key-values value persisted into upper level.
+ level.put(segment).right.right.value.right.value should contain only level.levelNumber
+
+ assertGetFromThisLevelOnly(keyValues, level) //all key-values value persisted into upper level.
+ assertGetFromThisLevelOnly(keyValues2, level) //all key-values value persisted into upper level.
+ }
}
}
}
diff --git a/core/src/test/scala/swaydb/core/level/LevelSpec.scala b/core/src/test/scala/swaydb/core/level/LevelSpec.scala
index 0bbee1ad7..959bdf06c 100644
--- a/core/src/test/scala/swaydb/core/level/LevelSpec.scala
+++ b/core/src/test/scala/swaydb/core/level/LevelSpec.scala
@@ -30,10 +30,8 @@ import org.scalamock.scalatest.MockFactory
import org.scalatest.PrivateMethodTester
import swaydb.IO
import swaydb.IOValues._
-import swaydb.core.RunThis._
import swaydb.core.TestData._
-import swaydb.core.actor.FileSweeper.FileSweeperActor
-import swaydb.core.actor.{FileSweeper, MemorySweeper}
+import swaydb.core.TestCaseSweeper._
import swaydb.core.data._
import swaydb.core.io.file.Effect
import swaydb.core.io.file.Effect._
@@ -42,7 +40,7 @@ import swaydb.core.map.MapEntry
import swaydb.core.segment.Segment
import swaydb.core.segment.format.a.block.segment.SegmentBlock
import swaydb.core.util.{Extension, ReserveRange}
-import swaydb.core.{TestBase, TestSweeper, TestTimer}
+import swaydb.core.{TestBase, TestCaseSweeper, TestTimer}
import swaydb.data.config.{Dir, MMAP}
import swaydb.data.order.{KeyOrder, TimeOrder}
import swaydb.data.slice.Slice
@@ -84,8 +82,6 @@ sealed trait LevelSpec extends TestBase with MockFactory with PrivateMethodTeste
// override def deleteFiles: Boolean =
// false
- implicit val maxOpenSegmentsCacheImplicitLimiter: FileSweeperActor = TestSweeper.fileSweeper
- implicit val memorySweeperImplicitSweeper: Option[MemorySweeper.All] = TestSweeper.memorySweeperMax
implicit val skipListMerger = LevelZeroSkipListMerger
"acquireLock" should {
@@ -119,55 +115,58 @@ sealed trait LevelSpec extends TestBase with MockFactory with PrivateMethodTeste
"apply" should {
"create level" in {
- val level = TestLevel()
- if (memory) {
- //memory level always have one folder
- level.dirs should have size 1
- level.existsOnDisk shouldBe false
- level.inMemory shouldBe true
- // level.valuesConfig.compressDuplicateValues shouldBe true
- } else {
- level.existsOnDisk shouldBe true
- level.inMemory shouldBe false
-
- //there shouldBe at least one path
- level.dirs should not be empty
-
- //appendix path gets added to the head path
- val appendixPath = level.pathDistributor.headPath.resolve("appendix")
- appendixPath.exists shouldBe true
- appendixPath.resolve("0.log").exists shouldBe true
-
- //all paths should exists
- level.dirs.foreach(_.path.exists shouldBe true)
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel().clean()
+
+ if (memory) {
+ //memory level always have one folder
+ level.dirs should have size 1
+ level.existsOnDisk shouldBe false
+ level.inMemory shouldBe true
+ // level.valuesConfig.compressDuplicateValues shouldBe true
+ } else {
+ level.existsOnDisk shouldBe true
+ level.inMemory shouldBe false
+
+ //there shouldBe at least one path
+ level.dirs should not be empty
+
+ //appendix path gets added to the head path
+ val appendixPath = level.pathDistributor.headPath.resolve("appendix")
+ appendixPath.exists shouldBe true
+ appendixPath.resolve("0.log").exists shouldBe true
+
+ //all paths should exists
+ level.dirs.foreach(_.path.exists shouldBe true)
+ }
+
+ level.segmentsInLevel() shouldBe empty
+ level.removeDeletedRecords shouldBe true
}
-
- level.segmentsInLevel() shouldBe empty
- level.removeDeletedRecords shouldBe true
-
- level.delete.runRandomIO.right.value
- level.existsOnDisk shouldBe false
}
"report error if appendix file and folder does not exists" in {
if (persistent) {
- //create a non empty level
- val level = TestLevel()
- val segment = TestSegment(randomKeyValues(keyValuesCount))
+ TestCaseSweeper {
+ implicit sweeper =>
+ //create a non empty level
+ val level = TestLevel().clean()
- level.put(segment).right.right.value.right.value
+ val segment = TestSegment(randomKeyValues(keyValuesCount)).clean()
- //delete the appendix file
- level.pathDistributor.headPath.resolve("appendix").files(Extension.Log) map Effect.delete
- //expect failure when file does not exists
- level.tryReopen.left.get.exception shouldBe a[IllegalStateException]
+ level.put(segment).right.right.value.right.value
- //delete folder
- Effect.delete(level.pathDistributor.headPath.resolve("appendix")).runRandomIO.right.value
- //expect failure when folder does not exist
- level.tryReopen.left.get.exception shouldBe a[IllegalStateException]
+ //delete the appendix file
+ level.pathDistributor.headPath.resolve("appendix").files(Extension.Log) map Effect.delete
+ //expect failure when file does not exists
+ level.tryReopen.left.get.exception shouldBe a[IllegalStateException]
- level.delete.runRandomIO.right.value
+ //delete folder
+ Effect.delete(level.pathDistributor.headPath.resolve("appendix")).runRandomIO.right.value
+ //expect failure when folder does not exist
+ level.tryReopen.left.get.exception shouldBe a[IllegalStateException]
+ }
}
}
}
@@ -177,181 +176,199 @@ sealed trait LevelSpec extends TestBase with MockFactory with PrivateMethodTeste
if (memory) {
// memory Level do not have uncommitted Segments
} else {
- val level = TestLevel()
- level.putKeyValuesTest(randomPutKeyValues()).runRandomIO.right.value
- val segmentsIdsBeforeInvalidSegments = level.segmentFilesOnDisk
- segmentsIdsBeforeInvalidSegments should have size 1
-
- val currentSegmentId = segmentsIdsBeforeInvalidSegments.head.fileId.runRandomIO.right.value._1
-
- //create 3 invalid segments in all the paths of the Level
- level.dirs.foldLeft(currentSegmentId) {
- case (currentSegmentId, dir) =>
- TestSegment(path = dir.path.resolve((currentSegmentId + 1).toSegmentFileId)).runRandomIO.right.value
- TestSegment(path = dir.path.resolve((currentSegmentId + 2).toSegmentFileId)).runRandomIO.right.value
- TestSegment(path = dir.path.resolve((currentSegmentId + 3).toSegmentFileId)).runRandomIO.right.value
- currentSegmentId + 3
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel().clean()
+
+ level.putKeyValuesTest(randomPutKeyValues()).runRandomIO.right.value
+ val segmentsIdsBeforeInvalidSegments = level.segmentFilesOnDisk
+ segmentsIdsBeforeInvalidSegments should have size 1
+
+ val currentSegmentId = segmentsIdsBeforeInvalidSegments.head.fileId.runRandomIO.right.value._1
+
+ //create 3 invalid segments in all the paths of the Level
+ level.dirs.foldLeft(currentSegmentId) {
+ case (currentSegmentId, dir) =>
+ TestSegment(path = dir.path.resolve((currentSegmentId + 1).toSegmentFileId)).runRandomIO.right.value.clean()
+ TestSegment(path = dir.path.resolve((currentSegmentId + 2).toSegmentFileId)).runRandomIO.right.value.clean()
+ TestSegment(path = dir.path.resolve((currentSegmentId + 3).toSegmentFileId)).runRandomIO.right.value.clean()
+ currentSegmentId + 3
+ }
+ //every level folder has 3 uncommitted Segments plus 1 valid Segment
+ level.segmentFilesOnDisk should have size (level.dirs.size * 3) + 1
+
+ Level.deleteUncommittedSegments(level.dirs, level.segmentsInLevel()).runRandomIO.right.value
+
+ level.segmentFilesOnDisk should have size 1
+ level.segmentFilesOnDisk should contain only segmentsIdsBeforeInvalidSegments.head
+ level.reopen.segmentFilesOnDisk should contain only segmentsIdsBeforeInvalidSegments.head
}
- //every level folder has 3 uncommitted Segments plus 1 valid Segment
- level.segmentFilesOnDisk should have size (level.dirs.size * 3) + 1
-
- Level.deleteUncommittedSegments(level.dirs, level.segmentsInLevel()).runRandomIO.right.value
-
- level.segmentFilesOnDisk should have size 1
- level.segmentFilesOnDisk should contain only segmentsIdsBeforeInvalidSegments.head
- level.reopen.segmentFilesOnDisk should contain only segmentsIdsBeforeInvalidSegments.head
-
- level.delete.runRandomIO.right.value
}
}
}
"largestSegmentId" should {
"value the largest segment in the Level when the Level is not empty" in {
- val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb))
- level.putKeyValuesTest(randomizedKeyValues(2000)).runRandomIO.right.value
-
- val largeSegmentId = Level.largestSegmentId(level.segmentsInLevel())
- largeSegmentId shouldBe level.segmentsInLevel().map(_.path.fileId.runRandomIO.right.value._1).max
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb)).clean()
+ level.putKeyValuesTest(randomizedKeyValues(2000)).runRandomIO.right.value
- level.delete.runRandomIO.right.value
+ val largeSegmentId = Level.largestSegmentId(level.segmentsInLevel())
+ largeSegmentId shouldBe level.segmentsInLevel().map(_.path.fileId.runRandomIO.right.value._1).max
+ }
}
"return 0 when the Level is empty" in {
- val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb))
- Level.largestSegmentId(level.segmentsInLevel()) shouldBe 0
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb)).clean()
- level.delete.runRandomIO.right.value
+ Level.largestSegmentId(level.segmentsInLevel()) shouldBe 0
+ }
}
}
"optimalSegmentsToPushForward" should {
"return empty if there Levels are empty" in {
- val nextLevel = TestLevel()
- val level = TestLevel()
- implicit val reserve = ReserveRange.create[Unit]()
-
- Level.optimalSegmentsToPushForward(
- level = level,
- nextLevel = nextLevel,
- take = 10
- ) shouldBe Level.emptySegmentsToPush
-
- level.close.runRandomIO.right.value
- nextLevel.close.runRandomIO.right.value
-
- level.delete.runRandomIO.right.value
+ TestCaseSweeper {
+ implicit sweeper =>
+ val nextLevel = TestLevel()
+ val level = TestLevel().clean()
+ implicit val reserve = ReserveRange.create[Unit]()
+
+ Level.optimalSegmentsToPushForward(
+ level = level,
+ nextLevel = nextLevel,
+ take = 10
+ ) shouldBe Level.emptySegmentsToPush
+ }
}
"return all Segments to copy if next Level is empty" in {
- val nextLevel = TestLevel()
- val level = TestLevel(keyValues = randomizedKeyValues(count = 10000, startId = Some(1)), segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb))
- // level.segmentsCount() should be >= 2
-
- implicit val reserve = ReserveRange.create[Unit]()
-
- val (toCopy, toMerge) =
- Level.optimalSegmentsToPushForward(
- level = level,
- nextLevel = nextLevel,
- take = 10
- )
-
- toMerge shouldBe empty
- toCopy.map(_.path) shouldBe level.segmentsInLevel().take(10).map(_.path)
-
- level.close.runRandomIO.right.value
- nextLevel.close.runRandomIO.right.value
+ TestCaseSweeper {
+ implicit sweeper =>
+ val nextLevel = TestLevel()
+ val level = TestLevel(keyValues = randomizedKeyValues(count = 10000, startId = Some(1)), segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb)).clean()
+ // level.segmentsCount() should be >= 2
+
+ implicit val reserve = ReserveRange.create[Unit]()
+
+ val (toCopy, toMerge) =
+ Level.optimalSegmentsToPushForward(
+ level = level,
+ nextLevel = nextLevel,
+ take = 10
+ )
+
+ toMerge shouldBe empty
+ toCopy.map(_.path) shouldBe level.segmentsInLevel().take(10).map(_.path)
+ }
}
"return all unreserved Segments to copy if next Level is empty" in {
- val nextLevel = TestLevel()
- val level = TestLevel(keyValues = randomizedKeyValues(count = 10000, startId = Some(1)), segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb))
- level.segmentsCount() should be >= 2
+ TestCaseSweeper {
+ implicit sweeper =>
+ val nextLevel = TestLevel()
+ val level = TestLevel(keyValues = randomizedKeyValues(count = 10000, startId = Some(1)), segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb)).clean()
- implicit val reserve = ReserveRange.create[Unit]()
- val firstSegment = level.segmentsInLevel().head
+ level.segmentsCount() should be >= 2
- ReserveRange.reserveOrGet(firstSegment.minKey, firstSegment.maxKey.maxKey, firstSegment.maxKey.inclusive, ()) shouldBe empty //reserve first segment
+ implicit val reserve = ReserveRange.create[Unit]()
+ val firstSegment = level.segmentsInLevel().head
- val (toCopy, toMerge) =
- Level.optimalSegmentsToPushForward(
- level = level,
- nextLevel = nextLevel,
- take = 10
- )
+ ReserveRange.reserveOrGet(firstSegment.minKey, firstSegment.maxKey.maxKey, firstSegment.maxKey.inclusive, ()) shouldBe empty //reserve first segment
- toMerge shouldBe empty
- toCopy.map(_.path) shouldBe level.segmentsInLevel().drop(1).take(10).map(_.path)
+ val (toCopy, toMerge) =
+ Level.optimalSegmentsToPushForward(
+ level = level,
+ nextLevel = nextLevel,
+ take = 10
+ )
- level.delete.runRandomIO.right.value
- nextLevel.delete.runRandomIO.right.value
+ toMerge shouldBe empty
+ toCopy.map(_.path) shouldBe level.segmentsInLevel().drop(1).take(10).map(_.path)
+ }
}
}
"optimalSegmentsToCollapse" should {
"return empty if there Levels are empty" in {
- val level = TestLevel()
- implicit val reserve = ReserveRange.create[Unit]()
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel().clean()
- Level.optimalSegmentsToCollapse(
- level = level,
- take = 10
- ) shouldBe empty
+ implicit val reserve = ReserveRange.create[Unit]()
- level.delete.runRandomIO.right.value
+ Level.optimalSegmentsToCollapse(
+ level = level,
+ take = 10
+ ) shouldBe empty
+ }
}
"return empty if all segments were reserved" in {
- val keyValues = randomizedKeyValues(count = 10000, startId = Some(1))
- val level = TestLevel(keyValues = keyValues, segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb))
- level.segmentsCount() should be >= 2
+ TestCaseSweeper {
+ implicit sweeper =>
+ val keyValues = randomizedKeyValues(count = 10000, startId = Some(1))
+ val level = TestLevel(keyValues = keyValues, segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb)).clean()
- implicit val reserve = ReserveRange.create[Unit]()
+ level.segmentsCount() should be >= 2
- val minKey = keyValues.head.key
- val maxKey = Segment.minMaxKey(level.segmentsInLevel()).get
- ReserveRange.reserveOrGet(minKey, maxKey._2, maxKey._3, ()) shouldBe empty
+ implicit val reserve = ReserveRange.create[Unit]()
- Level.optimalSegmentsToCollapse(
- level = level,
- take = 10
- ) shouldBe empty
+ val minKey = keyValues.head.key
+ val maxKey = Segment.minMaxKey(level.segmentsInLevel()).get
+ ReserveRange.reserveOrGet(minKey, maxKey._2, maxKey._3, ()) shouldBe empty
- level.delete.runRandomIO.right.value
+ Level.optimalSegmentsToCollapse(
+ level = level,
+ take = 10
+ ) shouldBe empty
+ }
}
}
"reserve" should {
"reserve keys for compaction where Level is empty" in {
- val level = TestLevel()
- val keyValues = randomizedKeyValues(keyValuesCount).groupedSlice(2)
- val segment1 = TestSegment(keyValues.head).runRandomIO.right.value
- val segment2 = TestSegment(keyValues.last).runRandomIO.right.value
- level.reserve(Seq(segment1, segment2)).get shouldBe IO.Right[Promise[Unit], Slice[Byte]](keyValues.head.head.key)(IO.ExceptionHandler.PromiseUnit)
-
- //cannot reserve again
- level.reserve(Seq(segment1, segment2)).get shouldBe a[IO.Left[_, _]]
- level.reserve(Seq(segment1)).get shouldBe a[IO.Left[_, _]]
- level.reserve(Seq(segment2)).get shouldBe a[IO.Left[_, _]]
-
- level.delete.runRandomIO.right.value
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel().clean()
+
+ val keyValues = randomizedKeyValues(keyValuesCount).groupedSlice(2)
+ val segment1 = TestSegment(keyValues.head).runRandomIO.right.value.clean()
+ val segment2 = TestSegment(keyValues.last).runRandomIO.right.value.clean()
+ level.reserve(Seq(segment1, segment2)).get shouldBe IO.Right[Promise[Unit], Slice[Byte]](keyValues.head.head.key)(IO.ExceptionHandler.PromiseUnit)
+
+ //cannot reserve again
+ level.reserve(Seq(segment1, segment2)).get shouldBe a[IO.Left[_, _]]
+ level.reserve(Seq(segment1)).get shouldBe a[IO.Left[_, _]]
+ level.reserve(Seq(segment2)).get shouldBe a[IO.Left[_, _]]
+ }
}
"return completed Future for empty Segments" in {
- val level = TestLevel()
- level.reserve(Seq.empty).get.left.get.isCompleted shouldBe true
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel().clean()
- level.delete.runRandomIO.right.value
+ level.reserve(Seq.empty).get.left.get.isCompleted shouldBe true
+ }
}
"reserve min and max keys" in {
- val level = TestLevel()
- val keyValues = randomizedKeyValues(keyValuesCount).groupedSlice(2)
- val segments = Seq(TestSegment(keyValues.head).runRandomIO.right.value, TestSegment(keyValues.last).runRandomIO.right.value)
- level.put(segments).right.right.value.right.value
-
- level.delete.runRandomIO.right.value
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel().clean()
+
+ val keyValues = randomizedKeyValues(keyValuesCount).groupedSlice(2)
+ val segments =
+ Seq(
+ TestSegment(keyValues.head).runRandomIO.right.value.clean(),
+ TestSegment(keyValues.last).runRandomIO.right.value.clean()
+ )
+ level.put(segments).right.right.value.right.value
+ }
}
}
@@ -359,60 +376,63 @@ sealed trait LevelSpec extends TestBase with MockFactory with PrivateMethodTeste
import swaydb.core.map.serializer.AppendixMapEntryWriter._
"build MapEntry.Put map for the first created Segment" in {
- val level = TestLevel()
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel().clean()
- val segments = TestSegment(Slice(Memory.put(1, "value1"), Memory.put(2, "value2"))).runRandomIO.right.value
- val actualMapEntry = level.buildNewMapEntry(Slice(segments), originalSegmentMayBe = Segment.Null, initialMapEntry = None).runRandomIO.right.value
- val expectedMapEntry = MapEntry.Put[Slice[Byte], Segment](segments.minKey, segments)
+ val segments = TestSegment(Slice(Memory.put(1, "value1"), Memory.put(2, "value2"))).runRandomIO.right.value.clean()
+ val actualMapEntry = level.buildNewMapEntry(Slice(segments), originalSegmentMayBe = Segment.Null, initialMapEntry = None).runRandomIO.right.value
+ val expectedMapEntry = MapEntry.Put[Slice[Byte], Segment](segments.minKey, segments)
- actualMapEntry.asString(_.read[Int].toString, segment => segment.path.toString + segment.maxKey.maxKey.read[Int]) shouldBe
- expectedMapEntry.asString(_.read[Int].toString, segment => segment.path.toString + segment.maxKey.maxKey.read[Int])
-
- level.delete.runRandomIO.right.value
+ actualMapEntry.asString(_.read[Int].toString, segment => segment.path.toString + segment.maxKey.maxKey.read[Int]) shouldBe
+ expectedMapEntry.asString(_.read[Int].toString, segment => segment.path.toString + segment.maxKey.maxKey.read[Int])
+ }
}
"build MapEntry.Put map for the newly merged Segments and not add MapEntry.Remove map " +
"for original Segment as it's minKey is replace by one of the new Segment" in {
- val level = TestLevel()
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel().clean()
- val originalSegment = TestSegment(Slice(Memory.put(1, "value"), Memory.put(5, "value"))).runRandomIO.right.value
- val mergedSegment1 = TestSegment(Slice(Memory.put(1, "value"), Memory.put(5, "value"))).runRandomIO.right.value
- val mergedSegment2 = TestSegment(Slice(Memory.put(6, "value"), Memory.put(10, "value"))).runRandomIO.right.value
- val mergedSegment3 = TestSegment(Slice(Memory.put(11, "value"), Memory.put(15, "value"))).runRandomIO.right.value
+ val originalSegment = TestSegment(Slice(Memory.put(1, "value"), Memory.put(5, "value"))).runRandomIO.right.value.clean()
+ val mergedSegment1 = TestSegment(Slice(Memory.put(1, "value"), Memory.put(5, "value"))).runRandomIO.right.value.clean()
+ val mergedSegment2 = TestSegment(Slice(Memory.put(6, "value"), Memory.put(10, "value"))).runRandomIO.right.value.clean()
+ val mergedSegment3 = TestSegment(Slice(Memory.put(11, "value"), Memory.put(15, "value"))).runRandomIO.right.value.clean()
- val actualMapEntry = level.buildNewMapEntry(Slice(mergedSegment1, mergedSegment2, mergedSegment3), originalSegment, initialMapEntry = None).runRandomIO.right.value
+ val actualMapEntry = level.buildNewMapEntry(Slice(mergedSegment1, mergedSegment2, mergedSegment3), originalSegment, initialMapEntry = None).runRandomIO.right.value
- val expectedMapEntry =
- MapEntry.Put[Slice[Byte], Segment](1, mergedSegment1) ++
- MapEntry.Put[Slice[Byte], Segment](6, mergedSegment2) ++
- MapEntry.Put[Slice[Byte], Segment](11, mergedSegment3)
+ val expectedMapEntry =
+ MapEntry.Put[Slice[Byte], Segment](1, mergedSegment1) ++
+ MapEntry.Put[Slice[Byte], Segment](6, mergedSegment2) ++
+ MapEntry.Put[Slice[Byte], Segment](11, mergedSegment3)
- actualMapEntry.asString(_.read[Int].toString, segment => segment.path.toString + segment.maxKey.maxKey.read[Int]) shouldBe
- expectedMapEntry.asString(_.read[Int].toString, segment => segment.path.toString + segment.maxKey.maxKey.read[Int])
-
- level.delete.runRandomIO.right.value
+ actualMapEntry.asString(_.read[Int].toString, segment => segment.path.toString + segment.maxKey.maxKey.read[Int]) shouldBe
+ expectedMapEntry.asString(_.read[Int].toString, segment => segment.path.toString + segment.maxKey.maxKey.read[Int])
+ }
}
"build MapEntry.Put map for the newly merged Segments and also add Remove map entry for original map when all minKeys are unique" in {
- val level = TestLevel()
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel().clean()
- val originalSegment = TestSegment(Slice(Memory.put(0, "value"), Memory.put(5, "value"))).runRandomIO.right.value
- val mergedSegment1 = TestSegment(Slice(Memory.put(1, "value"), Memory.put(5, "value"))).runRandomIO.right.value
- val mergedSegment2 = TestSegment(Slice(Memory.put(6, "value"), Memory.put(10, "value"))).runRandomIO.right.value
- val mergedSegment3 = TestSegment(Slice(Memory.put(11, "value"), Memory.put(15, "value"))).runRandomIO.right.value
+ val originalSegment = TestSegment(Slice(Memory.put(0, "value"), Memory.put(5, "value"))).runRandomIO.right.value.clean()
+ val mergedSegment1 = TestSegment(Slice(Memory.put(1, "value"), Memory.put(5, "value"))).runRandomIO.right.value.clean()
+ val mergedSegment2 = TestSegment(Slice(Memory.put(6, "value"), Memory.put(10, "value"))).runRandomIO.right.value.clean()
+ val mergedSegment3 = TestSegment(Slice(Memory.put(11, "value"), Memory.put(15, "value"))).runRandomIO.right.value.clean()
- val expectedMapEntry =
- MapEntry.Put[Slice[Byte], Segment](1, mergedSegment1) ++
- MapEntry.Put[Slice[Byte], Segment](6, mergedSegment2) ++
- MapEntry.Put[Slice[Byte], Segment](11, mergedSegment3) ++
- MapEntry.Remove[Slice[Byte]](0)
+ val expectedMapEntry =
+ MapEntry.Put[Slice[Byte], Segment](1, mergedSegment1) ++
+ MapEntry.Put[Slice[Byte], Segment](6, mergedSegment2) ++
+ MapEntry.Put[Slice[Byte], Segment](11, mergedSegment3) ++
+ MapEntry.Remove[Slice[Byte]](0)
- val actualMapEntry = level.buildNewMapEntry(Slice(mergedSegment1, mergedSegment2, mergedSegment3), originalSegment, initialMapEntry = None).runRandomIO.right.value
+ val actualMapEntry = level.buildNewMapEntry(Slice(mergedSegment1, mergedSegment2, mergedSegment3), originalSegment, initialMapEntry = None).runRandomIO.right.value
- actualMapEntry.asString(_.read[Int].toString, segment => segment.path.toString + segment.maxKey.maxKey.read[Int]) shouldBe
- expectedMapEntry.asString(_.read[Int].toString, segment => segment.path.toString + segment.maxKey.maxKey.read[Int])
-
- level.delete.runRandomIO.right.value
+ actualMapEntry.asString(_.read[Int].toString, segment => segment.path.toString + segment.maxKey.maxKey.read[Int]) shouldBe
+ expectedMapEntry.asString(_.read[Int].toString, segment => segment.path.toString + segment.maxKey.maxKey.read[Int])
+ }
}
}
}
diff --git a/core/src/test/scala/swaydb/core/level/TrashLevelSpec.scala b/core/src/test/scala/swaydb/core/level/TrashLevelSpec.scala
index d45f1b09e..3b5eee6c6 100644
--- a/core/src/test/scala/swaydb/core/level/TrashLevelSpec.scala
+++ b/core/src/test/scala/swaydb/core/level/TrashLevelSpec.scala
@@ -28,7 +28,8 @@ import org.scalamock.scalatest.MockFactory
import org.scalatest.PrivateMethodTester
import swaydb.IOValues._
import swaydb.core.RunThis._
-import swaydb.core.TestBase
+import swaydb.core.{TestBase, TestCaseSweeper}
+import swaydb.core.TestCaseSweeper._
import swaydb.core.TestData._
import swaydb.core.segment.format.a.block.segment.SegmentBlock
import swaydb.core.segment.{Segment, ThreadReadState}
@@ -69,19 +70,22 @@ sealed trait TrashLevelSpec extends TestBase with MockFactory with PrivateMethod
"A TrashLevel" should {
"delete Segments when Push from an upper level" in {
- val level = TestLevel(nextLevel = Some(TrashLevel), throttle = (_) => Throttle(1.seconds, 10), segmentConfig = SegmentBlock.Config.random(pushForward = true))
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel(nextLevel = Some(TrashLevel), throttle = (_) => Throttle(1.seconds, 10), segmentConfig = SegmentBlock.Config.random(pushForward = true)).clean()
- val segments = Seq(TestSegment(randomKeyValues(keyValuesCount)).runRandomIO.right.value, TestSegment(randomIntKeyStringValues(keyValuesCount)).runRandomIO.right.value)
- level.put(segments).right.right.value.right.value
+ val segments = Seq(TestSegment(randomKeyValues(keyValuesCount)).runRandomIO.right.value, TestSegment(randomIntKeyStringValues(keyValuesCount)).runRandomIO.right.value)
+ level.put(segments).right.right.value.right.value
- //throttle is Duration.Zero, Segments value merged to lower ExpiryLevel and deleted from Level.
- eventual(15.seconds)(level.isEmpty shouldBe true)
- //key values do not exist
- Segment.getAllKeyValues(segments).runRandomIO.right.value foreach {
- keyValue =>
- level.get(keyValue.key, ThreadReadState.random).runRandomIO.right.value.toOptionPut shouldBe empty
+ //throttle is Duration.Zero, Segments value merged to lower ExpiryLevel and deleted from Level.
+ eventual(15.seconds)(level.isEmpty shouldBe true)
+ //key values do not exist
+ Segment.getAllKeyValues(segments).runRandomIO.right.value foreach {
+ keyValue =>
+ level.get(keyValue.key, ThreadReadState.random).runRandomIO.right.value.toOptionPut shouldBe empty
+ }
+ if (persistent) level.reopen.isEmpty shouldBe true
}
- if (persistent) level.reopen.isEmpty shouldBe true
}
}
}
diff --git a/core/src/test/scala/swaydb/core/level/compaction/throttle/ThrottleCompactionSpec.scala b/core/src/test/scala/swaydb/core/level/compaction/throttle/ThrottleCompactionSpec.scala
index 6a69a78e1..163873f37 100644
--- a/core/src/test/scala/swaydb/core/level/compaction/throttle/ThrottleCompactionSpec.scala
+++ b/core/src/test/scala/swaydb/core/level/compaction/throttle/ThrottleCompactionSpec.scala
@@ -37,7 +37,7 @@ import swaydb.core.data.Memory
import swaydb.core.level.NextLevel
import swaydb.core.segment.Segment
import swaydb.core.segment.format.a.block.segment.SegmentBlock
-import swaydb.core.{TestBase, TestSweeper, TestTimer}
+import swaydb.core.{TestBase, TestCaseSweeper, TestSweeper, TestTimer}
import swaydb.data.config.MMAP
import swaydb.data.order.{KeyOrder, TimeOrder}
import swaydb.data.slice.Slice
@@ -93,55 +93,61 @@ sealed trait CompactionSpec extends TestBase with MockFactory {
"remove Segments" when {
"Segments from upper Level are merged into lower level" in {
- val thisLevel = mock[NextLevel]("thisLevel")
- val nextLevel = mock[NextLevel]("nextLevel")
-
- val keyValues = randomPutKeyValues(keyValueCount).groupedSlice(2)
- val segments = Seq(TestSegment(keyValues(0)), TestSegment(keyValues(1)))
-
- //next level should value a put for all the input Segments
- (nextLevel.put(_: Iterable[Segment])) expects * onCall {
- putSegments: Iterable[Segment] =>
- putSegments.map(_.path) shouldBe segments.map(_.path)
- implicit val nothingExceptionHandler = IO.ExceptionHandler.Nothing
- IO.Right[Nothing, IO[Nothing, Set[Int]]](IO.Right[Nothing, Set[Int]](Set(Int.MaxValue)))
+ TestCaseSweeper {
+ implicit sweeper =>
+ val thisLevel = mock[NextLevel]("thisLevel")
+ val nextLevel = mock[NextLevel]("nextLevel")
+
+ val keyValues = randomPutKeyValues(keyValueCount).groupedSlice(2)
+ val segments = Seq(TestSegment(keyValues(0)), TestSegment(keyValues(1)))
+
+ //next level should value a put for all the input Segments
+ (nextLevel.put(_: Iterable[Segment])) expects * onCall {
+ putSegments: Iterable[Segment] =>
+ putSegments.map(_.path) shouldBe segments.map(_.path)
+ implicit val nothingExceptionHandler = IO.ExceptionHandler.Nothing
+ IO.Right[Nothing, IO[Nothing, Set[Int]]](IO.Right[Nothing, Set[Int]](Set(Int.MaxValue)))
+ }
+
+ //segments value removed
+ (thisLevel.removeSegments(_: Iterable[Segment])) expects * onCall {
+ putSegments: Iterable[Segment] =>
+ putSegments.map(_.path) shouldBe segments.map(_.path)
+ IO.Right(segments.size)
+ }
+
+ ThrottleCompaction.putForward(segments, thisLevel, nextLevel).right.right.value.right.value shouldBe segments.size
}
-
- //segments value removed
- (thisLevel.removeSegments(_: Iterable[Segment])) expects * onCall {
- putSegments: Iterable[Segment] =>
- putSegments.map(_.path) shouldBe segments.map(_.path)
- IO.Right(segments.size)
- }
-
- ThrottleCompaction.putForward(segments, thisLevel, nextLevel).right.right.value.right.value shouldBe segments.size
}
}
"return success" when {
"it fails to remove Segments" in {
- val thisLevel = mock[NextLevel]("thisLevel")
- val nextLevel = mock[NextLevel]("nextLevel")
-
- val keyValues = randomPutKeyValues(keyValueCount).groupedSlice(2)
- val segments = Seq(TestSegment(keyValues(0)), TestSegment(keyValues(1)))
-
- //next level should value a put for all the input Segments
- (nextLevel.put(_: Iterable[Segment])) expects * onCall {
- putSegments: Iterable[Segment] =>
- putSegments.map(_.path) shouldBe segments.map(_.path)
- implicit val nothingExceptionHandler = IO.ExceptionHandler.Nothing
- IO.Right[Nothing, IO[Nothing, Set[Int]]](IO.Right[Nothing, Set[Int]](Set(Int.MaxValue)))
+ TestCaseSweeper {
+ implicit sweeper =>
+ val thisLevel = mock[NextLevel]("thisLevel")
+ val nextLevel = mock[NextLevel]("nextLevel")
+
+ val keyValues = randomPutKeyValues(keyValueCount).groupedSlice(2)
+ val segments = Seq(TestSegment(keyValues(0)), TestSegment(keyValues(1)))
+
+ //next level should value a put for all the input Segments
+ (nextLevel.put(_: Iterable[Segment])) expects * onCall {
+ putSegments: Iterable[Segment] =>
+ putSegments.map(_.path) shouldBe segments.map(_.path)
+ implicit val nothingExceptionHandler = IO.ExceptionHandler.Nothing
+ IO.Right[Nothing, IO[Nothing, Set[Int]]](IO.Right[Nothing, Set[Int]](Set(Int.MaxValue)))
+ }
+
+ //segments value removed
+ (thisLevel.removeSegments(_: Iterable[Segment])) expects * onCall {
+ putSegments: Iterable[Segment] =>
+ putSegments.map(_.path) shouldBe segments.map(_.path)
+ IO.failed("Failed!")
+ }
+
+ ThrottleCompaction.putForward(segments, thisLevel, nextLevel).right.right.value.right.value shouldBe segments.size
}
-
- //segments value removed
- (thisLevel.removeSegments(_: Iterable[Segment])) expects * onCall {
- putSegments: Iterable[Segment] =>
- putSegments.map(_.path) shouldBe segments.map(_.path)
- IO.failed("Failed!")
- }
-
- ThrottleCompaction.putForward(segments, thisLevel, nextLevel).right.right.value.right.value shouldBe segments.size
}
}
}
@@ -149,71 +155,83 @@ sealed trait CompactionSpec extends TestBase with MockFactory {
"copyForwardForEach" should {
"not copy" when {
"it's the last Level and is empty" in {
- ThrottleCompaction.copyForwardForEach(Slice(TestLevel())) shouldBe 0
+ TestCaseSweeper {
+ implicit sweeper =>
+ ThrottleCompaction.copyForwardForEach(Slice(TestLevel())) shouldBe 0
+ }
}
"it's the last Level and is non empty" in {
- val keyValues = randomPutKeyValues(keyValueCount)
- val level = TestLevel(keyValues = keyValues)
- level.isEmpty shouldBe false
- ThrottleCompaction.copyForwardForEach(level.reverseLevels.toSlice) shouldBe 0
- if (persistent)
- assertGet(keyValues, level.reopen)
+ TestCaseSweeper {
+ implicit sweeper =>
+ val keyValues = randomPutKeyValues(keyValueCount)
+ val level = TestLevel(keyValues = keyValues)
+ level.isEmpty shouldBe false
+ ThrottleCompaction.copyForwardForEach(level.reverseLevels.toSlice) shouldBe 0
+ if (persistent)
+ assertGet(keyValues, level.reopen)
+ }
}
}
"copy all Segments to last level" when {
"no Segments overlap" in {
- val allKeyValues = randomPutKeyValues(keyValueCount, startId = Some(1))
- val keyValues = allKeyValues.groupedSlice(5)
-
- val level5 = TestLevel(keyValues = keyValues(4), segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb, pushForward = true))
- val level4 = TestLevel(nextLevel = Some(level5), keyValues = keyValues(3), segmentConfig = SegmentBlock.Config.random(minSegmentSize = 10.bytes, pushForward = true))
- val level3 = TestLevel(nextLevel = Some(level4), keyValues = keyValues(2), segmentConfig = SegmentBlock.Config.random(minSegmentSize = 10.bytes, pushForward = true))
- val level2 = TestLevel(nextLevel = Some(level3), keyValues = keyValues(1), segmentConfig = SegmentBlock.Config.random(minSegmentSize = 10.bytes, pushForward = true))
- val level1 = TestLevel(nextLevel = Some(level2), keyValues = keyValues(0), segmentConfig = SegmentBlock.Config.random(minSegmentSize = 10.bytes, pushForward = true))
-
- // level1.foreachLevel(_.segmentsCount() should be > 1)
-
- val expectedCopiedSegments = level1.foldLeftLevels(0)(_ + _.segmentsCount()) - level5.segmentsCount()
- val actualCopied = ThrottleCompaction.copyForwardForEach(level1.reverseLevels.toSlice)
- actualCopied shouldBe expectedCopiedSegments
- //all top levels shouldBe empty
- level1.mapLevels(level => level).dropRight(1).foreach(_.isEmpty shouldBe true)
-
- assertReads(allKeyValues, level1)
- assertReads(allKeyValues, level2)
- assertReads(allKeyValues, level3)
- assertReads(allKeyValues, level4)
- assertReads(allKeyValues, level5)
-
- assertGet(allKeyValues, level1.reopen)
+ TestCaseSweeper {
+ implicit sweeper =>
+ val allKeyValues = randomPutKeyValues(keyValueCount, startId = Some(1))
+ val keyValues = allKeyValues.groupedSlice(5)
+
+ val level5 = TestLevel(keyValues = keyValues(4), segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb, pushForward = true))
+ val level4 = TestLevel(nextLevel = Some(level5), keyValues = keyValues(3), segmentConfig = SegmentBlock.Config.random(minSegmentSize = 10.bytes, pushForward = true))
+ val level3 = TestLevel(nextLevel = Some(level4), keyValues = keyValues(2), segmentConfig = SegmentBlock.Config.random(minSegmentSize = 10.bytes, pushForward = true))
+ val level2 = TestLevel(nextLevel = Some(level3), keyValues = keyValues(1), segmentConfig = SegmentBlock.Config.random(minSegmentSize = 10.bytes, pushForward = true))
+ val level1 = TestLevel(nextLevel = Some(level2), keyValues = keyValues(0), segmentConfig = SegmentBlock.Config.random(minSegmentSize = 10.bytes, pushForward = true))
+
+ // level1.foreachLevel(_.segmentsCount() should be > 1)
+
+ val expectedCopiedSegments = level1.foldLeftLevels(0)(_ + _.segmentsCount()) - level5.segmentsCount()
+ val actualCopied = ThrottleCompaction.copyForwardForEach(level1.reverseLevels.toSlice)
+ actualCopied shouldBe expectedCopiedSegments
+ //all top levels shouldBe empty
+ level1.mapLevels(level => level).dropRight(1).foreach(_.isEmpty shouldBe true)
+
+ assertReads(allKeyValues, level1)
+ assertReads(allKeyValues, level2)
+ assertReads(allKeyValues, level3)
+ assertReads(allKeyValues, level4)
+ assertReads(allKeyValues, level5)
+
+ assertGet(allKeyValues, level1.reopen)
+ }
}
}
"copy Segments to last level" when {
"some Segments overlap" in {
- val allKeyValues = randomPutKeyValues(keyValueCount, addPutDeadlines = false, startId = Some(1))
+ TestCaseSweeper {
+ implicit sweeper =>
+ val allKeyValues = randomPutKeyValues(keyValueCount, addPutDeadlines = false, startId = Some(1))
- val keyValues = allKeyValues.groupedSlice(5)
+ val keyValues = allKeyValues.groupedSlice(5)
- val level5 = TestLevel(keyValues = Slice(keyValues(3).last) ++ keyValues(4), segmentConfig = SegmentBlock.Config.random(minSegmentSize = 2.kb, pushForward = true))
- val level4 = TestLevel(nextLevel = Some(level5), keyValues = Slice(keyValues(2).last) ++ keyValues(3), segmentConfig = SegmentBlock.Config.random(minSegmentSize = 2.kb, pushForward = true))
- val level3 = TestLevel(nextLevel = Some(level4), keyValues = keyValues(2), segmentConfig = SegmentBlock.Config.random(minSegmentSize = 2.kb, pushForward = true))
- val level2 = TestLevel(nextLevel = Some(level3), keyValues = keyValues(1), segmentConfig = SegmentBlock.Config.random(minSegmentSize = 2.kb, pushForward = true))
- val level1 = TestLevel(nextLevel = Some(level2), keyValues = keyValues(0), segmentConfig = SegmentBlock.Config.random(minSegmentSize = 2.kb, pushForward = true))
+ val level5 = TestLevel(keyValues = Slice(keyValues(3).last) ++ keyValues(4), segmentConfig = SegmentBlock.Config.random(minSegmentSize = 2.kb, pushForward = true))
+ val level4 = TestLevel(nextLevel = Some(level5), keyValues = Slice(keyValues(2).last) ++ keyValues(3), segmentConfig = SegmentBlock.Config.random(minSegmentSize = 2.kb, pushForward = true))
+ val level3 = TestLevel(nextLevel = Some(level4), keyValues = keyValues(2), segmentConfig = SegmentBlock.Config.random(minSegmentSize = 2.kb, pushForward = true))
+ val level2 = TestLevel(nextLevel = Some(level3), keyValues = keyValues(1), segmentConfig = SegmentBlock.Config.random(minSegmentSize = 2.kb, pushForward = true))
+ val level1 = TestLevel(nextLevel = Some(level2), keyValues = keyValues(0), segmentConfig = SegmentBlock.Config.random(minSegmentSize = 2.kb, pushForward = true))
- ThrottleCompaction.copyForwardForEach(level1.reverseLevels.toSlice)
+ ThrottleCompaction.copyForwardForEach(level1.reverseLevels.toSlice)
- //top levels are level, second last level value all overlapping Segments, last Level gets the rest.
- level1.isEmpty shouldBe true
- level2.isEmpty shouldBe true
- level3.isEmpty shouldBe true
- level4.isEmpty shouldBe false
- level5.isEmpty shouldBe false
+ //top levels are level, second last level value all overlapping Segments, last Level gets the rest.
+ level1.isEmpty shouldBe true
+ level2.isEmpty shouldBe true
+ level3.isEmpty shouldBe true
+ level4.isEmpty shouldBe false
+ level5.isEmpty shouldBe false
- assertReads(allKeyValues, level1)
- assertGet(allKeyValues, level1.reopen)
+ assertReads(allKeyValues, level1)
+ assertGet(allKeyValues, level1.reopen)
+ }
}
}
}
@@ -251,83 +269,89 @@ sealed trait CompactionSpec extends TestBase with MockFactory {
"keep invoking refresh" when {
"remaining compactions are non zero" in {
- val segments: ListBuffer[Segment] =
- (1 to 10).flatMap({
- i =>
- if (i % 2 == 0)
- Some(
- TestSegment(
- Slice(
- Memory.put(i, i, Some(expiredDeadline())),
- Memory.put(i + 1, i + 1, Some(expiredDeadline()))
+ TestCaseSweeper {
+ implicit sweeper =>
+ val segments: ListBuffer[Segment] =
+ (1 to 10).flatMap({
+ i =>
+ if (i % 2 == 0)
+ Some(
+ TestSegment(
+ Slice(
+ Memory.put(i, i, Some(expiredDeadline())),
+ Memory.put(i + 1, i + 1, Some(expiredDeadline()))
+ )
+ )
)
- )
- )
- else
- None
- }).to(ListBuffer)
-
- val level = mock[NextLevel]("level")
- (level.hasNextLevel _).expects() returns false repeat 6.times
- (level.segmentsInLevel _).expects() returning segments repeat 5.times
-
- (level.refresh(_: Segment)) expects * onCall {
- segment: Segment =>
- segments find (_.path == segment.path) shouldBe defined
- segments -= segment
- IO.Right(IO(segment.delete))(IO.ExceptionHandler.PromiseUnit)
- } repeat 5.times
-
- ThrottleCompaction.runLastLevelCompaction(
- level = level,
- checkExpired = true,
- remainingCompactions = 5,
- segmentsCompacted = 0
- ) shouldBe IO.Right(5)
+ else
+ None
+ }).to(ListBuffer)
+
+ val level = mock[NextLevel]("level")
+ (level.hasNextLevel _).expects() returns false repeat 6.times
+ (level.segmentsInLevel _).expects() returning segments repeat 5.times
+
+ (level.refresh(_: Segment)) expects * onCall {
+ segment: Segment =>
+ segments find (_.path == segment.path) shouldBe defined
+ segments -= segment
+ IO.Right(IO(segment.delete))(IO.ExceptionHandler.PromiseUnit)
+ } repeat 5.times
+
+ ThrottleCompaction.runLastLevelCompaction(
+ level = level,
+ checkExpired = true,
+ remainingCompactions = 5,
+ segmentsCompacted = 0
+ ) shouldBe IO.Right(5)
+ }
}
}
"invoke collapse" when {
"checkExpired is false" in {
- val segments: ListBuffer[Segment] =
- (1 to 10).flatMap({
- i =>
- if (i % 2 == 0)
- Some(
- TestSegment(
- Slice(
- Memory.put(i, i, Some(expiredDeadline())),
- Memory.put(i + 1, i + 1, Some(expiredDeadline()))
+ TestCaseSweeper {
+ implicit sweeper =>
+ val segments: ListBuffer[Segment] =
+ (1 to 10).flatMap({
+ i =>
+ if (i % 2 == 0)
+ Some(
+ TestSegment(
+ Slice(
+ Memory.put(i, i, Some(expiredDeadline())),
+ Memory.put(i + 1, i + 1, Some(expiredDeadline()))
+ )
+ )
)
- )
- )
- else
- None
- }).to(ListBuffer)
-
- val level = mock[NextLevel]("level")
- (level.hasNextLevel _).expects() returns false repeated 2.times
-
- (level.optimalSegmentsToCollapse _).expects(*) onCall {
- count: Int =>
- segments.take(count)
+ else
+ None
+ }).to(ListBuffer)
+
+ val level = mock[NextLevel]("level")
+ (level.hasNextLevel _).expects() returns false repeated 2.times
+
+ (level.optimalSegmentsToCollapse _).expects(*) onCall {
+ count: Int =>
+ segments.take(count)
+ }
+
+ (level.collapse(_: Iterable[Segment])) expects * onCall {
+ segmentsToCollapse: Iterable[Segment] =>
+ segmentsToCollapse foreach (segment => segments find (_.path == segment.path) shouldBe defined)
+ segments --= segmentsToCollapse
+ IO.Right(IO(segmentsToCollapse.size))(IO.ExceptionHandler.PromiseUnit)
+ }
+
+ // level.levelNumber _ expects() returns 1 repeat 3.times
+
+ ThrottleCompaction.runLastLevelCompaction(
+ level = level,
+ checkExpired = false,
+ remainingCompactions = 5,
+ segmentsCompacted = 0
+ ) shouldBe IO.Right(5)
}
-
- (level.collapse(_: Iterable[Segment])) expects * onCall {
- segmentsToCollapse: Iterable[Segment] =>
- segmentsToCollapse foreach (segment => segments find (_.path == segment.path) shouldBe defined)
- segments --= segmentsToCollapse
- IO.Right(IO(segmentsToCollapse.size))(IO.ExceptionHandler.PromiseUnit)
- }
-
- // level.levelNumber _ expects() returns 1 repeat 3.times
-
- ThrottleCompaction.runLastLevelCompaction(
- level = level,
- checkExpired = false,
- remainingCompactions = 5,
- segmentsCompacted = 0
- ) shouldBe IO.Right(5)
}
}
}
diff --git a/core/src/test/scala/swaydb/core/level/compaction/throttle/ThrottleCompactorSpec.scala b/core/src/test/scala/swaydb/core/level/compaction/throttle/ThrottleCompactorSpec.scala
index 18f659fa7..3622c2dfa 100644
--- a/core/src/test/scala/swaydb/core/level/compaction/throttle/ThrottleCompactorSpec.scala
+++ b/core/src/test/scala/swaydb/core/level/compaction/throttle/ThrottleCompactorSpec.scala
@@ -31,8 +31,9 @@ import swaydb.core.RunThis._
import swaydb.core.TestData._
import swaydb.core.actor.FileSweeper.FileSweeperActor
import swaydb.core.actor.{FileSweeper, MemorySweeper}
+import swaydb.core.level.Level
import swaydb.core.level.compaction.{Compaction, Compactor}
-import swaydb.core.{TestBase, TestExecutionContext, TestSweeper, TestTimer}
+import swaydb.core.{TestBase, TestExecutionContext, TestCaseSweeper, TestSweeper, TestTimer}
import swaydb.data.compaction.CompactionExecutionContext
import swaydb.data.config.MMAP
import swaydb.data.order.{KeyOrder, TimeOrder}
@@ -84,371 +85,409 @@ sealed trait ThrottleCompactorSpec extends TestBase with MockFactory {
"createActor" should {
"build compaction hierarchy" when {
"there are two Levels and one new ExecutionContext" in {
- val nextLevel = TestLevel()
- val zero = TestLevelZero(nextLevel = Some(nextLevel))
-
- val actor =
- ThrottleCompactor.createActor(
- List(zero, nextLevel),
- List(
- CompactionExecutionContext.Create(TestExecutionContext.executionContext),
- CompactionExecutionContext.Shared
- )
- ).get
-
- actor.state.await.compactionStates shouldBe empty
- actor.state.await.levels.map(_.rootPath) shouldBe Slice(zero.rootPath, nextLevel.rootPath)
- actor.state.await.child shouldBe empty
+ TestCaseSweeper {
+ implicit sweeper =>
+ val nextLevel = TestLevel()
+ val zero = TestLevelZero(nextLevel = Some(nextLevel))
+
+ val actor =
+ ThrottleCompactor.createActor(
+ List(zero, nextLevel),
+ List(
+ CompactionExecutionContext.Create(TestExecutionContext.executionContext),
+ CompactionExecutionContext.Shared
+ )
+ ).get
+
+ actor.state.await.compactionStates shouldBe empty
+ actor.state.await.levels.map(_.rootPath) shouldBe Slice(zero.rootPath, nextLevel.rootPath)
+ actor.state.await.child shouldBe empty
- zero.delete.get
+ }
}
"there are two Levels and two new ExecutionContext" in {
- val nextLevel = TestLevel()
- val zero = TestLevelZero(nextLevel = Some(nextLevel))
-
- val actor =
- ThrottleCompactor.createActor(
- List(zero, nextLevel),
- List(
- CompactionExecutionContext.Create(TestExecutionContext.executionContext),
- CompactionExecutionContext.Create(TestExecutionContext.executionContext)
- )
- ).get
-
- actor.state.await.compactionStates shouldBe empty
- actor.state.await.levels.map(_.rootPath) should contain only zero.rootPath
- actor.state.await.child shouldBe defined
-
- val childActor = actor.state.await.child.get.state.await
- childActor.child shouldBe empty
- childActor.levels.map(_.rootPath) should contain only nextLevel.rootPath
-
- zero.delete.get
+ TestCaseSweeper {
+ implicit sweeper =>
+ val nextLevel = TestLevel()
+ val zero = TestLevelZero(nextLevel = Some(nextLevel))
+
+ val actor =
+ ThrottleCompactor.createActor(
+ List(zero, nextLevel),
+ List(
+ CompactionExecutionContext.Create(TestExecutionContext.executionContext),
+ CompactionExecutionContext.Create(TestExecutionContext.executionContext)
+ )
+ ).get
+
+ actor.state.await.compactionStates shouldBe empty
+ actor.state.await.levels.map(_.rootPath) should contain only zero.rootPath
+ actor.state.await.child shouldBe defined
+
+ val childActor = actor.state.await.child.get.state.await
+ childActor.child shouldBe empty
+ childActor.levels.map(_.rootPath) should contain only nextLevel.rootPath
+ }
}
"there are three Levels and one new ExecutionContext" in {
- val nextLevel2 = TestLevel()
- val nextLevel = TestLevel(nextLevel = Some(nextLevel2))
- val zero = TestLevelZero(nextLevel = Some(nextLevel))
-
- val actor =
- ThrottleCompactor.createActor(
- List(zero, nextLevel, nextLevel2),
- List(
- CompactionExecutionContext.Create(TestExecutionContext.executionContext),
- CompactionExecutionContext.Shared,
- CompactionExecutionContext.Shared
- )
- ).get
-
- actor.state.await.compactionStates shouldBe empty
- actor.state.await.levels.map(_.rootPath) shouldBe Slice(zero.rootPath, nextLevel.rootPath, nextLevel2.rootPath)
- actor.state.await.child shouldBe empty
-
- zero.delete.get
+ TestCaseSweeper {
+ implicit sweeper =>
+ val nextLevel2 = TestLevel()
+ val nextLevel = TestLevel(nextLevel = Some(nextLevel2))
+ val zero = TestLevelZero(nextLevel = Some(nextLevel))
+
+ val actor =
+ ThrottleCompactor.createActor(
+ List(zero, nextLevel, nextLevel2),
+ List(
+ CompactionExecutionContext.Create(TestExecutionContext.executionContext),
+ CompactionExecutionContext.Shared,
+ CompactionExecutionContext.Shared
+ )
+ ).get
+
+ actor.state.await.compactionStates shouldBe empty
+ actor.state.await.levels.map(_.rootPath) shouldBe Slice(zero.rootPath, nextLevel.rootPath, nextLevel2.rootPath)
+ actor.state.await.child shouldBe empty
+ }
}
"there are three Levels and two new ExecutionContext" in {
- val nextLevel2 = TestLevel()
- val nextLevel = TestLevel(nextLevel = Some(nextLevel2))
- val zero = TestLevelZero(nextLevel = Some(nextLevel))
-
- val actor =
- ThrottleCompactor.createActor(
- List(zero, nextLevel, nextLevel2),
- List(
- CompactionExecutionContext.Create(TestExecutionContext.executionContext),
- CompactionExecutionContext.Shared,
- CompactionExecutionContext.Create(TestExecutionContext.executionContext)
- )
- ).get
-
- actor.state.await.compactionStates shouldBe empty
- actor.state.await.levels.map(_.rootPath) shouldBe Slice(zero.rootPath, nextLevel.rootPath)
- actor.state.await.child shouldBe defined
-
- val childActor = actor.state.await.child.get.state.await
- childActor.child shouldBe empty
- childActor.levels.map(_.rootPath) should contain only nextLevel2.rootPath
-
- zero.delete.get
+ TestCaseSweeper {
+ implicit sweeper =>
+ val nextLevel2 = TestLevel()
+ val nextLevel = TestLevel(nextLevel = Some(nextLevel2))
+ val zero = TestLevelZero(nextLevel = Some(nextLevel))
+
+ val actor =
+ ThrottleCompactor.createActor(
+ List(zero, nextLevel, nextLevel2),
+ List(
+ CompactionExecutionContext.Create(TestExecutionContext.executionContext),
+ CompactionExecutionContext.Shared,
+ CompactionExecutionContext.Create(TestExecutionContext.executionContext)
+ )
+ ).get
+
+ actor.state.await.compactionStates shouldBe empty
+ actor.state.await.levels.map(_.rootPath) shouldBe Slice(zero.rootPath, nextLevel.rootPath)
+ actor.state.await.child shouldBe defined
+
+ val childActor = actor.state.await.child.get.state.await
+ childActor.child shouldBe empty
+ childActor.levels.map(_.rootPath) should contain only nextLevel2.rootPath
+ }
}
}
}
"scheduleNextWakeUp" should {
- val nextLevel = TestLevel()
- val level = TestLevel(nextLevel = Some(nextLevel))
-
- val testState =
- ThrottleState(
- levels = Slice(level, nextLevel),
- child = None,
- // ordering = CompactionOrdering.ordering(_ => ThrottleLevelState.Sleeping(1.day.fromNow, 0)),
- executionContext = TestExecutionContext.executionContext,
- compactionStates = mutable.Map.empty
- )
+ def createTestLevel()(implicit sweeper: TestCaseSweeper): (Level, Level, ThrottleState) = {
+ val nextLevel = TestLevel()
+ val level = TestLevel(nextLevel = Some(nextLevel))
+
+ val testState =
+ ThrottleState(
+ levels = Slice(level, nextLevel),
+ child = None,
+ // ordering = CompactionOrdering.ordering(_ => ThrottleLevelState.Sleeping(1.day.fromNow, 0)),
+ executionContext = TestExecutionContext.executionContext,
+ compactionStates = mutable.Map.empty
+ )
+
+ (level, nextLevel, testState)
+ }
"not trigger wakeUp" when {
"level states were empty" in {
- val compactor = mock[Compactor[ThrottleState]]
- implicit val scheduler = Scheduler()
-
- val actor =
- Actor.wire[Compactor[ThrottleState], ThrottleState](
- name = "test",
- impl = compactor,
- state = testState
- )
+ TestCaseSweeper {
+ implicit sweeper =>
+ val (level, nextLevel, testState) = createTestLevel()
+
+ val compactor = mock[Compactor[ThrottleState]]
+ implicit val scheduler = Scheduler()
+
+ val actor =
+ Actor.wire[Compactor[ThrottleState], ThrottleState](
+ name = "test",
+ impl = compactor,
+ state = testState
+ )
- ThrottleCompactor.scheduleNextWakeUp(
- state = testState,
- self = actor
- )
+ ThrottleCompactor.scheduleNextWakeUp(
+ state = testState,
+ self = actor
+ )
- sleep(5.seconds)
+ sleep(5.seconds)
+ }
}
"level states were non-empty but level's state is unchanged and has scheduled task" in {
- val compactor = mock[Compactor[ThrottleState]]
- implicit val scheduler = Scheduler()
-
- val state =
- testState.copy(
- compactionStates =
- mutable.Map(
- level -> ThrottleLevelState.Sleeping(5.seconds.fromNow, level.stateId),
- nextLevel -> ThrottleLevelState.Sleeping(5.seconds.fromNow, nextLevel.stateId)
+ TestCaseSweeper {
+ implicit sweeper =>
+ val (level, nextLevel, testState) = createTestLevel()
+
+ val compactor = mock[Compactor[ThrottleState]]
+ implicit val scheduler = Scheduler()
+
+ val state =
+ testState.copy(
+ compactionStates =
+ mutable.Map(
+ level -> ThrottleLevelState.Sleeping(5.seconds.fromNow, level.stateId),
+ nextLevel -> ThrottleLevelState.Sleeping(5.seconds.fromNow, nextLevel.stateId)
+ )
)
- )
- state.sleepTask = Some(null)
+ state.sleepTask = Some(null)
- val actor =
- Actor.wire[Compactor[ThrottleState], ThrottleState](
- name = "test",
- impl = compactor,
- state = state
- )
+ val actor =
+ Actor.wire[Compactor[ThrottleState], ThrottleState](
+ name = "test",
+ impl = compactor,
+ state = state
+ )
- state.sleepTask shouldBe defined
+ state.sleepTask shouldBe defined
- ThrottleCompactor.scheduleNextWakeUp(
- state = state,
- self = actor
- )
+ ThrottleCompactor.scheduleNextWakeUp(
+ state = state,
+ self = actor
+ )
- sleep(3.seconds)
+ sleep(3.seconds)
+ }
}
"level states were non-empty but level's state is unchanged and task is undefined" in {
- val compactor = mock[Compactor[ThrottleState]]
- implicit val scheduler = Scheduler()
-
- val state =
- testState.copy(
- compactionStates =
- mutable.Map(
- level -> ThrottleLevelState.Sleeping(4.seconds.fromNow, level.stateId),
- nextLevel -> ThrottleLevelState.Sleeping(eitherOne(7.seconds.fromNow, 4.seconds.fromNow), nextLevel.stateId)
+ TestCaseSweeper {
+ implicit sweeper =>
+ val (level, nextLevel, testState) = createTestLevel()
+
+ val compactor = mock[Compactor[ThrottleState]]
+ implicit val scheduler = Scheduler()
+
+ val state =
+ testState.copy(
+ compactionStates =
+ mutable.Map(
+ level -> ThrottleLevelState.Sleeping(4.seconds.fromNow, level.stateId),
+ nextLevel -> ThrottleLevelState.Sleeping(eitherOne(7.seconds.fromNow, 4.seconds.fromNow), nextLevel.stateId)
+ )
)
- )
- val actor =
- Actor.wire[Compactor[ThrottleState], ThrottleState](
- name = "",
- impl = compactor,
- state = state
- )
+ val actor =
+ Actor.wire[Compactor[ThrottleState], ThrottleState](
+ name = "",
+ impl = compactor,
+ state = state
+ )
- state.sleepTask shouldBe empty
+ state.sleepTask shouldBe empty
- ThrottleCompactor.scheduleNextWakeUp(
- state = state,
- self = actor
- )
+ ThrottleCompactor.scheduleNextWakeUp(
+ state = state,
+ self = actor
+ )
- eventual(state.sleepTask shouldBe defined)
+ eventual(state.sleepTask shouldBe defined)
- sleep(2.seconds)
+ sleep(2.seconds)
- //eventually
- compactor.wakeUp _ expects(*, *, *) onCall {
- (throttle, copyForward, actor) =>
- copyForward shouldBe false
- throttle shouldBe state
- actor shouldBe actor
- }
+ //eventually
+ compactor.wakeUp _ expects(*, *, *) onCall {
+ (throttle, copyForward, actor) =>
+ copyForward shouldBe false
+ throttle shouldBe state
+ actor shouldBe actor
+ }
- sleep(3.seconds)
+ sleep(3.seconds)
+ }
}
}
"trigger wakeUp" when {
"one of level states is awaiting pull and successfully received read" in {
- implicit val scheduler = Scheduler()
-
- //create IO.Later that is busy
- val promise = Promise[Unit]()
-
- val awaitingPull = ThrottleLevelState.AwaitingPull(promise, 1.minute.fromNow, 0)
- awaitingPull.listenerInvoked shouldBe false
- //set the state to be awaiting pull
- val state =
- testState.copy(
- compactionStates =
- mutable.Map(
- level -> awaitingPull
+ TestCaseSweeper {
+ implicit sweeper =>
+ val (level, nextLevel, testState) = createTestLevel()
+
+ implicit val scheduler = Scheduler()
+
+ //create IO.Later that is busy
+ val promise = Promise[Unit]()
+
+ val awaitingPull = ThrottleLevelState.AwaitingPull(promise, 1.minute.fromNow, 0)
+ awaitingPull.listenerInvoked shouldBe false
+ //set the state to be awaiting pull
+ val state =
+ testState.copy(
+ compactionStates =
+ mutable.Map(
+ level -> awaitingPull
+ )
+ )
+ //mock the compaction that should expect a wakeUp call
+ val compactor = mock[Compactor[ThrottleState]]
+ compactor.wakeUp _ expects(*, *, *) onCall {
+ (callState, doCopy, _) =>
+ callState shouldBe state
+ doCopy shouldBe false
+ }
+
+ //initialise Compactor with the mocked class
+ val actor =
+ Actor.wire[Compactor[ThrottleState], ThrottleState](
+ name = "test",
+ impl = compactor,
+ state = state
)
- )
- //mock the compaction that should expect a wakeUp call
- val compactor = mock[Compactor[ThrottleState]]
- compactor.wakeUp _ expects(*, *, *) onCall {
- (callState, doCopy, _) =>
- callState shouldBe state
- doCopy shouldBe false
- }
-
- //initialise Compactor with the mocked class
- val actor =
- Actor.wire[Compactor[ThrottleState], ThrottleState](
- name = "test",
- impl = compactor,
- state = state
- )
- ThrottleCompactor.scheduleNextWakeUp(
- state = state,
- self = actor
- )
- //after calling scheduleNextWakeUp listener should be initialised.
- //this ensures that multiple wakeUp callbacks do not value registered for the same pull.
- awaitingPull.listenerInitialised shouldBe true
-
- //free the reserve and compaction should expect a message.
- scheduler.future(1.second)(promise.success(()))
-
- eventual(3.seconds) {
- //eventually is set to be ready.
- awaitingPull.listenerInvoked shouldBe true
- //next sleep task is initialised & it's the await's timeout.
- state.sleepTask.value._2 shouldBe awaitingPull.timeout
+ ThrottleCompactor.scheduleNextWakeUp(
+ state = state,
+ self = actor
+ )
+ //after calling scheduleNextWakeUp listener should be initialised.
+ //this ensures that multiple wakeUp callbacks do not value registered for the same pull.
+ awaitingPull.listenerInitialised shouldBe true
+
+ //free the reserve and compaction should expect a message.
+ scheduler.future(1.second)(promise.success(()))
+
+ eventual(3.seconds) {
+ //eventually is set to be ready.
+ awaitingPull.listenerInvoked shouldBe true
+ //next sleep task is initialised & it's the await's timeout.
+ state.sleepTask.value._2 shouldBe awaitingPull.timeout
+ }
}
}
"one of level states is awaiting pull and other Level's sleep is shorter" in {
- implicit val scheduler = Scheduler()
-
- val promise = Promise[Unit]()
-
- val level1AwaitingPull = ThrottleLevelState.AwaitingPull(promise, 1.minute.fromNow, 0)
- level1AwaitingPull.listenerInvoked shouldBe false
-
- //level 2's sleep is shorter than level1's awaitPull timeout sleep.
- val level2Sleep = ThrottleLevelState.Sleeping(5.seconds.fromNow, 0)
- //set the state to be awaiting pull
- val state =
- testState.copy(
- compactionStates =
- mutable.Map(
- level -> level1AwaitingPull,
- nextLevel -> level2Sleep
+ TestCaseSweeper {
+ implicit sweeper =>
+ val (level, nextLevel, testState) = createTestLevel()
+
+ implicit val scheduler = Scheduler()
+
+ val promise = Promise[Unit]()
+
+ val level1AwaitingPull = ThrottleLevelState.AwaitingPull(promise, 1.minute.fromNow, 0)
+ level1AwaitingPull.listenerInvoked shouldBe false
+
+ //level 2's sleep is shorter than level1's awaitPull timeout sleep.
+ val level2Sleep = ThrottleLevelState.Sleeping(5.seconds.fromNow, 0)
+ //set the state to be awaiting pull
+ val state =
+ testState.copy(
+ compactionStates =
+ mutable.Map(
+ level -> level1AwaitingPull,
+ nextLevel -> level2Sleep
+ )
+ )
+ //mock the compaction that should expect a wakeUp call
+ val compactor = mock[Compactor[ThrottleState]]
+ compactor.wakeUp _ expects(*, *, *) onCall {
+ (callState, doCopy, _) =>
+ callState shouldBe state
+ doCopy shouldBe false
+ }
+
+ //initialise Compactor with the mocked class
+ val actor =
+ Actor.wire[Compactor[ThrottleState], ThrottleState](
+ name = "test",
+ impl = compactor,
+ state = state
)
- )
- //mock the compaction that should expect a wakeUp call
- val compactor = mock[Compactor[ThrottleState]]
- compactor.wakeUp _ expects(*, *, *) onCall {
- (callState, doCopy, _) =>
- callState shouldBe state
- doCopy shouldBe false
- }
-
- //initialise Compactor with the mocked class
- val actor =
- Actor.wire[Compactor[ThrottleState], ThrottleState](
- name = "test",
- impl = compactor,
- state = state
- )
- ThrottleCompactor.scheduleNextWakeUp(
- state = state,
- self = actor
- )
- //a callback for awaiting pull should always be initialised.
- level1AwaitingPull.listenerInitialised shouldBe true
- state.sleepTask shouldBe defined
- //Level2's sleep is ending earlier than Level1's so task should be set for Level2's deadline.
- state.sleepTask.get._2 shouldBe level2Sleep.sleepDeadline
-
- //give it sometime and wakeUp call initialised by Level2 will be triggered.
- sleep(level2Sleep.sleepDeadline.timeLeft + 1.second)
+ ThrottleCompactor.scheduleNextWakeUp(
+ state = state,
+ self = actor
+ )
+ //a callback for awaiting pull should always be initialised.
+ level1AwaitingPull.listenerInitialised shouldBe true
+ state.sleepTask shouldBe defined
+ //Level2's sleep is ending earlier than Level1's so task should be set for Level2's deadline.
+ state.sleepTask.get._2 shouldBe level2Sleep.sleepDeadline
+
+ //give it sometime and wakeUp call initialised by Level2 will be triggered.
+ sleep(level2Sleep.sleepDeadline.timeLeft + 1.second)
+ }
}
}
}
"doWakeUp" should {
- val nextLevel = TestLevel()
- val level = TestLevel(nextLevel = Some(nextLevel))
-
- val testState =
- ThrottleState(
- levels = Slice(level, nextLevel),
- child = None,
- // ordering = CompactionOrdering.ordering(_ => ThrottleLevelState.Sleeping(1.day.fromNow, 0)),
- executionContext = TestExecutionContext.executionContext,
- compactionStates = mutable.Map.empty
- )
"run compaction and postCompaction" in {
- implicit val compaction = mock[Compaction[ThrottleState]]
-
- val parentCompactor = mock[Compactor[ThrottleState]]
- val childCompactor = mock[Compactor[ThrottleState]]
+ TestCaseSweeper {
+ implicit sweeper =>
+ val nextLevel = TestLevel()
+ val level = TestLevel(nextLevel = Some(nextLevel))
+
+ val testState =
+ ThrottleState(
+ levels = Slice(level, nextLevel),
+ child = None,
+ // ordering = CompactionOrdering.ordering(_ => ThrottleLevelState.Sleeping(1.day.fromNow, 0)),
+ executionContext = TestExecutionContext.executionContext,
+ compactionStates = mutable.Map.empty
+ )
- val copyForward = randomBoolean()
+ implicit val compaction = mock[Compaction[ThrottleState]]
- val childActor =
- Actor.wire[Compactor[ThrottleState], ThrottleState](
- name = "test",
- impl = childCompactor,
- state = testState
- )
+ val parentCompactor = mock[Compactor[ThrottleState]]
+ val childCompactor = mock[Compactor[ThrottleState]]
- val state: ThrottleState =
- testState.copy(compactionStates = mutable.Map.empty, child = Some(childActor))
+ val copyForward = randomBoolean()
- val actor =
- Actor.wire[Compactor[ThrottleState], ThrottleState](
- name = "test",
- impl = parentCompactor,
- state = state
- )
+ val childActor =
+ Actor.wire[Compactor[ThrottleState], ThrottleState](
+ name = "test",
+ impl = childCompactor,
+ state = testState
+ )
- //parent gets a compaction call
- compaction.run _ expects(*, *) onCall {
- (throttle, copy) =>
- throttle shouldBe state
- copy shouldBe copyForward
- ()
- }
+ val state: ThrottleState =
+ testState.copy(compactionStates = mutable.Map.empty, child = Some(childActor))
- //child get a compaction call.
- childCompactor.wakeUp _ expects(*, *, *) onCall {
- (state, copy, actor) =>
- copy shouldBe false
- ()
- }
+ val actor =
+ Actor.wire[Compactor[ThrottleState], ThrottleState](
+ name = "test",
+ impl = parentCompactor,
+ state = state
+ )
- ThrottleCompactor.doWakeUp(
- state = state,
- forwardCopyOnAllLevels = copyForward,
- self = actor
- )
+ //parent gets a compaction call
+ compaction.run _ expects(*, *) onCall {
+ (throttle, copy) =>
+ throttle shouldBe state
+ copy shouldBe copyForward
+ ()
+ }
+
+ //child get a compaction call.
+ childCompactor.wakeUp _ expects(*, *, *) onCall {
+ (state, copy, actor) =>
+ copy shouldBe false
+ ()
+ }
+
+ ThrottleCompactor.doWakeUp(
+ state = state,
+ forwardCopyOnAllLevels = copyForward,
+ self = actor
+ )
- sleep(5.seconds)
+ sleep(5.seconds)
+ }
}
}
}
diff --git a/core/src/test/scala/swaydb/core/level/tool/AppendixRepairerSpec.scala b/core/src/test/scala/swaydb/core/level/tool/AppendixRepairerSpec.scala
index e950794c1..366b8b689 100644
--- a/core/src/test/scala/swaydb/core/level/tool/AppendixRepairerSpec.scala
+++ b/core/src/test/scala/swaydb/core/level/tool/AppendixRepairerSpec.scala
@@ -36,14 +36,14 @@ import swaydb.core.io.file.Effect
import swaydb.core.io.file.Effect._
import swaydb.core.segment.Segment
import swaydb.core.segment.format.a.block.segment.SegmentBlock
-import swaydb.core.{TestBase, TestSweeper}
+import swaydb.core.{TestBase, TestCaseSweeper, TestSweeper}
import swaydb.data.compaction.Throttle
import swaydb.data.order.{KeyOrder, TimeOrder}
import swaydb.data.repairAppendix.{AppendixRepairStrategy, OverlappingSegmentsException}
import swaydb.data.slice.Slice
import swaydb.data.util.StorageUnits._
-import scala.concurrent.duration.Duration
+import scala.concurrent.duration.{Duration, DurationInt}
import scala.util.Random
class AppendixRepairerSpec extends TestBase {
@@ -60,129 +60,141 @@ class AppendixRepairerSpec extends TestBase {
}
"create new appendix file if all the Segments in the Level are non-overlapping Segments" in {
- val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb, deleteEventually = false, pushForward = false))
- level.putKeyValuesTest(randomizedKeyValues(10000)).runRandomIO.right.value
-
- level.segmentsCount() should be > 2
- val segmentsBeforeRepair = level.segmentsInLevel()
-
- //repair appendix
- AppendixRepairer(level.rootPath, AppendixRepairStrategy.ReportFailure).runRandomIO.right.value
- level.appendixPath.exists shouldBe true //appendix is created
-
- //reopen level and it should contain all the Segment
- val reopenedLevel = level.reopen
- reopenedLevel.segmentsInLevel().map(_.path) shouldBe segmentsBeforeRepair.map(_.path)
- reopenedLevel.close.runRandomIO.right.value
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb, deleteEventually = false, pushForward = false))
+ level.putKeyValuesTest(randomizedKeyValues(10000)).runRandomIO.right.value
+
+ level.segmentsCount() should be > 2
+ val segmentsBeforeRepair = level.segmentsInLevel()
+
+ //repair appendix
+ AppendixRepairer(level.rootPath, AppendixRepairStrategy.ReportFailure).runRandomIO.right.value
+ level.appendixPath.exists shouldBe true //appendix is created
+
+ //reopen level and it should contain all the Segment
+ val reopenedLevel = level.reopen
+ reopenedLevel.segmentsInLevel().map(_.path) shouldBe segmentsBeforeRepair.map(_.path)
+ reopenedLevel.closeNoSweep().runRandomIO.right.value
+ }
}
"create empty appendix file if the Level is empty" in {
- //create empty Level
- val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb, deleteEventually = false, pushForward = false))
-
- //delete appendix
- Effect.walkDelete(level.appendixPath).runRandomIO.right.value
- level.appendixPath.exists shouldBe false
-
- //repair appendix
- AppendixRepairer(level.rootPath, AppendixRepairStrategy.ReportFailure).runRandomIO.right.value
- level.appendixPath.exists shouldBe true //appendix is created
-
- //reopen level, the Level is empty
- val reopenedLevel = level.reopen
- reopenedLevel.isEmpty shouldBe true
- reopenedLevel.close.runRandomIO.right.value
+ TestCaseSweeper {
+ implicit sweeper =>
+ //create empty Level
+ val level = TestLevel(segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb, deleteEventually = false, pushForward = false))
+
+ //delete appendix
+ Effect.walkDelete(level.appendixPath).runRandomIO.right.value
+ level.appendixPath.exists shouldBe false
+
+ //repair appendix
+ AppendixRepairer(level.rootPath, AppendixRepairStrategy.ReportFailure).runRandomIO.right.value
+ level.appendixPath.exists shouldBe true //appendix is created
+
+ //reopen level, the Level is empty
+ val reopenedLevel = level.reopen
+ reopenedLevel.isEmpty shouldBe true
+ reopenedLevel.closeNoSweep().runRandomIO.right.value
+ }
}
"report duplicate Segments" in {
- //create a Level with a sub-level and disable throttling so that compaction does not delete expired key-values
- val level = TestLevel(
- segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb, deleteEventually = false, pushForward = false),
- nextLevel = Some(TestLevel()),
- throttle = (_) => Throttle(Duration.Zero, 0)
- )
-
- val keyValues = randomizedKeyValues(1000)
- level.putKeyValuesTest(keyValues).runRandomIO.right.value
-
- level.segmentsCount() should be > 2
- val segmentsBeforeRepair = level.segmentsInLevel()
- level.segmentsInLevel().foldLeft(segmentsBeforeRepair.last.path.fileId.runRandomIO.right.value._1 + 1) {
- case (segmentId, segment) =>
- //create a duplicate Segment
- val duplicateSegment = segment.path.getParent.resolve(segmentId.toSegmentFileId)
- Effect.copy(segment.path, duplicateSegment).runRandomIO.right.value
- //perform repair
- AppendixRepairer(level.rootPath, AppendixRepairStrategy.ReportFailure).left.runRandomIO.right.value.exception shouldBe a[OverlappingSegmentsException]
- //perform repair with DeleteNext. This will delete the newest duplicate Segment.
- AppendixRepairer(level.rootPath, AppendixRepairStrategy.KeepOld).runRandomIO.right.value
- //newer duplicate Segment is deleted
- duplicateSegment.exists shouldBe false
-
- //copy again
- Effect.copy(segment.path, duplicateSegment).runRandomIO.right.value
- //now use delete previous instead
- AppendixRepairer(level.rootPath, AppendixRepairStrategy.KeepNew).runRandomIO.right.value
- //newer duplicate Segment exists
- duplicateSegment.exists shouldBe true
- //older duplicate Segment is deleted
- segment.existsOnDisk shouldBe false
- segmentId + 1
+ TestCaseSweeper {
+ implicit sweeper =>
+ //create a Level with a sub-level and disable throttling so that compaction does not delete expired key-values
+ val level = TestLevel(
+ segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb, deleteEventually = false, pushForward = false),
+ nextLevel = Some(TestLevel()),
+ throttle = (_) => Throttle(Duration.Zero, 0)
+ )
+
+ val keyValues = randomizedKeyValues(1000)
+ level.putKeyValuesTest(keyValues).runRandomIO.right.value
+
+ level.segmentsCount() should be > 2
+ val segmentsBeforeRepair = level.segmentsInLevel()
+ level.segmentsInLevel().foldLeft(segmentsBeforeRepair.last.path.fileId.runRandomIO.right.value._1 + 1) {
+ case (segmentId, segment) =>
+ //create a duplicate Segment
+ val duplicateSegment = segment.path.getParent.resolve(segmentId.toSegmentFileId)
+ Effect.copy(segment.path, duplicateSegment).runRandomIO.right.value
+ //perform repair
+ AppendixRepairer(level.rootPath, AppendixRepairStrategy.ReportFailure).left.runRandomIO.right.value.exception shouldBe a[OverlappingSegmentsException]
+ //perform repair with DeleteNext. This will delete the newest duplicate Segment.
+ AppendixRepairer(level.rootPath, AppendixRepairStrategy.KeepOld).runRandomIO.right.value
+ //newer duplicate Segment is deleted
+ duplicateSegment.exists shouldBe false
+
+ //copy again
+ Effect.copy(segment.path, duplicateSegment).runRandomIO.right.value
+ //now use delete previous instead
+ AppendixRepairer(level.rootPath, AppendixRepairStrategy.KeepNew).runRandomIO.right.value
+ //newer duplicate Segment exists
+ duplicateSegment.exists shouldBe true
+ //older duplicate Segment is deleted
+ segment.existsOnDisk shouldBe false
+ segmentId + 1
+ }
+ //level still contains the same key-values
+ val reopenedLevel = level.reopen
+ Segment.getAllKeyValues(reopenedLevel.segmentsInLevel()).runRandomIO.right.value shouldBe keyValues
+ reopenedLevel.closeNoSweep().runRandomIO.right.value
}
- //level still contains the same key-values
- val reopenedLevel = level.reopen
- Segment.getAllKeyValues(reopenedLevel.segmentsInLevel()).runRandomIO.right.value shouldBe keyValues
- reopenedLevel.close.runRandomIO.right.value
}
"report overlapping min & max key Segments & delete newer overlapping Segment if KeepOld is selected" in {
- //create empty Level
- val keyValues = randomizedKeyValues(1000)
-
- val level = TestLevel(
- segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb, deleteEventually = false, pushForward = false),
- nextLevel = Some(TestLevel()),
- throttle = (_) => Throttle(Duration.Zero, 0)
- )
-
- level.putKeyValuesTest(keyValues).runRandomIO.right.value
-
- level.segmentsCount() should be > 2
- val segmentsBeforeRepair = level.segmentsInLevel()
- level.segmentsInLevel().foldLeft(segmentsBeforeRepair.last.path.fileId.runRandomIO.right.value._1 + 1) {
- case (overlappingSegmentId, segment) =>
- val overlappingLevelSegmentPath = level.rootPath.resolve(overlappingSegmentId.toSegmentFileId)
-
- def createOverlappingSegment() = {
- val numberOfKeyValuesToOverlap = randomNextInt(3) max 1
- val keyValuesToOverlap = Random.shuffle(segment.toSlice().runRandomIO.right.value.toList).take(numberOfKeyValuesToOverlap).map(_.toMemory).toSlice
- //create overlapping Segment
- val overlappingSegment = TestSegment(keyValuesToOverlap).runRandomIO.right.value
- Effect.copy(overlappingSegment.path, overlappingLevelSegmentPath).runRandomIO.right.value
- overlappingSegment.close.runRandomIO.right.value //gotta close the new segment create after it's copied over.
+ TestCaseSweeper {
+ implicit sweeper =>
+ //create empty Level
+ val keyValues = randomizedKeyValues(1000)
+
+ val level = TestLevel(
+ segmentConfig = SegmentBlock.Config.random(minSegmentSize = 1.kb, deleteEventually = false, pushForward = false),
+ nextLevel = Some(TestLevel()),
+ throttle = (_) => Throttle(Duration.Zero, 0)
+ )
+
+ level.putKeyValuesTest(keyValues).runRandomIO.right.value
+
+ level.segmentsCount() should be > 2
+ val segmentsBeforeRepair = level.segmentsInLevel()
+ level.segmentsInLevel().foldLeft(segmentsBeforeRepair.last.path.fileId.runRandomIO.right.value._1 + 1) {
+ case (overlappingSegmentId, segment) =>
+ val overlappingLevelSegmentPath = level.rootPath.resolve(overlappingSegmentId.toSegmentFileId)
+
+ def createOverlappingSegment() = {
+ val numberOfKeyValuesToOverlap = randomNextInt(3) max 1
+ val keyValuesToOverlap = Random.shuffle(segment.toSlice().runRandomIO.right.value.toList).take(numberOfKeyValuesToOverlap).map(_.toMemory).toSlice
+ //create overlapping Segment
+ val overlappingSegment = TestSegment(keyValuesToOverlap).runRandomIO.right.value
+ Effect.copy(overlappingSegment.path, overlappingLevelSegmentPath).runRandomIO.right.value
+ overlappingSegment.close.runRandomIO.right.value //gotta close the new segment create after it's copied over.
+ }
+
+ createOverlappingSegment()
+ //perform repair with Report
+ AppendixRepairer(level.rootPath, AppendixRepairStrategy.ReportFailure).left.runRandomIO.right.value.exception shouldBe a[OverlappingSegmentsException]
+ //perform repair with DeleteNext. This will delete the newest overlapping Segment.
+ AppendixRepairer(level.rootPath, AppendixRepairStrategy.KeepOld).runRandomIO.right.value
+ //overlapping Segment does not exist.
+ overlappingLevelSegmentPath.exists shouldBe false
+
+ //create overlapping Segment again but this time do DeletePrevious
+ createOverlappingSegment()
+ AppendixRepairer(level.rootPath, AppendixRepairStrategy.KeepNew).runRandomIO.right.value
+ //newer overlapping Segment exists
+ overlappingLevelSegmentPath.exists shouldBe true
+ //older overlapping Segment is deleted
+ segment.existsOnDisk shouldBe false
+
+ overlappingSegmentId + 1
}
-
- createOverlappingSegment()
- //perform repair with Report
- AppendixRepairer(level.rootPath, AppendixRepairStrategy.ReportFailure).left.runRandomIO.right.value.exception shouldBe a[OverlappingSegmentsException]
- //perform repair with DeleteNext. This will delete the newest overlapping Segment.
- AppendixRepairer(level.rootPath, AppendixRepairStrategy.KeepOld).runRandomIO.right.value
- //overlapping Segment does not exist.
- overlappingLevelSegmentPath.exists shouldBe false
-
- //create overlapping Segment again but this time do DeletePrevious
- createOverlappingSegment()
- AppendixRepairer(level.rootPath, AppendixRepairStrategy.KeepNew).runRandomIO.right.value
- //newer overlapping Segment exists
- overlappingLevelSegmentPath.exists shouldBe true
- //older overlapping Segment is deleted
- segment.existsOnDisk shouldBe false
-
- overlappingSegmentId + 1
+ val reopenedLevel = level.reopen
+ reopenedLevel.segmentsCount() shouldBe segmentsBeforeRepair.size
+ reopenedLevel.closeNoSweep().runRandomIO.right.value
}
- val reopenedLevel = level.reopen
- reopenedLevel.segmentsCount() shouldBe segmentsBeforeRepair.size
- reopenedLevel.close.runRandomIO.right.value
}
}
}
diff --git a/core/src/test/scala/swaydb/core/level/zero/LevelZeroSpec.scala b/core/src/test/scala/swaydb/core/level/zero/LevelZeroSpec.scala
index e6ea81c91..31d34bc04 100644
--- a/core/src/test/scala/swaydb/core/level/zero/LevelZeroSpec.scala
+++ b/core/src/test/scala/swaydb/core/level/zero/LevelZeroSpec.scala
@@ -33,7 +33,8 @@ import swaydb.core.TestData._
import swaydb.core.data.Memory
import swaydb.core.io.file.Effect
import swaydb.core.segment.ThreadReadState
-import swaydb.core.{TestBase, TestTimer}
+import swaydb.core.{TestBase, TestCaseSweeper, TestTimer}
+import TestCaseSweeper._
import swaydb.data.compaction.Throttle
import swaydb.data.config.MMAP
import swaydb.data.order.{KeyOrder, TimeOrder}
@@ -80,123 +81,146 @@ sealed trait LevelZeroSpec extends TestBase with MockFactory {
"LevelZero" should {
"initialise" in {
- val nextLevel = TestLevel()
- val zero = TestLevelZero(Some(nextLevel))
- if (persistent) {
- zero.existsOnDisk shouldBe true
- nextLevel.existsOnDisk shouldBe true
- //maps folder is initialised
- Effect.exists(zero.path.resolve("0/0.log")) shouldBe true
- zero.reopen.existsOnDisk shouldBe true
- } else {
- zero.existsOnDisk shouldBe false
- nextLevel.existsOnDisk shouldBe false
+ TestCaseSweeper {
+ implicit sweeper =>
+ val nextLevel = TestLevel()
+ val zero = TestLevelZero(Some(nextLevel))
+ if (persistent) {
+ zero.existsOnDisk shouldBe true
+ nextLevel.existsOnDisk shouldBe true
+ //maps folder is initialised
+ Effect.exists(zero.path.resolve("0/0.log")) shouldBe true
+ zero.reopen.existsOnDisk shouldBe true
+ } else {
+ zero.existsOnDisk shouldBe false
+ nextLevel.existsOnDisk shouldBe false
+ }
}
}
}
"LevelZero.put" should {
"write key-value" in {
- def assert(zero: LevelZero): Unit = {
- zero.put(1, "one").runRandomIO
- zero.get(1, ThreadReadState.random).getPut.getOrFetchValue shouldBe ("one": Slice[Byte])
-
- zero.put("2", "two").runRandomIO
- zero.get("2", ThreadReadState.random).getPut.getOrFetchValue shouldBe ("two": Slice[Byte])
+ TestCaseSweeper {
+ implicit sweeper =>
+ def assert(zero: LevelZero): Unit = {
+ zero.put(1, "one").runRandomIO
+ zero.get(1, ThreadReadState.random).getPut.getOrFetchValue shouldBe ("one": Slice[Byte])
+
+ zero.put("2", "two").runRandomIO
+ zero.get("2", ThreadReadState.random).getPut.getOrFetchValue shouldBe ("two": Slice[Byte])
+ }
+
+ val zero = TestLevelZero(Some(TestLevel(throttle = (_) => Throttle(10.seconds, 0))))
+ assert(zero)
+ if (persistent) assert(zero.reopen)
}
-
- val zero = TestLevelZero(Some(TestLevel(throttle = (_) => Throttle(10.seconds, 0))))
- assert(zero)
- if (persistent) assert(zero.reopen)
}
"write key-values that have empty bytes but the Slices are closed" in {
- val level = TestLevel(throttle = (_) => Throttle(10.seconds, 0))
- val zero = TestLevelZero(Some(level))
- val one = Slice.create[Byte](10).addInt(1).close()
-
- zero.put(one, one).runRandomIO
-
- val gotFromLevelZero = zero.get(one, ThreadReadState.random).getPut.getOrFetchValue.getC
- gotFromLevelZero shouldBe one
- //ensure that key-values are not unsliced in LevelZero.
- gotFromLevelZero.underlyingArraySize shouldBe 10
-
- //the following does not apply to in-memory Levels
- //in-memory key-values are slice of the whole Segment.
- if (persistent) {
- //put the same key-value to Level1 and expect the key-values to be sliced
- level.putKeyValuesTest(Slice(Memory.put(one, one))).runRandomIO
- val gotFromLevelOne = level.get(one, ThreadReadState.random).getPut
- gotFromLevelOne.getOrFetchValue shouldBe one
- //ensure that key-values are not unsliced in LevelOne.
- gotFromLevelOne.getOrFetchValue.getC.underlyingArraySize shouldBe 4
+ TestCaseSweeper {
+ implicit sweeper =>
+ val level = TestLevel(throttle = (_) => Throttle(10.seconds, 0))
+ val zero = TestLevelZero(Some(level))
+ val one = Slice.create[Byte](10).addInt(1).close()
+
+ zero.put(one, one).runRandomIO
+
+ val gotFromLevelZero = zero.get(one, ThreadReadState.random).getPut.getOrFetchValue.getC
+ gotFromLevelZero shouldBe one
+ //ensure that key-values are not unsliced in LevelZero.
+ gotFromLevelZero.underlyingArraySize shouldBe 10
+
+ //the following does not apply to in-memory Levels
+ //in-memory key-values are slice of the whole Segment.
+ if (persistent) {
+ //put the same key-value to Level1 and expect the key-values to be sliced
+ level.putKeyValuesTest(Slice(Memory.put(one, one))).runRandomIO
+ val gotFromLevelOne = level.get(one, ThreadReadState.random).getPut
+ gotFromLevelOne.getOrFetchValue shouldBe one
+ //ensure that key-values are not unsliced in LevelOne.
+ gotFromLevelOne.getOrFetchValue.getC.underlyingArraySize shouldBe 4
+ }
}
}
"not write empty key-value" in {
- val zero = TestLevelZero(Some(TestLevel()))
- IO(zero.put(Slice.empty, Slice.empty)).left.value shouldBe a[IllegalArgumentException]
+ TestCaseSweeper {
+ implicit sweeper =>
+ val zero = TestLevelZero(Some(TestLevel()))
+ IO(zero.put(Slice.empty, Slice.empty)).left.value shouldBe a[IllegalArgumentException]
+ }
}
"write empty values" in {
- val zero = TestLevelZero(Some(TestLevel()))
- zero.put(1, Slice.empty).runRandomIO
- zero.get(1, ThreadReadState.random).getPut.getOrFetchValue shouldBe Slice.Null
+ TestCaseSweeper {
+ implicit sweeper =>
+ val zero = TestLevelZero(Some(TestLevel()))
+ zero.put(1, Slice.empty).runRandomIO
+ zero.get(1, ThreadReadState.random).getPut.getOrFetchValue shouldBe Slice.Null
+ }
}
"write large keys and values and reopen the database and re-read key-values" in {
//approx 2 mb key and values
+ TestCaseSweeper {
+ implicit sweeper =>
+ val key1 = "a" + Random.nextString(750000): Slice[Byte]
+ val key2 = "b" + Random.nextString(750000): Slice[Byte]
- val key1 = "a" + Random.nextString(750000): Slice[Byte]
- val key2 = "b" + Random.nextString(750000): Slice[Byte]
+ val value1 = Random.nextString(750000): Slice[Byte]
+ val value2 = Random.nextString(750000): Slice[Byte]
- val value1 = Random.nextString(750000): Slice[Byte]
- val value2 = Random.nextString(750000): Slice[Byte]
+ def assertWrite(zero: LevelZero): Unit = {
+ zero.put(key1, value1).runRandomIO
+ zero.put(key2, value2).runRandomIO
+ }
- def assertWrite(zero: LevelZero): Unit = {
- zero.put(key1, value1).runRandomIO
- zero.put(key2, value2).runRandomIO
- }
-
- def assertRead(zero: LevelZero): Unit = {
- zero.get(key1, ThreadReadState.random).getPut.getOrFetchValue shouldBe value1
- zero.get(key2, ThreadReadState.random).getPut.getOrFetchValue shouldBe value2
- }
+ def assertRead(zero: LevelZero): Unit = {
+ zero.get(key1, ThreadReadState.random).getPut.getOrFetchValue shouldBe value1
+ zero.get(key2, ThreadReadState.random).getPut.getOrFetchValue shouldBe value2
+ }
- val zero = TestLevelZero(Some(TestLevel(throttle = _ => Throttle(10.seconds, 0))))
+ val zero = TestLevelZero(Some(TestLevel(throttle = _ => Throttle(10.seconds, 0))))
- assertWrite(zero)
- assertRead(zero)
+ assertWrite(zero)
+ assertRead(zero)
- //allow compaction to do it's work
- sleep(2.seconds)
- if (persistent) assertRead(zero.reopen)
+ //allow compaction to do it's work
+ sleep(2.seconds)
+ if (persistent) assertRead(zero.reopen)
+ }
}
"write keys only" in {
- val zero = TestLevelZero(Some(TestLevel()))
+ TestCaseSweeper {
+ implicit sweeper =>
+ val zero = TestLevelZero(Some(TestLevel()))
- zero.put("one").runRandomIO
- zero.put("two").runRandomIO
+ zero.put("one").runRandomIO
+ zero.put("two").runRandomIO
- zero.get("one", ThreadReadState.random).getPut.getOrFetchValue.toOptionC shouldBe empty
- zero.get("two", ThreadReadState.random).getPut.getOrFetchValue.toOptionC shouldBe empty
+ zero.get("one", ThreadReadState.random).getPut.getOrFetchValue.toOptionC shouldBe empty
+ zero.get("two", ThreadReadState.random).getPut.getOrFetchValue.toOptionC shouldBe empty
- zero.contains("one", ThreadReadState.random).runRandomIO.right.value shouldBe true
- zero.contains("two", ThreadReadState.random).runRandomIO.right.value shouldBe true
- zero.contains("three", ThreadReadState.random).runRandomIO.right.value shouldBe false
+ zero.contains("one", ThreadReadState.random).runRandomIO.right.value shouldBe true
+ zero.contains("two", ThreadReadState.random).runRandomIO.right.value shouldBe true
+ zero.contains("three", ThreadReadState.random).runRandomIO.right.value shouldBe false
+ }
}
"batch write key-values" in {
- val keyValues = randomIntKeyStringValues(keyValuesCount)
+ TestCaseSweeper {
+ implicit sweeper =>
+ val keyValues = randomIntKeyStringValues(keyValuesCount)
- val zero = TestLevelZero(Some(TestLevel()))
- zero.put(_ => keyValues.toMapEntry.get).runRandomIO
+ val zero = TestLevelZero(Some(TestLevel()))
+ zero.put(_ => keyValues.toMapEntry.get).runRandomIO
- assertGet(keyValues, zero)
+ assertGet(keyValues, zero)
- zero.keyValueCount.runRandomIO.right.value shouldBe keyValues.size
+ zero.keyValueCount.runRandomIO.right.value shouldBe keyValues.size
+ }
}
//removed test - empty check are performed at the source where the MapEntry is created.
@@ -218,146 +242,169 @@ sealed trait LevelZeroSpec extends TestBase with MockFactory {
"LevelZero.remove" should {
"remove key-values" in {
- val zero = TestLevelZero(Some(TestLevel(throttle = (_) => Throttle(10.seconds, 0))), mapSize = 1.byte)
- val keyValues = randomIntKeyStringValues(keyValuesCount)
- keyValues foreach {
- keyValue =>
- zero.put(keyValue.key, keyValue.getOrFetchValue).runRandomIO
+ TestCaseSweeper {
+ implicit sweeper =>
+ val zero = TestLevelZero(Some(TestLevel(throttle = (_) => Throttle(10.seconds, 0))), mapSize = 1.byte)
+ val keyValues = randomIntKeyStringValues(keyValuesCount)
+ keyValues foreach {
+ keyValue =>
+ zero.put(keyValue.key, keyValue.getOrFetchValue).runRandomIO
+ }
+
+ if (unexpiredPuts(keyValues).nonEmpty)
+ zero.head(ThreadReadState.random).runRandomIO.get.toOptionPut shouldBe defined
+
+ keyValues foreach {
+ keyValue =>
+ zero.remove(keyValue.key).runRandomIO
+ }
+
+ zero.head(ThreadReadState.random).toOptionPut shouldBe empty
+ zero.last(ThreadReadState.random).toOptionPut shouldBe empty
}
-
- if (unexpiredPuts(keyValues).nonEmpty)
- zero.head(ThreadReadState.random).runRandomIO.get.toOptionPut shouldBe defined
-
- keyValues foreach {
- keyValue =>
- zero.remove(keyValue.key).runRandomIO
- }
-
- zero.head(ThreadReadState.random).toOptionPut shouldBe empty
- zero.last(ThreadReadState.random).toOptionPut shouldBe empty
}
"batch remove key-values" in {
- val keyValues = randomIntKeyStringValues(keyValuesCount)
- val zero = TestLevelZero(Some(TestLevel()))
- zero.put(_ => keyValues.toMapEntry.get).runRandomIO
+ TestCaseSweeper {
+ implicit sweeper =>
+ val keyValues = randomIntKeyStringValues(keyValuesCount)
+ val zero = TestLevelZero(Some(TestLevel()))
+ zero.put(_ => keyValues.toMapEntry.get).runRandomIO
- assertGet(keyValues, zero)
+ assertGet(keyValues, zero)
- val removeKeyValues = Slice(keyValues.map(keyValue => Memory.remove(keyValue.key)).toArray)
- zero.put(_ => removeKeyValues.toMapEntry.get).runRandomIO
+ val removeKeyValues = Slice(keyValues.map(keyValue => Memory.remove(keyValue.key)).toArray)
+ zero.put(_ => removeKeyValues.toMapEntry.get).runRandomIO
- assertGetNone(keyValues, zero)
- zero.head(ThreadReadState.random).toOptionPut shouldBe empty
+ assertGetNone(keyValues, zero)
+ zero.head(ThreadReadState.random).toOptionPut shouldBe empty
+ }
}
}
"LevelZero.clear" should {
"a database with single key-value" in {
- val zero = TestLevelZero(Some(TestLevel(throttle = (_) => Throttle(10.seconds, 0))), mapSize = 1.byte)
- val keyValues = randomIntKeyStringValues(1)
- keyValues foreach {
- keyValue =>
- zero.put(keyValue.key, keyValue.getOrFetchValue).runRandomIO
- }
+ TestCaseSweeper {
+ implicit sweeper =>
+ val zero = TestLevelZero(Some(TestLevel(throttle = (_) => Throttle(10.seconds, 0))), mapSize = 1.byte)
+ val keyValues = randomIntKeyStringValues(1)
+ keyValues foreach {
+ keyValue =>
+ zero.put(keyValue.key, keyValue.getOrFetchValue).runRandomIO
+ }
- zero.keyValueCount shouldBe 1
+ zero.keyValueCount shouldBe 1
- zero.clear(ThreadReadState.random).runRandomIO.get
+ zero.clear(ThreadReadState.random).runRandomIO.get
- zero.head(ThreadReadState.random).toOptionPut shouldBe empty
- zero.last(ThreadReadState.random).toOptionPut shouldBe empty
+ zero.head(ThreadReadState.random).toOptionPut shouldBe empty
+ zero.last(ThreadReadState.random).toOptionPut shouldBe empty
+ }
}
"remove all key-values" in {
- val zero = TestLevelZero(Some(TestLevel(throttle = (_) => Throttle(10.seconds, 0))), mapSize = 1.byte)
- val keyValues = randomIntKeyStringValues(keyValuesCount)
- keyValues foreach {
- keyValue =>
- zero.put(keyValue.key, keyValue.getOrFetchValue).runRandomIO
+ TestCaseSweeper {
+ implicit sweeper =>
+ val zero = TestLevelZero(Some(TestLevel(throttle = (_) => Throttle(10.seconds, 0))), mapSize = 1.byte)
+ val keyValues = randomIntKeyStringValues(keyValuesCount)
+ keyValues foreach {
+ keyValue =>
+ zero.put(keyValue.key, keyValue.getOrFetchValue).runRandomIO
+ }
+
+ zero.clear(ThreadReadState.random).runRandomIO.get
+
+ zero.head(ThreadReadState.random).toOptionPut shouldBe empty
+ zero.last(ThreadReadState.random).toOptionPut shouldBe empty
}
-
- zero.clear(ThreadReadState.random).runRandomIO.get
-
- zero.head(ThreadReadState.random).toOptionPut shouldBe empty
- zero.last(ThreadReadState.random).toOptionPut shouldBe empty
}
}
"LevelZero.head" should {
"return the first key-value" in {
//disable throttle
- val zero = TestLevelZero(Some(TestLevel(throttle = (_) => Throttle(10.seconds, 0))), mapSize = 1.byte)
-
- zero.put(1, "one").runRandomIO.value
- zero.put(2, "two").runRandomIO.value
- zero.put(3, "three").runRandomIO.value
- zero.put(4, "four").runRandomIO.value
- zero.put(5, "five").runRandomIO.value
-
- val head = zero.head(ThreadReadState.random).getPut
- head.key shouldBe (1: Slice[Byte])
- head.getOrFetchValue shouldBe ("one": Slice[Byte])
-
- //remove 1
- zero.remove(1).runRandomIO
- println
- zero.head(ThreadReadState.random).getPut.getOrFetchValue shouldBe ("two": Slice[Byte])
-
- zero.remove(2).runRandomIO
- zero.remove(3).runRandomIO
- zero.remove(4).runRandomIO
-
- zero.head(ThreadReadState.random).getPut.getOrFetchValue shouldBe ("five": Slice[Byte])
-
- zero.remove(5).runRandomIO
- zero.head(ThreadReadState.random).toOptionPut shouldBe empty
- zero.last(ThreadReadState.random).toOptionPut shouldBe empty
+ TestCaseSweeper {
+ implicit sweeper =>
+ val zero = TestLevelZero(Some(TestLevel(throttle = (_) => Throttle(10.seconds, 0))), mapSize = 1.byte)
+
+ zero.put(1, "one").runRandomIO.value
+ zero.put(2, "two").runRandomIO.value
+ zero.put(3, "three").runRandomIO.value
+ zero.put(4, "four").runRandomIO.value
+ zero.put(5, "five").runRandomIO.value
+
+ val head = zero.head(ThreadReadState.random).getPut
+ head.key shouldBe (1: Slice[Byte])
+ head.getOrFetchValue shouldBe ("one": Slice[Byte])
+
+ //remove 1
+ zero.remove(1).runRandomIO
+ println
+ zero.head(ThreadReadState.random).getPut.getOrFetchValue shouldBe ("two": Slice[Byte])
+
+ zero.remove(2).runRandomIO
+ zero.remove(3).runRandomIO
+ zero.remove(4).runRandomIO
+
+ zero.head(ThreadReadState.random).getPut.getOrFetchValue shouldBe ("five": Slice[Byte])
+
+ zero.remove(5).runRandomIO
+ zero.head(ThreadReadState.random).toOptionPut shouldBe empty
+ zero.last(ThreadReadState.random).toOptionPut shouldBe empty
+ }
}
}
"LevelZero.last" should {
"return the last key-value" in {
- val zero = TestLevelZero(Some(TestLevel()), mapSize = 1.byte)
+ TestCaseSweeper {
+ implicit sweeper =>
+ val zero = TestLevelZero(Some(TestLevel()), mapSize = 1.byte)
- zero.put(1, "one").runRandomIO
- zero.put(2, "two").runRandomIO
- zero.put(3, "three").runRandomIO
- zero.put(4, "four").runRandomIO
- zero.put(5, "five").runRandomIO
+ zero.put(1, "one").runRandomIO
+ zero.put(2, "two").runRandomIO
+ zero.put(3, "three").runRandomIO
+ zero.put(4, "four").runRandomIO
+ zero.put(5, "five").runRandomIO
- zero.last(ThreadReadState.random).getPut.getOrFetchValue shouldBe ("five": Slice[Byte])
+ zero.last(ThreadReadState.random).getPut.getOrFetchValue shouldBe ("five": Slice[Byte])
- //remove 5
- zero.remove(5).runRandomIO
- zero.last(ThreadReadState.random).getPut.getOrFetchValue shouldBe ("four": Slice[Byte])
+ //remove 5
+ zero.remove(5).runRandomIO
+ zero.last(ThreadReadState.random).getPut.getOrFetchValue shouldBe ("four": Slice[Byte])
- zero.remove(2).runRandomIO
- zero.remove(3).runRandomIO
- zero.remove(4).runRandomIO
+ zero.remove(2).runRandomIO
+ zero.remove(3).runRandomIO
+ zero.remove(4).runRandomIO
- println
- zero.last(ThreadReadState.random).getPut.getOrFetchValue shouldBe ("one": Slice[Byte])
+ zero.last(ThreadReadState.random).getPut.getOrFetchValue shouldBe ("one": Slice[Byte])
- zero.remove(1).runRandomIO
- zero.last(ThreadReadState.random).toOptionPut shouldBe empty
- zero.head(ThreadReadState.random).toOptionPut shouldBe empty
+ zero.remove(1).runRandomIO
+ zero.last(ThreadReadState.random).toOptionPut shouldBe empty
+ zero.head(ThreadReadState.random).toOptionPut shouldBe empty
+ }
}
}
"LevelZero.remove range" should {
"not allow from key to be > than to key" in {
- val zero = TestLevelZero(Some(TestLevel()), mapSize = 1.byte)
- IO(zero.remove(10, 1)).left.value.getMessage shouldBe "fromKey should be less than toKey."
- IO(zero.remove(2, 1)).left.value.getMessage shouldBe "fromKey should be less than toKey."
+ TestCaseSweeper {
+ implicit sweeper =>
+ val zero = TestLevelZero(Some(TestLevel()), mapSize = 1.byte)
+ IO(zero.remove(10, 1)).left.value.getMessage shouldBe "fromKey should be less than toKey."
+ IO(zero.remove(2, 1)).left.value.getMessage shouldBe "fromKey should be less than toKey."
+ }
}
}
"LevelZero.update range" should {
"not allow from key to be > than to key" in {
- val zero = TestLevelZero(Some(TestLevel()), mapSize = 1.byte)
- IO(zero.update(10, 1, value = "value")).left.value.getMessage shouldBe "fromKey should be less than toKey."
- IO(zero.update(2, 1, value = "value")).left.value.getMessage shouldBe "fromKey should be less than toKey."
+ TestCaseSweeper {
+ implicit sweeper =>
+ val zero = TestLevelZero(Some(TestLevel()), mapSize = 1.byte)
+ IO(zero.update(10, 1, value = "value")).left.value.getMessage shouldBe "fromKey should be less than toKey."
+ IO(zero.update(2, 1, value = "value")).left.value.getMessage shouldBe "fromKey should be less than toKey."
+ }
}
}
}
diff --git a/core/src/test/scala/swaydb/core/map/package.scala b/core/src/test/scala/swaydb/core/map/package.scala
index 87028a5a0..3fec927b2 100644
--- a/core/src/test/scala/swaydb/core/map/package.scala
+++ b/core/src/test/scala/swaydb/core/map/package.scala
@@ -75,7 +75,7 @@ package object map {
*/
def ensureClose(): Unit = {
map.close()
- map.bufferCleaner.actor.receiveAllBlocking(Int.MaxValue).get
+ map.bufferCleaner.actor.receiveAllBlocking(Int.MaxValue, 1.second).get
val isShut = (map.bufferCleaner.actor ask ByteBufferSweeper.Command.IsTerminatedAndCleaned[Unit]).await(10.seconds)
assert(isShut, "Is not shut")
}
diff --git a/core/src/test/scala/swaydb/core/map/serializer/AppendixMapEntrySpec.scala b/core/src/test/scala/swaydb/core/map/serializer/AppendixMapEntrySpec.scala
index 14036ceea..654046f8f 100644
--- a/core/src/test/scala/swaydb/core/map/serializer/AppendixMapEntrySpec.scala
+++ b/core/src/test/scala/swaydb/core/map/serializer/AppendixMapEntrySpec.scala
@@ -55,11 +55,12 @@ class AppendixMapEntrySpec extends TestBase {
implicit def segmentIO: SegmentIO = SegmentIO.random
val appendixReader = AppendixMapEntryReader(MMAP.Enabled(OperatingSystem.isWindows))
- val segment = TestSegment()
"MapEntryWriterAppendix & MapEntryReaderAppendix" should {
"write Add segment" in {
+ val segment = TestSegment()
+
import AppendixMapEntryWriter.AppendixPutWriter
val entry = MapEntry.Put[Slice[Byte], Segment](segment.minKey, segment)
diff --git a/core/src/test/scala/swaydb/core/segment/format/a/SegmentSearcherSpec.scala b/core/src/test/scala/swaydb/core/segment/format/a/SegmentSearcherSpec.scala
index 3f6694761..88d754555 100644
--- a/core/src/test/scala/swaydb/core/segment/format/a/SegmentSearcherSpec.scala
+++ b/core/src/test/scala/swaydb/core/segment/format/a/SegmentSearcherSpec.scala
@@ -262,7 +262,7 @@ class SegmentSearcherSpec extends TestBase with MockFactory {
Benchmark(s"Creating Segment for ${keyValues.size}") {
getBlocks(
keyValues = keyValues,
- segmentConfig = SegmentBlock.Config.random.copy(minSize = Int.MaxValue)
+ segmentConfig = SegmentBlock.Config.random.copy(minSize = Int.MaxValue, maxCount = Int.MaxValue)
).get
}
diff --git a/core/src/test/scala/swaydb/core/segment/format/a/SegmentWriteSpec.scala b/core/src/test/scala/swaydb/core/segment/format/a/SegmentWriteSpec.scala
index e3454053c..16a01a8a8 100644
--- a/core/src/test/scala/swaydb/core/segment/format/a/SegmentWriteSpec.scala
+++ b/core/src/test/scala/swaydb/core/segment/format/a/SegmentWriteSpec.scala
@@ -582,7 +582,7 @@ sealed trait SegmentWriteSpec extends TestBase {
//finally also close the segment to close the file.
close
- fileSweeper.terminateAndRecover[Future](10.seconds).await(10.seconds)
+ fileSweeper.terminateAndRecoverAsync[Future](10.seconds).await(10.seconds)
}
}
@@ -700,7 +700,7 @@ sealed trait SegmentWriteSpec extends TestBase {
segment1.close
segment2.close
- fileSweeper.terminateAndRecover[Future](10.seconds).await(10.seconds)
+ fileSweeper.terminateAndRecoverAsync[Future](10.seconds).await(10.seconds)
}
}
}
diff --git a/core/src/test/scala/swaydb/core/segment/format/a/block/SegmentBlockCacheSpec.scala b/core/src/test/scala/swaydb/core/segment/format/a/block/SegmentBlockCacheSpec.scala
index 03018ad29..a1eeeaf8a 100644
--- a/core/src/test/scala/swaydb/core/segment/format/a/block/SegmentBlockCacheSpec.scala
+++ b/core/src/test/scala/swaydb/core/segment/format/a/block/SegmentBlockCacheSpec.scala
@@ -7,7 +7,6 @@ import swaydb.IOValues._
import swaydb.core.CommonAssertions._
import swaydb.core.RunThis._
import swaydb.core.TestData._
-import swaydb.core.TestSweeper.level0PushDownPool
import swaydb.core.actor.MemorySweeper
import swaydb.core.data.Memory
import swaydb.core.segment.PersistentSegmentOne
@@ -18,7 +17,7 @@ import swaydb.core.segment.format.a.block.reader.UnblockedReader
import swaydb.core.segment.format.a.block.segment.{SegmentBlock, SegmentBlockCache}
import swaydb.core.segment.format.a.block.sortedindex.SortedIndexBlock
import swaydb.core.segment.format.a.block.values.ValuesBlock
-import swaydb.core.{TestBase, TestTimer}
+import swaydb.core.{TestBase, TestExecutionContext, TestTimer}
import swaydb.data.config.{ActorConfig, IOAction, MemoryCache}
import swaydb.data.order.KeyOrder
import swaydb.data.slice.Slice
@@ -30,6 +29,7 @@ import scala.jdk.CollectionConverters._
import scala.concurrent.duration._
import scala.util.Random
import scala.collection.parallel.CollectionConverters._
+import scala.concurrent.ExecutionContext
class SegmentBlockCacheSpec extends TestBase {
implicit val order = KeyOrder.default
@@ -117,7 +117,7 @@ class SegmentBlockCacheSpec extends TestBase {
"not add un-cached blocks and readers to memory sweeper" in {
runThis(10.times) {
implicit val sweeper: Option[MemorySweeper.Block] =
- MemorySweeper(MemoryCache.ByteCacheOnly(4098, 50000.bytes, 600.mb, ActorConfig.random()(level0PushDownPool)))
+ MemorySweeper(MemoryCache.ByteCacheOnly(4098, 50000.bytes, 600.mb, ActorConfig.random()(TestExecutionContext.executionContext)))
.map(_.asInstanceOf[MemorySweeper.Block])
val actor = sweeper.value.actor.value
@@ -194,7 +194,7 @@ class SegmentBlockCacheSpec extends TestBase {
"add cached blocks to memory sweeper" in {
runThis(10.times) {
implicit val sweeper: Option[MemorySweeper.Block] =
- MemorySweeper(MemoryCache.ByteCacheOnly(4098, 50000.bytes, 600.mb, ActorConfig.random(10.seconds)(level0PushDownPool)))
+ MemorySweeper(MemoryCache.ByteCacheOnly(4098, 50000.bytes, 600.mb, ActorConfig.random(10.seconds)(TestExecutionContext.executionContext)))
.map(_.asInstanceOf[MemorySweeper.Block])
val actor = sweeper.value.actor.value
diff --git a/data/src/main/scala/swaydb/Actor.scala b/data/src/main/scala/swaydb/Actor.scala
index 6def5ec45..5541756bc 100644
--- a/data/src/main/scala/swaydb/Actor.scala
+++ b/data/src/main/scala/swaydb/Actor.scala
@@ -74,7 +74,7 @@ sealed trait ActorRef[-T, S] { self =>
def receiveAllForce[BAG[_]](retryOnBusyDelay: FiniteDuration)(implicit scheduler: Scheduler,
bag: Bag.Async[BAG]): BAG[Unit]
- def receiveAllBlocking(retryCounts: Int): Try[Unit]
+ def receiveAllBlocking(retryCounts: Int, block: FiniteDuration): Try[Unit]
def terminate(): Unit
@@ -86,8 +86,10 @@ sealed trait ActorRef[-T, S] { self =>
def terminateAndClear(): Unit
- def terminateAndRecover[BAG[_]](retryOnBusyDelay: FiniteDuration)(implicit scheduler: Scheduler,
- bag: Bag.Async[BAG]): BAG[Unit]
+ def terminateAndRecoverAsync[BAG[_]](retryOnBusyDelay: FiniteDuration)(implicit scheduler: Scheduler,
+ bag: Bag.Async[BAG]): BAG[Unit]
+
+ def terminateAndRecoverSync[BAG[_]](retryOnBusyDelay: FiniteDuration)(implicit bag: Bag.Sync[BAG]): BAG[Unit]
}
object Actor {
@@ -114,13 +116,14 @@ object Actor {
override def isEmpty: Boolean = throw new Exception("Dead Actor")
override def recover[M <: T, E: ExceptionHandler](f: (M, IO[E, Error], Actor[T, S]) => Unit): ActorRef[T, S] = throw new Exception("Dead Actor")
override def receiveAllForce[BAG[_]](retryOnBusyDelay: FiniteDuration)(implicit scheduler: Scheduler, bag: Bag.Async[BAG]): BAG[Unit] = throw new Exception("Dead Actor")
- override def receiveAllBlocking(retryCounts: Int): Try[Unit] = throw new Exception("Dead Actor")
+ override def receiveAllBlocking(retryCounts: Int, block: FiniteDuration): Try[Unit] = throw new Exception("Dead Actor")
override def terminate(): Unit = throw new Exception("Dead Actor")
- override def terminateAfter(timeout: FiniteDuration)(implicit scheduler: Scheduler): ActorRef[T, S] = throw new Exception("Dead Actor")
override def isTerminated: Boolean = throw new Exception("Dead Actor")
override def clear(): Unit = throw new Exception("Dead Actor")
override def terminateAndClear(): Unit = throw new Exception("Dead Actor")
- override def terminateAndRecover[BAG[_]](retryOnBusyDelay: FiniteDuration)(implicit scheduler: Scheduler, bag: Bag.Async[BAG]): BAG[Unit] = throw new Exception("Dead Actor")
+ override def terminateAndRecoverAsync[BAG[_]](retryOnBusyDelay: FiniteDuration)(implicit scheduler: Scheduler, bag: Bag.Async[BAG]): BAG[Unit] = throw new Exception("Dead Actor")
+ override def terminateAfter(timeout: FiniteDuration)(implicit scheduler: Scheduler): ActorRef[T, S] = throw new Exception("Dead Actor")
+ override def terminateAndRecoverSync[BAG[_]](retryOnBusyDelay: FiniteDuration)(implicit bag: Bag.Sync[BAG]): BAG[Unit] = throw new Exception("Dead Actor")
}
def cacheFromConfig[T](config: ActorConfig,
@@ -615,26 +618,30 @@ class Actor[-T, S](val name: String,
bag: Bag.Async[BAG]): BAG[Unit] =
bag.suspend(bag.fromFuture(receiveAllInFuture(retryOnBusyDelay)))
- override def receiveAllBlocking(retryCounts: Int): Try[Unit] =
- receiveAllBlocking(0, retryCounts)
+ override def receiveAllBlocking(retryCounts: Int, block: FiniteDuration): Try[Unit] =
+ receiveAllBlocking(0, retryCounts, block)
@tailrec
- private def receiveAllBlocking(retires: Int, maxRetries: Int): Try[Unit] =
- if (retires > maxRetries)
+ private def receiveAllBlocking(retires: Int, maxRetries: Int, block: FiniteDuration): Try[Unit] =
+ if (retires > maxRetries) {
Failure(new Exception(s"Retries timeout. Retries: $retires. maxRetries: $maxRetries"))
- else if (busy.compareAndSet(false, true))
+ } else if (busy.compareAndSet(false, true)) {
Try(receive(Int.MaxValue, wakeUpOnComplete = false)) match {
case success @ Success(_) =>
- if (messageCount <= 0)
+ if (messageCount <= 0) {
success
- else
- receiveAllBlocking(0, maxRetries)
+ } else {
+ Thread.sleep(block.toMillis)
+ receiveAllBlocking(0, maxRetries, block)
+ }
case failure @ Failure(_) =>
failure
}
- else
- receiveAllBlocking(retires + 1, maxRetries)
+ } else {
+ Thread.sleep(block.toMillis)
+ receiveAllBlocking(retires + 1, maxRetries, block)
+ }
private def receive(overflow: Int, wakeUpOnComplete: Boolean): Unit = {
@@ -723,8 +730,8 @@ class Actor[-T, S](val name: String,
*
* If [[recover]] function is not specified then all queues messages are cleared.
*/
- def terminateAndRecover[BAG[_]](retryOnBusyDelay: FiniteDuration)(implicit scheduler: Scheduler,
- bag: Bag.Async[BAG]): BAG[Unit] =
+ def terminateAndRecoverAsync[BAG[_]](retryOnBusyDelay: FiniteDuration)(implicit scheduler: Scheduler,
+ bag: Bag.Async[BAG]): BAG[Unit] =
bag.suspend {
terminate()
if (recovery.isDefined) {
@@ -736,6 +743,24 @@ class Actor[-T, S](val name: String,
}
}
+ override def terminateAndRecoverSync[BAG[_]](retryOnBusyDelay: FiniteDuration)(implicit bag: Bag.Sync[BAG]): BAG[Unit] =
+ bag.suspend {
+ terminate()
+ if (recovery.isDefined) {
+ receiveAllBlocking(100, retryOnBusyDelay) match {
+ case Success(_) =>
+ bag.unit
+
+ case Failure(exception) =>
+ bag.failure(exception)
+ }
+ } else {
+ logger.error(s"""terminateAndRecover invoked on Actor("$name") with no recovery defined. Messages cleared.""")
+ clear()
+ bag.unit
+ }
+ }
+
override def terminateAndClear(): Unit = {
terminate()
clear()
diff --git a/data/src/main/scala/swaydb/IO.scala b/data/src/main/scala/swaydb/IO.scala
index 3a638a22a..26cc6dc78 100644
--- a/data/src/main/scala/swaydb/IO.scala
+++ b/data/src/main/scala/swaydb/IO.scala
@@ -756,9 +756,40 @@ object IO {
}
- def fromFuture[L: IO.ExceptionHandler, R](future: Future[R])(implicit ec: ExecutionContext): IO.Defer[L, R] = {
- val reserve = Reserve.busy((), "Future reserve")
- future onComplete {
+ /**
+ * There is a bug here. The Future should not be executed from within [[IO.Defer]].
+ *
+ * The following will lead to an infinite loop because future will get invoked each time
+ * the [[Reserve]] returns free but the future is incomplete which will result in the Defer
+ * being created again which will result in a new future.
+ *
+ * But using this as root defer is ok.
+ *
+ * {{{
+ * def future = Future(println("Running future")
+ *
+ * IO
+ * .Defer
+ * .unit
+ * .and(IO.fromFuture(future, "My test"))
+ * }}}
+ *
+ *
+ * Use val instead where new future is not created every time [[IO.Defer]] is retried on failure/busy.
+ *
+ * {{{
+ * val future = Future(println("Running future")
+ *
+ * IO
+ * .Defer
+ * .and(IO.fromFuture(future, "My test"))
+ * .unit
+ * }}}
+ */
+ private[swaydb] def fromFuture[R](executedFuture: Future[R], reserveName: String = "Future reserve")(implicit ec: ExecutionContext): IO.Defer[swaydb.Error.Level, R] = {
+ val reserve = Reserve.busy((), reserveName)
+
+ executedFuture onComplete {
_ =>
Reserve.setFree(reserve)
}
@@ -769,33 +800,30 @@ object IO {
*
* This is necessary to avoid blocking Futures.
*/
- def deferredValue =
- future.value map {
- case scala.util.Success(value) =>
- //success
- value
+ def deferredValue(): R =
+ executedFuture.value match {
+ case Some(result) =>
+ result match {
+ case scala.util.Success(value) =>
+ //success
+ value
+
+ case scala.util.Failure(exception) =>
+ //throw Future's failure so deferred can react to this.
+ //wrap it in another Exception in-case the inner exception is recoverable because the future is complete
+ //and cannot be recovered.
+ throw new Exception(exception)
+ }
- case scala.util.Failure(exception) =>
- //throw Future's failure so deferred can react to this.
- //wrap it in another Exception incase the inner exception is recoverable because the future is complete
- //and cannot be recovered.
- throw new Exception(exception)
- } getOrElse {
- //Accessing Future when its incomplete.
- throw swaydb.Exception.GetOnIncompleteDeferredFutureIO(reserve)
+ case None =>
+ //Accessing Future when its incomplete.
+ throw swaydb.Exception.GetOnIncompleteDeferredFutureIO(reserve)
}
- val error = swaydb.Error.ReservedResource(reserve)
-
- //Deferred instance that will handle the outcome of the Future
- val recoverableDeferred =
- IO.Defer[swaydb.Error.Segment, R](
- value = deferredValue,
- error = error
- )
+ import swaydb.Error.Level.ExceptionHandler
//Deferred that returns the result of the above deferred when completed.
- IO.Defer[L, R](recoverableDeferred.toIO.get)
+ IO.Defer[swaydb.Error.Level, R](deferredValue(), swaydb.Error.ReservedResource(reserve))
}
/** **********************************
@@ -804,7 +832,7 @@ object IO {
* ************ DEFERRED ************
* **********************************
* **********************************
- * **********************************/
+ * ********************************* */
object Defer extends LazyLogging {
@@ -864,7 +892,7 @@ object IO {
def isFailure: Boolean =
isPending && toIO.isLeft
- private[Defer] def getUnsafe: A = {
+ private[IO] def getUnsafe: A = {
//Runs composed functions does not perform any recovery.
def forceGet: A =
getValue getOrElse {
@@ -876,10 +904,13 @@ object IO {
if (_value.isDefined || !isBusy)
forceGet
else
- error map {
- error =>
+ error match {
+ case Some(error) =>
throw IO.ExceptionHandler.toException[E](error)
- } getOrElse forceGet
+
+ case None =>
+ forceGet
+ }
}
def toIO: IO[E, A] =
@@ -929,8 +960,8 @@ object IO {
case IO.Left(error) =>
logger.trace(s"Run! isCached: ${getValue.isDefined}. ${io.getClass.getSimpleName}")
if (recovery.isDefined) //pattern matching is not allowing @tailrec. So .get is required here.
- doRun(recovery.get.asInstanceOf[E => IO.Defer[E, A]](error), 0)
- else
+ doRun(recovery.get.asInstanceOf[E => IO.Defer[E, A]](error), 0)
+ else
io
}
@@ -975,8 +1006,8 @@ object IO {
case IO.Left(error) =>
logger.trace(s"Run! isCached: ${getValue.isDefined}. ${io.getClass.getSimpleName}")
if (recovery.isDefined) //pattern matching is not allowing @tailrec. So .get is required here.
- doRun(recovery.get.asInstanceOf[E => IO.Defer[E, B]](error), 0)
- else
+ doRun(recovery.get.asInstanceOf[E => IO.Defer[E, B]](error), 0)
+ else
bag.fromIO(io)
}
@@ -1043,8 +1074,8 @@ object IO {
case IO.Left(error) =>
if (recovery.isDefined) //pattern matching is not allowing @tailrec. So .get is required here.
- runNow(recovery.get.asInstanceOf[E => IO.Defer[E, B]](error), 0)
- else
+ runNow(recovery.get.asInstanceOf[E => IO.Defer[E, B]](error), 0)
+ else
bag.fromIO(io)
}
@@ -1089,12 +1120,32 @@ object IO {
error = error
)
+ def and[F >: E : IO.ExceptionHandler, B](f: => IO.Defer[F, B]): IO.Defer[F, B] =
+ IO.Defer[F, B](
+ operation =
+ () => {
+ getUnsafe
+ f.getUnsafe
+ },
+ error = error
+ )
+
def flatMapIO[F >: E : IO.ExceptionHandler, B](f: A => IO[F, B]): IO.Defer[F, B] =
IO.Defer[F, B](
operation = () => f(getUnsafe).get,
error = error
)
+ def andIO[F >: E : IO.ExceptionHandler, B](f: => IO[F, B]): IO.Defer[F, B] =
+ IO.Defer[F, B](
+ operation =
+ () => {
+ getUnsafe
+ f.get
+ },
+ error = error
+ )
+
def recover[B >: A](f: PartialFunction[E, B]): IO.Defer[E, B] =
copy(
recovery =
diff --git a/data/src/main/scala/swaydb/data/util/Futures.scala b/data/src/main/scala/swaydb/data/util/Futures.scala
index b4b11c9c9..5c8dd4a98 100644
--- a/data/src/main/scala/swaydb/data/util/Futures.scala
+++ b/data/src/main/scala/swaydb/data/util/Futures.scala
@@ -24,7 +24,9 @@
package swaydb.data.util
-import scala.concurrent.Future
+import swaydb.IO
+
+import scala.concurrent.{ExecutionContext, Future}
object Futures {
@@ -32,4 +34,12 @@ object Futures {
val unit: Future[Unit] = Future.successful(())
val `true` = Future.successful(true)
val `false` = Future.successful(false)
+
+ implicit class FutureImplicits[T](future1: Future[T]) {
+ def and(future2: Future[T])(implicit executionContext: ExecutionContext) =
+ future1.flatMap(_ => future2)
+
+ def and[L, R](io: IO[L, T])(implicit executionContext: ExecutionContext) =
+ future1.flatMap(_ => io.toFuture)
+ }
}
diff --git a/data/src/test/scala/swaydb/data/IODeferSpec.scala b/data/src/test/scala/swaydb/data/IODeferSpec.scala
index 239509e27..c058674a3 100644
--- a/data/src/test/scala/swaydb/data/IODeferSpec.scala
+++ b/data/src/test/scala/swaydb/data/IODeferSpec.scala
@@ -62,7 +62,7 @@ class IODeferSpec extends AnyWordSpec with Matchers with Eventually with MockFac
val timeBeforeDeferred = System.currentTimeMillis()
future.isCompleted shouldBe false
- val defer = IO.fromFuture[swaydb.Error.Segment, A](future)
+ val defer = IO.fromFuture[A](future)
future.isCompleted shouldBe false
defer.isPending shouldBe true
defer.isReady shouldBe true
@@ -127,11 +127,11 @@ class IODeferSpec extends AnyWordSpec with Matchers with Eventually with MockFac
}
}
- val defer1 = IO.fromFuture[Error.Segment, Int](futures(0))
- val defer2 = IO.fromFuture[Error.Segment, Int](futures(1))
- val defer3 = IO.fromFuture[Error.Segment, Int](futures(2))
- val defer4 = IO.fromFuture[Error.Segment, Int](futures(3))
- val defer5 = IO.fromFuture[Error.Segment, Int](futures(4))
+ val defer1 = IO.fromFuture[Int](futures(0))
+ val defer2 = IO.fromFuture[Int](futures(1))
+ val defer3 = IO.fromFuture[Int](futures(2))
+ val defer4 = IO.fromFuture[Int](futures(3))
+ val defer5 = IO.fromFuture[Int](futures(4))
val createDefers = {
defer1 flatMap {
@@ -160,6 +160,35 @@ class IODeferSpec extends AnyWordSpec with Matchers with Eventually with MockFac
}
}
}
+
+ "concurrent success" when {
+ "future is initialised within deferred" in {
+ (1 to 5) foreach {
+ _ =>
+ def future: Future[Int] =
+ Future {
+ val sleeping = Random.nextInt(10)
+ println(s"Sleep for $sleeping.seconds")
+ Thread.sleep(sleeping.seconds.toMillis)
+ println(s"Completed sleep $sleeping.seconds")
+ 1
+ }
+
+ //this re-creates Future again and again on reboot.
+ val createDefers =
+ IO.Defer.unit.and(IO.fromFuture[Int](future))
+
+ if (Random.nextBoolean()) {
+ createDefers.runIO shouldBe IO.Right(1)
+ createDefers.runFutureIO shouldBe IO.Right(1)
+ } else {
+ createDefers.runFutureIO shouldBe IO.Right(1)
+ createDefers.runIO shouldBe IO.Right(1)
+ }
+ }
+ }
+ }
+
}
"runIO" when {
diff --git a/swaydb/src/main/scala/swaydb/Map.scala b/swaydb/src/main/scala/swaydb/Map.scala
index 3fbe792c1..86a944456 100644
--- a/swaydb/src/main/scala/swaydb/Map.scala
+++ b/swaydb/src/main/scala/swaydb/Map.scala
@@ -37,7 +37,7 @@ import swaydb.data.util.TupleOrNone
import swaydb.serializers.{Serializer, _}
import scala.collection.mutable
-import scala.concurrent.duration.{Deadline, FiniteDuration}
+import scala.concurrent.duration.{Deadline, DurationInt, FiniteDuration}
object Map {
@@ -433,11 +433,11 @@ case class Map[K, V, F, BAG[_]] private(private[swaydb] val core: Core[BAG],
def asScala: scala.collection.mutable.Map[K, V] =
ScalaMap[K, V, F](toBag[Bag.Less](Bag.less))
- def close(): BAG[Unit] =
- bag.suspend(core.close())
+ def close(retryInterval: FiniteDuration = 1.second): BAG[Unit] =
+ bag.suspend(core.close(retryInterval))
- def delete(): BAG[Unit] =
- bag.suspend(core.delete())
+ def delete(retryInterval: FiniteDuration = 1.second): BAG[Unit] =
+ bag.suspend(core.delete(retryInterval))
override def toString(): String =
classOf[Map[_, _, _, BAG]].getSimpleName
diff --git a/swaydb/src/main/scala/swaydb/MapT.scala b/swaydb/src/main/scala/swaydb/MapT.scala
index 230b5fe9b..da0095254 100644
--- a/swaydb/src/main/scala/swaydb/MapT.scala
+++ b/swaydb/src/main/scala/swaydb/MapT.scala
@@ -29,7 +29,7 @@ import java.nio.file.Path
import swaydb.data.accelerate.LevelZeroMeter
import swaydb.data.compaction.LevelMeter
-import scala.concurrent.duration.{Deadline, FiniteDuration}
+import scala.concurrent.duration.{Deadline, DurationInt, FiniteDuration}
/**
* Base trait for a basic SwayDB Map type.
@@ -177,7 +177,7 @@ trait MapT[K, V, F, BAG[_]] extends SetMapT[K, V, F, BAG] { self =>
def asScala: scala.collection.mutable.Map[K, V]
- def close(): BAG[Unit]
+ def close(retryInterval: FiniteDuration = 1.second): BAG[Unit]
- def delete(): BAG[Unit]
+ def delete(retryInterval: FiniteDuration = 1.second): BAG[Unit]
}
\ No newline at end of file
diff --git a/swaydb/src/main/scala/swaydb/MultiMap.scala b/swaydb/src/main/scala/swaydb/MultiMap.scala
index 6b50e5245..3c63145e0 100644
--- a/swaydb/src/main/scala/swaydb/MultiMap.scala
+++ b/swaydb/src/main/scala/swaydb/MultiMap.scala
@@ -37,7 +37,7 @@ import swaydb.serializers.{Serializer, _}
import scala.collection.compat._
import scala.collection.mutable
-import scala.concurrent.duration.{Deadline, FiniteDuration}
+import scala.concurrent.duration.{Deadline, DurationInt, FiniteDuration}
object MultiMap {
@@ -755,11 +755,11 @@ case class MultiMap[M, K, V, F, BAG[_]] private(private[swaydb] val innerMap: Ma
def asScala: scala.collection.mutable.Map[K, V] =
ScalaMap[K, V, F](toBag[Bag.Less](Bag.less))
- def close(): BAG[Unit] =
- innerMap.close()
+ def close(retryInterval: FiniteDuration = 1.second): BAG[Unit] =
+ innerMap.close(retryInterval)
- def delete(): BAG[Unit] =
- innerMap.delete()
+ def delete(retryInterval: FiniteDuration = 1.second): BAG[Unit] =
+ innerMap.delete(retryInterval)
override def toString(): String =
classOf[Map[_, _, _, BAG]].getSimpleName
diff --git a/swaydb/src/main/scala/swaydb/Queue.scala b/swaydb/src/main/scala/swaydb/Queue.scala
index 425b28487..b042c5be0 100644
--- a/swaydb/src/main/scala/swaydb/Queue.scala
+++ b/swaydb/src/main/scala/swaydb/Queue.scala
@@ -35,7 +35,7 @@ import swaydb.data.slice.Slice
import swaydb.serializers.Serializer
import scala.annotation.tailrec
-import scala.concurrent.duration.{Deadline, FiniteDuration}
+import scala.concurrent.duration.{Deadline, DurationInt, FiniteDuration}
object Queue {
/**
@@ -211,9 +211,9 @@ case class Queue[A] private(private val set: Set[(Long, A), Nothing, Bag.Less],
.stream
.map(_._2)
- def close(): Unit =
- set.close()
+ def close(retryInterval: FiniteDuration = 1.second): Unit =
+ set.close(retryInterval)
- def delete(): Unit =
- set.delete()
+ def delete(retryInterval: FiniteDuration = 1.second): Unit =
+ set.delete(retryInterval)
}
diff --git a/swaydb/src/main/scala/swaydb/Set.scala b/swaydb/src/main/scala/swaydb/Set.scala
index c4d9ed7f3..92bbb8120 100644
--- a/swaydb/src/main/scala/swaydb/Set.scala
+++ b/swaydb/src/main/scala/swaydb/Set.scala
@@ -34,7 +34,7 @@ import swaydb.data.compaction.LevelMeter
import swaydb.data.slice.{Slice, SliceOption}
import swaydb.serializers.{Serializer, _}
-import scala.concurrent.duration.{Deadline, FiniteDuration}
+import scala.concurrent.duration.{Deadline, DurationInt, FiniteDuration}
import swaydb.core.function.{FunctionStore => CoreFunctionStore}
object Set {
@@ -329,11 +329,11 @@ case class Set[A, F, BAG[_]] private(private[swaydb] val core: Core[BAG],
def asScala: scala.collection.mutable.Set[A] =
ScalaSet[A, F](toBag[Bag.Less](Bag.less))
- def close(): BAG[Unit] =
- bag.suspend(core.close())
+ def close(retryInterval: FiniteDuration = 1.second): BAG[Unit] =
+ bag.suspend(core.close(retryInterval))
- def delete(): BAG[Unit] =
- bag.suspend(core.delete())
+ def delete(retryInterval: FiniteDuration = 1.second): BAG[Unit] =
+ bag.suspend(core.delete(retryInterval))
override def toString(): String =
classOf[Map[_, _, _, BAG]].getSimpleName
diff --git a/swaydb/src/main/scala/swaydb/SetMap.scala b/swaydb/src/main/scala/swaydb/SetMap.scala
index b9eb6f95f..dbc408258 100644
--- a/swaydb/src/main/scala/swaydb/SetMap.scala
+++ b/swaydb/src/main/scala/swaydb/SetMap.scala
@@ -34,7 +34,7 @@ import swaydb.data.slice.Slice
import swaydb.serializers.Serializer
import scala.collection.mutable
-import scala.concurrent.duration.{Deadline, FiniteDuration}
+import scala.concurrent.duration.{Deadline, DurationInt, FiniteDuration}
object SetMap {
/**
@@ -271,11 +271,11 @@ case class SetMap[K, V, F, BAG[_]] private(set: Set[(K, V), F, BAG])(implicit ba
def toBag[X[_]](implicit bag: Bag[X]): SetMap[K, V, F, X] =
SetMap(set = set.toBag[X])
- def close(): BAG[Unit] =
- set.close()
+ def close(retryInterval: FiniteDuration = 1.second): BAG[Unit] =
+ set.close(retryInterval)
- def delete(): BAG[Unit] =
- set.delete()
+ def delete(retryInterval: FiniteDuration = 1.second): BAG[Unit] =
+ set.delete(retryInterval)
override def toString(): String =
classOf[SetMap[_, _, _, BAG]].getSimpleName
diff --git a/swaydb/src/main/scala/swaydb/SetMapT.scala b/swaydb/src/main/scala/swaydb/SetMapT.scala
index 1db0b19d4..4bbcf149a 100644
--- a/swaydb/src/main/scala/swaydb/SetMapT.scala
+++ b/swaydb/src/main/scala/swaydb/SetMapT.scala
@@ -30,7 +30,7 @@ import swaydb.data.accelerate.LevelZeroMeter
import swaydb.data.compaction.LevelMeter
import scala.collection.mutable
-import scala.concurrent.duration.{Deadline, FiniteDuration}
+import scala.concurrent.duration.{Deadline, DurationInt, FiniteDuration}
/**
* Base trait for a basic SwayDB SetMap type.
@@ -132,9 +132,9 @@ trait SetMapT[K, V, F, BAG[_]] {
def asScala: scala.collection.mutable.Map[K, V]
- def close(): BAG[Unit]
+ def close(retryInterval: FiniteDuration = 1.second): BAG[Unit]
- def delete(): BAG[Unit]
+ def delete(retryInterval: FiniteDuration = 1.second): BAG[Unit]
override def toString(): String =
classOf[SetMapT[_, _, _, BAG]].getSimpleName