├── project ├── build.properties └── plugins.sbt ├── doc ├── img │ ├── compaction.jpeg │ ├── spec │ │ ├── data-flow.jpg │ │ ├── find-key.jpg │ │ ├── get-all.jpg │ │ ├── merge-iterator.jpg │ │ ├── log-after-merge.jpg │ │ ├── log-before-merge.jpg │ │ ├── log-after-rollback.jpg │ │ └── log-over-two-files.jpg │ └── journal-shards.jpeg ├── store_format.md ├── README.md ├── store_spec.md └── design_spec.md ├── .travis.yml ├── src ├── test │ └── scala │ │ └── io │ │ └── iohk │ │ └── iodb │ │ ├── QuickStoreRefTest.scala │ │ ├── TestWithTempDir.scala │ │ ├── ForkExecutor.scala │ │ ├── bench │ │ ├── BlockProcessing.scala │ │ ├── CompactionBench.scala │ │ ├── Benchmark.scala │ │ ├── InitialProcessing.scala │ │ ├── RocksStore.scala │ │ ├── LevelDBStore.scala │ │ ├── SimpleKVBench.scala │ │ └── LongBench.scala │ │ ├── ReferenceStore.scala │ │ ├── ShardedStoreTest.scala │ │ ├── ShardedIteratorTest.scala │ │ ├── TestUtils.scala │ │ ├── smoke │ │ ├── RandomRollbackTest.scala │ │ └── M1Test.scala │ │ ├── prop │ │ ├── IODBSpecification.scala │ │ └── StoreSpecification.scala │ │ ├── LogStoreTest.scala │ │ └── StoreTest.scala └── main │ └── scala │ └── io │ └── iohk │ └── iodb │ ├── DataCorruptionException.scala │ ├── ByteArrayWrapper.scala │ ├── ShardedIterator.scala │ ├── Store.scala │ ├── Utils.java │ ├── QuickStore.scala │ └── ShardedStore.scala ├── lock.sbt ├── .gitignore ├── changelog.md ├── README.md └── LICENSE /project/build.properties: -------------------------------------------------------------------------------- 1 | sbt.version=0.13.13 2 | -------------------------------------------------------------------------------- /doc/img/compaction.jpeg: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/input-output-hk/iodb/HEAD/doc/img/compaction.jpeg -------------------------------------------------------------------------------- /doc/img/spec/data-flow.jpg: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/input-output-hk/iodb/HEAD/doc/img/spec/data-flow.jpg -------------------------------------------------------------------------------- /doc/img/spec/find-key.jpg: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/input-output-hk/iodb/HEAD/doc/img/spec/find-key.jpg -------------------------------------------------------------------------------- /doc/img/spec/get-all.jpg: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/input-output-hk/iodb/HEAD/doc/img/spec/get-all.jpg -------------------------------------------------------------------------------- /doc/img/journal-shards.jpeg: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/input-output-hk/iodb/HEAD/doc/img/journal-shards.jpeg -------------------------------------------------------------------------------- /.travis.yml: -------------------------------------------------------------------------------- 1 | language: scala 2 | jdk: 3 | - oraclejdk8 4 | scala: 5 | - 2.12.1 6 | script: 7 | - sbt -v -d test 8 | -------------------------------------------------------------------------------- /doc/img/spec/merge-iterator.jpg: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/input-output-hk/iodb/HEAD/doc/img/spec/merge-iterator.jpg -------------------------------------------------------------------------------- /doc/img/spec/log-after-merge.jpg: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/input-output-hk/iodb/HEAD/doc/img/spec/log-after-merge.jpg -------------------------------------------------------------------------------- /doc/img/spec/log-before-merge.jpg: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/input-output-hk/iodb/HEAD/doc/img/spec/log-before-merge.jpg -------------------------------------------------------------------------------- /doc/img/spec/log-after-rollback.jpg: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/input-output-hk/iodb/HEAD/doc/img/spec/log-after-rollback.jpg -------------------------------------------------------------------------------- /doc/img/spec/log-over-two-files.jpg: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/input-output-hk/iodb/HEAD/doc/img/spec/log-over-two-files.jpg -------------------------------------------------------------------------------- /src/test/scala/io/iohk/iodb/QuickStoreRefTest.scala: -------------------------------------------------------------------------------- 1 | package io.iohk.iodb 2 | 3 | 4 | class QuickStoreRefTest extends StoreTest { 5 | override def open(keySize: Int): Store = new QuickStore(dir) 6 | 7 | 8 | } -------------------------------------------------------------------------------- /project/plugins.sbt: -------------------------------------------------------------------------------- 1 | logLevel := Level.Warn 2 | // 3 | //addSbtPlugin("com.jsuereth" % "sbt-pgp" % "1.0.0") 4 | // 5 | addSbtPlugin("com.github.tkawachi" % "sbt-lock" % "0.3.0") 6 | // 7 | //addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "0.8.0") 8 | -------------------------------------------------------------------------------- /src/main/scala/io/iohk/iodb/DataCorruptionException.scala: -------------------------------------------------------------------------------- 1 | package io.iohk.iodb 2 | 3 | /** 4 | * Exception if data in files are corrupted 5 | */ 6 | class DataCorruptionException(msg: String) 7 | extends RuntimeException(msg) { 8 | 9 | } 10 | -------------------------------------------------------------------------------- /lock.sbt: -------------------------------------------------------------------------------- 1 | // DON'T EDIT THIS FILE. 2 | // This file is auto generated by sbt-lock 0.3.0. 3 | // https://github.com/tkawachi/sbt-lock/ 4 | dependencyOverrides in ThisBuild ++= Set( 5 | "com.google.guava" % "guava" % "19.0", 6 | "net.jpountz.lz4" % "lz4" % "1.3.0", 7 | "org.slf4j" % "slf4j-api" % "1.8.0-beta0" 8 | ) 9 | // LIBRARY_DEPENDENCIES_HASH ce13340a74dc3cbf832944a27d30edfe95339aeb 10 | -------------------------------------------------------------------------------- /src/test/scala/io/iohk/iodb/TestWithTempDir.scala: -------------------------------------------------------------------------------- 1 | package io.iohk.iodb 2 | 3 | import java.io.File 4 | 5 | import org.junit.{After, Before} 6 | import org.scalatest.Assertions 7 | 8 | trait TestWithTempDir extends Assertions { 9 | 10 | var dir: File = _ 11 | 12 | def storeSize: Long = TestUtils.dirSize(dir) 13 | 14 | @Before def init() { 15 | dir = TestUtils.tempDir() 16 | } 17 | 18 | @After def deleteFiles(): Unit = { 19 | if (dir == null) return 20 | TestUtils.deleteRecur(dir) 21 | } 22 | } 23 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | # use glob syntax. 2 | syntax: glob 3 | *.ser 4 | *.class 5 | *~ 6 | *.bak 7 | #*.off 8 | *.old 9 | 10 | # eclipse conf file 11 | .settings 12 | .classpath 13 | .project 14 | .manager 15 | .scala_dependencies 16 | 17 | # idea 18 | .idea 19 | *.iml 20 | 21 | # building 22 | target 23 | build 24 | null 25 | tmp* 26 | temp* 27 | dist 28 | test-output 29 | build.log 30 | 31 | # other scm 32 | .svn 33 | .CVS 34 | .hg* 35 | 36 | # switch to regexp syntax. 37 | # syntax: regexp 38 | # ^\.pc/ 39 | 40 | #SHITTY output not in target directory 41 | build.log 42 | -------------------------------------------------------------------------------- /src/test/scala/io/iohk/iodb/ForkExecutor.scala: -------------------------------------------------------------------------------- 1 | package io.iohk.iodb 2 | 3 | import java.util.concurrent.atomic.AtomicReference 4 | import java.util.concurrent.{Executors, TimeUnit} 5 | 6 | /** executes multiple tasks in background thread, waits until they all finish, fails if any tasks throws exception */ 7 | class ForkExecutor(duration: Long) { 8 | 9 | val endTime = System.currentTimeMillis() + duration 10 | 11 | val exception = new AtomicReference[Throwable]() 12 | 13 | val executor = Executors.newCachedThreadPool() 14 | 15 | def keepRunning: Boolean = System.currentTimeMillis() < endTime 16 | 17 | 18 | def execute(task: => Unit): Unit = { 19 | executor.submit(TestUtils.runnable { 20 | try { 21 | task 22 | } catch { 23 | case e: Throwable => exception.compareAndSet(null, e) 24 | } 25 | }) 26 | } 27 | 28 | def finish(): Unit = { 29 | executor.shutdown() 30 | 31 | def rethrow(): Unit = { 32 | if (exception.get() != null) { 33 | throw new RuntimeException(exception.get()) 34 | } 35 | } 36 | 37 | while (!executor.awaitTermination(1, TimeUnit.SECONDS)) { 38 | rethrow() 39 | } 40 | rethrow() 41 | } 42 | 43 | 44 | } 45 | -------------------------------------------------------------------------------- /src/main/scala/io/iohk/iodb/ByteArrayWrapper.scala: -------------------------------------------------------------------------------- 1 | package io.iohk.iodb 2 | 3 | import java.io.Serializable 4 | import java.nio.ByteBuffer 5 | import java.util 6 | 7 | object ByteArrayWrapper { 8 | 9 | def fromLong(id: Long): ByteArrayWrapper = { 10 | val b = ByteBuffer.allocate(8) 11 | b.putLong(0, id) 12 | ByteArrayWrapper(b.array()) 13 | } 14 | } 15 | 16 | /** 17 | * Wraps byte array and provides hashCode, equals and compare methods. 18 | */ 19 | case class ByteArrayWrapper(data: Array[Byte]) 20 | extends Serializable 21 | with Comparable[ByteArrayWrapper] 22 | with Ordered[ByteArrayWrapper] { 23 | 24 | /** alternative constructor which takes array size and creates new empty array */ 25 | def this(size:Int) = this(new Array[Byte](size)) 26 | 27 | def size = data.length 28 | 29 | require(data != null) 30 | 31 | //TODO wrapped data immutable? 32 | 33 | override def equals(o: Any): Boolean = 34 | o.isInstanceOf[ByteArrayWrapper] && 35 | util.Arrays.equals(data, o.asInstanceOf[ByteArrayWrapper].data) 36 | 37 | override def hashCode: Int = Utils.byteArrayHashCode(data) 38 | 39 | override def compareTo(o: ByteArrayWrapper): Int = Utils.BYTE_ARRAY_COMPARATOR.compare(this.data, o.data) 40 | 41 | override def compare(that: ByteArrayWrapper): Int = compareTo(that) 42 | 43 | override def toString: String = { 44 | val v = if (size == 8) { 45 | //if size is 8, display as a number 46 | Utils.getLong(data, 0).toString + "L" 47 | } else { 48 | javax.xml.bind.DatatypeConverter.printHexBinary(data) 49 | } 50 | getClass.getSimpleName + "[" + v + "]" 51 | } 52 | } -------------------------------------------------------------------------------- /src/test/scala/io/iohk/iodb/bench/BlockProcessing.scala: -------------------------------------------------------------------------------- 1 | package io.iohk.iodb.bench 2 | 3 | import java.io.File 4 | import java.util.concurrent.atomic.AtomicLong 5 | 6 | import io.iohk.iodb.{ByteArrayWrapper, ShardedStore, Store, TestUtils} 7 | 8 | object BlockProcessing extends Benchmark with App { 9 | 10 | val InitialSize = 5000000 11 | 12 | val InputsPerBlock = 5500 13 | val OutputsPerBlock = 6000 14 | val Blocks = 2000 15 | 16 | val keysCache = Seq[ByteArrayWrapper]() 17 | 18 | var version = new AtomicLong(1) 19 | 20 | def bench(store: Store, dir: File): Unit = { 21 | (1 to 1000).foreach { v => 22 | val toInsert = (1 to InitialSize / 1000).map(_ => randomKV()) 23 | if (v % Blocks == 0) keysCache ++ toInsert.map(_._1) 24 | store.update(version.incrementAndGet(), Seq.empty, toInsert) 25 | } 26 | 27 | println("Initial data is loaded into the store") 28 | 29 | val (_, ts) = (1L to Blocks).foldLeft((Seq[ByteArrayWrapper](), Seq[Long]())) { case ((cache, times), v) => 30 | val (time, newCache) = TestUtils.runningTime(processBlock(version.incrementAndGet, store, InputsPerBlock, OutputsPerBlock, cache).get) 31 | println(s"Block processing time for block# $v: " + time) 32 | (newCache, times ++ Seq(time)) 33 | } 34 | 35 | val totalTime = ts.sum 36 | 37 | println(s"Store: $store") 38 | println(s"Total processing time: $totalTime") 39 | println(s"Avg block processing time: ${totalTime / Blocks.toFloat}") 40 | store.close() 41 | TestUtils.deleteRecur(dir) 42 | } 43 | 44 | 45 | var dir = TestUtils.tempDir() 46 | bench(new ShardedStore(dir, keySize = KeySize), dir) 47 | 48 | println("===========================") 49 | 50 | dir = TestUtils.tempDir() 51 | bench(new RocksStore(dir), dir) 52 | } -------------------------------------------------------------------------------- /src/test/scala/io/iohk/iodb/bench/CompactionBench.scala: -------------------------------------------------------------------------------- 1 | package io.iohk.iodb.bench 2 | 3 | import java.util.concurrent.atomic.AtomicLong 4 | 5 | import io.iohk.iodb.TestUtils._ 6 | import io.iohk.iodb._ 7 | 8 | /** 9 | * Benchmark to test how background compaction is blocking updates. 10 | * We should get decent number of updates, if the compaction does not block writer threads. 11 | */ 12 | object CompactionBench { 13 | 14 | def main(args: Array[String]): Unit = { 15 | val time: Long = 60 16 | val endTime = System.currentTimeMillis() + time * 1000 17 | 18 | //start background thread with updates, while compaction runs on foreground 19 | val dir = TestUtils.tempDir() 20 | val store = new LogStore(dir = dir, keySize = 8, keepVersions = 0) 21 | val updateCounter = new AtomicLong(1) 22 | var compactionCounter = 0L 23 | 24 | val updateThread = new Thread(runnable { 25 | while (System.currentTimeMillis() < endTime) { 26 | val value = updateCounter.incrementAndGet() 27 | val toUpdate = List((fromLong(1L), fromLong(value))) 28 | store.update(versionID = fromLong(1L), toUpdate = toUpdate, toRemove = Nil) 29 | } 30 | }) 31 | updateThread.setDaemon(false) 32 | updateThread.start() 33 | 34 | 35 | while (System.currentTimeMillis() < endTime) { 36 | store.taskCompact() 37 | store.clean(0) 38 | compactionCounter += 1 39 | } 40 | 41 | //wait until Update Thread finishes 42 | while (updateThread.isAlive) { 43 | Thread.sleep(1) 44 | } 45 | 46 | 47 | println("Runtime: " + (time / 1000) + " seconds") 48 | println("Update count: " + updateCounter.get()) 49 | println("Compaction count: " + compactionCounter) 50 | 51 | store.close() 52 | deleteRecur(dir) 53 | } 54 | 55 | } 56 | -------------------------------------------------------------------------------- /src/test/scala/io/iohk/iodb/bench/Benchmark.scala: -------------------------------------------------------------------------------- 1 | package io.iohk.iodb.bench 2 | 3 | import io.iohk.iodb.{ByteArrayWrapper, Store, TestUtils} 4 | 5 | import scala.util.{Random, Try} 6 | 7 | 8 | trait Benchmark { 9 | type Key = Array[Byte] 10 | 11 | val KeySize = 32 12 | val ValueSize = 256 13 | 14 | def randomBytes(howMany: Int): Array[Byte] = { 15 | val res = new Array[Byte](howMany) 16 | Random.nextBytes(res) 17 | res 18 | } 19 | 20 | def randomKV(): (ByteArrayWrapper, ByteArrayWrapper) = { 21 | val key = new Array[Byte](KeySize) 22 | Random.nextBytes(key) 23 | val value = new Array[Byte](ValueSize) 24 | Random.nextBytes(value) 25 | ByteArrayWrapper(key) -> ByteArrayWrapper(value) 26 | } 27 | 28 | /** 29 | * Imitation of one block processing. We take random keys previously inserted 30 | * (for now, just a range of them), read them, remove, and append new objects. 31 | * 32 | * @return updated keys cache 33 | */ 34 | def processBlock(version: Long, 35 | store: Store, 36 | inputs: Int, 37 | outputs: Int, 38 | keysCache: Seq[ByteArrayWrapper]): Try[Seq[ByteArrayWrapper]] = { 39 | val cacheSize = keysCache.size 40 | 41 | val (toRemove, cacheRem) = if (cacheSize > inputs) { 42 | val sliceStart = Random.nextInt(cacheSize - inputs) 43 | keysCache.slice(sliceStart, sliceStart + inputs) -> 44 | (keysCache.take(sliceStart) ++ keysCache.drop(sliceStart + inputs)) 45 | } else (keysCache, Seq()) 46 | 47 | val toAppend = (1 to outputs).map(_ => randomKV()) 48 | 49 | toRemove.foreach(store.get) 50 | 51 | Try(store.update(TestUtils.fromLong(version), toRemove, toAppend)).map(_ => 52 | cacheRem ++ toAppend.map(_._1) 53 | ) 54 | } 55 | } -------------------------------------------------------------------------------- /src/test/scala/io/iohk/iodb/ReferenceStore.scala: -------------------------------------------------------------------------------- 1 | package io.iohk.iodb 2 | 3 | import io.iohk.iodb.Store.{K, V, VersionID} 4 | 5 | import scala.collection.mutable 6 | 7 | /** 8 | * Trivial on-heap implementation of Store, used for unit tests. 9 | */ 10 | class ReferenceStore extends Store { 11 | 12 | protected var current: Map[K, V] = Map() 13 | protected var history = mutable.LinkedHashMap[VersionID, Map[K, V]]() 14 | protected var curVersiondId: Option[VersionID] = None 15 | 16 | override def get(key: K): Option[V] = current.get(key) 17 | 18 | override def clean(count: Int): Unit = { 19 | //remove all but last N versions from history 20 | history = history.takeRight(count) 21 | } 22 | 23 | override def lastVersionID: Option[VersionID] = curVersiondId 24 | 25 | override def update(versionID: VersionID, toRemove: Iterable[K], toUpdate: Iterable[(K, V)]): Unit = { 26 | assert(history.get(versionID) == None) 27 | current = current ++ (toUpdate) -- (toRemove) 28 | history.put(versionID, current) 29 | curVersiondId = Some(versionID) 30 | } 31 | 32 | override def rollback(versionID: VersionID): Unit = { 33 | current = history(versionID) 34 | curVersiondId = Some(versionID) 35 | //drop non used versions 36 | var found = false 37 | for (ver <- history.keys) { 38 | if (found) { 39 | //remove all keys after current version 40 | history.remove(ver) 41 | } 42 | found = found || ver == versionID 43 | } 44 | } 45 | 46 | override def close(): Unit = {} 47 | 48 | override def getAll(consumer: (K, V) => Unit) = { 49 | for ((k, v) <- current) { 50 | consumer(k, v) 51 | } 52 | } 53 | 54 | override def rollbackVersions(): Iterable[VersionID] = history.keys 55 | 56 | override def verify(): Unit = ??? 57 | } 58 | -------------------------------------------------------------------------------- /changelog.md: -------------------------------------------------------------------------------- 1 | 0.4 release 2 | ---------------- 3 | 4 | - Rewrite LogStore and ShardedStore (former LSMStore) 5 | 6 | - Reduce internal state to fix concurrency issues 7 | 8 | - Change design and compaction model. Journal is now source of durability, rather than short term cache 9 | 10 | - Added background threads and background compaction 11 | 12 | Issues: 13 | 14 | - See [list on github](https://github.com/input-output-hk/iodb/issues?q=is%3Aissue+is%3Aopen+label%3A0.5) targeted for `0.5` release 15 | 16 | 17 | 18 | 0.3.1 19 | ---------------- 20 | 21 | - Improve performance of a task distributing content between shards. 22 | 23 | - Add `QuickStore`. It stores data in-memory, but also has durability and rollbacks. 24 | 25 | - Improve sorting performance, updates and background tasks are 5x faster. 26 | 27 | - Reduce memory footprint, background task no longer load entire shard into memory. 28 | 29 | - Reduce number of opened file handles. 30 | 31 | 0.3 release 32 | ------------- 33 | 34 | - Use 64 bit non-cryptographic checksum to protect from data corruption. [Issue #7](https://github.com/input-output-hk/iodb/issues/7). 35 | 36 | - Rollback would not handle deleted keys correctly. [Issue #17](https://github.com/input-output-hk/iodb/issues/17). 37 | 38 | - Reintroduce memory-mapped and Unsafe file access methods. 39 | 40 | - Add `Store.getAll()` to access all key-value pairs at given version. 41 | 42 | - Fix `get()` after `close()` causes JVM crash. [Issue #16](https://github.com/input-output-hk/iodb/issues/16). 43 | 44 | - Add smoke tests to randomly test insert, delete and rollback. 45 | 46 | - Fix NPE in background thread, if folder gets deleted after unit tests finishes. [Issue #15](https://github.com/input-output-hk/iodb/issues/15). 47 | 48 | 49 | 0.2 release 50 | --------------- 51 | 52 | Rework shards and journal design. Better scalability. -------------------------------------------------------------------------------- /src/test/scala/io/iohk/iodb/bench/InitialProcessing.scala: -------------------------------------------------------------------------------- 1 | package io.iohk.iodb.bench 2 | 3 | import java.io.File 4 | 5 | import ch.qos.logback.classic.LoggerContext 6 | import io.iohk.iodb.{ByteArrayWrapper, ShardedStore, Store, TestUtils} 7 | import org.slf4j.LoggerFactory 8 | 9 | /** 10 | * Performance benchmark utility simulating initial blockchain processing 11 | */ 12 | object InitialProcessing extends Benchmark { 13 | val Milestones = Seq(1000, 5000, 10000, 50000, 100000, 250000, 500000, 750000, 1000000) 14 | 15 | val Inputs = 5500 16 | //average number of inputs per block 17 | val Outputs = 6000 //average number of outputs per block 18 | 19 | def bench(store: Store, dir: File): Unit = { 20 | println(s"Store: $store") 21 | 22 | Milestones.foldLeft((0, 0L, Seq[ByteArrayWrapper]())) { 23 | case ((prevMilestone, prevTime, prevCache), milestone) => 24 | val (time, newCache) = TestUtils.runningTime { 25 | (prevMilestone + 1 to milestone).foldLeft(prevCache) { case (cache, version) => 26 | processBlock(version, store, Inputs, Outputs, cache).get.take(Inputs * 100) 27 | } 28 | } 29 | val newTime = prevTime + time 30 | println(s"Time to get to $milestone: $time") 31 | (milestone, newTime, newCache) 32 | } 33 | 34 | store.close() 35 | TestUtils.deleteRecur(dir) 36 | } 37 | 38 | def main(args: Array[String]): Unit = { 39 | //switching off logging 40 | val context = LoggerFactory.getILoggerFactory.asInstanceOf[LoggerContext] 41 | context.stop() 42 | 43 | var dir = TestUtils.tempDir() 44 | bench(new ShardedStore(dir, keySize = KeySize), dir) 45 | 46 | System.gc() 47 | Thread.sleep(15000) 48 | println("======================================") 49 | 50 | dir = TestUtils.tempDir() 51 | bench(new RocksStore(dir), dir) 52 | } 53 | } -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # IODB - database engine for blockchain 2 | 3 | [![Build Status](https://travis-ci.org/input-output-hk/iodb.svg?branch=master)](https://travis-ci.org/input-output-hk/iodb) 4 | 5 | IODB is embedded storage engine designed for blockchain applications. 6 | It is inspired by [RocksDB](http://rocksdb.org). 7 | It provides ordered key-value store, it is similar to `SortedMap`. 8 | Its main advantage are snapshots with branching and fast rollbacks. 9 | 10 | Main features include: 11 | - Ordered key-value store 12 | - Written in Scala, functional interface 13 | - Multi-threaded background compaction 14 | - Very fast durable commits 15 | - Atomic updates with MVCC isolation and crash protection 16 | - Snapshots with branching and rollbacks 17 | - Log structured storage, old data are never overwritten for improved crash protection 18 | 19 | Getting started 20 | --------------------- 21 | 22 | IODB builds are available in [Maven repository](https://mvnrepository.com/artifact/org.scorexfoundation/iodb_2.12). Maven dependency snippet is bellow, replace `$VERSION` with 23 | [![Maven Central](https://maven-badges.herokuapp.com/maven-central/org.scorexfoundation/iodb_2.12/badge.svg)](https://mvnrepository.com/artifact/org.scorexfoundation/iodb_2.12) 24 | : 25 | 26 | ```xml 27 | 28 | org.scorexfoundation 29 | iodb_2.12 30 | $VERSION 31 | 32 | ``` 33 | 34 | Code examples are in the [src/test/scala/examples](src/test/scala/examples) folder. 35 | 36 | Documentation is in the [doc](doc) folder. 37 | 38 | Compile 39 | --------- 40 | 41 | IODB works with Intellij IDEA with Scala plugin. 42 | 43 | - Checkout IODB: 44 | ``` 45 | git clone https://github.com/input-output-hk/iodb.git 46 | ``` 47 | - Install [SBT](http://www.scala-sbt.org/release/docs/Setup.html) 48 | 49 | 50 | - Compile IODB and install JAR files into local repository: 51 | ``` 52 | sbt publish 53 | ``` 54 | 55 | 56 | -------------------------------------------------------------------------------- /src/test/scala/io/iohk/iodb/bench/RocksStore.scala: -------------------------------------------------------------------------------- 1 | package io.iohk.iodb.bench 2 | 3 | import java.io.File 4 | import java.util.concurrent.locks.ReentrantReadWriteLock 5 | 6 | import io.iohk.iodb.{ByteArrayWrapper, Store} 7 | import org.rocksdb._ 8 | 9 | /** 10 | * Uses RocksDB as backend 11 | */ 12 | class RocksStore(val dir: File) extends Store { 13 | 14 | import Store._ 15 | { 16 | RocksDB.loadLibrary() 17 | } 18 | 19 | protected val db: RocksDB = { 20 | val options: Options = new Options().setCreateIfMissing(true) 21 | RocksDB.open(options, dir.getPath) 22 | } 23 | 24 | //TODO snapshots? 25 | protected var snapshots = new java.util.TreeMap[Long, Snapshot](java.util.Collections.reverseOrder[Long]()) 26 | 27 | //TODO thread safe 28 | private val lock = new ReentrantReadWriteLock() 29 | 30 | //TODO: versioning 31 | private var version: VersionID = null 32 | 33 | 34 | /** returns value associated with key */ 35 | override def get(key: K): Option[V] = { 36 | val ret = db.get(key.data) 37 | if (ret == null) None else Some(ByteArrayWrapper(ret)) 38 | } 39 | 40 | /** returns versionID from last update, used when Scorex starts */ 41 | override def lastVersionID: Option[VersionID] = Some(version) 42 | 43 | /** update records and move to new version */ 44 | override def update(versionID: VersionID, toRemove: Iterable[K], toUpdate: Iterable[(K, V)]): Unit = { 45 | for (key <- toRemove) { 46 | db.remove(key.data) 47 | } 48 | 49 | for ((key, value) <- toUpdate) { 50 | db.put(key.data, value.data) 51 | } 52 | db.flush(new FlushOptions().setWaitForFlush(true)) 53 | version = versionID 54 | } 55 | 56 | 57 | override def close(): Unit = { 58 | db.close() 59 | } 60 | 61 | 62 | 63 | override def clean(count: Int): Unit = ??? 64 | 65 | override def rollback(versionID: VersionID): Unit = ??? 66 | 67 | override def getAll(consumer: (K, V) => Unit) = ??? 68 | 69 | override def rollbackVersions(): Iterable[VersionID] = ??? 70 | 71 | override def verify(): Unit = ??? 72 | } 73 | -------------------------------------------------------------------------------- /src/test/scala/io/iohk/iodb/bench/LevelDBStore.scala: -------------------------------------------------------------------------------- 1 | package io.iohk.iodb.bench 2 | 3 | import java.io._ 4 | 5 | import io.iohk.iodb.Store.{K, V, VersionID} 6 | import io.iohk.iodb.{ByteArrayWrapper, Store} 7 | import org.iq80.leveldb._ 8 | import org.iq80.leveldb.impl._ 9 | 10 | /** 11 | * Uses LevelDB backend 12 | */ 13 | class LevelDBStore(val dir: File, val storeName: String = "leveldb") extends Store { 14 | 15 | private val db: DB = { 16 | val op = new Options() 17 | op.createIfMissing(true) 18 | Iq80DBFactory.factory.open(new File(dir, storeName), op) 19 | } 20 | 21 | private var lastVersion: Option[VersionID] = None 22 | 23 | override def get(key: K): Option[V] = { 24 | val b = db.get(key.data) 25 | return if (b == null) None else Some(ByteArrayWrapper(b)) 26 | } 27 | 28 | override def getAll(consumer: (K, V) => Unit): Unit = { 29 | val iterator = db.iterator(); 30 | iterator.seekToFirst() 31 | try { 32 | while (iterator.hasNext) { 33 | val n = iterator.next() 34 | consumer(ByteArrayWrapper(n.getKey), ByteArrayWrapper(n.getValue)) 35 | } 36 | } finally { 37 | iterator.close(); 38 | } 39 | 40 | } 41 | 42 | override def clean(count: Int): Unit = ??? 43 | 44 | override def lastVersionID: Option[VersionID] = lastVersion 45 | 46 | override def update(versionID: VersionID, 47 | toRemove: Iterable[K], 48 | toUpdate: Iterable[(K, V)]): Unit = { 49 | 50 | val batch = db.createWriteBatch(); 51 | try { 52 | toRemove.foreach(b => batch.delete(b.data)) 53 | for ((k, v) <- toUpdate) { 54 | batch.put(k.data, v.data) 55 | } 56 | db.write(batch); 57 | } finally { 58 | // Make sure you close the batch to avoid resource leaks. 59 | batch.close(); 60 | } 61 | 62 | } 63 | 64 | override def rollback(versionID: VersionID): Unit = ??? 65 | 66 | override def close(): Unit = { 67 | db.close() 68 | } 69 | 70 | 71 | override def rollbackVersions(): Iterable[VersionID] = ??? 72 | 73 | 74 | override def verify(): Unit = ??? 75 | } 76 | -------------------------------------------------------------------------------- /src/test/scala/io/iohk/iodb/ShardedStoreTest.scala: -------------------------------------------------------------------------------- 1 | package io.iohk.iodb 2 | 3 | import io.iohk.iodb.Store._ 4 | import org.junit.Test 5 | import org.scalatest.Matchers._ 6 | 7 | import scala.collection.JavaConverters._ 8 | 9 | class ShardedStoreTest extends StoreTest { 10 | 11 | override def open(keySize: Int) = new ShardedStore(dir = dir, keySize = keySize) 12 | 13 | @Test def distribute_flushes_log(): Unit = { 14 | val store = open(keySize = 8) 15 | 16 | for (i <- (0L until 10L)) { 17 | val b = TestUtils.fromLong(i) 18 | store.update(versionID = b, toRemove = Nil, toUpdate = List((b, b))) 19 | } 20 | assert(store.journal.loadUpdateOffsets(stopAtMerge = true, stopAtDistribute = false).size > 1) 21 | store.taskDistribute() 22 | assert(store.journal.loadUpdateOffsets(stopAtMerge = true, stopAtDistribute = true).size == 1) 23 | store.close() 24 | } 25 | 26 | @Test def get_after_dist(): Unit = { 27 | val store = open(keySize = 8) 28 | 29 | def check(shift: Int, inc: Int): Unit = { 30 | for (i <- (0L until 10L)) { 31 | val b = TestUtils.fromLong(i + shift) 32 | assert(Some(TestUtils.fromLong(i + inc)) == store.get(b)) 33 | } 34 | 35 | } 36 | 37 | def update(shift: Int, inc: Int): Unit = { 38 | for (i <- (0L until 10L)) { 39 | val b = TestUtils.fromLong(i + shift) 40 | val value = TestUtils.fromLong(i + inc) 41 | store.update(versionID = b, toRemove = Nil, toUpdate = List((b, value))) 42 | } 43 | } 44 | 45 | update(0, 0) 46 | check(0, 0) 47 | store.taskDistribute() 48 | check(0, 0) 49 | update(100, 1) 50 | check(0, 0) 51 | check(100, 1) 52 | update(0, 10) 53 | check(0, 10) 54 | store.close() 55 | } 56 | 57 | 58 | @Test def shard_count(): Unit = { 59 | val s = new ShardedStore(dir = dir, shardCount = 10, keySize = 8, executor = null) 60 | s.shards.size() shouldBe 10 61 | s.shards.asScala.values.toSet.size shouldBe 10 62 | 63 | //insert random data 64 | val data: Seq[(K, V)] = (1 until 10000).map(i => (TestUtils.randomA(8), TestUtils.randomA(3))).toBuffer 65 | val sorted = data.sortBy(_._1) 66 | s.update(TestUtils.fromLong(1), toUpdate = data, toRemove = Nil) 67 | s.getAll().toBuffer.size shouldBe sorted.size 68 | s.getAll().toBuffer shouldBe sorted 69 | 70 | s.taskDistribute() 71 | 72 | s.getAll().toBuffer.size shouldBe sorted.size 73 | s.getAll().toBuffer shouldBe sorted 74 | 75 | //test all shards are not empty 76 | for (shard <- s.shards.values().asScala) { 77 | assert(shard.getAll().toBuffer.size > 0) 78 | } 79 | s.close() 80 | } 81 | } 82 | -------------------------------------------------------------------------------- /src/test/scala/io/iohk/iodb/ShardedIteratorTest.scala: -------------------------------------------------------------------------------- 1 | package io.iohk.iodb 2 | 3 | import io.iohk.iodb.ShardedIterator._ 4 | import io.iohk.iodb.Store._ 5 | import io.iohk.iodb.TestUtils._ 6 | import org.junit.Test 7 | import org.scalatest.Matchers._ 8 | 9 | class ShardedIteratorTest { 10 | 11 | val k1 = fromLong(1) 12 | val k2 = fromLong(2) 13 | val k3 = fromLong(3) 14 | val k4 = fromLong(4) 15 | 16 | var iter: Iterator[(K, V)] = null 17 | 18 | def assertEmpty(): Unit = { 19 | iter.hasNext shouldBe false 20 | assertThrows[NoSuchElementException] { 21 | iter.next() 22 | } 23 | } 24 | 25 | @Test def empty(): Unit = { 26 | iter = distIter(journalIter = None.iterator, shardIters = List()) 27 | assertEmpty() 28 | } 29 | 30 | 31 | @Test def empty_journal_tombstone(): Unit = { 32 | iter = distIter( 33 | journalIter = List((k1, tombstone)).iterator, 34 | shardIters = List()) 35 | assertEmpty() 36 | } 37 | 38 | 39 | @Test def one(): Unit = { 40 | val list = List((k1, k1)) 41 | iter = distIter( 42 | journalIter = list.iterator, 43 | shardIters = List()) 44 | iter.toList shouldBe list 45 | assertEmpty() 46 | } 47 | 48 | 49 | @Test def two(): Unit = { 50 | val list = List((k1, k1), (k2, k2)) 51 | iter = distIter( 52 | journalIter = list.iterator, 53 | shardIters = List()) 54 | iter.toList shouldBe list 55 | assertEmpty() 56 | } 57 | 58 | 59 | @Test def oneShard(): Unit = { 60 | val list = List((k1, k1)) 61 | iter = distIter( 62 | journalIter = None.iterator, 63 | shardIters = List(list.iterator)) 64 | iter.toList shouldBe list 65 | assertEmpty() 66 | } 67 | 68 | 69 | @Test def twoShard(): Unit = { 70 | val list = List((k1, k1), (k2, k2)) 71 | iter = distIter( 72 | journalIter = None.iterator, 73 | shardIters = List(list.iterator)) 74 | iter.toList shouldBe list 75 | assertEmpty() 76 | } 77 | 78 | 79 | @Test def twoShards(): Unit = { 80 | val list1 = List((k1, k1), (k2, k2)) 81 | val list2 = List((k3, k3), (k4, k4)) 82 | iter = distIter( 83 | journalIter = None.iterator, 84 | shardIters = List(list1.iterator, list2.iterator)) 85 | iter.toList shouldBe list1 ::: list2 86 | assertEmpty() 87 | } 88 | 89 | 90 | @Test def twoShardsPlus(): Unit = { 91 | val list1 = List((k1, k1), (k2, k2)) 92 | val list2 = List((k3, k3), (k4, k4)) 93 | iter = distIter( 94 | journalIter = List((k3, tombstone)).iterator, 95 | shardIters = List(list1.iterator, list2.iterator)) 96 | iter.toList shouldBe List((k1, k1), (k2, k2), (k4, k4)) 97 | assertEmpty() 98 | } 99 | } 100 | -------------------------------------------------------------------------------- /src/test/scala/io/iohk/iodb/bench/SimpleKVBench.scala: -------------------------------------------------------------------------------- 1 | package io.iohk.iodb.bench 2 | 3 | import java.io.File 4 | import java.util.Random 5 | 6 | import io.iohk.iodb._ 7 | import org.junit.Test 8 | 9 | class SimpleKVBench{ 10 | @Test def run(): Unit ={ 11 | SimpleKVBench.main(Array("1000", "100")) 12 | } 13 | 14 | } 15 | 16 | case class BenchResult(storage: String, insertTime: Long, getTime: Long, storeSizeMb: Long) 17 | 18 | /** 19 | * Benchmark for IODB 20 | */ 21 | object SimpleKVBench extends Benchmark{ 22 | 23 | var defaultUpdates = 1000 24 | var defaultKeyCount = 100 25 | 26 | def main(args: Array[String]) { 27 | val updates = if (args.length > 0) args(0).toInt else defaultUpdates 28 | val keyCount = if (args.length > 1) args(1).toInt else defaultKeyCount 29 | var dir = TestUtils.tempDir() 30 | dir.mkdirs() 31 | val lb = bench( 32 | store = new ShardedStore(dir, keySize = KeySize, shardCount = 10), 33 | dir = dir, 34 | updates = updates, 35 | keyCount = keyCount) 36 | TestUtils.deleteRecur(dir) 37 | printlnResult(lb) 38 | 39 | dir = TestUtils.tempDir() 40 | dir.mkdirs() 41 | val rb = bench( 42 | store = new RocksStore(dir), 43 | dir = dir, 44 | updates = updates, 45 | keyCount = keyCount) 46 | printlnResult(rb) 47 | TestUtils.deleteRecur(dir) 48 | 49 | // dir = TestUtils.tempDir() 50 | // val lvb = bench( 51 | // store = new LevelDBStore(dir), 52 | // dir = dir, 53 | // updates = updates, 54 | // keyCount = keyCount) 55 | // printlnResult(lvb) 56 | // TestUtils.deleteRecur(dir) 57 | 58 | printf("Commit count: %,d \n", updates) 59 | printf("Keys per update: %,d \n", keyCount) 60 | 61 | } 62 | 63 | def bench(store: Store, dir: File, updates: Int, keyCount: Int): BenchResult = { 64 | val r = new Random(1) 65 | var version = 0 66 | //insert random values 67 | val insertTime = TestUtils.runningTimeUnit { 68 | for (i <- 0 until updates) { 69 | val toInsert = (0 until keyCount).map { a => 70 | val k = randomKey(r) 71 | (k, k) 72 | } 73 | version += 1 74 | store.update(version, List.empty, toInsert) 75 | } 76 | } 77 | 78 | Thread.sleep(10000) 79 | 80 | val getTime = TestUtils.runningTimeUnit { 81 | val r = new Random(1) 82 | for (i <- 0 until updates) { 83 | val toGet = (0 until keyCount).map { j => 84 | randomKey(r) 85 | } 86 | 87 | version += 1 88 | 89 | toGet.foreach { k => 90 | assert(null != store.get(k)) 91 | } 92 | } 93 | } 94 | 95 | val br = BenchResult(store.getClass.toString, insertTime, getTime, TestUtils.dirSize(dir) / (1024 * 1024)) 96 | 97 | store.close() 98 | TestUtils.deleteRecur(dir) 99 | br 100 | } 101 | 102 | def randomKey(r: Random): ByteArrayWrapper = { 103 | val key = new Array[Byte](KeySize) 104 | r.nextBytes(key) 105 | ByteArrayWrapper(key) 106 | } 107 | 108 | def printlnResult(res: BenchResult): Unit = { 109 | println("Store: " + res.storage) 110 | printf("Insert time: %,d \n", res.insertTime / 1000) 111 | printf("Get time: %,d \n", res.getTime / 1000) 112 | printf("Store size: %,d MB \n", res.storeSizeMb) 113 | } 114 | } 115 | -------------------------------------------------------------------------------- /src/test/scala/io/iohk/iodb/TestUtils.scala: -------------------------------------------------------------------------------- 1 | package io.iohk.iodb 2 | 3 | import java.io.File 4 | import java.nio.ByteBuffer 5 | import java.util.concurrent.{ExecutorService, TimeUnit} 6 | import java.util.logging.Level 7 | 8 | import scala.util.Random 9 | 10 | /** 11 | * test utilities 12 | */ 13 | object TestUtils { 14 | 15 | 16 | def dirSize(dir: File): Long = dir.listFiles().map(_.length()).sum 17 | 18 | def tempDir(): File = { 19 | val dir = new File(System.getProperty("java.io.tmpdir") + File.separator + "iodb" + Math.random()) 20 | dir.mkdirs() 21 | dir 22 | } 23 | 24 | 25 | def tempFile(): File = { 26 | val ret = new File(System.getProperty("java.io.tmpdir") + File.separator + "iodb" + Math.random()) 27 | ret.deleteOnExit() 28 | return ret 29 | } 30 | 31 | def deleteRecur(dir: File): Unit = { 32 | if (dir == null) return 33 | val files = dir.listFiles() 34 | if (files != null) 35 | files.foreach(deleteRecur) 36 | dir.delete() 37 | } 38 | 39 | /** generates random byte[] of given size */ 40 | def randomA(size: Int = 32, random: Random = new Random()): ByteArrayWrapper = { 41 | val b = new Array[Byte](size) 42 | random.nextBytes(b) 43 | ByteArrayWrapper(b) 44 | } 45 | 46 | /** return value of `-DlongTest=1` property, 0 if value is not defined, or 1 if its defined but not a number */ 47 | def longTest(): Long = { 48 | var ret = 0L 49 | try { 50 | val prop = System.getProperty("longTest") 51 | if (prop != null) { 52 | ret = 1 53 | if (prop.matches("[0-9]+")) { 54 | ret = prop.toLong 55 | } 56 | } 57 | } catch { 58 | case _: Exception => 59 | } 60 | 61 | ret 62 | } 63 | 64 | /** Duration of long running tests. 65 | * Its value is controlled by `-DlongTest=1` property, where the value is test runtime in minutes. 66 | * By default tests run 5 seconds, but it can be increased by setting this property. 67 | */ 68 | def endTimestamp(): Long = 69 | 5 * 1000 + longTest() * 60 * 1000 + System.currentTimeMillis() 70 | 71 | /** measures time it takes to execute function */ 72 | def runningTimeUnit(computation: => Unit): Long = { 73 | val s = System.currentTimeMillis() 74 | computation 75 | System.currentTimeMillis() - s 76 | } 77 | 78 | /** measures time it takes to execute function */ 79 | def runningTime[A](computation: => A): (Long, A) = { 80 | val s = System.currentTimeMillis() 81 | val res = computation 82 | (System.currentTimeMillis() - s, res) 83 | } 84 | 85 | def fromLong(id: Long): ByteArrayWrapper = { 86 | val b = ByteBuffer.allocate(8) 87 | b.putLong(0, id) 88 | ByteArrayWrapper(b.array()) 89 | } 90 | 91 | 92 | def runnable(f: => Unit): Runnable = 93 | return () => { 94 | try { 95 | f 96 | } catch { 97 | case e: Throwable => { 98 | Utils.LOG.log(Level.SEVERE, "Background task failed", e) 99 | } 100 | } 101 | } 102 | 103 | def waitForFinish(exec: ExecutorService): Unit = { 104 | exec.shutdown() 105 | exec.awaitTermination(400, TimeUnit.DAYS) 106 | } 107 | 108 | 109 | def withTempDir(ff: (File) => Unit) { 110 | val iFile = TestUtils.tempDir() 111 | try { 112 | ff(iFile) 113 | } finally { 114 | TestUtils.deleteRecur(iFile) 115 | } 116 | } 117 | } 118 | -------------------------------------------------------------------------------- /doc/store_format.md: -------------------------------------------------------------------------------- 1 | Store format spec 2 | ===================== 3 | 4 | 5 | LSM Store has two types of log files: 6 | 7 | * Journal file stores recent modifications 8 | * Shard files store older data 9 | 10 | They both share the same format and are composed of log entries 11 | 12 | ## Types of Log Entries 13 | 14 | 15 | ### Header and tail 16 | Each Log Entry has following header: 17 | 18 | * **entry size** - int - number of bytes consumed by this entry, includes checksum 19 | * **Update Type** - byte - identifies type of Log Entry that follows 20 | * 1 - Update Entry 21 | * 2 - Distribute Entry 22 | * 3 - Offset Alias 23 | * 4 - Merge Entry 24 | 25 | This is followed by other data from Log Entry 26 | 27 | Each Log Entry ends with checksum: 28 | 29 | * **checksum** - long - checksum for update entry 30 | - it uses proprietary [64bit XXHash](https://github.com/jpountz/lz4-java/tree/master/src/java/net/jpountz/xxhash) 31 | - is calculated from entire entry including header, but excluding checksum 32 | 33 | 34 | ### Update Entry 35 | Is written to Journal or Shard on each Update 36 | 37 | * **prev update file number** - long - file number of file where previous Update is 38 | * **prev update file offset** - long - offset of previous Update in file 39 | * **key count** - int - number of keys in this Update 40 | * **key size** - int - number of bytes in each key in this Update 41 | * **versionID size** - int - number of bytes used in versionID 42 | * **is merged** - boolean - true if this is merged update 43 | 44 | * section with keys, its size is **number of keys** * **key size** 45 | 46 | * section with value size & offset pair, 47 | * section size is **key count** * (4+4) 48 | * value size is int, -1 size is tombstone (deleted key) 49 | * value offset is int, 50 | * single update can not be larger than 2GB 51 | * value offset is counted from start of this update, not from start of the file 52 | 53 | 54 | - **versionID** - byte[] - version of current update 55 | 56 | - section with values 57 | 58 | ### Distribute Entry 59 | 60 | Distribute Entry is written to Journal after Distribute Task finishes. 61 | Distribute Entry finishes log traversal (while finding keys) similar way Merge Entry does. 62 | When Distribute Entry is found in Journal, its traversal should finish and traversal continues in Shards 63 | 64 | * **prev update file number** - long - file number of file where previous Update is 65 | * **prev update file offset** - long - offset of previous Update in file 66 | 67 | 68 | * **shard count** - int - number of shards in this Distribute Entry 69 | 70 | * section with Shard Prefix and file offsets 71 | * **shard prefix** - int - first 4 bytes of shard key 72 | * **shard pos file number** - long - filenumber in Shard where traversal continues for given version 73 | * **shard pos file offset** - long - position in Shard where traversal continues for given version 74 | 75 | 76 | ### Offset Alias 77 | 78 | Stored in Journal. Is used to reconstruct Offset Alias table when Journal is reopened. 79 | 80 | * **old update file number** - long - file number of file where previous Update is 81 | * **old prev update file offset** - long - offset of previous Update in file 82 | * **new prev update file number** - long - new file number, replaces old file number 83 | * **new prev update file offset** - long - new file offset, replaces old offset 84 | 85 | 86 | ### Merge Entry 87 | 88 | TODO is there difference between Update Entry and this? -------------------------------------------------------------------------------- /src/main/scala/io/iohk/iodb/ShardedIterator.scala: -------------------------------------------------------------------------------- 1 | package io.iohk.iodb 2 | 3 | import io.iohk.iodb.Store._ 4 | 5 | 6 | object ShardedIterator { 7 | 8 | /** iterator that merges key-value iterators from Journal and Shards */ 9 | def distIter( 10 | journalIter: Iterator[(K, V)], 11 | shardIters: Seq[Iterator[(K, V)]] 12 | ): Iterator[(K, V)] = { 13 | 14 | if (journalIter.isEmpty && shardIters.forall(_.isEmpty)) 15 | return None.iterator 16 | 17 | 18 | if (shardIters.isEmpty) 19 | return journalIter.filterNot(_._2 eq tombstone) 20 | 21 | object o extends Iterator[(K, V)] { 22 | 23 | val shards = shardIters.iterator 24 | var shard = shards.next() 25 | 26 | var j = journalNext() 27 | var s = shardNext() 28 | 29 | var next2: Option[(K, V)] = null 30 | 31 | def journalNext(): (K, V) = { 32 | if (journalIter.hasNext) journalIter.next() 33 | else null 34 | } 35 | 36 | def shardNext(): (K, V) = { 37 | while (!shard.hasNext) { 38 | //advance to shard with data 39 | if (shards.isEmpty) { 40 | return null 41 | } 42 | shard = shards.next() 43 | } 44 | 45 | return shard.next() 46 | } 47 | 48 | 49 | def advance(): Option[(K, V)] = { 50 | //FIXME this method is called recursively, could cause StackOverflow with too many tombstones, tailrec fails 51 | 52 | if (j == null && s == null) { 53 | //both empty 54 | return None 55 | } else if (j == null && s != null) { 56 | val s2 = s 57 | s = shardNext() 58 | if (s2._2 eq tombstone) 59 | return advance() 60 | return Some(s2) 61 | } else if (j != null && s == null) { 62 | val j2 = j 63 | j = journalNext() 64 | if (j2._2 eq tombstone) 65 | return advance() 66 | return Some(j2) 67 | } else if (s._1 == j._1) { 68 | //both are at the same key, take newer value from journal, advance both 69 | val j2 = j 70 | j = journalNext() 71 | s = shardNext() 72 | if (j2._2 eq tombstone) 73 | return advance() 74 | return Some(j2) 75 | } else if (j._1 < s._1) { 76 | //take from journal, advance journal 77 | val j2 = j 78 | j = journalNext() 79 | if (j2._2 eq tombstone) 80 | return advance() 81 | return Some(j2) 82 | } else if (j._1 > s._1) { 83 | //take from shard, advance shard 84 | val s2 = s 85 | s = shardNext() 86 | if (s2._2 eq tombstone) 87 | return advance() 88 | return Some(s2) 89 | } else { 90 | throw new IllegalStateException() 91 | } 92 | 93 | } 94 | 95 | override def hasNext: Boolean = { 96 | if (next2 == null) 97 | next2 = advance() 98 | 99 | return next2.isDefined 100 | } 101 | 102 | override def next(): (K, V) = { 103 | if (next2 == null) 104 | next2 = advance() 105 | 106 | if (next2.isEmpty) 107 | throw new NoSuchElementException() 108 | 109 | val next3 = next2 110 | next2 = null 111 | return next3.get 112 | } 113 | } 114 | 115 | return o 116 | 117 | 118 | } 119 | 120 | } -------------------------------------------------------------------------------- /src/test/scala/io/iohk/iodb/smoke/RandomRollbackTest.scala: -------------------------------------------------------------------------------- 1 | package io.iohk.iodb.smoke 2 | 3 | import io.iohk.iodb.Store._ 4 | import io.iohk.iodb.TestUtils._ 5 | import io.iohk.iodb.{ByteArrayWrapper, Store} 6 | import org.scalatest.Matchers._ 7 | 8 | import scala.collection.mutable 9 | import scala.util.Random 10 | 11 | /** 12 | * Randomly inserts data and performs rollback 13 | */ 14 | object RandomRollbackTest { 15 | 16 | val randomSeed = 111 17 | val loops = 100 18 | val keepVersions = 100 19 | 20 | val maxRemoveBatchSize = 20 21 | 22 | val maxInsertBatchSize = 200 23 | 24 | def test(store:Store): Unit = { 25 | 26 | val r = new Random(randomSeed) 27 | 28 | def randomBuf: ByteArrayWrapper = { 29 | val b = new ByteArrayWrapper(32) 30 | r.nextBytes(b.data) 31 | b 32 | } 33 | 34 | val history = new mutable.TreeMap[Long, (Map[K, V], Set[K])]() 35 | var inserted = Map[K, V]() 36 | var removed = Set[K]() 37 | var version = 1L 38 | 39 | // , keepVersions = keepVersions) 40 | // maxJournalEntryCount = 1000, splitSize = 1000, maxFileSize = 64000, executor = null) 41 | 42 | for (i <- 0 until loops) { 43 | //randomly switch between actions 44 | val a = r.nextInt(100) 45 | if (a < 3 && history.size > 2) { 46 | //perform rollback 47 | version = r.shuffle(history.keys.dropRight(1)).head 48 | inserted = history(version)._1 49 | removed = history(version)._2 50 | 51 | //remove unused items from history 52 | history.keys.foreach(v => if (v > version) history.remove(v)) 53 | store.rollback(fromLong(version)) 54 | } else if (a < 6 && history.size > 4) { 55 | //perform cleanup 56 | store.clean(keepVersions) 57 | } else { 58 | //insert data 59 | val toUpdate = 60 | (0 until r.nextInt(maxInsertBatchSize)) 61 | .map(i => (randomBuf, randomBuf)) 62 | .filter(a => !removed.contains(a._1)) 63 | .toMap 64 | 65 | val toRemove = r.shuffle(inserted.keys) 66 | .take(r.nextInt(maxRemoveBatchSize)) 67 | .filter(!toUpdate.contains(_)) 68 | 69 | version += 1 70 | store.update(versionID = fromLong(version), toRemove = toRemove, toUpdate = toUpdate) 71 | 72 | for (k <- toRemove) 73 | assert(store.get(k) == None) 74 | 75 | inserted = inserted.++(toUpdate).--(toRemove) 76 | removed ++= toRemove 77 | //check this version is biggest 78 | assert(history.keys.forall(_ < version)) 79 | history.put(version, (inserted, removed)) 80 | //cut history so it has maximally 'keepVersions' entries 81 | history.keys 82 | .take(Math.max(0, history.size - keepVersions)) 83 | .toBuffer 84 | .foreach(history.remove(_)) 85 | } 86 | 87 | assert(Some(fromLong(version)) == store.lastVersionID) 88 | 89 | //check current state matches store 90 | for ((k, v) <- inserted) { 91 | assert(store.get(k) == Some(v)) 92 | } 93 | for (k <- removed) { 94 | assert(store.get(k) == None) 95 | } 96 | store.verify() 97 | 98 | val versionsFromStore = store.rollbackVersions().toBuffer.sorted.takeRight(history.size) 99 | val versionsFromHistory = history.keySet.map(fromLong(_)).toBuffer 100 | 101 | versionsFromStore shouldBe versionsFromHistory 102 | 103 | val getAll = store.getAll() 104 | val b1 = getAll.toBuffer.sortBy[ByteArrayWrapper](_._1).toBuffer 105 | val b2 = inserted.toBuffer.sortBy[ByteArrayWrapper](_._1).toBuffer 106 | b1.size shouldBe b2.size 107 | b1 shouldBe b2 108 | 109 | } 110 | store.close() 111 | } 112 | 113 | } 114 | -------------------------------------------------------------------------------- /doc/README.md: -------------------------------------------------------------------------------- 1 | IODB 2 | ========== 3 | 4 | IODB is a key-value store designed for blockchain applications. 5 | Main features include: 6 | 7 | - log-structured storage inspired by LevelDB. existing data are never overwritten. 8 | - versioned snapshots with rollback capability 9 | - sorted keys 10 | - atomic batch updates 11 | - multi-threaded background compaction 12 | 13 | 14 | Updates 15 | ---------- 16 | 17 | All updates are grouped into batches. 18 | Each update creates new snapshot with unique `VersionID` 19 | Example update is [here](https://github.com/input-output-hk/iodb/tree/master/src/test/scala/examples/Update.scala). 20 | 21 | Update method takes list of key and value pairs (`Iterable[(ByteArrayWrapper,ByteArrayWrapper)]`) for update. 22 | 23 | Each update also takes list of keys to delete (`Iterable[ByteArrayWrapper]`). 24 | IODB does not really delete entries, but places 'tombstone' into log. 25 | Delete is just special type of value. 26 | 27 | Main Log 28 | --------------- 29 | 30 | Each update is placed into single 'log file'. 31 | Log file is newer overwritten, but might be deleted by compaction process once it becomes obsolete. 32 | Sequence of updates is 'main log'. 33 | List of entries for given version (snapshot) can be always reconstructed by replaying all log files from oldest to newest. 34 | 35 | Key lookup (`get`) traverses log files (updates) from newest to oldest, until key (or its tombstone) is found. 36 | 37 | Main Log provides durability for store. 38 | Main Log should survive JVM or hardware crash. 39 | 40 | Merge Files 41 | ------------ 42 | 43 | Traversing entire log to find keys could take long time. 44 | So compaction process periodically creates Merge Files (with `.merge` extension). 45 | This file contains all key-value entries from older log files files. 46 | To create merge file, compaction replays updates from older log files. 47 | 48 | When key lookup reaches Merge File, it can stop log traversal. If key is not found in Merge File, it is not going to exist in older version. 49 | 50 | Merge File does not have to include key tombstones for deleted keys. 51 | So compaction process can exclude tombstones when creating merge files. 52 | 53 | Merge file is created by compaction process by replaying all older update files. 54 | Keys in log file are sorted, so lazy N-way merge with single traversal can be used. 55 | To create merge file it takes approximately O(N*log(M)) time, where N is the total number of entries and M is number of versions in merge. 56 | Usually M is small enough, not all versions are used for replay, 57 | but there is older merge file. 58 | 59 | Sharded Index 60 | ------------------ 61 | 62 | It is not practical to create merge files for large datasets. 63 | For that reason IODB maintains separate index for lookups. 64 | It slices (shards) keys into manageable intervals (typically 16 MB), 65 | and maintains separate log for each interval. 66 | 67 | Compaction process takes data from main log and distributes it across shards. 68 | It also periodically creates merge files on most updated shards and 69 | removes obsolete files. 70 | 71 | Number of shards is not fixed. When shard becomes too big (too many keys in interval), 72 | it is sliced into number of smaller shards with smaller intervals. 73 | If shard becomes too small (too little keys), it is merged with its neighbours. 74 | 75 | Shard Index can be reconstructed from main log. 76 | Its files do not have to be protected from corruption and can be safely deleted. 77 | 78 | Background compaction 79 | ------------------------ 80 | 81 | Updates in IODB are very fast. 82 | Each commit creates only single file and exits. 83 | Most maintenance is performed in background by multi-threaded compaction. 84 | 85 | Compaction in IODB is inspired by RocksDB. It is also multi-threaded and runs 86 | in background. But compaction in IODB solves one big problem: 87 | RocksDB compaction has long running tasks, at worse merge task might have to 88 | read and reinsert entire store. If this long running tasks is terminated, 89 | its progress is lost and it must start from beginning. 90 | 91 | Compaction in IODB is composed of small atomic tasks. 92 | In theory single task should always work with batch smaller than 100 MB. 93 | This greatly simplifies administration of IODB. 94 | It also makes it simple to share CPU with other tasks. 95 | For example compaction process can be temporary paused if other tasks require CPU, memory or disk IO. 96 | 97 | Rollback 98 | -------------- 99 | Updates can be rolled back. 100 | Rollback will discard all changes made in newer updates and restore store to older version. 101 | Rollback example is [here](https://github.com/input-output-hk/iodb/tree/master/src/test/scala/examples/Rollback.scala). 102 | 103 | Rollback will delete newer update files and release some disk space 104 | 105 | 106 | Clean 107 | ----------- 108 | IODB can clean old versions to save disk space. 109 | Clean example is [here](https://github.com/input-output-hk/iodb/tree/master/src/test/scala/examples/Clean.scala) 110 | 111 | Cleanup will remove all older snapshots, except last N versions (N is passed in parameter). 112 | It will no longer be possible to rollback to older versions (except last N versions). 113 | Data inserted at older updates will not be lost, but merged into oldest preserved snapshot. 114 | 115 | Cleanup will delete older log files, but will preserve their data in Merge Files. 116 | Cleanup qis necessary for releasing disk space. 117 | -------------------------------------------------------------------------------- /src/test/scala/io/iohk/iodb/bench/LongBench.scala: -------------------------------------------------------------------------------- 1 | package io.iohk.iodb.bench 2 | 3 | import java.io.File 4 | import java.util.concurrent.TimeUnit 5 | 6 | import io.iohk.iodb.{ByteArrayWrapper, QuickStore, Store, TestUtils} 7 | import org.junit.Assert._ 8 | 9 | import scala.util.Random 10 | 11 | /** 12 | * Long running benchmark to test store scalability under continuous use. 13 | */ 14 | object LongBench { 15 | 16 | case class Config( 17 | keyCount: Long = 1e7.toLong, 18 | batchSize: Int = 10000, 19 | keySize: Int = 32, 20 | minValueSize: Int = 32, 21 | maxValueSize: Int = 128, 22 | randomSeed: Int = new Random().nextInt(), 23 | duration: Long = TimeUnit.HOURS.toMillis(1), 24 | maxStoreDirSize: Double = 1e9 * 100) { 25 | assert(keyCount % batchSize == 0) 26 | } 27 | 28 | def randomPair(r: Random, config: Config): (ByteArrayWrapper, ByteArrayWrapper) = { 29 | val valueSize = config.minValueSize + r.nextInt(config.maxValueSize - config.minValueSize) 30 | (TestUtils.randomA(size = config.keySize, random = r), TestUtils.randomA(size = valueSize, random = r)) 31 | } 32 | 33 | def checkDirSize(dir: File, config: Config): Unit = { 34 | val dirSize = TestUtils.dirSize(dir) 35 | if (dirSize > config.maxStoreDirSize) { 36 | throw new Error("Store too big, size is " + (dirSize / 1e9) + " GB") 37 | } 38 | } 39 | 40 | def printVal(name: String, value: Long): Unit = { 41 | printf(name + " %,d \n", value) 42 | } 43 | 44 | def main(args: Array[String]): Unit = { 45 | println("max heap " + Runtime.getRuntime.maxMemory() / 1e9.toLong + " GB") 46 | val dir = TestUtils.tempDir() 47 | val dirClean = new File(dir, "clean") 48 | dirClean.mkdirs() 49 | 50 | val storeFab = { f: File => new QuickStore(f) } 51 | 52 | val config = new Config() 53 | 54 | printf("Key Count %,d \n", config.keyCount) 55 | 56 | //fill store with N keysunti l 57 | var time = System.currentTimeMillis() 58 | var store: Store = storeFab(dirClean) 59 | var r = new Random(config.randomSeed) 60 | for (i <- 0L until config.keyCount by config.batchSize) { 61 | val toUpdate = (i until i + config.batchSize).map { i => randomPair(r, config) }.toBuffer 62 | store.update( 63 | versionID = TestUtils.fromLong(i), 64 | toRemove = Nil, 65 | toUpdate = toUpdate 66 | ) 67 | checkDirSize(dirClean, config) 68 | } 69 | store.close() 70 | 71 | time = System.currentTimeMillis() - time 72 | printVal("Insert", time) 73 | printVal("Size", TestUtils.dirSize(dirClean)) 74 | 75 | 76 | // now we have store with data in `dirClean` 77 | // run readonly benchmark 78 | store = storeFab(dirClean) 79 | 80 | val readSpeed = readBench(store, config) 81 | printVal("Read: ", readSpeed) 82 | 83 | 84 | val reupdateSpeed = reupdateBench(store, config, dirClean) 85 | printVal("Reupdate: ", reupdateSpeed) 86 | 87 | TestUtils.deleteRecur(dir) 88 | 89 | } 90 | 91 | def readBench(store: Store, config: Config): Long = { 92 | val startTime = System.currentTimeMillis() 93 | var counter = 0L 94 | while (true) { 95 | val r = new Random(config.randomSeed) 96 | 97 | for (i <- 0L until config.keyCount) { 98 | counter += 1 99 | 100 | val curTime = System.currentTimeMillis() 101 | if (curTime > startTime + config.duration) 102 | return 1000 * counter / (curTime - startTime) 103 | 104 | val (key, value) = randomPair(r, config) 105 | assertEquals(Some(value), store.get(key)) 106 | } 107 | } 108 | return -1L 109 | } 110 | 111 | 112 | def reupdateBench(store: Store, config: Config, dir: File): Long = { 113 | val startTime = System.currentTimeMillis() 114 | var counter = 0L 115 | while (true) { 116 | var r = new Random(config.randomSeed) 117 | for (i <- 0L until config.keyCount by config.batchSize) { 118 | val toUpdate = (i until i + config.batchSize).map { i => randomPair(r, config) }.toBuffer 119 | store.update( 120 | versionID = TestUtils.fromLong(i), 121 | toRemove = Nil, 122 | toUpdate = toUpdate 123 | ) 124 | checkDirSize(dir, config) 125 | val curTime = System.currentTimeMillis() 126 | 127 | counter += 1 128 | if (curTime > startTime + config.duration) 129 | return 1000 * counter / (curTime - startTime) 130 | 131 | } 132 | 133 | } 134 | return -1L 135 | } 136 | 137 | 138 | def updateBench(store: Store, config: Config, dir: File): Long = { 139 | val startTime = System.currentTimeMillis() 140 | var counter = 0L 141 | while (true) { 142 | var r = new Random(config.randomSeed + 1) 143 | for (i <- 0L until config.keyCount by config.batchSize) { 144 | val toUpdate = (i until i + config.batchSize).map { i => randomPair(r, config) }.toBuffer 145 | store.update( 146 | versionID = TestUtils.fromLong(i), 147 | toRemove = Nil, 148 | toUpdate = toUpdate 149 | ) 150 | checkDirSize(dir, config) 151 | val curTime = System.currentTimeMillis() 152 | 153 | counter += 1 154 | if (curTime > startTime + config.duration) 155 | return 1000 * counter / (curTime - startTime) 156 | 157 | } 158 | 159 | } 160 | return -1L 161 | } 162 | } 163 | -------------------------------------------------------------------------------- /src/main/scala/io/iohk/iodb/Store.scala: -------------------------------------------------------------------------------- 1 | package io.iohk.iodb 2 | 3 | import java.util.logging.Level 4 | 5 | import scala.collection.mutable 6 | 7 | object Store { 8 | 9 | /** type of key */ 10 | type K = ByteArrayWrapper 11 | /** type of value */ 12 | type V = ByteArrayWrapper 13 | 14 | /** type used for versionID */ 15 | type VersionID = ByteArrayWrapper 16 | 17 | type FileNum = Long 18 | type FileOffset = Long 19 | 20 | val tombstoneByteArray = new Array[Byte](0) 21 | val tombstone = new ByteArrayWrapper(tombstoneByteArray) 22 | } 23 | 24 | /** 25 | * Interface for a key-value versioned database. 26 | * It has been created with a blockchain core needs in mind. 27 | */ 28 | trait Store { 29 | 30 | import Store._ 31 | 32 | /** 33 | * Finds key and returns value associated with the key. 34 | * If key is not found, it returns null. 35 | * 36 | * It uses lattest (most recent) version available in store. 37 | * 38 | * @param key to lookup 39 | * @return value associated with key or null 40 | */ 41 | def get(key: K): Option[V] 42 | 43 | /** Returns value associated with the key, or defualt value from user 44 | */ 45 | def getOrElse(key: K, default: => V): V = get(key).getOrElse(default) 46 | 47 | /** returns value associated with the key or throws `NoSuchElementException` */ 48 | def apply(key: K): V = getOrElse(key, { 49 | throw new NoSuchElementException() 50 | }) 51 | 52 | /** 53 | * Batch get. 54 | * 55 | * Finds all keys from given iterable. 56 | * Result is returned in an iterable of key-value pairs. 57 | * If key is not found, null value is included in result pair. 58 | * 59 | * It uses lattest (most recent) version available in store 60 | * 61 | * @param keys keys to loopup 62 | * @return iterable over key-value pairs found in store 63 | */ 64 | def get(keys: Iterable[K]): Iterable[(K, Option[V])] = { 65 | val ret = scala.collection.mutable.ArrayBuffer.empty[(K, Option[V])] 66 | get(keys, (key: K, value: Option[V]) => 67 | ret += ((key, value)) 68 | ) 69 | ret 70 | } 71 | 72 | /** 73 | * Batch get with callback for result value. 74 | * 75 | * Finds all keys from given iterable. 76 | * Results are passed to callable consumer. 77 | * 78 | * It uses lattest (most recent) version available in store 79 | * 80 | * @param keys keys to lookup 81 | * @param consumer callback method to consume results 82 | */ 83 | def get(keys: Iterable[K], consumer: (K, Option[V]) => Unit): Unit = { 84 | for (key <- keys) { 85 | val value = get(key) 86 | consumer(key, value) 87 | } 88 | } 89 | 90 | /** Get content of entire store. Result is not sorted. */ 91 | def getAll(): Iterator[(K, V)] = { 92 | val ret = new mutable.ArrayBuffer[(K, V)]() 93 | getAll { (k: K, v: V) => 94 | ret += ((k, v)) 95 | } 96 | return ret.iterator 97 | } 98 | 99 | /** 100 | * Get content of entire store and pass it to consumer. 101 | * There might be too many entries to fit on heap. 102 | * Iterators also cause problems for locking. 103 | * So the consumer is preferred way to fetch all entries. 104 | * 105 | * @param consumer 106 | */ 107 | def getAll(consumer: (K, V) => Unit) 108 | 109 | 110 | /** 111 | * Starts or resumes background compaction. 112 | * Compaction performs cleanup and runs in background process. 113 | * It removes older version and compacts index to consume less space. 114 | * 115 | * @param count how many past versions to keep 116 | * 117 | */ 118 | def clean(count: Int) 119 | 120 | /** 121 | * Returns current versionID used by Store. 122 | * It is last version store was update to with `update()` method. 123 | * 124 | * If store is empty, the last version does not exist yet and store returns `None` 125 | * 126 | * VersionID is persisted between restarts. 127 | */ 128 | def lastVersionID: Option[VersionID] 129 | 130 | /** 131 | * Batch update records. 132 | * 133 | * Each update increments versionID. New versionID is passed as an argument. 134 | * 135 | * Update might remove some key-value pairs, or can insert new key-value pairs. 136 | * Iterable of keys to be deleted, and iterable of key-value pairs to be updated is passed as an argument. 137 | * 138 | * @param versionID new versionID associated with this update 139 | * @param toRemove iterable over keys which will be deleted in this update 140 | * @param toUpdate iterable over key-value pairs which will be inserted in this update 141 | */ 142 | 143 | def update(versionID: VersionID, toRemove: Iterable[K], toUpdate: Iterable[(K, V)]) 144 | 145 | def update(version: Long, toRemove: Iterable[K], toUpdate: Iterable[(K, V)]): Unit = { 146 | update(ByteArrayWrapper.fromLong(version), toRemove, toUpdate) 147 | } 148 | 149 | /** 150 | * Reverts to an older versionID. 151 | * All key-value pairs are reverted to this older version (updates between two versionIDs are removed). 152 | * 153 | * Higher (newer) versionIDs are discarded and their versionID can be reused 154 | */ 155 | def rollback(versionID: VersionID) 156 | 157 | 158 | /** 159 | * Closes store. All resources associated with this store are closed and released (files, background threads...). 160 | * Any get/update operations invoked on closed store will throw an exception. 161 | */ 162 | def close(): Unit //TODO: Try[Unit] ? 163 | 164 | /** 165 | * Returns all versions known to this store. 166 | * User can rollback to versions in this list with rollback method 167 | */ 168 | def rollbackVersions(): Iterable[VersionID] 169 | 170 | 171 | /** 172 | * Checks storage for consistency, invalid checksums, data corruption... 173 | * Throws an exception if any error is found. 174 | * This method may run for very long time, depending on store size. 175 | */ 176 | def verify(): Unit 177 | 178 | 179 | 180 | 181 | def runnable(f: => Unit): Runnable = 182 | return () => { 183 | try { 184 | f 185 | } catch { 186 | case e: Throwable => { 187 | Utils.LOG.log(Level.SEVERE, "Background task failed", e) 188 | } 189 | } 190 | } 191 | 192 | } -------------------------------------------------------------------------------- /src/test/scala/io/iohk/iodb/prop/IODBSpecification.scala: -------------------------------------------------------------------------------- 1 | package io.iohk.iodb.prop 2 | 3 | import java.security.MessageDigest 4 | 5 | import io.iohk.iodb._ 6 | import org.scalatest.prop.{GeneratorDrivenPropertyChecks, PropertyChecks} 7 | import org.scalatest.{BeforeAndAfterAll, Matchers, PropSpec} 8 | 9 | import scala.annotation.tailrec 10 | import scala.util.Random 11 | 12 | @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) 13 | class IODBSpecification extends PropSpec 14 | with PropertyChecks 15 | with GeneratorDrivenPropertyChecks 16 | with Matchers 17 | with BeforeAndAfterAll { 18 | 19 | property("rollback test Sharded") { 20 | TestUtils.withTempDir { iFile => 21 | rollbackTest(blockStorage = new ShardedStore(iFile)) 22 | } 23 | } 24 | 25 | property("writeKey test Sharded") { 26 | TestUtils.withTempDir { iFile => 27 | writeKeyTest(blockStorage = new ShardedStore(iFile)) 28 | } 29 | } 30 | 31 | 32 | property("rollback test Log") { 33 | TestUtils.withTempDir { iFile => 34 | rollbackTest(blockStorage = new LogStore(iFile)) 35 | } 36 | } 37 | 38 | property("writeKey test Log") { 39 | TestUtils.withTempDir { iFile => 40 | writeKeyTest(blockStorage = new LogStore(iFile)) 41 | } 42 | } 43 | 44 | property("rollback test quick") { 45 | TestUtils.withTempDir { iFile => 46 | rollbackTest(blockStorage = new QuickStore(iFile, keepVersions = 1000)) 47 | } 48 | } 49 | 50 | property("writeKey test quick") { 51 | TestUtils.withTempDir { iFile => 52 | writeKeyTest(blockStorage = new QuickStore(iFile)) 53 | } 54 | } 55 | 56 | def rollbackTest(blockStorage:Store){ 57 | //initialize test 58 | val NumberOfBlocks = 100 59 | val NumberOfRollbacks = 100 60 | 61 | case class BlockChanges(id: ByteArrayWrapper, 62 | toRemove: Seq[ByteArrayWrapper], 63 | toInsert: Seq[(ByteArrayWrapper, ByteArrayWrapper)]) 64 | 65 | def hash(b: Array[Byte]): Array[Byte] = MessageDigest.getInstance("SHA-256").digest(b) 66 | 67 | def randomBytes(): ByteArrayWrapper = ByteArrayWrapper(hash(Random.nextString(16).getBytes)) 68 | 69 | def generateBytes(): Seq[(ByteArrayWrapper, ByteArrayWrapper)] = { 70 | (0 until Random.nextInt(100)).map(i => (randomBytes(), randomBytes())) 71 | } 72 | 73 | val (blockchain: IndexedSeq[BlockChanges], existingKeys: Seq[ByteArrayWrapper]) = { 74 | @tailrec 75 | def loop(acc: IndexedSeq[BlockChanges], existingKeys: Seq[ByteArrayWrapper]): (IndexedSeq[BlockChanges], Seq[ByteArrayWrapper]) = { 76 | if (acc.length < NumberOfBlocks) { 77 | val toInsert = generateBytes() 78 | val toRemove: Seq[ByteArrayWrapper] = existingKeys.filter(k => Random.nextBoolean()) 79 | val newExistingKeys = existingKeys.filter(ek => !toRemove.contains(ek)) ++ toInsert.map(_._1) 80 | val newBlock = BlockChanges(randomBytes(), toRemove, toInsert) 81 | loop(newBlock +: acc, newExistingKeys) 82 | } else { 83 | (acc, existingKeys) 84 | } 85 | } 86 | loop(IndexedSeq.empty, Seq.empty) 87 | } 88 | val allBlockchainKeys: Seq[ByteArrayWrapper] = blockchain.flatMap(_.toInsert.map(_._1)) 89 | 90 | def storageHash(storage: Store): Array[Byte] = { 91 | val valuesBytes = allBlockchainKeys.map(k => storage.get(k).map(_.toString).getOrElse("")).mkString.getBytes 92 | hash(valuesBytes) 93 | } 94 | 95 | //initialize blockchain 96 | blockchain.foreach(b => blockStorage.update(b.id, b.toRemove, b.toInsert)) 97 | 98 | val finalHash = storageHash(blockStorage) 99 | val finalVersion = blockStorage.lastVersionID.get 100 | 101 | (0 until NumberOfRollbacks) foreach { _ => 102 | val idToRollback: ByteArrayWrapper = blockchain.map(_.id).apply(Random.nextInt(blockchain.length)) 103 | val index: Int = blockchain.indexWhere(_.id == idToRollback) 104 | val removedBlocks = blockchain.takeRight(NumberOfBlocks - index - 1) 105 | blockStorage.rollback(idToRollback) 106 | 107 | removedBlocks.foreach(b => blockStorage.update(b.id, b.toRemove, b.toInsert)) 108 | blockStorage.lastVersionID.get shouldEqual finalVersion 109 | storageHash(blockStorage) shouldEqual finalHash 110 | } 111 | 112 | existingKeys.foreach(ek => blockStorage.get(ek).isDefined shouldBe true) 113 | blockStorage.close() 114 | } 115 | 116 | 117 | def writeKeyTest(blockStorage:Store){ 118 | var ids: Seq[ByteArrayWrapper] = Seq() 119 | var removed: Seq[ByteArrayWrapper] = Seq() 120 | var i = 0 121 | 122 | forAll { (key: String, value: Array[Byte], removing: Boolean) => 123 | val toRemove = if (removing && ids.nonEmpty) Seq(ids(Random.nextInt(ids.length))) else Seq() 124 | toRemove.foreach { tr => 125 | ids = ids.filter(i => i != tr) 126 | removed = tr +: removed 127 | } 128 | 129 | val id: ByteArrayWrapper = hash(i + key) 130 | val fValue: ByteArrayWrapper = ByteArrayWrapper(value) 131 | ids = id +: ids 132 | i = i + 1 133 | 134 | blockStorage.update( 135 | id, 136 | toRemove, 137 | Seq(id -> fValue)) 138 | } 139 | 140 | //old keys are defined 141 | ids.foreach { id => 142 | blockStorage.get(id) match { 143 | case None => throw new Error(s"Id $id} not found") 144 | case Some(v) => 145 | } 146 | } 147 | 148 | //removed keys are defined not defined 149 | removed.foreach { id => 150 | blockStorage.get(id) match { 151 | case None => 152 | case Some(v) => throw new Error(s"Id $id} is defined after delete") 153 | } 154 | } 155 | ids.foreach(id => blockStorage.rollbackVersions().exists(_ == id) shouldBe true) 156 | 157 | blockStorage.close() 158 | } 159 | 160 | 161 | case class BlockChanges(id: ByteArrayWrapper, 162 | toRemove: Seq[ByteArrayWrapper], 163 | toInsert: Seq[(ByteArrayWrapper, ByteArrayWrapper)]) 164 | 165 | def hash(b: Array[Byte]): Array[Byte] = MessageDigest.getInstance("SHA-256").digest(b) 166 | 167 | def randomBytes(): ByteArrayWrapper = ByteArrayWrapper(hash(Random.nextString(16).getBytes)) 168 | 169 | def generateBytes(howMany: Int): Seq[(ByteArrayWrapper, ByteArrayWrapper)] = { 170 | (0 until howMany).map(i => (randomBytes(), randomBytes())) 171 | } 172 | 173 | 174 | def hash(s: String) = ByteArrayWrapper(MessageDigest.getInstance("SHA-256").digest(s.getBytes)) 175 | } -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | CC0 1.0 Universal 2 | 3 | Statement of Purpose 4 | 5 | The laws of most jurisdictions throughout the world automatically confer 6 | exclusive Copyright and Related Rights (defined below) upon the creator and 7 | subsequent owner(s) (each and all, an "owner") of an original work of 8 | authorship and/or a database (each, a "Work"). 9 | 10 | Certain owners wish to permanently relinquish those rights to a Work for the 11 | purpose of contributing to a commons of creative, cultural and scientific 12 | works ("Commons") that the public can reliably and without fear of later 13 | claims of infringement build upon, modify, incorporate in other works, reuse 14 | and redistribute as freely as possible in any form whatsoever and for any 15 | purposes, including without limitation commercial purposes. These owners may 16 | contribute to the Commons to promote the ideal of a free culture and the 17 | further production of creative, cultural and scientific works, or to gain 18 | reputation or greater distribution for their Work in part through the use and 19 | efforts of others. 20 | 21 | For these and/or other purposes and motivations, and without any expectation 22 | of additional consideration or compensation, the person associating CC0 with a 23 | Work (the "Affirmer"), to the extent that he or she is an owner of Copyright 24 | and Related Rights in the Work, voluntarily elects to apply CC0 to the Work 25 | and publicly distribute the Work under its terms, with knowledge of his or her 26 | Copyright and Related Rights in the Work and the meaning and intended legal 27 | effect of CC0 on those rights. 28 | 29 | 1. Copyright and Related Rights. A Work made available under CC0 may be 30 | protected by copyright and related or neighboring rights ("Copyright and 31 | Related Rights"). Copyright and Related Rights include, but are not limited 32 | to, the following: 33 | 34 | i. the right to reproduce, adapt, distribute, perform, display, communicate, 35 | and translate a Work; 36 | 37 | ii. moral rights retained by the original author(s) and/or performer(s); 38 | 39 | iii. publicity and privacy rights pertaining to a person's image or likeness 40 | depicted in a Work; 41 | 42 | iv. rights protecting against unfair competition in regards to a Work, 43 | subject to the limitations in paragraph 4(a), below; 44 | 45 | v. rights protecting the extraction, dissemination, use and reuse of data in 46 | a Work; 47 | 48 | vi. database rights (such as those arising under Directive 96/9/EC of the 49 | European Parliament and of the Council of 11 March 1996 on the legal 50 | protection of databases, and under any national implementation thereof, 51 | including any amended or successor version of such directive); and 52 | 53 | vii. other similar, equivalent or corresponding rights throughout the world 54 | based on applicable law or treaty, and any national implementations thereof. 55 | 56 | 2. Waiver. To the greatest extent permitted by, but not in contravention of, 57 | applicable law, Affirmer hereby overtly, fully, permanently, irrevocably and 58 | unconditionally waives, abandons, and surrenders all of Affirmer's Copyright 59 | and Related Rights and associated claims and causes of action, whether now 60 | known or unknown (including existing as well as future claims and causes of 61 | action), in the Work (i) in all territories worldwide, (ii) for the maximum 62 | duration provided by applicable law or treaty (including future time 63 | extensions), (iii) in any current or future medium and for any number of 64 | copies, and (iv) for any purpose whatsoever, including without limitation 65 | commercial, advertising or promotional purposes (the "Waiver"). Affirmer makes 66 | the Waiver for the benefit of each member of the public at large and to the 67 | detriment of Affirmer's heirs and successors, fully intending that such Waiver 68 | shall not be subject to revocation, rescission, cancellation, termination, or 69 | any other legal or equitable action to disrupt the quiet enjoyment of the Work 70 | by the public as contemplated by Affirmer's express Statement of Purpose. 71 | 72 | 3. Public License Fallback. Should any part of the Waiver for any reason be 73 | judged legally invalid or ineffective under applicable law, then the Waiver 74 | shall be preserved to the maximum extent permitted taking into account 75 | Affirmer's express Statement of Purpose. In addition, to the extent the Waiver 76 | is so judged Affirmer hereby grants to each affected person a royalty-free, 77 | non transferable, non sublicensable, non exclusive, irrevocable and 78 | unconditional license to exercise Affirmer's Copyright and Related Rights in 79 | the Work (i) in all territories worldwide, (ii) for the maximum duration 80 | provided by applicable law or treaty (including future time extensions), (iii) 81 | in any current or future medium and for any number of copies, and (iv) for any 82 | purpose whatsoever, including without limitation commercial, advertising or 83 | promotional purposes (the "License"). The License shall be deemed effective as 84 | of the date CC0 was applied by Affirmer to the Work. Should any part of the 85 | License for any reason be judged legally invalid or ineffective under 86 | applicable law, such partial invalidity or ineffectiveness shall not 87 | invalidate the remainder of the License, and in such case Affirmer hereby 88 | affirms that he or she will not (i) exercise any of his or her remaining 89 | Copyright and Related Rights in the Work or (ii) assert any associated claims 90 | and causes of action with respect to the Work, in either case contrary to 91 | Affirmer's express Statement of Purpose. 92 | 93 | 4. Limitations and Disclaimers. 94 | 95 | a. No trademark or patent rights held by Affirmer are waived, abandoned, 96 | surrendered, licensed or otherwise affected by this document. 97 | 98 | b. Affirmer offers the Work as-is and makes no representations or warranties 99 | of any kind concerning the Work, express, implied, statutory or otherwise, 100 | including without limitation warranties of title, merchantability, fitness 101 | for a particular purpose, non infringement, or the absence of latent or 102 | other defects, accuracy, or the present or absence of errors, whether or not 103 | discoverable, all to the greatest extent permissible under applicable law. 104 | 105 | c. Affirmer disclaims responsibility for clearing rights of other persons 106 | that may apply to the Work or any use thereof, including without limitation 107 | any person's Copyright and Related Rights in the Work. Further, Affirmer 108 | disclaims responsibility for obtaining any necessary consents, permissions 109 | or other rights required for any use of the Work. 110 | 111 | d. Affirmer understands and acknowledges that Creative Commons is not a 112 | party to this document and has no duty or obligation with respect to this 113 | CC0 or use of the Work. 114 | 115 | For more information, please see 116 | 117 | -------------------------------------------------------------------------------- /src/test/scala/io/iohk/iodb/smoke/M1Test.scala: -------------------------------------------------------------------------------- 1 | package io.iohk.iodb.smoke 2 | 3 | import java.io.{BufferedOutputStream, FileOutputStream} 4 | import java.nio.ByteBuffer 5 | import java.nio.channels.FileChannel 6 | import java.nio.channels.FileChannel.MapMode 7 | import java.nio.file.StandardOpenOption 8 | 9 | import io.iohk.iodb.{ByteArrayWrapper, ShardedStore, Store, TestUtils} 10 | import io.iohk.iodb.Store._ 11 | import TestUtils._ 12 | import io.iohk.iodb.Utils._ 13 | 14 | import scala.util.Random 15 | import org.scalatest.Matchers._ 16 | 17 | import scala.collection.mutable 18 | import scala.collection.mutable.ArrayBuffer 19 | 20 | 21 | /** tests inserts, updates, deletes and rollbacks */ 22 | object M1Test { 23 | 24 | val defaultKeyCount = 1000000 25 | val defaultDuration = 12L 26 | 27 | val defaultShardCount = 16 28 | 29 | val batchSize = 1e5.toLong 30 | 31 | val defaultMaxReadsIncrement = 10000 32 | 33 | 34 | def main(args: Array[String]): Unit = { 35 | 36 | 37 | val keyCount = if(args.length>=1) args(0).toInt else defaultKeyCount 38 | val duration = if(args.length>=2) args(1).toLong else defaultDuration 39 | val shardCount = if(args.length>=3) args(2).toInt else defaultShardCount 40 | val maxReads = if(args.length>=4) args(3).toInt else defaultMaxReadsIncrement 41 | 42 | val dir = TestUtils.tempDir() 43 | 44 | def dirSize() = dir.listFiles().map(_.length()).sum 45 | 46 | def dirSizeGiga(): String = (dirSize().toDouble / 1e9) + " GB" 47 | 48 | 49 | val store = new ShardedStore(dir = dir, keySize = 8, shardCount = shardCount) 50 | 51 | try { 52 | println("KeyCount: " + keyCount) 53 | println("Shard count: " + shardCount) 54 | println("Dir: " + dir.getPath) 55 | 56 | //insert initial data 57 | for (i <- 0L until keyCount by batchSize) { 58 | val value = valueFromSeed(1) 59 | val keyVals = (i until Math.min(keyCount, i + batchSize)).map(k => (fromLong(longHash(k)), value)) 60 | 61 | store.update(versionID = fromLong(-i), toUpdate = keyVals, toRemove = Nil) 62 | } 63 | 64 | val initRollbackVersionsSize = store.rollbackVersions().size 65 | 66 | println("Store populated, dir size: " + dirSizeGiga()) 67 | 68 | //initialize reference file with given size: 69 | // Reference file is used to calculate content of store. 70 | // Each byte in file represents one key (byte offset is key value). 71 | // Byte value at given offset represents seed value associated with given key, where 0 is deleted key 72 | val refFile = createRefFile(keyCount, 1) 73 | 74 | val endTime = System.currentTimeMillis() + duration * 3600 * 1000 75 | 76 | var history = new mutable.ArrayBuffer[Long]() 77 | 78 | val r = new Random() 79 | while (System.currentTimeMillis() < endTime) { 80 | if (history.size > 5 && r.nextInt(20) == 0) { 81 | //rollback 82 | var cutOffset = r.nextInt(history.size - 2) 83 | var versionID = fromLong(history(cutOffset)) 84 | history = history.take(cutOffset + 1) 85 | store.rollback(versionID) 86 | println("rollback") 87 | 88 | //replay actions on reference file to match content of store 89 | replayRefFile(refFile, history, keyCount) 90 | } else { 91 | val vlong = Math.abs(r.nextLong) 92 | var version = fromLong(vlong) 93 | //update some keys and delete some existing keys 94 | 95 | history.append(vlong) 96 | val (toUpdate, toRemove) = alterFile(vlong, refFile, keyCount) 97 | 98 | store.update(version, toUpdate = toUpdate, toRemove = toRemove) 99 | } 100 | 101 | if(r.nextInt(100)==0) { 102 | 103 | //verify store 104 | store.rollbackVersions().toBuffer.drop(initRollbackVersionsSize) shouldBe history.map(fromLong(_)) 105 | 106 | val increment = Math.max(1.0, 1.0*keyCount/maxReads).toInt 107 | //iterate over all keys in ref file, ensure that store content is identical 108 | for (offset <- 0 until keyCount) { 109 | val seed = refFile.get(offset) 110 | val value = if (seed == 0) None else Some(valueFromSeed(seed)) 111 | if (offset % increment == 0) { 112 | val key = fromLong(longHash(offset)) 113 | // only check Nth elements 114 | store.get(key) shouldBe value 115 | } 116 | } 117 | 118 | println("Ver: " + history.size + " - disk size: " + dirSizeGiga()) 119 | } 120 | } 121 | 122 | }finally{ 123 | deleteRecur(dir) 124 | store.close() 125 | } 126 | 127 | } 128 | 129 | 130 | def valueFromSeed(seed:Byte):ByteArrayWrapper = { 131 | assert(seed!=0) 132 | val random = new Random(seed) 133 | val size = random.nextInt(100) 134 | randomA(size=size, random) 135 | } 136 | 137 | 138 | /* 139 | */ 140 | def createRefFile(keyCount:Int, valueSeed:Byte):ByteBuffer = { 141 | assert(keyCount<=Integer.MAX_VALUE, "keyCount must be < 2G") 142 | val f = tempFile() 143 | val fout = new FileOutputStream(f) 144 | val bout = new BufferedOutputStream(fout) 145 | 146 | //fill with data 147 | for(i <- 0L until keyCount){ 148 | bout.write(valueSeed) 149 | } 150 | bout.flush() 151 | fout.close() 152 | 153 | //memory map 154 | val channel = FileChannel.open(f.toPath, StandardOpenOption.READ, StandardOpenOption.WRITE) 155 | val buf = channel.map(MapMode.READ_WRITE, 0, keyCount) 156 | channel.close() 157 | 158 | f.delete() 159 | f.deleteOnExit() 160 | 161 | return buf 162 | } 163 | 164 | 165 | def alterFile(v:Long, ref:ByteBuffer, keyCount:Int): (Iterable[(K,V)], Iterable[K]) ={ 166 | val r = new Random(v) 167 | val updateSize = r.nextInt(2000) 168 | val removeSize = r.nextInt(1000) 169 | 170 | val keys = (0 until updateSize).map(i=>r.nextInt(keyCount)).toSet 171 | val keyVals = keys.toBuffer.sorted.map { k => 172 | var seed = r.nextInt().toByte 173 | if(seed==0) seed=1 174 | ref.put(k, seed) //update reference file 175 | (fromLong(longHash(k)), valueFromSeed(seed)) 176 | } 177 | 178 | //find some keys for deletion 179 | val keysToRemove = (0 until removeSize).map(i=>r.nextInt(keyCount)) 180 | .filter(ref.get(_)!=0) 181 | .filter(!keys.contains(_)) 182 | .toSet.toBuffer.sorted.map { k => 183 | ref.put(k, 0) //update reference file 184 | fromLong(longHash(k)) 185 | } 186 | 187 | (keyVals, keysToRemove) 188 | } 189 | 190 | def replayRefFile(refFile: ByteBuffer, history: ArrayBuffer[FileNum], keyCount:Int) = { 191 | //first fill with 1 192 | for(offset <- 0 until keyCount){ 193 | refFile.put(offset, 1) 194 | } 195 | //now replay all actions 196 | for(version<-history){ 197 | alterFile(version, refFile, keyCount) 198 | } 199 | } 200 | 201 | 202 | } 203 | -------------------------------------------------------------------------------- /src/test/scala/io/iohk/iodb/prop/StoreSpecification.scala: -------------------------------------------------------------------------------- 1 | package io.iohk.iodb.prop 2 | 3 | import java.io.File 4 | 5 | import io.iohk.iodb._ 6 | import org.junit.Test 7 | import org.scalacheck.Test._ 8 | import org.scalacheck.commands.Commands 9 | import org.scalacheck.{Gen, Prop} 10 | import org.scalatest.BeforeAndAfterAll 11 | import org.scalatest.junit.JUnitSuite 12 | import org.scalatest.prop.Checkers 13 | 14 | import scala.util.{Failure, Random, Success, Try} 15 | 16 | 17 | class StoreSpecification extends JUnitSuite with Checkers with BeforeAndAfterAll { 18 | 19 | val params = Parameters.default 20 | .withMinSize(100) 21 | .withMaxSize(200) 22 | .withMinSuccessfulTests(2) 23 | .withWorkers(2) 24 | 25 | //todo: pass initial set size? for now the set is only about 1 element 26 | 27 | @Test 28 | def testLsm1(): Unit = { 29 | check(new LSMCommands(maxJournalEntryCount = 1000, keepVersion = 1500).property(), params) 30 | } 31 | 32 | @Test 33 | def testLsm2(): Unit = { 34 | check(new LSMCommands(maxJournalEntryCount = 10, keepVersion = 1500).property(), params) 35 | } 36 | 37 | } 38 | 39 | //todo: comments 40 | class LSMCommands(val maxJournalEntryCount: Int, val keepVersion: Int) extends Commands { 41 | 42 | type Version = Int 43 | 44 | type Appended = IndexedSeq[(ByteArrayWrapper, ByteArrayWrapper)] 45 | type Removed = IndexedSeq[ByteArrayWrapper] 46 | 47 | type AppendsIndex = Map[Version, Int] 48 | type RemovalsIndex = Map[Version, Int] 49 | 50 | // type State = (Version, AppendsIndex, RemovalsIndex, Appended, Removed) 51 | case class State(version: Version, appendsIndex: AppendsIndex, removalsIndex: RemovalsIndex, appended: Appended, removed: Removed) 52 | 53 | var folder: File = null 54 | type Sut = Store 55 | 56 | val initialState: State = State(0, Map(0 -> 1), Map(), IndexedSeq(ByteArrayWrapper(Array.fill(32)(0: Byte)) -> ByteArrayWrapper.fromLong(5)), IndexedSeq()) 57 | 58 | override def canCreateNewSut(newState: State, 59 | initSuts: Traversable[State], 60 | runningSuts: Traversable[Store]): Boolean = true 61 | 62 | override def newSut(state: State): Store = { 63 | folder = TestUtils.tempDir() 64 | val s = new LogStore(folder, keepVersions = keepVersion /*, executor = null*/) 65 | s.update(state.version, state.removed, state.appended) 66 | s 67 | } 68 | 69 | override def initialPreCondition(state: State): Boolean = true 70 | 71 | override def genCommand(state: State): Gen[Command] = { 72 | 73 | lazy val appendsCount = Random.nextInt(500) + 100 74 | 75 | lazy val toAppend = (0 until appendsCount).map { _ => 76 | val k = Array.fill(32)(Random.nextInt(Byte.MaxValue).toByte) 77 | val v = Array.fill(Random.nextInt(100) + 5)(Random.nextInt(Byte.MaxValue).toByte) 78 | ByteArrayWrapper(k) -> ByteArrayWrapper(v) 79 | } 80 | 81 | lazy val remCount = Math.min(Random.nextInt(100), state.appended.size) 82 | 83 | lazy val toRemove = (0 until remCount).map { _ => 84 | val ap = Random.nextInt(state.appended.size) 85 | state.appended(ap)._1 86 | }.filter(k => !state.removed.contains(k)).toSet.toSeq 87 | 88 | lazy val genFwd = { 89 | Gen.const(AppendForward(state.version + 1, toAppend, toRemove)) 90 | } 91 | 92 | lazy val genGetExisting = { 93 | var existingIdOpt: Option[ByteArrayWrapper] = None 94 | 95 | do { 96 | val ap = Random.nextInt(state.appended.size) 97 | val eId = state.appended(ap)._1 98 | if (!state.removed.contains(eId)) existingIdOpt = Some(eId) 99 | } while (existingIdOpt.isEmpty) 100 | 101 | val existingId = existingIdOpt.get 102 | 103 | Gen.const(new GetExisting(existingId)) 104 | } 105 | 106 | lazy val genGetRemoved = { 107 | val rp = Random.nextInt(state.removed.size) 108 | val removedId = state.removed(rp) 109 | 110 | Gen.const(new GetRemoved(removedId)) 111 | } 112 | 113 | lazy val genCleanup = Gen.const(CleanUp) 114 | 115 | lazy val genRollback = Gen.choose(2, 10).map(d => new Rollback(state.version - d)) 116 | 117 | val gf = Seq(500 -> genFwd, 50 -> genGetExisting, 1 -> genCleanup) 118 | val gfr = if (state.removed.isEmpty) gf else gf ++ Seq(30 -> genGetRemoved) 119 | val gfrr = if (state.version > 20) gfr ++ Seq(2 -> genRollback) else gfr 120 | Gen.frequency(gfrr: _*) 121 | } 122 | 123 | override def destroySut(sut: Store): Unit = { 124 | sut.close() 125 | TestUtils.deleteRecur(folder) 126 | } 127 | 128 | override def genInitialState: Gen[State] = Gen.const(initialState) 129 | 130 | case class AppendForward(version: Version, toAppend: Seq[(ByteArrayWrapper, ByteArrayWrapper)], toRemove: Seq[ByteArrayWrapper]) extends Command { 131 | type Result = Try[Unit] 132 | 133 | override def run(sut: Store): Try[Unit] = { 134 | // println("appending: " + toAppend.size + " removing: " + toRemove.size) 135 | Try(sut.update(ByteArrayWrapper.fromLong(version), toRemove, toAppend)) 136 | } 137 | 138 | override def nextState(state: State): State = { 139 | assert(state.appendsIndex.get(version) == None) 140 | assert(state.removalsIndex.get(version) == None) 141 | State(version, 142 | state.appendsIndex ++ Seq((version, state.appended.size + toAppend.size)), 143 | state.removalsIndex ++ Seq((version, state.removed.size + toRemove.size)), 144 | state.appended ++ toAppend, 145 | state.removed ++ toRemove 146 | ) 147 | } 148 | 149 | override def preCondition(state: State): Boolean = { 150 | val keys = toAppend.map(_._1) ++ toRemove 151 | (version == state.version + 1) && (keys.toSet.size == keys.size) 152 | } 153 | 154 | override def postCondition(state: State, result: Try[Try[Unit]]): Prop = { 155 | result.flatten match { 156 | case Success(_) => true 157 | case Failure(e) => println(e.getMessage); false 158 | } 159 | } 160 | } 161 | 162 | //todo: check that created after a rollback version element not exist anymore and that created before the rollback and deleted after element exists 163 | class Rollback(version: Version) extends Command { 164 | type Result = Try[Unit] 165 | 166 | override def run(sut: Store): Try[Unit] = { 167 | Try(sut.rollback(ByteArrayWrapper.fromLong(version))) 168 | } 169 | 170 | override def nextState(state: State): State = { 171 | val ap = state.appendsIndex(version) 172 | val rp = state.removalsIndex(version) 173 | State(version, state.appendsIndex.filterKeys(_ <= version), state.removalsIndex.filterKeys(_ <= version), state.appended.take(ap), state.removed.take(rp)) 174 | } 175 | 176 | override def preCondition(state: State): Boolean = state.version > version 177 | 178 | override def postCondition(state: State, result: Try[Try[Unit]]): Prop = { 179 | val res = result.flatten.isSuccess 180 | if (!res) println("rollback failed: " + result.flatten) 181 | res 182 | } 183 | } 184 | 185 | class GetExisting(key: ByteArrayWrapper) extends Command { 186 | override type Result = Option[ByteArrayWrapper] 187 | 188 | override def run(sut: Store): Option[ByteArrayWrapper] = sut.get(key) 189 | 190 | override def nextState(state: State): State = state 191 | 192 | override def preCondition(state: State): Boolean = true 193 | 194 | override def postCondition(state: State, result: Try[Option[ByteArrayWrapper]]): Prop = { 195 | val v = state.appended.find { case (k, _) => key == k }.get._2 196 | val res = result.toOption.flatten.contains(v) 197 | if (!res) println(s"key not found: $key") 198 | res 199 | } 200 | } 201 | 202 | class GetRemoved(key: ByteArrayWrapper) extends Command { 203 | override type Result = Option[ByteArrayWrapper] 204 | 205 | override def run(sut: Store): Option[ByteArrayWrapper] = sut.get(key) 206 | 207 | override def nextState(state: State): State = state 208 | 209 | override def preCondition(state: State): Boolean = true 210 | 211 | override def postCondition(state: State, result: Try[Option[ByteArrayWrapper]]): Prop = { 212 | state.removed.contains(key) && result.map(_.isEmpty).getOrElse(false) 213 | } 214 | } 215 | 216 | object CleanUp extends UnitCommand { 217 | override def postCondition(state: State, success: Boolean): Prop = success 218 | 219 | override def run(sut: Store): Unit = { 220 | sut.clean(1000) 221 | // sut.taskCleanup() 222 | // sut.verify() 223 | } 224 | 225 | override def nextState(state: State): State = state 226 | 227 | override def preCondition(state: State): Boolean = true 228 | } 229 | 230 | } -------------------------------------------------------------------------------- /src/test/scala/io/iohk/iodb/LogStoreTest.scala: -------------------------------------------------------------------------------- 1 | package io.iohk.iodb 2 | 3 | import java.io.FileOutputStream 4 | 5 | import io.iohk.iodb.Store._ 6 | import io.iohk.iodb.TestUtils._ 7 | import org.junit.Assert._ 8 | import org.junit.Test 9 | 10 | import scala.collection.JavaConverters._ 11 | import scala.collection.mutable 12 | import scala.util.Random 13 | import org.scalatest.Matchers._ 14 | 15 | class LogStoreTest extends StoreTest { 16 | 17 | override def open(keySize: Int) = new LogStore(dir = dir, keySize = keySize) 18 | 19 | @Test def binarySearch() { 20 | val store = new LogStore(dir = dir, keySize = 32) 21 | 22 | //random testing 23 | val r = new Random() 24 | val data = (0 until 1000).map { i => 25 | val key = randomA(store.keySize) 26 | val valSize = 10 + r.nextInt(100) 27 | val value = randomA(valSize) 28 | (key, value) 29 | }.sortBy(_._1) 30 | 31 | //serialize update entry 32 | val b = store.serializeUpdate(Store.tombstone, data, true, prevFileNumber = 0, prevFileOffset = 0) 33 | 34 | //write to file 35 | val f = tempFile() 36 | val fout = new FileOutputStream(f) 37 | val foffset = 10000 38 | fout.write(new Array[Byte](foffset)) 39 | fout.write(b) 40 | fout.close() 41 | 42 | //try to read all values 43 | val fa = store.fileChannelOpen(f) 44 | for ((key, value) <- data) { 45 | val value2 = store.fileGetValue(fa, key, store.keySize, foffset) 46 | assertEquals(value2, Some(value)) 47 | } 48 | 49 | //try non existent 50 | val nonExistentKey = randomA(32) 51 | assertEquals(null, store.fileGetValue(fa, nonExistentKey, store.keySize, foffset)) 52 | store.verify() 53 | store.close() 54 | } 55 | 56 | @Test def get2(): Unit = { 57 | val store = new LogStore(dir = dir, keySize = 8) 58 | 59 | val updated = mutable.HashMap[K, V]() 60 | val removed = mutable.HashSet[K]() 61 | for (i <- 0 until 10) { 62 | //generate random data 63 | var toUpdate = (0 until 10).map(a => (randomA(8), randomA(40))) 64 | var toRemove: List[K] = if (updated.isEmpty) Nil else updated.keys.take(2).toList 65 | toRemove.foreach(updated.remove(_)) 66 | 67 | //modify 68 | store.update(fromLong(i), toUpdate = toUpdate, toRemove = toRemove) 69 | 70 | removed ++= toRemove 71 | updated ++= toUpdate 72 | 73 | removed.foreach { k => 74 | assertEquals(store.get(k), None) 75 | } 76 | for ((k, v) <- updated) { 77 | assertEquals(store.get(k), Some(v)) 78 | } 79 | } 80 | 81 | //try to iterate over all items in store 82 | val updated2 = mutable.HashMap[K, V]() 83 | val removed2 = mutable.HashSet[K]() 84 | 85 | store.loadKeyValues(store.loadUpdateOffsets(stopAtMerge = true, stopAtDistribute = false), store.fileHandles.asScala.toMap, false).foreach { case (k, v) => 86 | if (v eq tombstone) 87 | removed2.add(k) 88 | else 89 | updated2.put(k, v) 90 | } 91 | 92 | //and compare result 93 | assertEquals(removed, removed2) 94 | assertEquals(updated, updated2) 95 | store.verify() 96 | store.close() 97 | } 98 | 99 | @Test def fileAccess_getAll(): Unit = { 100 | val store = new LogStore(dir = dir, keySize = 32) 101 | for (i <- 0 until 10) { 102 | //generate random data 103 | val toUpdate = (0 until 10).map(a => (randomA(32), randomA(40))).toMap 104 | val toRemove = (0 until 10).map(a => randomA(32)).toSet 105 | store.update(fromLong(i), toUpdate = toUpdate, toRemove = toRemove) 106 | 107 | //try to get all key/vals from last update 108 | val keyVals = store.fileReadKeyValues(store.fileHandles.firstEntry().getValue, store.getValidPos().offset, keySize = 32).toBuffer 109 | val toUpdate2 = keyVals.filterNot(_._2 eq tombstone).toMap 110 | val toRemove2 = keyVals.filter(_._2 eq tombstone).map(_._1).toSet 111 | assertEquals(toUpdate, toUpdate2) 112 | assertEquals(toRemove, toRemove2) 113 | } 114 | store.verify() 115 | store.close() 116 | } 117 | 118 | @Test def startNewFile(): Unit = { 119 | val store = new LogStore(dir = dir, keySize = 8) 120 | for (i <- 1L until 10) { 121 | store.update(fromLong(i), toUpdate = makeKeyVal(i, i), toRemove = Nil) 122 | assert(dir.listFiles().filter(_.length() > 0).size == i) 123 | store.startNewFile() 124 | } 125 | store.close() 126 | } 127 | 128 | @Test def offset_allias(): Unit = { 129 | var store = new LogStore(dir = dir, keySize = 8) 130 | 131 | def update(i: Long) = store.update(fromLong(i), toUpdate = makeKeyVal(i, i), toRemove = Nil) 132 | 133 | update(1) 134 | assert(store.eof.fileNum == 1) 135 | store.startNewFile() 136 | update(2) 137 | store.startNewFile() 138 | update(3) 139 | store.startNewFile() 140 | update(4) 141 | 142 | //skip over #2 143 | store.appendFileAlias(3, 0, 2, 0) 144 | 145 | def check() { 146 | assert(Map(FilePos(3, 0) -> FilePos(2, 0)).asJava == store.offsetAliases) 147 | assert(List(fromLong(1), fromLong(2), fromLong(4)) == store.rollbackVersions()) 148 | assert(Some(fromLong(1)) == store.get(fromLong(1))) 149 | assert(Some(fromLong(2)) == store.get(fromLong(2))) 150 | assert(None == store.get(fromLong(3))) 151 | assert(Some(fromLong(4)) == store.get(fromLong(4))) 152 | } 153 | 154 | val aliases = store.offsetAliases 155 | check() 156 | //reopen 157 | store.close() 158 | store = new LogStore(dir = dir, keySize = 8) 159 | assert(aliases == store.offsetAliases) 160 | check() 161 | } 162 | 163 | 164 | @Test def get_stops_at_merge(): Unit = { 165 | var store = new LogStore(dir = dir, keySize = 8) 166 | 167 | def update(i: Long) = store.update(fromLong(i), toUpdate = makeKeyVal(i, i), toRemove = Nil) 168 | 169 | update(1L) 170 | update(2L) 171 | //insert compacted entry 172 | val eof = store.eof 173 | val pos = store.getValidPos() 174 | val compactedData = store.serializeUpdate(fromLong(3L), data = makeKeyVal(3L, 3L), 175 | isMerged = true, prevFileNumber = pos.fileNum, prevFileOffset = pos.offset) 176 | store.append(compactedData) 177 | 178 | //and update positions 179 | store.setValidPos(eof) 180 | // store.eof = store.eof.copy(offset = store.eof.offset + compactedData.size) 181 | //add extra entry after compacted entry 182 | update(4L) 183 | 184 | //older entries should be ignored, latest entry is merged 185 | def check() { 186 | assert(None == store.get(fromLong(1L))) 187 | assert(None == store.get(fromLong(2L))) 188 | assert(Some(fromLong(3L)) == store.get(fromLong(3L))) 189 | assert(Some(fromLong(4L)) == store.get(fromLong(4L))) 190 | } 191 | 192 | check() 193 | store.close() 194 | store = new LogStore(dir = dir, keySize = 8) 195 | check() 196 | store.close() 197 | } 198 | 199 | @Test def readerIncrement(): Unit = { 200 | val store = new LogStore(dir = dir, keySize = 8) 201 | Utils.fileReaderIncrement(store.fileSemaphore, 22L) 202 | assert(store.fileSemaphore.get(22L) == 1L) 203 | Utils.fileReaderIncrement(store.fileSemaphore, 22L) 204 | assert(store.fileSemaphore.get(22L) == 2L) 205 | Utils.fileReaderDecrement(store.fileSemaphore, 22L) 206 | assert(store.fileSemaphore.get(22L) == 1L) 207 | Utils.fileReaderDecrement(store.fileSemaphore, 22L) 208 | assert(store.fileSemaphore.get(22L) == null) 209 | 210 | store.close() 211 | } 212 | 213 | 214 | @Test def loadOffsets_stops_at_merge(): Unit = { 215 | val store = new LogStore(dir = dir, keySize = 8) 216 | 217 | for (i <- 1 to 10) { 218 | val b = fromLong(i) 219 | store.update(versionID = b, toRemove = Nil, toUpdate = List((b, b))) 220 | } 221 | 222 | store.taskCompact() 223 | assert(1 == store.loadUpdateOffsets(stopAtMerge = true, stopAtDistribute = true).size) 224 | 225 | for (i <- 1 to 10) { 226 | val b = fromLong(i + 20) 227 | store.update(versionID = b, toRemove = Nil, toUpdate = List((b, b))) 228 | assert(i + 1 == store.loadUpdateOffsets(stopAtMerge = true, stopAtDistribute = true).size) 229 | } 230 | store.close() 231 | } 232 | 233 | @Test def file_handle_leaks(): Unit = { 234 | val store = open(keySize = 8) 235 | for (i <- 1 to 10) { 236 | val b = fromLong(i) 237 | store.update(versionID = b, toRemove = Nil, toUpdate = List((b, b))) 238 | } 239 | 240 | store.fileSemaphore shouldBe empty 241 | 242 | for (i <- 1 to 10) { 243 | val b = fromLong(i) 244 | store.get(b) shouldBe Some(b) 245 | } 246 | store.fileSemaphore shouldBe empty 247 | 248 | store.loadUpdateOffsets(stopAtMerge = true, stopAtDistribute = false) 249 | store.fileSemaphore shouldBe empty 250 | 251 | store.getAll { (k, v) => 252 | } 253 | store.fileSemaphore shouldBe empty 254 | 255 | store.taskCompact() 256 | store.fileSemaphore shouldBe empty 257 | 258 | store.clean(1) 259 | store.fileSemaphore shouldBe empty 260 | 261 | store.close() 262 | store.fileSemaphore shouldBe empty 263 | } 264 | 265 | @Test def reopen(): Unit ={ 266 | var store = open(keySize = 8) 267 | var i = 0 268 | while(i<1000) { 269 | for(j<-0 until 10) { 270 | i += 1 271 | val b = fromLong(i) 272 | store.update(versionID = b, toRemove = Nil, toUpdate = List((b, b))) 273 | if(Math.random()<0.1) 274 | store.taskCompact() 275 | } 276 | 277 | val validPos = store._validPos.get() 278 | val eof = store.eof 279 | 280 | store.close() 281 | store = open(keySize = 8) 282 | if(validPos.offset!=0) 283 | store._validPos.get() shouldBe validPos 284 | else 285 | validPos shouldBe new FilePos(store._validPos.get().fileNum+1, 0) 286 | 287 | eof shouldBe store.eof 288 | } 289 | } 290 | } -------------------------------------------------------------------------------- /doc/store_spec.md: -------------------------------------------------------------------------------- 1 | IODB Specification 2 | ====================== 3 | 4 | Features 5 | ------------ 6 | 7 | IODB is embedded storage engine designed for blockchain applications. 8 | It is inspired by [RocksDB](http://rocksdb.org). 9 | It provides ordered key-value store, it is similar to `SortedMap`. 10 | Its main advantage are snapshots with branching and fast rollbacks. 11 | 12 | Main features include: 13 | * Ordered key-value store 14 | * Written in Scala, functional interface 15 | * Multi-threaded background compaction 16 | * Very fast durable commits 17 | * Atomic updates with MVCC isolation and crash protection 18 | * Snapshots with branching and rollbacks 19 | * Log structured storage, old data are never overwritten for improved crash protection 20 | 21 | Overview 22 | ----------------------- 23 | 24 | 25 | * Key-value store is backed by Log-Structured Merge-Tree inspired by RocksDB and SSTables from Cassandra 26 | * Some general ideas about LSMTrees are in separate [blog posts](http://www.mapdb.org/blog/lsm_store_and_updates/) 27 | * Keys are ordered, store allows range queries 28 | 29 | * Modifications are appended to end of file 30 | * old data are never overwritten 31 | * Append-only files are more durable than traditional Write-Ahead-Log 32 | * Writes and commits are faster, only single fsync is needed 33 | 34 | * There is compaction background process, it merges changes and removes duplicate updates 35 | * Compared to similar databases (LevelDB, RocksDB) compaction does not consume too much memory 36 | * Compaction reads data sequentially and is cache friendly 37 | * Compaction process does not block reads or writes, store can be modified while compaction is running 38 | 39 | * Some features are designed for blockchain applications 40 | * Keys in store have high entropy, compression would not work and is not implemented 41 | * Version IDs (identifies snapshots) are variable sized `byte[]` (64bit longs would not be enough) 42 | * Versioning, branching and rollbacks are natively supported without performance overhead 43 | 44 | 45 | Dictionary 46 | ------------ 47 | Here is explanation of some terms used in this document: 48 | 49 | **Update** - Key updates, modifications and deletes are grouped into batches, all updates are executed in single atomic **Update** (synonym is a commit, transaction or snapshot) 50 | 51 | **Version ID** - Every Update has assigned VersionID . Is supplied by user on each Update, latter can be used for rollback, or to query old snapshot. 52 | 53 | **Update Entry** - Section of Log, it contains modifications performed in single Update 54 | 55 | **Merge Entry** - Section of Log, it contains merged content of multiple Update Operations. (result of all inserts and deletes) 56 | 57 | **Log** - File that contains multiple Update or Merge Entries. More updates can be appended to the end of file. 58 | Basic Log structure is described in this [blog post](http://www.mapdb.org/blog/lsm_store_and_updates/). 59 | 60 | **Journal** - One or more Log files. It stores most recent modifications. 61 | 62 | **Shard** - One or more Log files. It contains older data. There are multiple Shards, each contains keys from its interval. Shard intervals are not overlapping. 63 | 64 | **Link** - Points to Update, is composed from file number and file offset 65 | 66 | Log 67 | ---------------- 68 | 69 | * Log is sequence of Updates 70 | * Log 71 | * Update stores Keys in sorted table, binary search is used to find keys in Update 72 | * Deleted keys are represented by tombstones 73 | * Each Update Entry has a Link (file number and offset) to previous Update 74 | * Update Entries are usually organized in single linked list (newer Updates are after older)q 75 | * Rollback can create branched tree, where last Update Entry in file points somewhere at start of the file 76 | 77 | 78 | Data lifecycle 79 | ---------------------- 80 | 81 | * All modifications (updates and deletes) are inserted into Journal. 82 | 83 | * There is Distribute Task, it runs in background and distributes data from Journal into Shards. 84 | * This operation inserts new Update Entries into Shard Log File 85 | * After data are written to Shards, Journal content might get discarded if their Snapshots are expired 86 | 87 | * There is Shard Compaction Task, it runs in background and compacts Shards 88 | * It chooses most fragmented Shard (with most Update Entries) 89 | * It merges all Update Entries from given shard 90 | * It writes new Merge Entry into Shard Log 91 | * Older Update Entries might get discarded (if in separate file, and their Snapshots are expired) 92 | 93 | 94 | 95 | ![IODB](img/journal-shards.jpeg) 96 | 97 | ### Key search 98 | 99 | Key (or value) is searched in following way 100 | 101 | * Every Update Entry (or Compaction Entry) stores data in sorted order, so the binary search is used to find Key in single Update Entry 102 | 103 | * Search starts by traversing Journal from newest to older Update Entry 104 | * If the Key is found in Journal the search stops 105 | 106 | * Once Distribute Entry is found in Journal the Journal traversal stops, and search continues in Shard 107 | * Distribute Entry contains last valid Link (file number and offset) to Shard 108 | 109 | * Traversal continues in Shard until Merge Entry is found 110 | * If the Key was not found at this point, it does not exist (Merge Entry contains union of all older Updates) 111 | 112 | Distribute Task 113 | --------------------- 114 | 115 | * Distribute Task takes data from Journal and splits them into Shards 116 | 117 | * It runs in background thread 118 | * Journal can be updated while it is running, new Update Entries are added to end of Journal file 119 | 120 | * It merges content from multiple Update Entries from Journal into single sorted iterator 121 | * Does not have to load entire data set into memory 122 | 123 | * Once Distribute Task finishes, it updates Journal with *Distribute Entry*, this contains pointers to Shard Update Entries 124 | * Distribute Entry in Journal indicates that key search should continue in Shard 125 | 126 | * Once journal content is written into Shards, some Update Entries in Journal can be discarded. 127 | * But not if they are needed for rollback 128 | 129 | ### Concurrency 130 | 131 | Distribute task needs to run without blocking appends into Journal. 132 | In this case it uses read-only data. File is modified only at the end, but its content remains unchanged. 133 | 134 | 135 | Compaction 136 | ---------------- 137 | 138 | * Compaction merges several Update Entries into single Compaction Entry 139 | * It removes duplicates and saves space when single Key was modified several times 140 | * It removes deleted keys (tombstones) from Log and saves space 141 | 142 | * Compaction runs in background processes and does not block updates 143 | * IODB only performs Compaction on Shards, it does not block Distribute Task which updates Shards 144 | 145 | * IODB compaction tasks work on small data chunks 146 | * Each compaction task should finish within seconds 147 | * Small data chunks decrease disk space overhead 148 | * That improves concurrency 149 | * Long running tasks are problem in RocksDB and similar storage engines 150 | 151 | ![IODB](img/compaction.jpeg) 152 | 153 | * Compaction works following way 154 | * Compaction takes Entry Updates from Log 155 | * It merges their content into single sorted iterator using Merge Sort 156 | * This content is saved to Log as new Merge Entry 157 | * Link Alias is than inserted to Log, to make Merge Entry visible 158 | 159 | ### Concurrency 160 | 161 | Compaction needs to progress without blocking other writes. That means that Log must be writeable both for compaction and another updates. 162 | This is done by using two files and Link Alias: 163 | 164 | * Lets say there is file **A** with several Updates, which needs to be compacted 165 | * Compaction closes **A**, no more updates will be appended to it. 166 | * It opens two new append-only files: **B**, **C** 167 | * **B** is not part of Journal yet, links from **C** are pointing to **B** 168 | * Compaction will output its content into file **B** 169 | * **C** is part of Journal, new Updates are appended there 170 | * Once Compaction finishes it fsyncs file **B** and inserts Link Alias into **C** to replace content from **A** with **B** 171 | * At this point file **A** can be deleted, if its content is not needed for rollback 172 | 173 | Storage format 174 | ---------------- 175 | 176 | Storage format is described in [separate doc](store_format.md). 177 | 178 | 179 | Solutions for problems in older versions 180 | -------------------------------------- 181 | 182 | Older IODB designs had following problems: 183 | 184 | * It required too much memory, often it would fail with OutOfMemoryExceptions 185 | * It required too many file handles 186 | * It was not really concurrent, background operations would block reads and writes 187 | * It took long time to reopen the storage, because all log files needed to be replayed to find latest Updates. 188 | 189 | Here is how it is addressed in IODB 190 | 191 | ### Memory Usage 192 | * Older version kept HashMap of all Updates in form of `HashMap BYTE_ARRAY_COMPARATOR = (o1, o2) -> compare(o1, o2); 55 | 56 | public static int compare(byte[] o1, byte[] o2) { 57 | // if (o1 == o2) return 0; 58 | final int len = Math.min(o1.length, o2.length); 59 | for (int i = 0; i < len; i++) { 60 | int b1 = o1[i] & 0xFF; 61 | int b2 = o2[i] & 0xFF; 62 | if (b1 != b2) 63 | return b1 - b2; 64 | } 65 | return o1.length - o2.length; 66 | } 67 | 68 | 69 | public static int byteArrayHashCode(byte[] data) { 70 | //do not use Arrays.hashCode, it generates too many collisions (31 is too low) 71 | int h = 1; 72 | for (byte b : data) { 73 | h = h * (-1640531527) + b; 74 | } 75 | return h; 76 | } 77 | 78 | /** returns byte array, which is greater than any other array of given size */ 79 | public static byte[] greatest(int size){ 80 | byte[] ret = new byte[size]; 81 | for(int i=0;i 0) 132 | hi = mid - 1; 133 | else 134 | return mid; 135 | } 136 | return -1; 137 | } 138 | 139 | private static int unsafeCompare(long bufAddress, int mid, int keySize, long[] keyParsed) { 140 | bufAddress = bufAddress + mid * (keySize); 141 | int offset = -1; 142 | for (long keyPart : keyParsed) { 143 | long v = unsafeGetLong(bufAddress + offset) & 0xFFFFFFFFFFFFFFL; 144 | if (v < keyPart) 145 | return -1; 146 | else if (v > keyPart) 147 | return 1; 148 | offset = Math.min(keySize - 8, offset + 7); //TODO this does not work with small keys 149 | } 150 | return 0; 151 | } 152 | 153 | 154 | /** 155 | * Hack to unmap MappedByteBuffer. 156 | * Unmap is necessary on Windows, otherwise file is locked until JVM exits or BB is GCed. 157 | * There is no public JVM API to unmap buffer, so this tries to use SUN proprietary API for unmap. 158 | * Any error is silently ignored (for example SUN API does not exist on Android). 159 | */ 160 | protected static boolean unmap(ByteBuffer b) { 161 | if (!(b instanceof DirectBuffer)) 162 | return false; 163 | 164 | // need to dispose old direct buffer, see bug 165 | // http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=4724038 166 | DirectBuffer bb = (DirectBuffer) b; 167 | Cleaner c = bb.cleaner(); 168 | if (c != null) { 169 | c.clean(); 170 | return true; 171 | } 172 | Object attachment = bb.attachment(); 173 | return attachment != null && attachment instanceof DirectBuffer && unmap(b); 174 | 175 | } 176 | 177 | protected static File[] listFiles(File dir, String extension) { 178 | return dir.listFiles(pathname -> pathname.isFile() && pathname.getName().endsWith("." + extension)); 179 | } 180 | 181 | 182 | 183 | static void writeFully(FileChannel channel, long offset, ByteBuffer buf) throws IOException { 184 | int remaining = buf.limit() - buf.position(); 185 | 186 | while (remaining > 0) { 187 | int written = channel.write(buf, offset); 188 | if (written < 0) 189 | throw new EOFException(); 190 | remaining -= written; 191 | } 192 | } 193 | 194 | static void writeFully(FileChannel channel, long offset, File from) throws IOException { 195 | long fromLen = from.length(); 196 | FileChannel fromc = FileChannel.open(from.toPath(), StandardOpenOption.READ); 197 | 198 | long remaining = fromLen; 199 | 200 | while (remaining > 0) { 201 | long written = channel.transferFrom(fromc, offset+ fromLen-remaining, remaining); 202 | if (written < 0) 203 | throw new EOFException(); 204 | remaining -= written; 205 | } 206 | fromc.close(); 207 | } 208 | 209 | 210 | 211 | static void readFully(FileChannel channel, long offset, ByteBuffer buf) throws IOException { 212 | int remaining = buf.limit() - buf.position(); 213 | 214 | while (remaining > 0) { 215 | int read = channel.read(buf, offset); 216 | if (read < 0) 217 | throw new EOFException(); 218 | remaining -= read; 219 | } 220 | } 221 | 222 | public static int getInt(byte[] buf, int pos) { 223 | return 224 | (((int) buf[pos++]) << 24) | 225 | (((int) buf[pos++] & 0xFF) << 16) | 226 | (((int) buf[pos++] & 0xFF) << 8) | 227 | (((int) buf[pos] & 0xFF)); 228 | } 229 | 230 | public static void putInt(byte[] buf, int pos, int v) { 231 | buf[pos++] = (byte) (0xff & (v >> 24)); //TODO PERF is >>> faster here? Also invert 0xFF &? 232 | buf[pos++] = (byte) (0xff & (v >> 16)); 233 | buf[pos++] = (byte) (0xff & (v >> 8)); 234 | buf[pos] = (byte) (0xff & (v)); 235 | } 236 | 237 | 238 | public static long getLong(byte[] buf, int pos) { 239 | return 240 | ((((long) buf[pos++]) << 56) | 241 | (((long) buf[pos++] & 0xFF) << 48) | 242 | (((long) buf[pos++] & 0xFF) << 40) | 243 | (((long) buf[pos++] & 0xFF) << 32) | 244 | (((long) buf[pos++] & 0xFF) << 24) | 245 | (((long) buf[pos++] & 0xFF) << 16) | 246 | (((long) buf[pos++] & 0xFF) << 8) | 247 | (((long) buf[pos] & 0xFF))); 248 | 249 | } 250 | 251 | public static void putLong(byte[] buf, int pos, long v) { 252 | buf[pos++] = (byte) (0xff & (v >> 56)); 253 | buf[pos++] = (byte) (0xff & (v >> 48)); 254 | buf[pos++] = (byte) (0xff & (v >> 40)); 255 | buf[pos++] = (byte) (0xff & (v >> 32)); 256 | buf[pos++] = (byte) (0xff & (v >> 24)); 257 | buf[pos++] = (byte) (0xff & (v >> 16)); 258 | buf[pos++] = (byte) (0xff & (v >> 8)); 259 | buf[pos] = (byte) (0xff & (v)); 260 | } 261 | 262 | public static void putLong(byte[] buf, int pos, long v, int vSize) { 263 | for (int i = vSize - 1; i >= 0; i--) { 264 | buf[i + pos] = (byte) (0xff & v); 265 | v >>>= 8; 266 | } 267 | } 268 | 269 | private static final XXHash64 hash64 = XXHashFactory.fastestJavaInstance().hash64(); 270 | 271 | public static long checksum(byte[] data) { 272 | return checksum(data, 0, data.length); 273 | } 274 | 275 | 276 | public static long checksum(byte[] data, int startOffset, int size) { 277 | int seed = 0x3289989d; 278 | return hash64.hash(data, startOffset, size, seed); 279 | } 280 | 281 | public static void fileReaderIncrement(ConcurrentMap readers, Long fileNum) { 282 | readers.compute(fileNum, (key, value) -> value == null ? 1L : value + 1L); 283 | } 284 | 285 | 286 | public static void fileReaderDecrement(ConcurrentMap readers, Long fileNum) { 287 | readers.compute(fileNum, (key, value) -> { 288 | if (value == null) 289 | throw new IllegalMonitorStateException("file not locked: " + fileNum); 290 | if (value.longValue() == 1L) { 291 | return null; 292 | } else { 293 | return value - 1L; 294 | } 295 | }); 296 | } 297 | 298 | public static byte[] shardPrefix(int shardCount, int shardNum, int keySize) { 299 | assert (shardCount > shardNum); 300 | assert (shardCount > 0); 301 | assert (shardNum >= 0); 302 | assert (keySize >= 4); 303 | long d = (0xFFFFFFL * shardNum) / shardCount; 304 | assert (d < 0xFFFFFFL + 1); 305 | 306 | int d2 = (int) (d << 8); 307 | byte[] ret = new byte[keySize]; 308 | putInt(ret, 0, d2); 309 | 310 | return ret; 311 | } 312 | 313 | 314 | public static long longHash(long h) { 315 | return h * -7046029254386353131L; 316 | } 317 | 318 | 319 | } 320 | -------------------------------------------------------------------------------- /src/main/scala/io/iohk/iodb/QuickStore.scala: -------------------------------------------------------------------------------- 1 | package io.iohk.iodb 2 | 3 | import java.io._ 4 | import java.nio.ByteBuffer 5 | import java.nio.file.{Files, StandardOpenOption} 6 | import java.util.concurrent.locks.ReentrantReadWriteLock 7 | 8 | import com.sun.xml.internal.messaging.saaj.util.ByteOutputStream 9 | import io.iohk.iodb.Store.{K, V, VersionID} 10 | 11 | import scala.collection.mutable 12 | import scala.collection.mutable.ArrayBuffer 13 | 14 | /** 15 | * Keeps all data in-memory. Also uses log file to provide durability and rollbacks 16 | */ 17 | class QuickStore( 18 | val dir: File, 19 | val filePrefix: String = "quickStore", 20 | val keepVersions: Int = 0 21 | ) extends Store { 22 | 23 | protected val lock = new ReentrantReadWriteLock() 24 | protected val keyvals = new mutable.HashMap[K, V]() 25 | 26 | protected val path = new File(dir, filePrefix + "-1").toPath 27 | 28 | protected var versionID: VersionID = Store.tombstone 29 | 30 | { 31 | if (!path.toFile.exists()) 32 | path.toFile.createNewFile() 33 | 34 | //get all updates from file 35 | val updates = new mutable.HashMap[VersionID, QuickUpdate]() 36 | var lastVersionID: VersionID = null 37 | deserializeAllUpdates { u => 38 | if (!u.isRollbackMarker) 39 | updates.put(u.versionID, u) 40 | lastVersionID = u.versionID 41 | } 42 | //construct sequence of updates 43 | val seq = new mutable.ArrayBuffer[QuickUpdate] 44 | var counter = 0L 45 | while (lastVersionID != null) { 46 | //cyclic ref protection 47 | counter += 1 48 | if (counter > 1e9) 49 | throw new DataCorruptionException("too many versions, most likely cyclic ref") 50 | val lastUpdate = updates(lastVersionID) 51 | seq += lastUpdate 52 | lastVersionID = 53 | if (lastUpdate.prevVersionID eq Store.tombstone) null 54 | else lastUpdate.prevVersionID 55 | } 56 | 57 | versionID = seq.map(_.versionID).headOption.getOrElse(Store.tombstone) 58 | 59 | //replay updates to get current map 60 | replayChanges(seq.reverse) { (c: QuickChange, u: QuickUpdate) => 61 | if (c.newValue eq Store.tombstone) { 62 | keyvals.remove(c.key) 63 | } else { 64 | keyvals.update(c.key, c.newValue) 65 | } 66 | } 67 | } 68 | 69 | override def get(key: K): Option[V] = { 70 | lock.readLock().lock() 71 | try { 72 | return keyvals.get(key) 73 | } finally { 74 | lock.readLock().unlock() 75 | } 76 | } 77 | 78 | override def getAll(consumer: (K, V) => Unit): Unit = { 79 | lock.readLock().lock() 80 | try { 81 | for ((key, value) <- keyvals) { 82 | consumer(key, value) 83 | } 84 | } finally { 85 | lock.readLock().unlock() 86 | } 87 | } 88 | 89 | override def update(versionID: VersionID, toRemove: Iterable[K], toUpdate: Iterable[(K, V)]): Unit = { 90 | val changes = new ArrayBuffer[QuickChange]() 91 | for ((key, value) <- toUpdate) { 92 | changes += new QuickChange(key = key, 93 | oldValue = keyvals.getOrElse(key, Store.tombstone), 94 | newValue = value) 95 | } 96 | 97 | for ((key) <- toRemove) { 98 | changes += new QuickChange(key = key, 99 | oldValue = keyvals.getOrElse(key, Store.tombstone), 100 | newValue = Store.tombstone) 101 | } 102 | 103 | lock.writeLock().lock() 104 | try { 105 | 106 | val binaryUpdate = serializeUpdate(versionID = versionID, prevVersionID = this.versionID, isRollbackMarker = false, changes = changes) 107 | 108 | this.versionID = versionID 109 | val fout = Files.newOutputStream(path, 110 | StandardOpenOption.APPEND, StandardOpenOption.WRITE, 111 | StandardOpenOption.DSYNC) 112 | try { 113 | fout.write(binaryUpdate) 114 | } finally { 115 | fout.flush() 116 | fout.close() 117 | } 118 | 119 | 120 | for ((key, value) <- toUpdate) { 121 | keyvals.put(key, value) 122 | } 123 | for ((key) <- toRemove) { 124 | keyvals.remove(key) 125 | } 126 | } finally { 127 | lock.writeLock().unlock() 128 | } 129 | } 130 | 131 | protected[iodb] def serializeUpdate(versionID: VersionID, prevVersionID: VersionID, isRollbackMarker: Boolean, changes: Iterable[QuickChange]): Array[Byte] = { 132 | val out = new ByteOutputStream() 133 | val out2 = new DataOutputStream(out) 134 | //skip place for update size and checksum 135 | out2.writeLong(0L) //checksum 136 | out2.writeInt(0) //update size 137 | 138 | def write(b: ByteArrayWrapper): Unit = { 139 | if (b eq Store.tombstone) { 140 | out2.writeInt(-1) 141 | } else { 142 | out2.writeInt(b.data.size) 143 | out2.write(b.data) 144 | } 145 | } 146 | 147 | out2.writeBoolean(isRollbackMarker) 148 | write(versionID) 149 | write(prevVersionID) 150 | out2.writeInt(changes.size) 151 | 152 | changes.foreach { c => 153 | write(c.key) 154 | write(c.oldValue) 155 | write(c.newValue) 156 | } 157 | val b = out.getBytes 158 | val out3 = ByteBuffer.wrap(b) 159 | out3.putInt(8, b.size) 160 | out3.putLong(0, Utils.checksum(b)) 161 | return b 162 | } 163 | 164 | 165 | protected def readByteArray(in: DataInput): ByteArrayWrapper = { 166 | val size = in.readInt() 167 | if (size == -1) 168 | return Store.tombstone 169 | val r = new ByteArrayWrapper(size) 170 | in.readFully(r.data) 171 | return r 172 | } 173 | 174 | protected[iodb] def deserializeUpdate(in: DataInputStream, offset: Long, skipChecksum: Boolean = false): QuickUpdate = { 175 | 176 | val checksum = in.readLong() 177 | val length = in.readInt() 178 | 179 | //verify checksum, must read byte[] to do that 180 | val b = new Array[Byte](length) 181 | in.readFully(b, 12, length - 12) 182 | 183 | if (!skipChecksum) { 184 | Utils.putInt(b, 8, length) 185 | 186 | val calculatedChecksum = Utils.checksum(b) 187 | if (checksum != calculatedChecksum) 188 | throw new DataCorruptionException("wrong checksum") 189 | } 190 | 191 | val in2 = new DataInputStream(new ByteArrayInputStream(b)) 192 | in2.readLong() 193 | in2.readInt() 194 | 195 | val isRollbackMarker = in2.readBoolean() 196 | val versionID = readByteArray(in2) 197 | val prevVersionID = readByteArray(in2) 198 | 199 | val keyCount = in2.readInt() 200 | // val changes = 201 | // if(skipData) Nil 202 | // else (0 until keyCount).map(i => new QuickChange(read(), read(), read())).toBuffer 203 | return new QuickUpdate(offset = offset, length = length, 204 | versionID = versionID, prevVersionID = prevVersionID, 205 | isRollbackMarker = isRollbackMarker, 206 | changeCount = keyCount) 207 | } 208 | 209 | 210 | protected[iodb] def deserializeAllUpdates(consumer: (QuickUpdate) => Unit): Unit = { 211 | val fin = Files.newInputStream(path, StandardOpenOption.READ) 212 | var offset = 0L 213 | try { 214 | val din = new DataInputStream(fin) 215 | while (din.available() > 0) { 216 | val u = deserializeUpdate(in = din, offset = offset) 217 | consumer(u) 218 | offset += u.length 219 | } 220 | } finally { 221 | fin.close() 222 | } 223 | } 224 | 225 | protected def deserializeChange(in: DataInput) = 226 | new QuickChange( 227 | key = readByteArray(in), 228 | oldValue = readByteArray(in), 229 | newValue = readByteArray(in)) 230 | 231 | 232 | protected def replayChanges(updates: Iterable[QuickUpdate])(consumer: (QuickChange, QuickUpdate) => Unit): Unit = { 233 | val fin = new FileInputStream(path.toFile) 234 | try { 235 | for (update <- updates; if (update.changeCount > 0)) { 236 | //seek to offset where data are starting 237 | fin.getChannel.position(update.offset + 4 + 8 + 4 + 4 + 1 + 4 + update.versionID.size + update.prevVersionID.size) 238 | 239 | //create buffered stream, it can not be reused, `fin` will seek and buffer would become invalid 240 | val din = new DataInputStream(new BufferedInputStream(fin)) 241 | for (i <- 0L until update.changeCount) { 242 | val change = deserializeChange(din) 243 | consumer(change, update) 244 | } 245 | } 246 | } finally { 247 | fin.close() 248 | } 249 | } 250 | 251 | override def lastVersionID: Option[VersionID] = { 252 | lock.readLock().lock() 253 | try { 254 | val versionID2 = versionID 255 | return ( 256 | if ((versionID2 eq null) || (versionID2 eq Store.tombstone)) None 257 | else Some(versionID2) 258 | ) 259 | } finally { 260 | lock.readLock().unlock() 261 | } 262 | } 263 | 264 | override def rollback(versionID: VersionID): Unit = { 265 | lock.writeLock().lock() 266 | try { 267 | val updatesMap = mutable.HashMap[VersionID, QuickUpdate]() 268 | var lastUpdate: QuickUpdate = null 269 | deserializeAllUpdates { u => 270 | if (!u.isRollbackMarker) 271 | updatesMap.put(u.versionID, u) 272 | lastUpdate = u 273 | } 274 | 275 | if (!updatesMap.contains(versionID) || lastUpdate == null) 276 | throw new IllegalArgumentException("VersionID not found") 277 | 278 | 279 | //replace rollback marker, if is here 280 | if(lastUpdate!=null) 281 | lastUpdate = updatesMap.getOrElse(lastUpdate.versionID, lastUpdate) 282 | 283 | 284 | //TODO check for duplicate links? 285 | //create linked list of updates 286 | val updates = new ArrayBuffer[QuickUpdate]() 287 | var counter = 0L 288 | while (lastUpdate.versionID != versionID) { 289 | updates += lastUpdate 290 | lastUpdate = updatesMap(lastUpdate.prevVersionID) 291 | //cyclic ref protection 292 | counter += 1 293 | if (counter > 1e7) 294 | throw new DataCorruptionException("rollback over too many versions, most likely cyclic ref") 295 | } 296 | 297 | this.versionID = versionID 298 | 299 | //replay in reverse order (insert old values) 300 | replayChanges(updates) { (c: QuickChange, u: QuickUpdate) => 301 | if (c.oldValue eq Store.tombstone) 302 | keyvals.remove(c.key) 303 | else 304 | keyvals.put(c.key, c.oldValue) 305 | } 306 | 307 | //update file to mark rollback 308 | 309 | val binaryUpdate = serializeUpdate(versionID = versionID, prevVersionID = this.versionID, 310 | isRollbackMarker = true, changes = Nil) 311 | 312 | val fout = Files.newOutputStream(path, 313 | StandardOpenOption.APPEND, StandardOpenOption.WRITE, 314 | StandardOpenOption.DSYNC) 315 | try { 316 | fout.write(binaryUpdate) 317 | } finally { 318 | fout.flush() 319 | fout.close() 320 | } 321 | } finally { 322 | lock.writeLock().unlock() 323 | } 324 | } 325 | 326 | 327 | override def close(): Unit = {} 328 | 329 | override def clean(count: Int): Unit = {} 330 | 331 | override def rollbackVersions(): Iterable[VersionID] = { 332 | lock.readLock().lock() 333 | try { 334 | val updatesMap = mutable.HashMap[VersionID, QuickUpdate]() 335 | var lastUpdate: QuickUpdate = null 336 | deserializeAllUpdates { u => 337 | if (!u.isRollbackMarker) 338 | updatesMap.put(u.versionID, u) 339 | lastUpdate = u 340 | } 341 | 342 | //replace rollback marker, if is here 343 | if(lastUpdate!=null) 344 | lastUpdate = updatesMap.getOrElse(lastUpdate.versionID, lastUpdate) 345 | 346 | //create linked list of updates 347 | val updates = new ArrayBuffer[QuickUpdate]() 348 | var counter = 0L 349 | while (lastUpdate != null) { 350 | updates += lastUpdate 351 | lastUpdate = updatesMap.getOrElse(lastUpdate.prevVersionID, null) 352 | //cyclic ref protection 353 | counter += 1 354 | if (counter > 1e7) 355 | throw new DataCorruptionException("rollback over too many versions, most likely cyclic ref") 356 | } 357 | 358 | return updates.map(_.versionID).reverse 359 | } finally { 360 | lock.readLock().unlock() 361 | } 362 | } 363 | 364 | 365 | override def verify(): Unit = { 366 | 367 | } 368 | } 369 | 370 | 371 | protected case class QuickUpdate( 372 | offset: Long, 373 | length: Long, //number of bytes consumed by this update 374 | isRollbackMarker: Boolean, 375 | versionID: VersionID, 376 | prevVersionID: VersionID, 377 | changeCount: Long) 378 | 379 | protected case class QuickChange( 380 | key: K, 381 | oldValue: V, 382 | newValue: V) -------------------------------------------------------------------------------- /src/test/scala/io/iohk/iodb/StoreTest.scala: -------------------------------------------------------------------------------- 1 | package io.iohk.iodb 2 | 3 | import java.security.MessageDigest 4 | import java.util.concurrent.atomic.{AtomicInteger, AtomicLong} 5 | 6 | import io.iohk.iodb.Store.{K, V} 7 | import io.iohk.iodb.TestUtils._ 8 | import org.junit.Assert.assertEquals 9 | import org.junit.Test 10 | 11 | import scala.collection.mutable 12 | import scala.util.Random 13 | import org.scalatest._ 14 | import Matchers._ 15 | import io.iohk.iodb.smoke.RandomRollbackTest 16 | 17 | abstract class StoreTest extends TestWithTempDir { 18 | 19 | def open(keySize: Int = 32): Store 20 | 21 | 22 | def testReopen(): Unit = { 23 | var store = open(keySize = 8) 24 | store.update(fromLong(1L), toUpdate = (1L to 100L).map(i => (fromLong(i), fromLong(i))), toRemove = Nil) 25 | store.update(fromLong(2L), toUpdate = Nil, toRemove = (90L to 100L).map(fromLong)) 26 | 27 | def check(): Unit = { 28 | (1L to 89L).foreach(i => assert(Some(fromLong(i)) == store.get(fromLong(i)))) 29 | store.verify() 30 | } 31 | 32 | check() 33 | store.close() 34 | store = open(keySize = 8) 35 | 36 | check() 37 | 38 | store.close() 39 | } 40 | 41 | @Test def none_last_versionID(): Unit = { 42 | var s = open() 43 | assert(s.lastVersionID == None) 44 | s.close() 45 | s = open() 46 | assert(s.lastVersionID == None) 47 | s.close() 48 | } 49 | 50 | @Test def get() { 51 | val store = open(keySize = 32) 52 | 53 | //random testing 54 | val r = new Random() 55 | val data = (0 until 1000).map { i => 56 | val key = randomA(32) 57 | val valSize = 10 + r.nextInt(100) 58 | val value = randomA(valSize) 59 | (key, value) 60 | }.sortBy(_._1) 61 | 62 | //put 63 | store.update(versionID = fromLong(1L), toUpdate = data, toRemove = Nil) 64 | 65 | //try to read all values 66 | for ((key, value) <- data) { 67 | assertEquals(Some(value), store.get(key)) 68 | } 69 | //try non existent 70 | val nonExistentKey = randomA(32) 71 | assertEquals(None, store.get(nonExistentKey)) 72 | store.verify() 73 | store.close() 74 | } 75 | 76 | 77 | @Test def getMulti(): Unit = { 78 | val store = open(keySize = 8) 79 | for (i <- 1 to 1000) { 80 | val b = fromLong(i) 81 | store.update(versionID = b, toRemove = Nil, toUpdate = List((b, b))) 82 | } 83 | 84 | for (i <- 1 to 1000) { 85 | val b = fromLong(i) 86 | store.get(b) shouldBe Some(b) 87 | } 88 | store.close() 89 | } 90 | 91 | 92 | @Test def get_getAll(): Unit = { 93 | val store = open(keySize = 8) 94 | 95 | val updated = mutable.HashMap[K, V]() 96 | val removed = mutable.HashSet[K]() 97 | for (i <- 0 until 10) { 98 | //generate random data 99 | var toUpdate = (0 until 10).map(a => (randomA(8), randomA(40))) 100 | var toRemove: List[K] = if (updated.isEmpty) Nil else updated.keys.take(2).toList 101 | toRemove.foreach(updated.remove(_)) 102 | 103 | //modify 104 | store.update(fromLong(i), toUpdate = toUpdate, toRemove = toRemove) 105 | 106 | removed ++= toRemove 107 | updated ++= toUpdate 108 | 109 | removed.foreach { k => 110 | assertEquals(store.get(k), None) 111 | } 112 | for ((k, v) <- updated) { 113 | assertEquals(store.get(k), Some(v)) 114 | } 115 | } 116 | 117 | //try to iterate over all items in store 118 | val updated2 = mutable.HashMap[K, V]() 119 | 120 | for ((key, value) <- store.getAll()) { 121 | updated2.put(key, value) 122 | } 123 | updated.size shouldBe updated2.size 124 | assertEquals(updated, updated2) 125 | store.verify() 126 | store.close() 127 | } 128 | 129 | 130 | def makeKeyVal(key: Long, value: Long) = List((fromLong(key), fromLong(value))) 131 | 132 | @Test def getVersions(): Unit = { 133 | val store = open(keySize = 8) 134 | 135 | val versions = (0L until 100).map(fromLong).toBuffer 136 | val updates = makeKeyVal(1, 1) 137 | 138 | for (version <- versions) { 139 | store.update(versionID = version, toUpdate = updates, toRemove = Nil) 140 | assertEquals(Some(fromLong(1L)), store.get(fromLong(1L))) 141 | assertEquals(Some(version), store.lastVersionID) 142 | } 143 | val versions2 = store.rollbackVersions().toBuffer 144 | assertEquals(versions, versions2) 145 | store.verify() 146 | store.close() 147 | } 148 | 149 | @Test def ropen(): Unit = { 150 | var store = open(keySize = 8) 151 | 152 | store.update(versionID = fromLong(1), toUpdate = makeKeyVal(1, 1), toRemove = Nil) 153 | assertEquals(Some(fromLong(1)), store.lastVersionID) 154 | assertEquals(Some(fromLong(1)), store.get(fromLong(1))) 155 | 156 | store.verify() 157 | store.close() 158 | store = open(keySize = 8) 159 | assertEquals(Some(fromLong(1)), store.lastVersionID) 160 | assertEquals(Some(fromLong(1)), store.get(fromLong(1))) 161 | store.verify() 162 | store.close() 163 | } 164 | 165 | @Test def rollback(): Unit = { 166 | var store = open(keySize = 8) 167 | 168 | store.update(versionID = fromLong(1), toUpdate = makeKeyVal(1, 1), toRemove = Nil) 169 | store.update(versionID = fromLong(2), toUpdate = makeKeyVal(1, 2), toRemove = Nil) 170 | assertEquals(Some(fromLong(2)), store.lastVersionID) 171 | assertEquals(Some(fromLong(2)), store.get(fromLong(1))) 172 | store.rollback(fromLong(1)) 173 | assertEquals(Some(fromLong(1)), store.lastVersionID) 174 | assertEquals(Some(fromLong(1)), store.get(fromLong(1))) 175 | 176 | store.verify() 177 | //reopen, rollback should be preserved 178 | store.close() 179 | store = open(keySize = 8) 180 | assertEquals(Some(fromLong(1)), store.lastVersionID) 181 | assertEquals(Some(fromLong(1)), store.get(fromLong(1))) 182 | store.verify() 183 | store.close() 184 | } 185 | 186 | @Test def longRunningUpdates(): Unit = { 187 | val cycles = 100 + 100000 * TestUtils.longTest() 188 | val store = open(keySize = 8) 189 | for (c <- 0L until cycles) { 190 | val toUpdate = (0 until 100).map { k => (fromLong(k), fromLong(1)) } 191 | store.update(versionID = fromLong(c), toUpdate = toUpdate, toRemove = Nil) 192 | if (c % 2000 == 0) { 193 | store.clean(1000) 194 | } 195 | assert(TestUtils.dirSize(dir) < 1e8) 196 | } 197 | store.verify() 198 | store.close() 199 | } 200 | 201 | @Test def concurrent_updates(): Unit = { 202 | if (TestUtils.longTest() <= 0) 203 | return 204 | 205 | val store = open(keySize = 8) 206 | val threadCount = 8 207 | val versionID = new AtomicLong(0) 208 | val key = new AtomicLong(0) 209 | val count: Long = 1e6.toLong 210 | 211 | val exec = new ForkExecutor(1) 212 | for (i <- 0 until threadCount) { 213 | exec.execute { 214 | var newVersion = versionID.incrementAndGet() 215 | while (newVersion <= count) { 216 | val newKey = key.incrementAndGet() 217 | store.update(fromLong(newVersion), toUpdate = List((fromLong(newKey), fromLong(newKey))), toRemove = Nil) 218 | 219 | newVersion = versionID.incrementAndGet() 220 | } 221 | } 222 | } 223 | 224 | //wait for tasks to finish 225 | exec.finish() 226 | 227 | //ensure all keys are present 228 | val keys = (1L to count).map(fromLong(_)).toSet 229 | val keys2 = store.getAll().map(_._1).toSet 230 | assert(keys == keys2) 231 | 232 | val versions = store.rollbackVersions().toSet 233 | assert(keys == versions) 234 | store.verify() 235 | store.close() 236 | } 237 | 238 | @Test def concurrent_key_update(): Unit = { 239 | val threadCount = 1 240 | val duration = 100 * 1000; // in milliseconds 241 | 242 | val exec = new ForkExecutor(duration) 243 | val store: Store = open(keySize = 8) 244 | 245 | for (i <- (1 to threadCount)) { 246 | val key = TestUtils.fromLong(i) 247 | exec.execute { 248 | var counter = 1L 249 | val r = new Random() 250 | while (exec.keepRunning) { 251 | val versionID2 = TestUtils.fromLong(r.nextLong()) 252 | val value = TestUtils.fromLong(counter) 253 | store.update(versionID2, Nil, List((key, value))) 254 | for (a <- (1 until 100)) { //try more times, higher chance to catch merge/distribute in progress 255 | assert(store.get(key) == Some(value)) 256 | } 257 | 258 | counter += 1 259 | } 260 | } 261 | } 262 | //start cleanup thread 263 | exec.execute { 264 | while (exec.keepRunning) { 265 | store.clean(1) 266 | } 267 | } 268 | 269 | exec.finish() 270 | store.close() 271 | } 272 | 273 | 274 | @Test def `quick store's lastVersionId should be None right after creation` { 275 | val s = open() 276 | assert(s.lastVersionID == None) 277 | s.close() 278 | } 279 | 280 | @Test def `empty update rollback versions test` { 281 | emptyUpdateRollbackVersions(blockStorage = open()) 282 | } 283 | 284 | @Test def `consistent data after rollbacks test` { 285 | dataAfterRollbackTest(blockStorage = open()) 286 | } 287 | 288 | 289 | 290 | def dataAfterRollbackTest(blockStorage: Store): Unit = { 291 | 292 | val data1 = generateBytes(20) 293 | val data2 = generateBytes(20) 294 | val data3 = generateBytes(20) 295 | 296 | val block1 = BlockChanges(data1.head._1, Seq(), data1) 297 | val block2 = BlockChanges(data2.head._1, Seq(), data2) 298 | val block3 = BlockChanges(data3.head._1, Seq(), data3) 299 | 300 | blockStorage.update(block1.id, block1.toRemove, block1.toInsert) 301 | blockStorage.update(block2.id, block2.toRemove, block2.toInsert) 302 | blockStorage.update(block3.id, block3.toRemove, block3.toInsert) 303 | 304 | assert(blockStorage.lastVersionID == Some(block3.id)) 305 | 306 | def checkBlockExists(block: BlockChanges): Unit = block.toInsert.foreach{ case (k , v) => 307 | val valueOpt = blockStorage.get(k) 308 | assert(valueOpt.isDefined) 309 | assert(valueOpt.contains(v)) 310 | } 311 | 312 | def checkBlockNotExists(block: BlockChanges): Unit = block.toInsert.foreach{ case (k , _) => 313 | assert(blockStorage.get(k) == None) 314 | } 315 | 316 | checkBlockExists(block1) 317 | checkBlockExists(block2) 318 | checkBlockExists(block3) 319 | 320 | blockStorage.rollback(block2.id) 321 | 322 | checkBlockExists(block1) 323 | checkBlockExists(block2) 324 | checkBlockNotExists(block3) 325 | 326 | blockStorage.update(block3.id, block3.toRemove, block3.toInsert) 327 | 328 | checkBlockExists(block1) 329 | checkBlockExists(block2) 330 | checkBlockExists(block3) 331 | 332 | blockStorage.rollback(block1.id) 333 | 334 | checkBlockExists(block1) 335 | checkBlockNotExists(block2) 336 | checkBlockNotExists(block3) 337 | 338 | blockStorage.update(block2.id, block2.toRemove, block2.toInsert) 339 | blockStorage.update(block3.id, block3.toRemove, block3.toInsert) 340 | 341 | checkBlockExists(block1) 342 | checkBlockExists(block2) 343 | checkBlockExists(block3) 344 | blockStorage.close() 345 | } 346 | 347 | 348 | def emptyUpdateRollbackVersions(blockStorage: Store): Unit = { 349 | assert(blockStorage.rollbackVersions().size == 0) 350 | 351 | val version1 = ByteArrayWrapper("version1".getBytes) 352 | blockStorage.update(version1, Seq(), Seq()) 353 | assert(blockStorage.rollbackVersions().size == 1) 354 | 355 | val version2 = ByteArrayWrapper("version2".getBytes) 356 | blockStorage.update(version2, Seq(), Seq()) 357 | assert(blockStorage.rollbackVersions().size == 2) 358 | blockStorage.close() 359 | } 360 | 361 | case class BlockChanges(id: ByteArrayWrapper, 362 | toRemove: Seq[ByteArrayWrapper], 363 | toInsert: Seq[(ByteArrayWrapper, ByteArrayWrapper)]) 364 | 365 | def hash(b: Array[Byte]): Array[Byte] = MessageDigest.getInstance("SHA-256").digest(b) 366 | 367 | def randomBytes(): ByteArrayWrapper = ByteArrayWrapper(hash(Random.nextString(16).getBytes)) 368 | 369 | def generateBytes(howMany: Int): Seq[(ByteArrayWrapper, ByteArrayWrapper)] = { 370 | (0 until howMany).map(i => (randomBytes(), randomBytes())) 371 | } 372 | 373 | 374 | @Test def doubleRollbackTest: Unit ={ 375 | val s = open() 376 | val data = generateBytes(100) 377 | val block1 = BlockChanges(data.head._1, Seq(), data.take(50)) 378 | val block2 = BlockChanges(data(51)._1, data.map(_._1).take(20), data.slice(51, 61)) 379 | val block3 = BlockChanges(data(61)._1, data.map(_._1).slice(20, 30), data.slice(61, 71)) 380 | s.update(block1.id, block1.toRemove, block1.toInsert) 381 | s.update(block2.id, block2.toRemove, block2.toInsert) 382 | s.get(block2.id) shouldBe Some(data(51)._2) 383 | s.rollbackVersions() shouldBe List(block1.id, block2.id) 384 | s.lastVersionID.get shouldBe block2.id 385 | s.rollback(block1.id) 386 | s.get(block1.id) shouldBe Some(data.head._2) 387 | s.get(block2.id) shouldBe None 388 | 389 | s.lastVersionID.get shouldBe block1.id 390 | s.rollbackVersions() shouldBe List(block1.id) 391 | 392 | s.update(block3.id, block3.toRemove, block3.toInsert) 393 | s.get(block3.id) shouldBe Some(data(61)._2) 394 | s.lastVersionID.get shouldBe block3.id 395 | s.rollbackVersions() shouldBe List(block1.id, block3.id) 396 | 397 | s.rollback(block1.id) 398 | s.get(block1.id) shouldBe Some(data.head._2) 399 | s.get(block2.id) shouldBe None 400 | s.get(block3.id) shouldBe None 401 | s.lastVersionID.get shouldBe block1.id 402 | s.rollbackVersions() shouldBe List(block1.id) 403 | s.close() 404 | } 405 | 406 | @Test def test_random_rollback(): Unit ={ 407 | RandomRollbackTest.test(open()) 408 | } 409 | 410 | @Test def idle_store(): Unit ={ 411 | val s = open() 412 | Thread.sleep(10000) 413 | s.close() 414 | } 415 | 416 | } 417 | -------------------------------------------------------------------------------- /src/main/scala/io/iohk/iodb/ShardedStore.scala: -------------------------------------------------------------------------------- 1 | package io.iohk.iodb 2 | 3 | import java.io._ 4 | import java.util.concurrent.atomic.AtomicLong 5 | import java.util.concurrent.locks.ReentrantLock 6 | import java.util.concurrent._ 7 | 8 | import com.google.common.base.Strings 9 | import com.google.common.io.Closeables 10 | import io.iohk.iodb.Store.{K, V, VersionID} 11 | 12 | import scala.collection.JavaConverters._ 13 | import scala.collection.mutable.ArrayBuffer 14 | 15 | class ShardedStore( 16 | val dir: File, 17 | val keySize: Int = 32, 18 | val shardCount: Int = 20, 19 | //unlimited thread executor, but the threads will exit faster to prevent memory leak 20 | val executor: Executor = new ThreadPoolExecutor(0, Integer.MAX_VALUE, 1, TimeUnit.SECONDS, new SynchronousQueue[Runnable]()) 21 | ) extends Store { 22 | 23 | val journal = new LogStore(keySize = keySize, dir = dir, filePrefix = "journal", executor = null) 24 | 25 | 26 | val shards = new java.util.TreeMap[K, LogStore]() 27 | 28 | /** ensures that only single distribute task runs at a time */ 29 | private val distributeLock = new ReentrantLock() 30 | 31 | @volatile private var isClosed = false 32 | 33 | //initialize shards 34 | assert(shardCount > 0) 35 | for (i <- 0 until shardCount) { 36 | val key = ByteArrayWrapper(Utils.shardPrefix(shardCount, i, keySize)) 37 | val shard = new LogStore(keySize = keySize, dir = dir, filePrefix = "shard_" + i + "_", executor = null, compactEnabled = false) 38 | shards.put(key, shard) 39 | } 40 | 41 | if (executor != null) { 42 | def waitForStart(): Unit = { 43 | //wait initial time until store is initialized 44 | var count = 60 45 | while (count > 0 && !isClosed) { 46 | count -= 1 47 | Thread.sleep(1000) 48 | } 49 | } 50 | 51 | //start background compaction task 52 | executor.execute(runnable { 53 | waitForStart() 54 | //start loop 55 | while (!isClosed) { 56 | try { 57 | //find the most fragmented shard 58 | val shard: LogStore = shards.values().asScala 59 | .map { s => (s, s.unmergedUpdatesCounter.get()) } 60 | .toBuffer 61 | .sortBy { s: (LogStore, Long) => s._2 } 62 | .reverse.head._1 63 | if (shard.unmergedUpdatesCounter.get > 4) { 64 | shard.taskCompact() 65 | } else { 66 | Thread.sleep(100) 67 | } 68 | } catch { 69 | case e: Throwable => new ExecutionException("Background shard compaction task failed", e).printStackTrace() 70 | } 71 | } 72 | }) 73 | 74 | executor.execute(runnable { 75 | waitForStart() 76 | while (!isClosed) { 77 | try { 78 | if (journalUpdateCounter.incrementAndGet() > 10) { 79 | journalUpdateCounter.set(0) 80 | taskDistribute() 81 | } else { 82 | Thread.sleep(100) 83 | } 84 | } catch { 85 | case e: Throwable => new ExecutionException("Background distribution task failed", e).printStackTrace() 86 | } 87 | } 88 | }) 89 | } 90 | 91 | 92 | val journalUpdateCounter = new AtomicLong(0) 93 | 94 | override def get(key: K): Option[V] = { 95 | assert(key.size > 7) 96 | val v = journal.getDistribute(key) 97 | if (v._1.isDefined) 98 | return v._1 //value was found in journal 99 | if (v._2.isEmpty) 100 | return None //map of shards was not found 101 | 102 | val shardEntry = v._2.get 103 | 104 | val shardPos = shardEntry.shards.floorEntry(key).getValue 105 | val shard = shards.floorEntry(key).getValue 106 | 107 | return shard.get(key = key, pos = shardPos) 108 | } 109 | 110 | override def getAll(consumer: (K, V) => Unit): Unit = { 111 | //FIXME content is loaded to heap 112 | val (data, shardEntry) = journal.getAllDistribute() 113 | 114 | if (shardEntry != null) { 115 | shardEntry.shards.asScala.foreach { a => 116 | val shardKey = a._1 117 | val shardPos = a._2 118 | val shard = shards.get(shardKey) 119 | 120 | shard.getAll({ (k, v) => 121 | data.putIfAbsent(k, v) 122 | }, dropTombstone = true, startPos = shardPos) 123 | } 124 | } 125 | 126 | data.asScala.foreach { p => 127 | if (!(p._2 eq Store.tombstone)) 128 | consumer(p._1, p._2) 129 | } 130 | } 131 | 132 | override def clean(count: Int): Unit = { 133 | //TODO cleanup 134 | } 135 | 136 | override def lastVersionID: Option[VersionID] = { 137 | journal.lastVersionID 138 | } 139 | 140 | override def update(versionID: VersionID, toRemove: Iterable[K], toUpdate: Iterable[(K, V)]): Unit = { 141 | journal.update(versionID = versionID, toRemove = toRemove, toUpdate = toUpdate) 142 | } 143 | 144 | override def rollback(versionID: VersionID): Unit = { 145 | //TODO: there is a race condition between rolling back shards and updating journal 146 | 147 | distributeLock.lock() 148 | journal.appendLock.lock() 149 | try { 150 | journal.rollback(versionID) 151 | //find last distribute entry 152 | val offsets = journal.loadUpdateOffsets(stopAtMerge = false, stopAtDistribute = false) 153 | val distOffsets = offsets.filter(_.entryType == LogStore.headDistributeFinished) 154 | // println(distOffsets.toBuffer) 155 | //if there is some, load its version ID and rollback all shards 156 | if (!distOffsets.isEmpty) { 157 | val de = journal.loadDistributeEntry(distOffsets.head.pos) 158 | shards.asScala.foreach { a => 159 | val shardKey = a._1 160 | val shard = a._2 161 | val shardOffset = de.shards.get(shardKey) 162 | shard.rollbackToOffset(shardOffset) 163 | } 164 | } else { 165 | //there is no distr entry, so remove all content from shards 166 | shards.values().asScala.foreach { s => 167 | s.rollbackToZero() 168 | assert(s.getAll().isEmpty) 169 | assert(s.lastVersionID == None) 170 | } 171 | 172 | } 173 | } finally { 174 | journal.appendLock.unlock() 175 | distributeLock.unlock() 176 | } 177 | } 178 | 179 | override def close(): Unit = { 180 | distributeLock.lock() 181 | try { 182 | isClosed = true 183 | journal.close() 184 | shards.values().asScala.foreach(_.close()) 185 | } finally { 186 | distributeLock.unlock() 187 | } 188 | } 189 | 190 | override def rollbackVersions(): Iterable[VersionID] = { 191 | journal.rollbackVersions() 192 | } 193 | 194 | override def verify(): Unit = { 195 | journal.verify() 196 | shards.values().asScala.foreach(_.verify()) 197 | } 198 | 199 | 200 | def taskDistribute(): Unit = { 201 | distributeLock.lock() 202 | try { 203 | if (isClosed) 204 | return 205 | 206 | val (prev, pos) = journal.appendDistributePlaceholder() 207 | 208 | val offsets = journal.loadUpdateOffsets(stopAtMerge = true, stopAtDistribute = true, startPos = pos) 209 | if (offsets.isEmpty) 210 | return 211 | //lock all files for reading 212 | val files = offsets.map(o => (o.pos.fileNum, journal.fileLock(o.pos.fileNum))).toMap 213 | try { 214 | 215 | val offsets2 = offsets.filter(t => t.entryType == LogStore.headUpdate || t.entryType == LogStore.headMerge) 216 | if (offsets2.isEmpty) 217 | return 218 | val versionID = journal.loadVersionID(offsets2.head.pos) 219 | 220 | val dataset = journal 221 | .loadKeyValues(offsets2, files, dropTombstones = false) 222 | if (dataset.isEmpty) 223 | return 224 | 225 | val tmpFile = new File(dir, "distribute" + System.currentTimeMillis()) 226 | var out1 = new FileOutputStream(tmpFile) 227 | var out2 = new DataOutputStream(new BufferedOutputStream(out1)) 228 | assert(tmpFile.exists()) 229 | 230 | val shardIter = shards.asScala.iterator 231 | var curr: (K, LogStore) = shardIter.next() 232 | 233 | val distributeEntryContent = new ArrayBuffer[(K, FilePos)] 234 | 235 | def appendPair(next: (K, V)) { 236 | def isTomb = next._2 eq Store.tombstone 237 | 238 | out2.writeInt(next._1.size) 239 | out2.writeInt(if (isTomb) -1 else next._2.size) 240 | out2.write(next._1.data) 241 | if (!isTomb) 242 | out2.write(next._2.data) 243 | } 244 | 245 | def flushShard(nextShardKey: K): Unit = { 246 | var next: (K, V) = null 247 | //collect keys, until next shard is reached 248 | do { 249 | next = if (dataset.hasNext) dataset.next() else null 250 | if (next != null && (nextShardKey == null || next._1 < nextShardKey)) { 251 | appendPair(next) 252 | } 253 | } while (next != null && (nextShardKey == null || next._1 < nextShardKey)) 254 | 255 | //reached end of data, or next shard, flush current shard 256 | out2.flush() 257 | out2.close() 258 | out1.close() 259 | 260 | object iter extends Iterable[(K, V)] { 261 | override def iterator(): Iterator[(K, V)] = { 262 | var in0 = new FileInputStream(tmpFile) 263 | val in = new DataInputStream(new BufferedInputStream(in0)) 264 | 265 | object i extends Iterator[(K, V)] { 266 | 267 | var _next: (K, V) = null 268 | 269 | override def hasNext = _next != null 270 | 271 | override def next(): (K, V) = { 272 | val ret = _next; 273 | if (_next == null) 274 | throw new NoSuchElementException() 275 | 276 | advance() 277 | return ret 278 | } 279 | 280 | def advance(): Unit = { 281 | if (in0 == null) { 282 | _next = null 283 | return 284 | } 285 | 286 | try { 287 | val keySize = in.readInt() 288 | assert(keySize >= 0) 289 | 290 | val valueSize = in.readInt() 291 | assert(valueSize >= -1) 292 | 293 | val key = new K(keySize) 294 | in.readFully(key.data) 295 | 296 | val value = if (valueSize == -1) Store.tombstone else new V(valueSize) 297 | if (valueSize > 0) 298 | in.readFully(value.data) 299 | 300 | _next = (key, value) 301 | } catch { 302 | case eof: EOFException => { 303 | //TODO better way to check for EOF 304 | in0.close() // release file resources 305 | in0 = null 306 | } 307 | } 308 | } 309 | } 310 | 311 | i.advance() 312 | 313 | return i 314 | } 315 | } 316 | //TODO versionID 317 | val shardOffset = curr._2.updateDistribute(versionID = new K(0), data = iter, triggerCompaction = false) 318 | 319 | distributeEntryContent += ((curr._1, shardOffset)) 320 | 321 | out1 = new FileOutputStream(tmpFile) 322 | out2 = new DataOutputStream(new BufferedOutputStream(out1)) 323 | assert(tmpFile.length() == 0) 324 | 325 | //next belongs to next shard 326 | if (next != null) { 327 | appendPair(next) 328 | } 329 | } 330 | 331 | //iterate over shards 332 | while (shardIter.hasNext) { 333 | val next = shardIter.next() 334 | flushShard(next._1) 335 | curr = next 336 | } 337 | //update last shard 338 | flushShard(null) 339 | 340 | out1.close() 341 | tmpFile.delete() 342 | 343 | //append distribute entry into journal 344 | val journalOffset = offsets.head.pos 345 | // insert distribute entry into journal 346 | val pos2: FilePos = journal.appendDistributeEntry(journalPos = pos, 347 | prevPos = prev, versionID = versionID, shards = distributeEntryContent) 348 | // insert alias, so , so it points to prev 349 | journal.appendFileAlias(pos.fileNum, pos.offset, pos2.fileNum, pos2.offset, updateEOF = true) 350 | 351 | journal.unmergedUpdatesCounter.set(0) 352 | } finally { 353 | for ((fileNum, fileHandle) <- files) { 354 | if (fileHandle != null) 355 | journal.fileUnlock(fileNum) 356 | } 357 | } 358 | } finally { 359 | distributeLock.unlock() 360 | } 361 | } 362 | 363 | 364 | def printDirContent(dir: File = dir, out: PrintStream = System.out): Unit = { 365 | if (!dir.exists() || !dir.isDirectory) { 366 | out.println("Not a directory: " + dir) 367 | return 368 | } 369 | 370 | 371 | def printE(name: String, e: Any): Unit = { 372 | out.println(Strings.padStart(name, 15, ' ') + " = " + e) 373 | } 374 | 375 | // loop over files in dir 376 | for (f <- dir.listFiles(); if (f.isFile)) { 377 | try { 378 | out.println("") 379 | out.println("=== " + f.getName + " ===") 380 | val fileLength = f.length() 381 | printE("file length", fileLength) 382 | 383 | // loop over log entries in file 384 | val r = new RandomAccessFile(f, "r") 385 | try { 386 | 387 | while (r.getFilePointer < fileLength) { 388 | out.println("----------------") 389 | 390 | printE("Entry offset", r.getFilePointer) 391 | val entrySize = r.readInt() 392 | val entryEndOffset = r.getFilePointer - 4 + entrySize 393 | printE("size", entrySize) 394 | val head = r.readByte() 395 | 396 | 397 | def printLink(name: String): Unit = { 398 | printE(name, r.readLong() + ":" + r.readLong()) 399 | } 400 | 401 | def printPrevLink(): Unit = { 402 | printLink("prev") 403 | } 404 | 405 | head match { 406 | case LogStore.headUpdate => { 407 | printE("head", head + " - Update") 408 | printPrevLink() 409 | val keyCount = r.readInt() 410 | val keySize = r.readInt() 411 | printE("key count", keyCount) 412 | printE("key size", keySize) 413 | } 414 | 415 | case LogStore.headMerge => { 416 | printE("head", head + " - Merge") 417 | printPrevLink() 418 | 419 | val keyCount = r.readInt() 420 | val keySize = r.readInt() 421 | printE("key count", keyCount) 422 | printE("key size", keySize) 423 | } 424 | 425 | case LogStore.headDistributePlaceholder => { 426 | printE("head", head + " - Distribute Placeholder") 427 | printPrevLink() 428 | } 429 | 430 | case LogStore.headDistributeFinished => { 431 | printE("head", head + " - Distribute Finished") 432 | printPrevLink() 433 | printLink("journal") 434 | val shardCount = r.readInt() 435 | 436 | for (i <- 0 until shardCount) { 437 | out.println("") 438 | printE("shard num", i) 439 | printLink("shard") 440 | val key = new K(keySize) 441 | r.readFully(key.data) 442 | printE("shard key", key) 443 | } 444 | } 445 | 446 | case LogStore.headAlias => { 447 | printE("head", head + " - Alias") 448 | printPrevLink() 449 | printLink("old") 450 | printLink("new") 451 | } 452 | 453 | case _ => { 454 | out.println(" !!! UNKNOWN HEADER !!!") 455 | } 456 | 457 | } 458 | 459 | // seek to end of entry 460 | r.seek(entryEndOffset) 461 | 462 | } 463 | } finally { 464 | Closeables.close(r, true) 465 | } 466 | 467 | 468 | } catch { 469 | case e: Exception => e.printStackTrace(out) 470 | } 471 | } 472 | 473 | 474 | } 475 | 476 | } 477 | -------------------------------------------------------------------------------- /doc/design_spec.md: -------------------------------------------------------------------------------- 1 | IODB design specification 2 | ============================ 3 | 4 | This document outlines design and implementation of IODB database at 0.4 release. 5 | 6 | 7 | 8 | 9 | Basic terms 10 | ----------- 11 | 12 | This spec assumes basic knowledge of log structured stores. 13 | Here are some basic terms 14 | 15 | ### Log 16 | - Log is sequence of updates organized by time 17 | - This sequence might not follow actual organization of data, it can be reorganized by rollback or compaction 18 | - Log can be spread over multiple files. 19 | - Each Log Entry contains: 20 | - list of update keys 21 | - list of deleted keys (see tombstone) 22 | - link to previous update 23 | 24 | - Key-Value pair is found by traversing Log until key is found 25 | - rollback or Compaction can reorganize sequence of updates 26 | - search follows link to previous update in Log Entry, rather than sequentially traversing log file in reverse order 27 | 28 | 29 | ![Log before merge](img/spec/log-before-merge.jpg) 30 | 31 | 32 | 33 | ### State of store 34 | - content of store for given VersionID (or most recent update) 35 | - all key-value pairs in store 36 | 37 | ### Tombstone 38 | 39 | - Already inserted data are immutable. It is not possible to delete keys directly from store. 40 | - Deleted keys are indicated by **tombstone** marker 41 | - it is special type of value 42 | - in binary storage is indicated by value with length `-1` 43 | - Compaction eventually removes tombstones from store and reclaims the disk space 44 | 45 | 46 | ### VersionID 47 | - Each Update takes VersionID as a parameter 48 | - Each Update creates snapshot of data 49 | - This snapshot is identified (and can be reverted to) by `byte[]` identifier 50 | 51 | ### Binary Search 52 | 53 | - Keys in each update entry are sorted 54 | - To find a key binary search is used 55 | - It compares `byte[]` (or `ByteArrayWrapper`) with content of file 56 | - All keys have equal size, that simplifies the search significantly 57 | - 0.4 version uses `FileChannel` which is very slow 58 | - memory mapped file can speedup binary search 10x 59 | - unsafe file access is even faster 60 | 61 | 62 | ### Merge iterator 63 | 64 | - state of store (all key-value pairs) is reconstructed by replaying all updates from start to end 65 | - easiest way is to traverse log and insert key-value pairs into in-memory `HashMap` 66 | - that consumes too much memory 67 | 68 | - IODB uses Merge iterator (lazy-N-way merge) instead 69 | 70 | - Inside each Update Entry (such as V1, V2...), keys are stored in sorted order. 71 | 72 | - Compaction reads content of all Updates in parallel, and produces the result by comparing keys from all updates. 73 | 74 | - The time complexity is `O(N*log(U))`, where `N` is the total number of keys in all Updates and `U` is the number of Updates (versions) in merge. 75 | 76 | - Compaction is streaming data; it never loads all keys into memory. But it needs to store `U` keys in memory for comparison. 77 | 78 | - If `U` is too large, the compaction process can be performed in stages. 79 | 80 | - No random IO, as all reads are sequential and all writes are append-only. 81 | 82 | 83 | ![Merge iterator](img/spec/merge-iterator.jpg) 84 | 85 | 86 | ### Merge in log 87 | 88 | As the number of updates grows and the linked-list gets longer, Find Key operation will become slow 89 | Also obsolete versions of keys are causing space overhead. 90 | 91 | Both problems are solved by merging older updates into a single update. 92 | The merge is typically performed by a compaction process, 93 | which runs on the background in a separate thread. 94 | 95 | - Merge process takes N Updates 96 | - from most recent 97 | - until previous Merge Entry or start of the log 98 | - It produces Merge Iterator over this data set 99 | - It inserts current state of store into Log Entry 100 | 101 | - 0.4 serializes Merge Entry into `byte[]` using `ByteArrayOutputStream` 102 | - it can run out of memory 103 | - temporary file should be used instead 104 | 105 | ![Log before merge](img/spec/log-before-merge.jpg) 106 | 107 | 108 | 109 | ![Log after merge](img/spec/log-after-merge.jpg) 110 | 111 | 112 | ### Shards 113 | 114 | - If store contains too many Key-Value pairs, Merge or binary search becomes slow 115 | - So the store is split into Shards, each Shard is compacted and managed separately 116 | - Newer data are stored in single log (Journal) latter moved into Shards 117 | 118 | 119 | Data lifecycle 120 | ------------------ 121 | 122 | IODB moves data around to ensure good performance 123 | 124 | ### Update 125 | 126 | Modifications (updated key-value pairs and keys to delete) are inserted in 127 | batches (here referred as updates). 128 | Each Update is identified by `VersionID` and creates new snapshot which 129 | can be rolled back to. 130 | 131 | Over time data (key-value pairs) move following way: 132 | 133 | - journal contains most recent updates 134 | - over time journal becomes too long, so Distribute Task is triggered 135 | - at start Distribute Task inserts Distribute Placeholder entry into Journal 136 | - Distribute Task moves data from Journal into Shards 137 | - after Distribute Task finishes, it puts file positions in Shards into Journal 138 | - over time (after some Distribute Tasks are finished) some Shards become too long, this triggers Shard Merge task 139 | - Shard Merge tasks select longest Shard and merges content from multiple Update Entries into single Merge Entry 140 | 141 | 142 | ![Data flow](img/spec/data-flow.jpg) 143 | 144 | 145 | ### find key (get) 146 | 147 | `Store.get()` returns value associated with the key, or `None` if key is not found 148 | 149 | - search traverses journal, until Distribute Placeholder is found (or key is found) 150 | - from Distribute Entry it takes position in Shard 151 | - search continues by traversing Shards from given position 152 | - search finishes when 153 | - key is found 154 | - Merge Entry is found in Shard 155 | - end of Shard is reached 156 | 157 | 158 | ![find key](img/spec/find-key.jpg) 159 | 160 | 161 | ### get all 162 | 163 | `Store.getAll()` returns content of store (all key-value pairs) for given `VersionID` (or msot recent version). 164 | 165 | Store content is spread over journal, shards, multiple files. 166 | So the dataset is produced by iterating over multiple files, and using lazy-N-way merge iterator: 167 | 168 | - traverse Journal until Distribute Placeholder is found 169 | - take all data in Journal from VersionID until Distribute Placeholder 170 | - merge them into single data set 171 | - store it in temporary file (is currently done in-memory and that causes out-of-memory exceptions) 172 | - this is referred here as Journal content 173 | - iterate over content of shards and merge them with Journal content 174 | - Shards are not overlapping 175 | - Journal content is sorted 176 | - Shard and Journal content is merged on single iteration 177 | 178 | ![get all](img/spec/get-all.jpg) 179 | 180 | 181 | Rollback 182 | ---------- 183 | 184 | Rollback reverts store into state at given VersionID. It discards data inserted after 185 | given VersionID. 186 | 187 | Log is chain of updates. It is not possible to overwrite new updates during rollback. 188 | So the new update (Offset Alias Log entry) is written into Journal. 189 | It instruct search and other operation to skip irrelevant data while traversing the log. 190 | 191 | Rollback is performed in following way 192 | - traverse Journal until matching VersionID is found 193 | - continue Journal traversal until Distribute Entry is found 194 | - this is first distribute entry after VersionID 195 | - we need Shard positions for given VersionID 196 | - insert Offset Alias Log Entry into journal 197 | - rollback Shards into state found in Distribe Entry 198 | 199 | 200 | ![Log after rollback](img/spec/log-after-rollback.jpg) 201 | 202 | 203 | Shards 204 | ------ 205 | 206 | - binary search on large tables is slow, also compaction on large tables is slow 207 | - only most recent data are stored in Journal 208 | - older data are distributed into shards 209 | - there is Distribute Task, it takes data from Journal and distributes them into Shards 210 | 211 | 212 | ### Shard Splitting 213 | 214 | - ideally Shards should be created dynamically 215 | - small or empty store starts with single shards 216 | - if Shard becomes too big, it is sliced into several smaller Shards 217 | - if Shard becomes too small, it is merged into its neighbours 218 | 219 | - dynamic Shard allocation has following advantages 220 | - less configuration, number of shard scales with store size 221 | - Shard Boundaries are self-balancing 222 | - if Key Space is sliced at constant intervals, and key distribution is not random (for example incremental keys), most keys can end in single interval 223 | - with dynamic Shard allocation new shards are created by slicing old Shards. 224 | 225 | - 0.4 release has **static sharding** 226 | - `ShardedStore` has `shardCount` parameter 227 | - this is hardcoded into store and can not be changed 228 | - key space is sliced at constant interval 229 | - dynamic Sharding was in 0.3 release, but caused concurrency issues 230 | on 231 | 232 | 233 | 234 | 235 | File naming 236 | ---------------------- 237 | 238 | - Log is composed of multiple files 239 | - IODB stores data in directory 240 | - this dir contains multiple logs (Journal and Shards) 241 | - it also contains temporary files 242 | 243 | - Each file has prefix, file ID and suffix 244 | 245 | - Prefix identifies type of file (journal, shard, temp) 246 | - It also assigns file into its Log (Journal or Shard) 247 | 248 | - File ID gives position of file within log 249 | - TODO File ID should have two parts; log file position and file version 250 | 251 | - Suffix is not used 252 | - TODO drop suffix or use it to identify temp files 253 | 254 | 255 | - Temporary files are used 256 | - To produce merged result during compaction and distribute (TODO currently uses `byte[]`) 257 | - To store large updates 258 | - To store result of `getAll()` operation (TODO currently loaded into in-memory `TreeMap`) 259 | 260 | 261 | 262 | File handles and File IO 263 | ------------------------ 264 | 265 | - IODB performs following IO operations on files 266 | - append at end of the file 267 | - sync; flush write cache on file (temp files are not flushed) 268 | - binary search read 269 | 270 | - IODB keeps number of files open 271 | - one writeable handle to append to Journal 272 | - one readonly handle for each file 273 | 274 | - File handle is limited resource 275 | - by default only 64K handles are allowed per process on Linux 276 | - exhausting file handles can cause JVM to crash (it can not open DLLs or JAR files) 277 | 278 | - IODB keeps file handles in `LogStore.fileHandles` map 279 | 280 | - File handle can be `FileChannel` or `MappedByteBuffer` depending on implementation 281 | 282 | 283 | - file delete 284 | - IODB needs to delete outdated files 285 | - files can not be deleted while it is opened for reading (race condition) 286 | - other threads could fail, while reading data from store 287 | - memory mapped file crashes (segfault) when accessing unmapped buffer 288 | - there is semaphore for each file 289 | - it protects file from deletion while it is read from 290 | - each read operation needs to lock files it will use 291 | - see `LogStore.fileSemaphore` for details 292 | 293 | 294 | Memory mapped files 295 | ------------------- 296 | 297 | - Current version (0.4) uses `FileChannel` to read files 298 | - that is very slow for binary search, 299 | - file caching does not work efficiently 300 | - `FileChannel` requires seeks and uses extra translation layer 301 | 302 | - better way is to use memory mapped files 303 | - it maps file into memory address space, reads do not require translation 304 | - binary search is very efficient (10x) faster 305 | - caching works very well 306 | 307 | 308 | - memory mapped (mmap) files have some downsides. Those are outlined [in this blog post](http://www.mapdb.org/blog/mmap_files_alloc_and_jvm_crash/) 309 | - JVM has no official way to close (unmap) mmap files 310 | - file handle is closed after GC, that can cause JVM process to run out of file handles 311 | - there is a hack to close mmap file forcibly 312 | - it is not supported in all JVMs (Java 9, Android etc are different) 313 | - after unmapping, mmap file can not be read from, it causes access to illegal address and JVM process crashes with SEGFAULT 314 | 315 | - memory mapped files can be speedup even more by replacing `byte[]` with primitive `long` for binary search 316 | 317 | 318 | 319 | 320 | Background Operations 321 | -------------------------- 322 | 323 | - IODB performs various maintenance tasks on background 324 | 325 | - background task are scheduled automatically in `Executor` 326 | - see `executor` constructor parameter at `ShardedStore` and `LogStore` 327 | - if you set `executor` param to null, background tasks are disabled 328 | - in that case user should perform compaction by calling background tasks directly 329 | - see `task*()` methods at `ShardedStore` and `LogStore` 330 | 331 | 332 | IODB performs following tasks 333 | 334 | 335 | ### Distribute Task 336 | 337 | - distribute task moves data from Journal into Shards 338 | - it is triggered when number of updates in journal is over limit (see `ShardedStore#journalUpdateCounter`) 339 | - update count is since last merge, not total count 340 | - TODO it should be triggered if total space of undistributed updates is over limit, not update count 341 | - can be triggered manually with `ShardedStore.taskDistribute()` 342 | 343 | Distribute tasks works in following way: 344 | 345 | - insert Distribute Placeholder Log Entry into journal 346 | - this is to prevent race conditions, journal might be updated while Distribute Task runs 347 | - iterate over Journal Updates, produce list of all updates since previous Distribute Placeholder 348 | - produce iterator over data in Journal Updates 349 | - slice content of iterator into shards 350 | - only single pass is needed, lazy-N-way merge produces sorted result 351 | - each Shard is updated with single update 352 | - Journal is updated with Distribute Log Entry 353 | - it contains pointers to latest updates in Shards 354 | - Distribute Placeholder is replaced by final version of Distribute Log Entry 355 | - Offset Alias is inserted into journal 356 | 357 | 358 | 359 | 360 | ### Compact Task 361 | 362 | - Distribute Task merges multiple Update Entries into single Merge Entry 363 | - it makes Shards faster and releases unused version data 364 | - it is triggered when number of unmerged updates in Shard is over limit (see `LogStore.umergedUpdatesCounter`) 365 | 366 | - can be triggered manually by `LogStore.taskCompact()` 367 | - TODO expose this in `ShardedStore` as well 368 | 369 | It works in following way: 370 | 371 | - `ShardedStore` runs background task 372 | - every N seconds it selects most fragmented Shard 373 | - if number of unmerged Update Entries (since last merge) is over limit, it calls `LogStore.taskCompact()` on this Shard 374 | - In LogStore it selects Most Recent Update 375 | - It traverses Update Entries until it reaches previous Merge Entry (or start of log) 376 | - It produces merge iterator over all updates in Shard 377 | - It inserts merged content of Update Entries as new Merge Log Entry into Shard 378 | - It replaces Most Recent Update with Merge Entry by using Offset Alias 379 | 380 | 381 | ### File Shrink Task 382 | 383 | This tasks replaces old file with new one, and reclaims space in file. It is not present in 0.4 release 384 | 385 | TODO current design (0.4) is not very effective to reclaim space released by rollback. 386 | 387 | TODO FIG shrink file 388 | 389 | 390 | 391 | Source code overview 392 | ------------------------ 393 | 394 | Some notes: 395 | 396 | - key, values and VersionIds in IODB are represented as `byte[]` 397 | - for practical purposes (hash,comparable) `ByteArrayWrapper` is used 398 | 399 | - Basic interface is `Store` 400 | - it is dictionary that maps key to value 401 | 402 | 403 | ### Quick Store 404 | - it stores all data in memory 405 | -uses sorted map 406 | - is used as a reference for writing unit tests (other stores should pass the same criteria as simple implementation does) 407 | - uses simple `ReadWriteLock` for thread safety 408 | - can store data on disk using log file 409 | - rollback method replays updates in reverse order 410 | - when opened it replays log file to reconstruct current state (all key-value pairs) into SortedMap 411 | 412 | 413 | ### Log Store 414 | 415 | - `LogStore` stores all data in multiple log files 416 | - it uses binary search to find keys 417 | - this class is used for Journal and Shard in `ShardedStore` 418 | - it has background operation that merges multiple Update Entries 419 | - merge operation uses lazy-N-way merge iterator, so it does not load all entries into heap 420 | 421 | 422 | - `serialize*()` methods convert updates and other modifications into their binary representation 423 | - `byte[]` is used right now 424 | - TODO use temporary files, large updates can cause OOEM exceptions 425 | 426 | - `LogStore#append` will take binary data and append it to end of the file 427 | 428 | #### concurrency 429 | 430 | - `LogStore` is thread safe, but does not have a global lock 431 | - it maintains most state in file 432 | 433 | - there is a race condition in file deletion 434 | - if file is deleted while other thread is reading it, it could cause JVM to crash (see memory mapped files) 435 | - to prevent that we use file semaphore 436 | - read operation must lock files it will use 437 | - delete operation is performed on background, and will not remove files that are locked for reading 438 | - see `fileSemaphore` and `fileToDelete` fields 439 | 440 | - there is a race condition in file append (expand log file) 441 | - multiple writers should be allowed at the same time, but there is only single file 442 | - ideally the content of update should be written into temporary file, and latter transferred into log using `FileChannel.transferFrom()` (avoids CPU cache) 443 | - in 0.4 release append lock is used, so LogStore does not handle concurrent updates 444 | 445 | - there is a race condition between `Store.get()` and `Store.update()` 446 | - `Store.get()` obtains last log file offset from atomic variable, and works with that 447 | - it can not see future updates 448 | 449 | #### Background tasks 450 | 451 | - `LogStore` has only one background task 452 | - it merges multiple Updates into single Merge entry 453 | - can be invoked manually with `LogStore.taskCompact()` 454 | 455 | ### Sharded Store 456 | 457 | `ShardedStore` is combination of multiple logs (`LogStore`). 458 | It is composed of Journal and Shards, all are represented by `LogStore`. 459 | 460 | - newer data are in Journal 461 | - data are moved into Shards by distribute task 462 | - Sharded Store controls compaction of Shards 463 | 464 | ### Concurrency 465 | 466 | 467 | - there is a race condition between rollback and distribute task 468 | - in 0.4 release rollback operation uses `distributeLock` so the distribute task and rollback can not run at the same time 469 | 470 | - there is a race condition between update and distribute task 471 | - it is solved by using two Distribute Log Entries 472 | - Distribute Task inserts Distribute Placeholder Log Entry into Journal 473 | - that is latter replaced by Distribute Log Entry 474 | 475 | 476 | 477 | 478 | 479 | File format overview 480 | ----------------------- 481 | 482 | Binary format is described in `store_format.md` 483 | 484 | In short log can contain following types of entries: 485 | 486 | - **update log entry** - Is inserted on each update 487 | - **merge log entry** - Is inserted by Compaction Task, merges content from multiple updates into single log entry 488 | - **distribute placeholder log entry** - Inserted into journal when distribute task starts. 489 | - **distribute log entry** - Inserted into journal after distribute task finishes. It contains pointers into shards. 490 | - **offset alias log entry** - inserted by Rollback or Compaction Task. It replaces Update Entry with new one. For example after compaction the most recent update is replaced by Compacted Entry. 491 | 492 | 493 | 494 | 495 | 496 | 497 | 498 | 499 | Unit tests 500 | ------------------- 501 | 502 | - IODB uses JUnit 4 runner to run unit tests 503 | - test written in ScalaTest should extend `org.scalatest.junit.JUnitSuite` 504 | 505 | 506 | #### io.iohk.iodb package 507 | - contains normal unit tests 508 | - there is abstract `StoreTest which tests general contract of `Store` interface 509 | - it has `QuickStoreRefTest`, `LogStoreTest` and `ShardedStoreTest` subclasses 510 | - most tests extends `TestWithTempDir` class that creates and deletes temporary directory 511 | 512 | ### bench package 513 | - contains benchmarks 514 | - those are not run automatically, but using `main` method 515 | - from command line you can trigger it using following command line `sbt 'test:run-main io.iohk.iodb.bench.ClassName'` 516 | 517 | ### prop package 518 | - properties testing 519 | - generates random data and tries to break store 520 | 521 | ### smoke 522 | - integration tests 523 | - generates random data and tries to break store 524 | - there is `M1Test` which is acceptance test for 0.4 release 525 | 526 | 527 | ### Long running tests 528 | - by default unit tests run only short time (10 minutes) to make development easier 529 | - however before release full test suite should run for several days to 530 | - find race condition 531 | - stress test store 532 | - test for memory and disk leaks 533 | - Long running test are trigged by `longTest=1` system property 534 | - from command line use `sbt test -DlongTest=1` to run acceptance --------------------------------------------------------------------------------