mirror of
https://github.com/Cuprate/cuprate.git
synced 2024-12-22 19:49:28 +00:00
Database (#35)
* rename `database` -> `old_database` Keeping it around for reference until new implementation is complete. * create new `database/` skeleton * add `DATABASE.md` design doc skeleton * move design doc to `database/README.md` * add rough code * `lib.rs` -> `gist.rs` * database: use workspace deps * workspace: include `database/` as member CI will now include this crate. * cargo fmt * database: `AGPL` -> `MIT` * readme: add `TODO`s * add base files * cargo.toml: add `heed` feature * fix clippy * lib.rs: add extremely pedantic lints * readme: add `# Backends` * cargo.toml: add `cfg-if` * add `backend/` structure * base `database.rs` * cargo.toml: add `borsh` * backend: add `DATABASE_BACKEND` * base `error.rs` * base `database.rs` * base `transaction.rs` * base `table.rs` * lib.rs: add imports * add `pod.rs` * pod: use `Read/Write`, add tests for all primitive numbers * pod: impl Pod for `Vec<u8>`, `[u8; N]` * pod: add docs, add `private::Sealed` * pod: `to_writer`, `from_reader` The new `as_bytes` + `from_bytes` now allows (de)serializing from bytes directly instead of going through Read/Write. Different array return sizes are abstracted away with `-> impl AsRef<[u8]>` * pod: impl Pod for `Box<[u8]>` * pod: return `Err` on incorrect length in `from_bytes()` * pod: docs * pod: impl Pod for `Arc<[u8]>` * readme: docs * database: add `create_table()`, `get_table()` * table: `Pod` bound * backend: move into directories * pod: add `into_bytes()` * heed: impl `BytesEncode`, `BytesDecode` * add `actor`, `service` features The thread/actor system used will be gated behind `actor`, and the `tower/tokio` integration will be gated behind `service`. * add `lib.rs` docs * service: add `service.rs` * service: add `reader.rs` * service: add `writer.rs` * service: add `request.rs` & `response.rs` * cargo.toml: add `crossbeam` * service: add/use `enum Request`, `enum Response` * service: basic signatures for thread-pools, `Writer` -> `Writers` * service: split `tower::Service<ReadRequest/WriteRequest>` * service: impl `tower::Service` for `Database(Reader|Writer)` * service: add `init()`, impl basic `Reader/Writer` pools * service: add example `Request`'s * service: add example `ReadRequest` handling * temporarily allow clippy lints * readme: update file structure * transaction: add `RoTx::get_range()` * service: module docs * cargo.toml: add `cuprate-helper` * service: scale readers/writers based on thread count * database: change lifetimes * heed: impl Database for `heed` * heed: add `ConcreteRoTx`, `ConcreteRwTx`, impl Tx traits * docs * service: `read.rs` docs * service: `write.rs` docs * service: request/response docs * service: use `OnceLock` in `init()`, add `db_read()`, `db_write()` * service: leak database into `&'static` * database: add `#[inline]`, `#[cold]` * service: `free.rs` docs, more `#[inline]` + `#[cold]` * service: add `shutdown()` * service: add `Request::Shutdown` * service: `shutdown()` docs * heed: hide concrete tx types * lib.rs: add terms * split `Env` <-> `Database` * cargo.toml: add `paste` * database: add `tables/` * impl `serde/borsh` where possible * tables: add `Tables`, add test docs * make db backend mutually exclusive to fix `--all-features` * tables: use `$()*` in `tables!()` * cargo.toml: add `sanakirja 1.4.0` * sanakirja: impl `Env` * sanakirja: impl `Database` * sanakirja: impl `Transaction` * table: temporarily fix `sanakirja` K/V bounds * table: fix `#[cfg]` * cargo.toml: fix deps * lib.rs: docs * service: docs * readme: add files, update `# Documentation`, add `# Layers` * readme: `src/` file purpose * readme: `src/service/` file purpose * readme: `src/backend/` file purpose * fix `Cargo.lock` merge conflict * database: remove `gist.rs` * add to `constants.rs` * add top `//! comments` for files/modules * constants: add sanity-check test * service: add `only_one_database` test in `free.rs` * service: add `tests.rs` * remove unneeded markers + imports * backend: fix `get_range()`'s trait `impl` return * env: add `create_tables_if_needed()`, don't return `Option<Db>` * sort imports by `CONTRIBUTING.md` rules oops sorry boog * add `monero.rs` * monero: docs * database: add missing `RoTx/RwTx` inputs * backend: add missing `RoTx/RwTx` inputs * `monero.rs` trait -> free functions in `ops/` * pod: make methods infallible * ci: add `rustup update` step * service: use `Arc` instead of leaking, remove `db_read/db_write` * service: use `InfallibleOneshotReceiver` for readers * service: shutdown on error, add todos * service: remove `Request` * service: combine `ReadResponse` and `WriteResponse` * service: use `InfallibleOneshotReceiver` for writer * service: only spawn 1 writer, don't allow cloning writer handle * table: add associated `const CONSTANT_SIZE` * add `key.rs` + `trait Key`, add bound to `Table` * fix typos
This commit is contained in:
parent
b7df133175
commit
331d4d3e7f
52 changed files with 3596 additions and 455 deletions
3
.github/workflows/ci.yml
vendored
3
.github/workflows/ci.yml
vendored
|
@ -88,6 +88,9 @@ jobs:
|
|||
if: matrix.os == 'windows-latest'
|
||||
run: rustup toolchain install stable-x86_64-pc-windows-gnu -c clippy && rustup set default-host x86_64-pc-windows-gnu && rustup default stable-x86_64-pc-windows-gnu
|
||||
|
||||
- name: Update Rust
|
||||
run: rustup update
|
||||
|
||||
- name: Clippy (fail on warnings)
|
||||
run: cargo clippy --workspace --all-targets --all-features -- -D warnings
|
||||
|
||||
|
|
298
Cargo.lock
generated
298
Cargo.lock
generated
|
@ -168,6 +168,15 @@ version = "1.6.0"
|
|||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "8c3c1a368f70d6cf7302d78f8f7093da241fb8e8807c05cc9e51a125895a6d5b"
|
||||
|
||||
[[package]]
|
||||
name = "bincode"
|
||||
version = "1.3.3"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "b1f45e9417d87227c7a56d22e471c6206462cba514c7590c09aff4cf6d1ddcad"
|
||||
dependencies = [
|
||||
"serde",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "bit-set"
|
||||
version = "0.5.3"
|
||||
|
@ -194,6 +203,9 @@ name = "bitflags"
|
|||
version = "2.4.2"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "ed570934406eb16438a4e976b1b4500774099c13b8cb96eec99f620f05090ddf"
|
||||
dependencies = [
|
||||
"serde",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "bitvec"
|
||||
|
@ -255,6 +267,12 @@ version = "3.14.0"
|
|||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "7f30e7476521f6f8af1a1c4c0b8cc94f0bee37d91763d0ca2665f299b6cd8aec"
|
||||
|
||||
[[package]]
|
||||
name = "bytemuck"
|
||||
version = "1.14.3"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "a2ef034f05691a48569bd920a96c81b9d91bbad1ab5ac7c4616c1f6ef36cb79f"
|
||||
|
||||
[[package]]
|
||||
name = "byteorder"
|
||||
version = "1.5.0"
|
||||
|
@ -290,11 +308,10 @@ dependencies = [
|
|||
|
||||
[[package]]
|
||||
name = "cc"
|
||||
version = "1.0.83"
|
||||
version = "1.0.85"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "f1174fb0b6ec23863f8b971027804a42614e347eafb0a95bf0b12cdae21fc4d0"
|
||||
checksum = "9b918671670962b48bc23753aef0c51d072dca6f52f01f800854ada6ddb7f7d3"
|
||||
dependencies = [
|
||||
"jobserver",
|
||||
"libc",
|
||||
]
|
||||
|
||||
|
@ -432,6 +449,19 @@ name = "crossbeam"
|
|||
version = "0.8.4"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "1137cd7e7fc0fb5d3c5a8678be38ec56e819125d8d7907411fe24ccb943faca8"
|
||||
dependencies = [
|
||||
"crossbeam-channel",
|
||||
"crossbeam-deque",
|
||||
"crossbeam-epoch",
|
||||
"crossbeam-queue",
|
||||
"crossbeam-utils",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "crossbeam-channel"
|
||||
version = "0.5.11"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "176dc175b78f56c0f321911d9c8eb2b77a78a4860b9c19db83835fea1a46649b"
|
||||
dependencies = [
|
||||
"crossbeam-utils",
|
||||
]
|
||||
|
@ -455,6 +485,15 @@ dependencies = [
|
|||
"crossbeam-utils",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "crossbeam-queue"
|
||||
version = "0.3.11"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "df0346b5d5e76ac2fe4e327c5fd1118d6be7c51dfb18f9b7922923f287471e35"
|
||||
dependencies = [
|
||||
"crossbeam-utils",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "crossbeam-utils"
|
||||
version = "0.8.19"
|
||||
|
@ -528,6 +567,23 @@ dependencies = [
|
|||
"tracing-subscriber",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "cuprate-database"
|
||||
version = "0.0.0"
|
||||
dependencies = [
|
||||
"borsh",
|
||||
"cfg-if",
|
||||
"crossbeam",
|
||||
"cuprate-helper",
|
||||
"heed",
|
||||
"paste",
|
||||
"sanakirja",
|
||||
"serde",
|
||||
"thiserror",
|
||||
"tokio",
|
||||
"tower",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "cuprate-helper"
|
||||
version = "0.1.0"
|
||||
|
@ -674,6 +730,15 @@ dependencies = [
|
|||
"zeroize",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "doxygen-rs"
|
||||
version = "0.4.2"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "415b6ec780d34dcf624666747194393603d0373b7141eef01d12ee58881507d9"
|
||||
dependencies = [
|
||||
"phf",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "either"
|
||||
version = "1.10.0"
|
||||
|
@ -770,7 +835,7 @@ checksum = "1ee447700ac8aa0b2f2bd7bc4462ad686ba06baa6727ac149a2d6277f0d240fd"
|
|||
dependencies = [
|
||||
"cfg-if",
|
||||
"libc",
|
||||
"redox_syscall",
|
||||
"redox_syscall 0.4.1",
|
||||
"windows-sys 0.52.0",
|
||||
]
|
||||
|
||||
|
@ -835,6 +900,16 @@ dependencies = [
|
|||
"percent-encoding",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "fs2"
|
||||
version = "0.4.3"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "9564fc758e15025b46aa6643b1b77d047d1a56a1aea6e01002ac0c7026876213"
|
||||
dependencies = [
|
||||
"libc",
|
||||
"winapi",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "funty"
|
||||
version = "2.0.0"
|
||||
|
@ -990,12 +1065,58 @@ dependencies = [
|
|||
"ahash",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "hdrhistogram"
|
||||
version = "7.5.4"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "765c9198f173dd59ce26ff9f95ef0aafd0a0fe01fb9d72841bc5066a4c06511d"
|
||||
dependencies = [
|
||||
"byteorder",
|
||||
"num-traits",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "heck"
|
||||
version = "0.4.1"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "95505c38b4572b2d910cecb0281560f54b440a19336cbbcb27bf6ce6adc6f5a8"
|
||||
|
||||
[[package]]
|
||||
name = "heed"
|
||||
version = "0.20.0-alpha.9"
|
||||
source = "git+https://github.com/Cuprate/heed?rev=5aa75b7#5aa75b7a44c8e572cf4957c35526e604e7a692ac"
|
||||
dependencies = [
|
||||
"bitflags 2.4.2",
|
||||
"bytemuck",
|
||||
"byteorder",
|
||||
"heed-traits",
|
||||
"heed-types",
|
||||
"libc",
|
||||
"lmdb-master-sys",
|
||||
"once_cell",
|
||||
"page_size",
|
||||
"serde",
|
||||
"synchronoise",
|
||||
"url",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "heed-traits"
|
||||
version = "0.20.0-alpha.9"
|
||||
source = "git+https://github.com/Cuprate/heed?rev=5aa75b7#5aa75b7a44c8e572cf4957c35526e604e7a692ac"
|
||||
|
||||
[[package]]
|
||||
name = "heed-types"
|
||||
version = "0.20.0-alpha.9"
|
||||
source = "git+https://github.com/Cuprate/heed?rev=5aa75b7#5aa75b7a44c8e572cf4957c35526e604e7a692ac"
|
||||
dependencies = [
|
||||
"bincode",
|
||||
"byteorder",
|
||||
"heed-traits",
|
||||
"serde",
|
||||
"serde_json",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "hermit-abi"
|
||||
version = "0.3.5"
|
||||
|
@ -1246,6 +1367,15 @@ dependencies = [
|
|||
"generic-array",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "instant"
|
||||
version = "0.1.12"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "7a5bbe824c507c5da5956355e86a746d82e0e1464f65d862cc5e71da70e94b2c"
|
||||
dependencies = [
|
||||
"cfg-if",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "ipnet"
|
||||
version = "2.9.0"
|
||||
|
@ -1258,15 +1388,6 @@ version = "1.0.10"
|
|||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "b1a46d1a171d865aa5f83f92695765caa047a9b4cbae2cbf37dbd613a793fd4c"
|
||||
|
||||
[[package]]
|
||||
name = "jobserver"
|
||||
version = "0.1.28"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "ab46a6e9526ddef3ae7f787c06f0f2600639ba80ea3eade3d8e670a2230f51d6"
|
||||
dependencies = [
|
||||
"libc",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "js-sys"
|
||||
version = "0.3.68"
|
||||
|
@ -1320,7 +1441,7 @@ checksum = "85c833ca1e66078851dba29046874e38f08b2c883700aa29a03ddd3b23814ee8"
|
|||
dependencies = [
|
||||
"bitflags 2.4.2",
|
||||
"libc",
|
||||
"redox_syscall",
|
||||
"redox_syscall 0.4.1",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
|
@ -1329,6 +1450,16 @@ version = "0.4.13"
|
|||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "01cda141df6706de531b6c46c3a33ecca755538219bd484262fa09410c13539c"
|
||||
|
||||
[[package]]
|
||||
name = "lmdb-master-sys"
|
||||
version = "0.1.0"
|
||||
source = "git+https://github.com/Cuprate/heed?rev=5aa75b7#5aa75b7a44c8e572cf4957c35526e604e7a692ac"
|
||||
dependencies = [
|
||||
"cc",
|
||||
"doxygen-rs",
|
||||
"libc",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "lock_api"
|
||||
version = "0.4.11"
|
||||
|
@ -1361,6 +1492,16 @@ version = "2.7.1"
|
|||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "523dc4f511e55ab87b694dc30d0f820d60906ef06413f93d4d7a1385599cc149"
|
||||
|
||||
[[package]]
|
||||
name = "memmap"
|
||||
version = "0.7.0"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "6585fd95e7bb50d6cc31e20d4cf9afb4e2ba16c5846fc76793f11218da9c475b"
|
||||
dependencies = [
|
||||
"libc",
|
||||
"winapi",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "merlin"
|
||||
version = "3.0.0"
|
||||
|
@ -1666,12 +1807,33 @@ version = "0.1.1"
|
|||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "b15813163c1d831bf4a13c3610c05c0d03b39feb07f7e09fa234dac9b15aaf39"
|
||||
|
||||
[[package]]
|
||||
name = "page_size"
|
||||
version = "0.6.0"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "30d5b2194ed13191c1999ae0704b7839fb18384fa22e49b57eeaa97d79ce40da"
|
||||
dependencies = [
|
||||
"libc",
|
||||
"winapi",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "parking"
|
||||
version = "2.2.0"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "bb813b8af86854136c6922af0598d719255ecb2179515e6e7730d468f05c9cae"
|
||||
|
||||
[[package]]
|
||||
name = "parking_lot"
|
||||
version = "0.11.2"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "7d17b78036a60663b797adeaee46f5c9dfebb86948d1255007a1d6be0271ff99"
|
||||
dependencies = [
|
||||
"instant",
|
||||
"lock_api",
|
||||
"parking_lot_core 0.8.6",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "parking_lot"
|
||||
version = "0.12.1"
|
||||
|
@ -1679,7 +1841,21 @@ source = "registry+https://github.com/rust-lang/crates.io-index"
|
|||
checksum = "3742b2c103b9f06bc9fff0a37ff4912935851bee6d36f3c02bcc755bcfec228f"
|
||||
dependencies = [
|
||||
"lock_api",
|
||||
"parking_lot_core",
|
||||
"parking_lot_core 0.9.9",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "parking_lot_core"
|
||||
version = "0.8.6"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "60a2cfe6f0ad2bfc16aefa463b497d5c7a5ecd44a23efa72aa342d90177356dc"
|
||||
dependencies = [
|
||||
"cfg-if",
|
||||
"instant",
|
||||
"libc",
|
||||
"redox_syscall 0.2.16",
|
||||
"smallvec",
|
||||
"winapi",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
|
@ -1690,7 +1866,7 @@ checksum = "4c42a9226546d68acdd9c0a280d17ce19bfe27a46bf68784e4066115788d008e"
|
|||
dependencies = [
|
||||
"cfg-if",
|
||||
"libc",
|
||||
"redox_syscall",
|
||||
"redox_syscall 0.4.1",
|
||||
"smallvec",
|
||||
"windows-targets 0.48.5",
|
||||
]
|
||||
|
@ -1753,6 +1929,48 @@ version = "2.3.1"
|
|||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "e3148f5046208a5d56bcfc03053e3ca6334e51da8dfb19b6cdc8b306fae3283e"
|
||||
|
||||
[[package]]
|
||||
name = "phf"
|
||||
version = "0.11.2"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "ade2d8b8f33c7333b51bcf0428d37e217e9f32192ae4772156f65063b8ce03dc"
|
||||
dependencies = [
|
||||
"phf_macros",
|
||||
"phf_shared",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "phf_generator"
|
||||
version = "0.11.2"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "48e4cc64c2ad9ebe670cb8fd69dd50ae301650392e81c05f9bfcb2d5bdbc24b0"
|
||||
dependencies = [
|
||||
"phf_shared",
|
||||
"rand",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "phf_macros"
|
||||
version = "0.11.2"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "3444646e286606587e49f3bcf1679b8cef1dc2c5ecc29ddacaffc305180d464b"
|
||||
dependencies = [
|
||||
"phf_generator",
|
||||
"phf_shared",
|
||||
"proc-macro2",
|
||||
"quote",
|
||||
"syn 2.0.48",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "phf_shared"
|
||||
version = "0.11.2"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "90fcb95eef784c2ac79119d1dd819e162b5da872ce6f3c3abe1e8ca1c082f72b"
|
||||
dependencies = [
|
||||
"siphasher",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "pin-project"
|
||||
version = "1.1.4"
|
||||
|
@ -1981,6 +2199,15 @@ dependencies = [
|
|||
"crossbeam-utils",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "redox_syscall"
|
||||
version = "0.2.16"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "fb5a58c1855b4b6819d59012155603f0b22ad30cad752600aadfcb695265519a"
|
||||
dependencies = [
|
||||
"bitflags 1.3.2",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "redox_syscall"
|
||||
version = "0.4.1"
|
||||
|
@ -2195,6 +2422,27 @@ version = "1.0.16"
|
|||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "f98d2aa92eebf49b69786be48e4477826b256916e84a57ff2a4f21923b48eb4c"
|
||||
|
||||
[[package]]
|
||||
name = "sanakirja"
|
||||
version = "1.4.1"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "7c385eb43079aa7dc6204e473b68b4305ceaea8048dda3a985a339bbb57cde72"
|
||||
dependencies = [
|
||||
"fs2",
|
||||
"log",
|
||||
"memmap",
|
||||
"parking_lot 0.11.2",
|
||||
"sanakirja-core",
|
||||
"serde",
|
||||
"thiserror",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "sanakirja-core"
|
||||
version = "1.4.1"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "8376db34ae3eac6e7bd91168bc638450073b708ce9fb46940de676f552238bf5"
|
||||
|
||||
[[package]]
|
||||
name = "schannel"
|
||||
version = "0.1.23"
|
||||
|
@ -2357,6 +2605,12 @@ dependencies = [
|
|||
"tower-service",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "siphasher"
|
||||
version = "0.3.11"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "38b58827f4464d87d377d175e90bf58eb00fd8716ff0a62f80356b5e61555d0d"
|
||||
|
||||
[[package]]
|
||||
name = "slab"
|
||||
version = "0.4.9"
|
||||
|
@ -2449,6 +2703,15 @@ version = "0.1.2"
|
|||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "2047c6ded9c721764247e62cd3b03c09ffc529b2ba5b10ec482ae507a4a70160"
|
||||
|
||||
[[package]]
|
||||
name = "synchronoise"
|
||||
version = "1.0.1"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "3dbc01390fc626ce8d1cffe3376ded2b72a11bb70e1c75f404a210e4daa4def2"
|
||||
dependencies = [
|
||||
"crossbeam-queue",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "system-configuration"
|
||||
version = "0.5.1"
|
||||
|
@ -2583,7 +2846,7 @@ dependencies = [
|
|||
"libc",
|
||||
"mio",
|
||||
"num_cpus",
|
||||
"parking_lot",
|
||||
"parking_lot 0.12.1",
|
||||
"pin-project-lite",
|
||||
"signal-hook-registry",
|
||||
"socket2",
|
||||
|
@ -2674,6 +2937,7 @@ checksum = "b8fa9be0de6cf49e536ce1851f987bd21a43b771b09473c3549a6c853db37c1c"
|
|||
dependencies = [
|
||||
"futures-core",
|
||||
"futures-util",
|
||||
"hdrhistogram",
|
||||
"indexmap 1.9.3",
|
||||
"pin-project",
|
||||
"pin-project-lite",
|
||||
|
|
|
@ -5,6 +5,7 @@ members = [
|
|||
"consensus",
|
||||
"consensus/rules",
|
||||
"cryptonight",
|
||||
"database",
|
||||
"helper",
|
||||
"net/epee-encoding",
|
||||
"net/fixed-bytes",
|
||||
|
@ -38,6 +39,7 @@ opt-level = 3
|
|||
async-trait = { version = "0.1.74", default-features = false }
|
||||
borsh = { version = "1.2.1", default-features = false }
|
||||
bytes = { version = "1.5.0", default-features = false }
|
||||
cfg-if = { version = "1.0.0", default-features = false }
|
||||
clap = { version = "4.4.7", default-features = false }
|
||||
chrono = { version = "0.4.31", default-features = false }
|
||||
crypto-bigint = { version = "0.5.5", default-features = false }
|
||||
|
@ -50,6 +52,7 @@ hex = { version = "0.4.3", default-features = false }
|
|||
hex-literal = { version = "0.4", default-features = false }
|
||||
monero-serai = { git = "https://github.com/Cuprate/serai.git", rev = "347d4cf", default-features = false }
|
||||
multiexp = { git = "https://github.com/Cuprate/serai.git", rev = "347d4cf", default-features = false }
|
||||
paste = { version = "1.0.14", default-features = false }
|
||||
pin-project = { version = "1.1.3", default-features = false }
|
||||
randomx-rs = { git = "https://github.com/Cuprate/randomx-rs.git", rev = "0028464", default-features = false }
|
||||
rand = { version = "0.8.5", default-features = false }
|
||||
|
@ -58,7 +61,7 @@ serde_bytes = { version = "0.11.12", default-features = false }
|
|||
serde_json = { version = "1.0.108", default-features = false }
|
||||
serde = { version = "1.0.190", default-features = false }
|
||||
thiserror = { version = "1.0.50", default-features = false }
|
||||
thread_local = { version = "1.1.7", default-features = false }
|
||||
thread_local = { version = "1.1.7", default-features = false }
|
||||
tokio-util = { version = "0.7.10", default-features = false }
|
||||
tokio-stream = { version = "0.1.14", default-features = false }
|
||||
tokio = { version = "1.33.0", default-features = false }
|
||||
|
@ -79,7 +82,6 @@ proptest-derive = { version = "0.4.0" }
|
|||
# notify = { version = "6.1.1" } # Filesystem watching | https://github.com/notify-rs/notify
|
||||
# once_cell = { version = "1.18.0" } # Lazy/one-time initialization | https://github.com/matklad/once_cell
|
||||
# open = { version = "5.0.0" } # Open PATH/URL, probably for binaries | https://github.com/byron/open-rs
|
||||
# paste = { version = "1.0.14" } # Macro `ident` pasting | https://github.com/dtolnay/paste
|
||||
# regex = { version = "1.10.2" } # Regular expressions | https://github.com/rust-lang/regex
|
||||
# ryu = { version = "1.0.15" } # Fast float to string formatting | https://github.com/dtolnay/ryu
|
||||
# strum = { version = "0.25.0" } # Enum macros/traits | https://github.com/Peternator7/strum
|
||||
|
|
|
@ -1,33 +1,33 @@
|
|||
[package]
|
||||
name = "cuprate-database"
|
||||
version = "0.0.1"
|
||||
edition = "2021"
|
||||
license = "AGPL-3.0-only"
|
||||
|
||||
# All Contributors on github
|
||||
authors=[
|
||||
"SyntheticBird45 <@someoneelse495495:matrix.org>",
|
||||
"Boog900"
|
||||
]
|
||||
name = "cuprate-database"
|
||||
version = "0.0.0"
|
||||
edition = "2021"
|
||||
description = "Cuprate's database abstraction"
|
||||
license = "MIT"
|
||||
authors = ["hinto-janai"]
|
||||
repository = "https://github.com/Cuprate/cuprate/tree/main/database"
|
||||
keywords = ["cuprate", "database"]
|
||||
|
||||
[features]
|
||||
mdbx = ["dep:libmdbx"]
|
||||
hse = []
|
||||
default = ["heed", "service"]
|
||||
# default = ["sanakirja", "service"] # For testing `sanakirja`.
|
||||
heed = ["dep:heed"]
|
||||
sanakirja = ["dep:sanakirja"]
|
||||
service = ["dep:cuprate-helper", "dep:crossbeam", "dep:tokio", "dep:tower"]
|
||||
|
||||
[dependencies]
|
||||
monero = {workspace = true, features = ["serde"]}
|
||||
tiny-keccak = { version = "2.0", features = ["sha3"] }
|
||||
serde = { workspace = true}
|
||||
thiserror = {workspace = true }
|
||||
bincode = { workspace = true }
|
||||
libmdbx = { version = "0.3.1", optional = true }
|
||||
cfg-if = { workspace = true }
|
||||
paste = { workspace = true }
|
||||
thiserror = { workspace = true }
|
||||
|
||||
[build]
|
||||
linker="clang"
|
||||
rustflags=[
|
||||
"-Clink-arg=-fuse-ld=mold",
|
||||
"-Zcf-protection=full",
|
||||
"-Zsanitizer=cfi",
|
||||
"-Crelocation-model=pie",
|
||||
"-Cstack-protector=all",
|
||||
]
|
||||
# `service` feature.
|
||||
cuprate-helper = { path = "../helper", features = ["thread"], optional = true }
|
||||
crossbeam = { workspace = true, features = ["std"], optional = true }
|
||||
tokio = { workspace = true, features = ["full"], optional = true }
|
||||
tower = { workspace = true, features = ["full"], optional = true }
|
||||
|
||||
# Optional features.
|
||||
borsh = { workspace = true, optional = true }
|
||||
heed = { git = "https://github.com/Cuprate/heed", rev = "5aa75b7", optional = true }
|
||||
sanakirja = { version = "1.4.0", optional = true }
|
||||
serde = { workspace = true, optional = true }
|
137
database/README.md
Normal file
137
database/README.md
Normal file
|
@ -0,0 +1,137 @@
|
|||
# Database
|
||||
Cuprate's database implementation.
|
||||
|
||||
<!-- Did you know markdown automatically increments number lists, even if they are all 1...? -->
|
||||
1. [Documentation](#documentation)
|
||||
1. [File Structure](#file-structure)
|
||||
- [`src/`](#src)
|
||||
- [`src/service/`](#src-service)
|
||||
- [`src/backend/`](#src-backend)
|
||||
1. [Backends](#backends)
|
||||
- [`heed`](#heed)
|
||||
- [`sanakirja`](#sanakirja)
|
||||
1. [Layers](#layers)
|
||||
- [Database](#database)
|
||||
- [Trait](#trait)
|
||||
- [ConcreteDatabase](#concretedatabase)
|
||||
- [Thread-pool](#thread-pool)
|
||||
- [Service](#service)
|
||||
|
||||
---
|
||||
|
||||
# Documentation
|
||||
In general, documentation for `database/` is split into 3:
|
||||
|
||||
| Documentation location | Purpose |
|
||||
|---------------------------|---------|
|
||||
| `database/README.md` | High level design of `cuprate-database`
|
||||
| `cuprate-database` | Practical usage documentation/warnings/notes/etc
|
||||
| Source file `// comments` | Implementation-specific details (e.g, how many reader threads to spawn?)
|
||||
|
||||
This README serves as the overview/design document.
|
||||
|
||||
For actual practical usage, `cuprate-database`'s types and general usage are documented via standard Rust tooling.
|
||||
|
||||
Run:
|
||||
```bash
|
||||
cargo doc --package cuprate-database --open
|
||||
```
|
||||
at the root of the repo to open/read the documentation.
|
||||
|
||||
If this documentation is too abstract, refer to any of the source files, they are heavily commented. There are many `// Regular comments` that explain more implementation specific details that aren't present here or in the docs. Use the file reference below to find what you're looking for.
|
||||
|
||||
The code within `src/` is also littered with some `grep`-able comments containing some keywords:
|
||||
|
||||
| Word | Meaning |
|
||||
|-------------|---------|
|
||||
| `INVARIANT` | This code makes an _assumption_ that must be upheld for correctness
|
||||
| `SAFETY` | This `unsafe` code is okay, for `x,y,z` reasons
|
||||
| `FIXME` | This code works but isn't ideal
|
||||
| `HACK` | This code is a brittle workaround
|
||||
| `PERF` | This code is weird for performance reasons
|
||||
| `TODO` | This has to be implemented
|
||||
| `SOMEDAY` | This should be implemented... someday
|
||||
|
||||
# File Structure
|
||||
A quick reference of the structure of the folders & files in `cuprate-database`.
|
||||
|
||||
Note that `lib.rs/mod.rs` files are purely for re-exporting/visibility/lints, and contain no code. Each sub-directory has a corresponding `mod.rs`.
|
||||
|
||||
## `src/`
|
||||
The top-level `src/` files.
|
||||
|
||||
| File/Folder | Purpose |
|
||||
|------------------|---------|
|
||||
| `constants.rs` | General constants used throughout `cuprate-database`
|
||||
| `database.rs` | Abstracted database; `trait Database`
|
||||
| `env.rs` | Abstracted database environment; `trait Env`
|
||||
| `error.rs` | Database error types
|
||||
| `free.rs` | General free functions (related to the database)
|
||||
| `key.rs` | Abstracted database keys; `trait Key`
|
||||
| `pod.rs` | Data (de)serialization; `trait Pod`
|
||||
| `table.rs` | Database table abstraction; `trait Table`
|
||||
| `tables.rs` | All the table definitions used by `cuprate-database`
|
||||
| `transaction.rs` | Database transaction abstraction; `trait RoTx`, `trait RwTx`
|
||||
|
||||
## `src/service/`
|
||||
This folder contains the `cupate_database::service` module.
|
||||
|
||||
| File/Folder | Purpose |
|
||||
|----------------|---------|
|
||||
| `free.rs` | General free functions used (related to `cuprate_database::service`)
|
||||
| `read.rs` | Read thread-pool definitions and logic
|
||||
| `request.rs` | Read/write `Request`s to the database
|
||||
| `response.rs` | Read/write `Response`'s from the database
|
||||
| `write.rs` | Write thread-pool definitions and logic
|
||||
|
||||
## `src/backend/`
|
||||
This folder contains the actual database crates used as the backend for `cuprate-database`.
|
||||
|
||||
Each backend has its own folder.
|
||||
|
||||
| File/Folder | Purpose |
|
||||
|--------------|---------|
|
||||
| `heed/` | Backend using using forked [`heed`](https://github.com/Cuprate/heed)
|
||||
| `sanakirja/` | Backend using [`sanakirja`](https://docs.rs/sanakirja)
|
||||
|
||||
### `src/backend/heed/`
|
||||
| File/Folder | Purpose |
|
||||
|------------------|---------|
|
||||
| `database.rs` | Implementation of `trait Database`
|
||||
| `env.rs` | Implementation of `trait Env`
|
||||
| `serde.rs` | Data (de)serialization implementations
|
||||
| `transaction.rs` | Implementation of `trait RoTx/RwTx`
|
||||
|
||||
### `src/backend/sanakirja/`
|
||||
| File/Folder | Purpose |
|
||||
|------------------|---------|
|
||||
| `database.rs` | Implementation of `trait Database`
|
||||
| `env.rs` | Implementation of `trait Env`
|
||||
| `transaction.rs` | Implementation of `trait RoTx/RwTx`
|
||||
|
||||
# Backends
|
||||
`cuprate-database`'s `trait`s abstract over various actual databases.
|
||||
|
||||
Each database's implementation is located in its respective file in `src/backend/${DATABASE_NAME}.rs`.
|
||||
|
||||
## `heed`
|
||||
The default database used is a modified fork of [`heed`](https://github.com/meilisearch/heed), located at [`Cuprate/heed`](https://github.com/Cuprate/heed).
|
||||
|
||||
To generate documentation of the fork for local use:
|
||||
```bash
|
||||
git clone --recursive https://github.com/Cuprate/heed
|
||||
cargo doc
|
||||
```
|
||||
`LMDB` should not need to be installed as `heed` has a build script that pulls it in automatically.
|
||||
|
||||
## `sanakirja`
|
||||
TODO
|
||||
|
||||
# Layers
|
||||
TODO: update with accurate information when ready, update image.
|
||||
|
||||
## Database
|
||||
## Trait
|
||||
## ConcreteDatabase
|
||||
## Thread
|
||||
## Service
|
47
database/src/backend/heed/database.rs
Normal file
47
database/src/backend/heed/database.rs
Normal file
|
@ -0,0 +1,47 @@
|
|||
//! Implementation of `trait Database` for `heed`.
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Import
|
||||
use crate::{database::Database, error::RuntimeError, table::Table};
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Database Impls
|
||||
impl<T: Table> Database<T> for heed::Database<T::Key, T::Value> {
|
||||
type RoTx<'db> = heed::RoTxn<'db>;
|
||||
type RwTx<'db> = heed::RwTxn<'db>;
|
||||
|
||||
fn get(&self, ro_tx: &Self::RoTx<'_>, key: &T::Key) -> Result<Option<T::Value>, RuntimeError> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
fn get_range(
|
||||
&self,
|
||||
ro_tx: &Self::RoTx<'_>,
|
||||
key: &T::Key,
|
||||
amount: usize,
|
||||
) -> Result<impl Iterator<Item = T::Value>, RuntimeError> {
|
||||
let iter: std::vec::Drain<'_, T::Value> = todo!();
|
||||
Ok(iter)
|
||||
}
|
||||
|
||||
fn put(
|
||||
&mut self,
|
||||
rw_tx: &mut Self::RwTx<'_>,
|
||||
key: &T::Key,
|
||||
value: &T::Value,
|
||||
) -> Result<(), RuntimeError> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
fn clear(&mut self, rw_tx: &mut Self::RwTx<'_>) -> Result<(), RuntimeError> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
fn delete(&mut self, rw_tx: &mut Self::RwTx<'_>, key: &T::Key) -> Result<bool, RuntimeError> {
|
||||
todo!()
|
||||
}
|
||||
}
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Tests
|
||||
#[cfg(test)]
|
||||
mod test {
|
||||
// use super::*;
|
||||
}
|
82
database/src/backend/heed/env.rs
Normal file
82
database/src/backend/heed/env.rs
Normal file
|
@ -0,0 +1,82 @@
|
|||
//! Implementation of `trait Env` for `heed`.
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Import
|
||||
use std::path::Path;
|
||||
|
||||
use crate::{database::Database, env::Env, error::RuntimeError, table::Table};
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Env
|
||||
/// A strongly typed, concrete database environment, backed by `heed`.
|
||||
pub struct ConcreteEnv(heed::Env);
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Env Impl
|
||||
impl Env for ConcreteEnv {
|
||||
/// TODO
|
||||
type RoTx<'db> = heed::RoTxn<'db>;
|
||||
|
||||
/// TODO
|
||||
type RwTx<'db> = heed::RwTxn<'db>;
|
||||
|
||||
//------------------------------------------------ Required
|
||||
#[cold]
|
||||
#[inline(never)] // called once.
|
||||
/// TODO
|
||||
/// # Errors
|
||||
/// TODO
|
||||
fn open<P: AsRef<Path>>(path: P) -> Result<Self, RuntimeError> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
/// # Errors
|
||||
/// TODO
|
||||
fn sync(&self) -> Result<(), RuntimeError> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
#[inline]
|
||||
/// TODO
|
||||
/// # Errors
|
||||
/// TODO
|
||||
fn ro_tx(&self) -> Result<Self::RoTx<'_>, RuntimeError> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
#[inline]
|
||||
/// TODO
|
||||
/// # Errors
|
||||
/// TODO
|
||||
fn rw_tx(&self) -> Result<Self::RwTx<'_>, RuntimeError> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
#[cold]
|
||||
#[inline(never)] // called infrequently?.
|
||||
/// TODO
|
||||
/// # Errors
|
||||
/// TODO
|
||||
fn create_tables_if_needed<T: Table>(
|
||||
&self,
|
||||
tx_rw: &mut Self::RwTx<'_>,
|
||||
) -> Result<(), RuntimeError> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
#[inline]
|
||||
/// TODO
|
||||
/// # Errors
|
||||
/// TODO
|
||||
fn open_database<T: Table>(
|
||||
&self,
|
||||
to_rw: &Self::RoTx<'_>,
|
||||
) -> Result<impl Database<T>, RuntimeError> {
|
||||
let tx: heed::Database<T::Key, T::Value> = todo!();
|
||||
Ok(tx)
|
||||
}
|
||||
}
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Tests
|
||||
#[cfg(test)]
|
||||
mod test {
|
||||
// use super::*;
|
||||
}
|
9
database/src/backend/heed/mod.rs
Normal file
9
database/src/backend/heed/mod.rs
Normal file
|
@ -0,0 +1,9 @@
|
|||
//! Database backend implementation backed by `heed`.
|
||||
|
||||
mod env;
|
||||
pub use env::ConcreteEnv;
|
||||
|
||||
mod database;
|
||||
|
||||
mod serde;
|
||||
mod transaction;
|
53
database/src/backend/heed/serde.rs
Normal file
53
database/src/backend/heed/serde.rs
Normal file
|
@ -0,0 +1,53 @@
|
|||
//! (De)serialization trait implementations for `heed`.
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Import
|
||||
use std::borrow::Cow;
|
||||
|
||||
use crate::pod::Pod;
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Serde
|
||||
/// Implement `heed` (de)serialization traits
|
||||
/// for anything that implements [`crate::pod::Pod`].
|
||||
///
|
||||
/// Blanket implementation breaks orphan impl rules, so this is used instead.
|
||||
macro_rules! impl_heed {
|
||||
($(
|
||||
$name:ident => // The name that implements [`crate::pod::Pod`]
|
||||
$t:ident // The type to (de)serialize into/from
|
||||
),* $(,)?) => {
|
||||
$(
|
||||
// `heed` Encode.
|
||||
impl<'a> heed::BytesEncode<'a> for $name {
|
||||
type EItem = $t;
|
||||
|
||||
#[inline]
|
||||
fn bytes_encode(item: &'a Self::EItem) -> Result<Cow<'a, [u8]>, heed::BoxedError> {
|
||||
Ok(item.into_bytes())
|
||||
}
|
||||
}
|
||||
|
||||
// `heed` Decode.
|
||||
impl<'a> heed::BytesDecode<'a> for $name {
|
||||
type DItem = $t;
|
||||
|
||||
#[inline]
|
||||
fn bytes_decode(bytes: &'a [u8]) -> Result<Self::DItem, heed::BoxedError> {
|
||||
Ok(Pod::from_bytes(bytes))
|
||||
}
|
||||
}
|
||||
)*
|
||||
};
|
||||
}
|
||||
|
||||
/// TODO
|
||||
struct Test;
|
||||
|
||||
impl_heed! {
|
||||
Test => u8,
|
||||
}
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Tests
|
||||
#[cfg(test)]
|
||||
mod test {
|
||||
// use super::*;
|
||||
}
|
35
database/src/backend/heed/transaction.rs
Normal file
35
database/src/backend/heed/transaction.rs
Normal file
|
@ -0,0 +1,35 @@
|
|||
//! Implementation of `trait RoTx/RwTx` for `heed`.
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Import
|
||||
use crate::{
|
||||
error::RuntimeError,
|
||||
transaction::{RoTx, RwTx},
|
||||
};
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- RoTx
|
||||
impl RoTx<'_> for heed::RoTxn<'_> {
|
||||
fn commit(self) -> Result<(), RuntimeError> {
|
||||
todo!()
|
||||
}
|
||||
}
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- RwTx
|
||||
impl RwTx<'_> for heed::RwTxn<'_> {
|
||||
/// TODO
|
||||
/// # Errors
|
||||
/// TODO
|
||||
fn commit(self) -> Result<(), RuntimeError> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
fn abort(self) {
|
||||
todo!()
|
||||
}
|
||||
}
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Tests
|
||||
#[cfg(test)]
|
||||
mod test {
|
||||
// use super::*;
|
||||
}
|
22
database/src/backend/mod.rs
Normal file
22
database/src/backend/mod.rs
Normal file
|
@ -0,0 +1,22 @@
|
|||
//! Database backends.
|
||||
//!
|
||||
//! TODO:
|
||||
//! Create a test backend backed by `std::collections::HashMap`.
|
||||
//!
|
||||
//! The full type could be something like `HashMap<&'static str, HashMap<K, V>>`.
|
||||
//! where the `str` is the table name, and the containing hashmap are are the
|
||||
//! key and values.
|
||||
//!
|
||||
//! Not sure how duplicate keys will work.
|
||||
|
||||
cfg_if::cfg_if! {
|
||||
// If both backends are enabled, fallback to `heed`.
|
||||
// This is useful when using `--all-features`.
|
||||
if #[cfg(all(feature = "sanakirja", not(feature = "heed")))] {
|
||||
mod sanakirja;
|
||||
pub use sanakirja::ConcreteEnv;
|
||||
} else {
|
||||
mod heed;
|
||||
pub use heed::ConcreteEnv;
|
||||
}
|
||||
}
|
47
database/src/backend/sanakirja/database.rs
Normal file
47
database/src/backend/sanakirja/database.rs
Normal file
|
@ -0,0 +1,47 @@
|
|||
//! Implementation of `trait Database` for `sanakirja`.
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Import
|
||||
use crate::{database::Database, error::RuntimeError, table::Table};
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Database Impls
|
||||
impl<T: Table> Database<T> for sanakirja::btree::Db<T::Key, T::Value> {
|
||||
type RoTx<'db> = sanakirja::Txn<&'_ sanakirja::Env>;
|
||||
type RwTx<'db> = sanakirja::MutTxn<&'_ sanakirja::Env, ()>;
|
||||
|
||||
fn get(&self, ro_tx: &Self::RoTx<'_>, key: &T::Key) -> Result<Option<T::Value>, RuntimeError> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
fn get_range(
|
||||
&self,
|
||||
ro_tx: &Self::RoTx<'_>,
|
||||
key: &T::Key,
|
||||
amount: usize,
|
||||
) -> Result<impl Iterator<Item = T::Value>, RuntimeError> {
|
||||
let iter: std::vec::Drain<'_, T::Value> = todo!();
|
||||
Ok(iter)
|
||||
}
|
||||
|
||||
fn put(
|
||||
&mut self,
|
||||
rx_tx: &mut Self::RwTx<'_>,
|
||||
key: &T::Key,
|
||||
value: &T::Value,
|
||||
) -> Result<(), RuntimeError> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
fn clear(&mut self, rx_tx: &mut Self::RwTx<'_>) -> Result<(), RuntimeError> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
fn delete(&mut self, rx_tx: &mut Self::RwTx<'_>, key: &T::Key) -> Result<bool, RuntimeError> {
|
||||
todo!()
|
||||
}
|
||||
}
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Tests
|
||||
#[cfg(test)]
|
||||
mod test {
|
||||
// use super::*;
|
||||
}
|
85
database/src/backend/sanakirja/env.rs
Normal file
85
database/src/backend/sanakirja/env.rs
Normal file
|
@ -0,0 +1,85 @@
|
|||
//! Implementation of `trait Env` for `sanakirja`.
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Import
|
||||
use std::path::Path;
|
||||
|
||||
use crate::{database::Database, env::Env, error::RuntimeError, table::Table};
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- ConcreteEnv
|
||||
/// A strongly typed, concrete database environment, backed by `sanakirja`.
|
||||
pub struct ConcreteEnv(sanakirja::Env);
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Env Impl
|
||||
impl Env for ConcreteEnv {
|
||||
/// TODO
|
||||
///
|
||||
/// We could also implement `Borrow<sanakirja::Env> for ConcreteEnv`
|
||||
/// instead of this reference.
|
||||
type RoTx<'db> = sanakirja::Txn<&'db sanakirja::Env>;
|
||||
|
||||
/// TODO
|
||||
type RwTx<'db> = sanakirja::MutTxn<&'db sanakirja::Env, ()>;
|
||||
|
||||
//------------------------------------------------ Required
|
||||
#[cold]
|
||||
#[inline(never)] // called once.
|
||||
/// TODO
|
||||
/// # Errors
|
||||
/// TODO
|
||||
fn open<P: AsRef<Path>>(path: P) -> Result<Self, RuntimeError> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
/// # Errors
|
||||
/// TODO
|
||||
fn sync(&self) -> Result<(), RuntimeError> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
#[inline]
|
||||
/// TODO
|
||||
/// # Errors
|
||||
/// TODO
|
||||
fn ro_tx(&self) -> Result<Self::RoTx<'_>, RuntimeError> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
#[inline]
|
||||
/// TODO
|
||||
/// # Errors
|
||||
/// TODO
|
||||
fn rw_tx(&self) -> Result<Self::RwTx<'_>, RuntimeError> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
#[cold]
|
||||
#[inline(never)] // called infrequently?.
|
||||
/// TODO
|
||||
/// # Errors
|
||||
/// TODO
|
||||
fn create_tables_if_needed<T: Table>(
|
||||
&self,
|
||||
tx_rw: &mut Self::RwTx<'_>,
|
||||
) -> Result<(), RuntimeError> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
#[inline]
|
||||
/// TODO
|
||||
/// # Errors
|
||||
/// TODO
|
||||
fn open_database<T: Table>(
|
||||
&self,
|
||||
to_rw: &Self::RoTx<'_>,
|
||||
) -> Result<Option<impl Database<T>>, RuntimeError> {
|
||||
let tx: sanakirja::btree::Db<T::Key, T::Value> = todo!();
|
||||
Ok(Some(tx))
|
||||
}
|
||||
}
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Tests
|
||||
#[cfg(test)]
|
||||
mod test {
|
||||
// use super::*;
|
||||
}
|
8
database/src/backend/sanakirja/mod.rs
Normal file
8
database/src/backend/sanakirja/mod.rs
Normal file
|
@ -0,0 +1,8 @@
|
|||
//! Database backend implementation backed by `sanakirja`.
|
||||
|
||||
mod env;
|
||||
pub use env::ConcreteEnv;
|
||||
|
||||
mod database;
|
||||
|
||||
mod transaction;
|
35
database/src/backend/sanakirja/transaction.rs
Normal file
35
database/src/backend/sanakirja/transaction.rs
Normal file
|
@ -0,0 +1,35 @@
|
|||
//! Implementation of `trait RoTx/RwTx` for `sanakirja`.
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Import
|
||||
use crate::{
|
||||
error::RuntimeError,
|
||||
transaction::{RoTx, RwTx},
|
||||
};
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- RoTx
|
||||
impl RoTx<'_> for sanakirja::Txn<&'_ sanakirja::Env> {
|
||||
fn commit(self) -> Result<(), RuntimeError> {
|
||||
todo!()
|
||||
}
|
||||
}
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- RwTx
|
||||
impl RwTx<'_> for sanakirja::MutTxn<&'_ sanakirja::Env, ()> {
|
||||
/// TODO
|
||||
/// # Errors
|
||||
/// TODO
|
||||
fn commit(self) -> Result<(), RuntimeError> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
fn abort(self) {
|
||||
todo!()
|
||||
}
|
||||
}
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Tests
|
||||
#[cfg(test)]
|
||||
mod test {
|
||||
// use super::*;
|
||||
}
|
51
database/src/constants.rs
Normal file
51
database/src/constants.rs
Normal file
|
@ -0,0 +1,51 @@
|
|||
//! General constants used throughout `cuprate-database`.
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Import
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Constants
|
||||
/// The directory that contains database-related files.
|
||||
///
|
||||
/// This is a sub-directory within the Cuprate folder, e.g:
|
||||
/// ```txt
|
||||
/// ~/.local/share/cuprate/
|
||||
/// ├─ database/ # <-
|
||||
/// ├─ data.mdb
|
||||
/// ├─ lock.mdb
|
||||
/// ```
|
||||
pub const CUPRATE_DATABASE_DIR: &str = "database";
|
||||
|
||||
/// The actual database file name.
|
||||
///
|
||||
/// This is a _file_ within [`CUPRATE_DATABASE_DIR`], e.g:
|
||||
/// ```txt
|
||||
/// ~/.local/share/cuprate/
|
||||
/// ├─ database/
|
||||
/// ├─ data.mdb # <-
|
||||
/// ├─ lock.mdb
|
||||
/// ```
|
||||
pub const CUPRATE_DATABASE_FILE: &str = "data";
|
||||
|
||||
cfg_if::cfg_if! {
|
||||
// If both backends are enabled, fallback to `heed`.
|
||||
// This is useful when using `--all-features`.
|
||||
if #[cfg(all(feature = "sanakirja", not(feature = "heed")))] {
|
||||
/// Static string of the `crate` being used as the database backend.
|
||||
pub const DATABASE_BACKEND: &str = "sanakirja";
|
||||
} else {
|
||||
/// Static string of the `crate` being used as the database backend.
|
||||
pub const DATABASE_BACKEND: &str = "heed";
|
||||
}
|
||||
}
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Tests
|
||||
#[cfg(test)]
|
||||
mod test {
|
||||
use super::*;
|
||||
|
||||
#[test]
|
||||
/// Sanity check that our PATHs aren't empty... (will cause disaster).
|
||||
fn non_empty_path() {
|
||||
assert!(!CUPRATE_DATABASE_DIR.is_empty());
|
||||
assert!(!CUPRATE_DATABASE_FILE.is_empty());
|
||||
}
|
||||
}
|
58
database/src/database.rs
Normal file
58
database/src/database.rs
Normal file
|
@ -0,0 +1,58 @@
|
|||
//! Abstracted database; `trait Database`.
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Import
|
||||
use crate::{
|
||||
error::RuntimeError,
|
||||
table::Table,
|
||||
transaction::{RoTx, RwTx},
|
||||
};
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Database
|
||||
/// Database (key-value store) abstraction.
|
||||
///
|
||||
/// TODO
|
||||
pub trait Database<T: Table> {
|
||||
//------------------------------------------------ Types
|
||||
/// TODO
|
||||
type RoTx<'db>: RoTx<'db>;
|
||||
|
||||
/// TODO
|
||||
type RwTx<'db>: RwTx<'db>;
|
||||
|
||||
//-------------------------------------------------------- Read
|
||||
/// TODO
|
||||
/// # Errors
|
||||
/// TODO
|
||||
fn get(&self, ro_tx: &Self::RoTx<'_>, key: &T::Key) -> Result<Option<T::Value>, RuntimeError>;
|
||||
|
||||
/// TODO
|
||||
/// # Errors
|
||||
/// TODO
|
||||
fn get_range(
|
||||
&self,
|
||||
ro_tx: &Self::RoTx<'_>,
|
||||
key: &T::Key,
|
||||
amount: usize,
|
||||
) -> Result<impl Iterator<Item = T::Value>, RuntimeError>;
|
||||
|
||||
//-------------------------------------------------------- Write
|
||||
/// TODO
|
||||
/// # Errors
|
||||
/// TODO
|
||||
fn put(
|
||||
&mut self,
|
||||
rw_tx: &mut Self::RwTx<'_>,
|
||||
key: &T::Key,
|
||||
value: &T::Value,
|
||||
) -> Result<(), RuntimeError>;
|
||||
|
||||
/// TODO
|
||||
/// # Errors
|
||||
/// TODO
|
||||
fn clear(&mut self, rw_tx: &mut Self::RwTx<'_>) -> Result<(), RuntimeError>;
|
||||
|
||||
/// TODO
|
||||
/// # Errors
|
||||
/// TODO
|
||||
fn delete(&mut self, rw_tx: &mut Self::RwTx<'_>, key: &T::Key) -> Result<bool, RuntimeError>;
|
||||
}
|
71
database/src/env.rs
Normal file
71
database/src/env.rs
Normal file
|
@ -0,0 +1,71 @@
|
|||
//! Abstracted database environment; `trait Env`.
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Import
|
||||
use std::path::Path;
|
||||
|
||||
use crate::{
|
||||
database::Database,
|
||||
error::RuntimeError,
|
||||
table::Table,
|
||||
transaction::{RoTx, RwTx},
|
||||
};
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Env
|
||||
/// Database environment abstraction.
|
||||
///
|
||||
/// TODO
|
||||
pub trait Env: Sized {
|
||||
//------------------------------------------------ Types
|
||||
/// TODO
|
||||
type RoTx<'db>: RoTx<'db>;
|
||||
|
||||
/// TODO
|
||||
type RwTx<'db>: RwTx<'db>;
|
||||
|
||||
//------------------------------------------------ Required
|
||||
/// TODO
|
||||
/// # Errors
|
||||
/// TODO
|
||||
fn open<P: AsRef<Path>>(path: P) -> Result<Self, RuntimeError>;
|
||||
|
||||
/// TODO
|
||||
/// # Errors
|
||||
/// TODO
|
||||
fn sync(&self) -> Result<(), RuntimeError>;
|
||||
|
||||
/// TODO
|
||||
/// # Errors
|
||||
/// TODO
|
||||
fn ro_tx(&self) -> Result<Self::RoTx<'_>, RuntimeError>;
|
||||
|
||||
/// TODO
|
||||
/// # Errors
|
||||
/// TODO
|
||||
fn rw_tx(&self) -> Result<Self::RwTx<'_>, RuntimeError>;
|
||||
|
||||
/// TODO
|
||||
/// # Errors
|
||||
/// TODO
|
||||
fn create_tables_if_needed<T: Table>(
|
||||
&self,
|
||||
rw_tx: &mut Self::RwTx<'_>,
|
||||
) -> Result<(), RuntimeError>;
|
||||
|
||||
/// TODO
|
||||
///
|
||||
/// # TODO: Invariant
|
||||
/// This should never panic the database because the table doesn't exist.
|
||||
///
|
||||
/// Opening/using the database env should have an invariant
|
||||
/// that it creates all the tables we need, such that this
|
||||
/// never returns `None`.
|
||||
///
|
||||
/// # Errors
|
||||
/// TODO
|
||||
fn open_database<T: Table>(
|
||||
&self,
|
||||
ro_tx: &Self::RoTx<'_>,
|
||||
) -> Result<impl Database<T>, RuntimeError>;
|
||||
|
||||
//------------------------------------------------ Provided
|
||||
}
|
|
@ -1,53 +1,62 @@
|
|||
//! ### Error module
|
||||
//! This module contains all errors abstraction used by the database crate. By implementing [`From<E>`] to the specific errors of storage engine crates, it let us
|
||||
//! handle more easily any type of error that can happen. This module does **NOT** contain interpretation of these errors, as these are defined for Blockchain abstraction. This is another difference
|
||||
//! from monerod which interpret these errors directly in its database functions:
|
||||
//! ```cpp
|
||||
//! /**
|
||||
//! * @brief A base class for BlockchainDB exceptions
|
||||
//! */
|
||||
//! class DB_EXCEPTION : public std::exception
|
||||
//! ```
|
||||
//! see `blockchain_db/blockchain_db.h` in monerod `src/` folder for more details.
|
||||
//! Database error types.
|
||||
//! TODO: `InitError/RuntimeError` are maybe bad names.
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Import
|
||||
use std::{borrow::Cow, fmt::Debug};
|
||||
|
||||
use crate::constants::DATABASE_BACKEND;
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- InitError
|
||||
/// Database errors that occur during initialization.
|
||||
///
|
||||
/// `BackendError` is an error specifically from the
|
||||
/// database backend being used. TODO: this may not
|
||||
/// be needed if we can convert all error types into
|
||||
/// "generic" database errors.
|
||||
#[derive(thiserror::Error, Debug)]
|
||||
/// `DB_FAILURES` is an enum for backend-agnostic, internal database errors. The `From` Trait must be implemented to the specific backend errors to match DB_FAILURES.
|
||||
pub enum DB_FAILURES {
|
||||
#[error("MDBX returned an error {0}")]
|
||||
MDBX_Error(#[from] libmdbx::Error),
|
||||
pub enum InitError<BackendError: Debug> {
|
||||
/// TODO
|
||||
#[error("database PATH is inaccessible: {0}")]
|
||||
Path(std::io::Error),
|
||||
|
||||
#[error("\n<DB_FAILURES::EncodingError> Failed to encode some data : `{0}`")]
|
||||
SerializeIssue(DB_SERIAL),
|
||||
/// TODO
|
||||
#[error("{DATABASE_BACKEND} error: {0}")]
|
||||
Backend(BackendError),
|
||||
|
||||
#[error("\nObject already exist in the database : {0}")]
|
||||
AlreadyExist(&'static str),
|
||||
|
||||
#[error("NotFound? {0}")]
|
||||
NotFound(&'static str),
|
||||
|
||||
#[error("\n<DB_FAILURES::Other> `{0}`")]
|
||||
Other(&'static str),
|
||||
|
||||
#[error(
|
||||
"\n<DB_FAILURES::FailedToCommit> A transaction tried to commit to the db, but failed."
|
||||
)]
|
||||
FailedToCommit,
|
||||
/// TODO
|
||||
///
|
||||
/// An unknown error occurred.
|
||||
#[error("unknown error: {0}")]
|
||||
Unknown(Cow<'static, str>),
|
||||
}
|
||||
|
||||
#[derive(thiserror::Error, Debug)]
|
||||
pub enum DB_SERIAL {
|
||||
#[error("An object failed to be serialized into bytes. It is likely an issue from monero-rs library. Please report this error on cuprate's github : https://github.com/Cuprate/cuprate/issues")]
|
||||
ConsensusEncode,
|
||||
//---------------------------------------------------------------------------------------------------- RuntimeError
|
||||
/// Database errors that occur _after_ successful initialization.
|
||||
#[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))]
|
||||
#[cfg_attr(
|
||||
feature = "borsh",
|
||||
derive(borsh::BorshSerialize, borsh::BorshDeserialize)
|
||||
)]
|
||||
#[derive(thiserror::Error, Debug, Clone, PartialEq, PartialOrd, Eq, Ord, Hash)]
|
||||
pub enum RuntimeError {
|
||||
// TODO: replace string with actual error type.
|
||||
///
|
||||
/// An error occurred when attempting to
|
||||
/// serialize the key data into bytes.
|
||||
#[error("serialize error: {0}")]
|
||||
Serialize(String),
|
||||
|
||||
#[error("Bytes failed to be deserialized into the requested object. It is likely an issue from monero-rs library. Please report this error on cuprate's github : https://github.com/Cuprate/cuprate/issues")]
|
||||
ConsensusDecode(Vec<u8>),
|
||||
// TODO: replace string with actual error type.
|
||||
///
|
||||
/// An error occurred when attempting to
|
||||
/// deserialize the response value from
|
||||
/// the database.
|
||||
#[error("deserialize error: {0}")]
|
||||
Deserialize(String),
|
||||
|
||||
#[error("monero-rs encoding|decoding logic failed : {0}")]
|
||||
MoneroEncode(#[from] monero::consensus::encode::Error),
|
||||
|
||||
#[error("Bincode failed to decode a type from the database : {0}")]
|
||||
BincodeDecode(#[from] bincode::error::DecodeError),
|
||||
|
||||
#[error("Bincode failed to encode a type for the database : {0}")]
|
||||
BincodeEncode(#[from] bincode::error::EncodeError),
|
||||
/// TODO
|
||||
///
|
||||
/// An unknown error occurred.
|
||||
#[error("unknown error: {0}")]
|
||||
Unknown(Cow<'static, str>),
|
||||
}
|
||||
|
|
17
database/src/free.rs
Normal file
17
database/src/free.rs
Normal file
|
@ -0,0 +1,17 @@
|
|||
//! General free functions (related to the database).
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Import
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Constants
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- TYPE
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- IMPL
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Trait Impl
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Tests
|
||||
#[cfg(test)]
|
||||
mod test {
|
||||
// use super::*;
|
||||
}
|
127
database/src/key.rs
Normal file
127
database/src/key.rs
Normal file
|
@ -0,0 +1,127 @@
|
|||
//! Database key abstraction; `trait Key`.
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Import
|
||||
#[allow(unused_imports)] // docs
|
||||
use crate::table::Table;
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Table
|
||||
/// Database [`Table`] key metadata.
|
||||
///
|
||||
/// Purely compile time information for database table keys, supporting duplicate keys.
|
||||
pub trait Key {
|
||||
/// Does this [`Key`] require multiple keys to reach a value?
|
||||
///
|
||||
/// If [`Key::DUPLICATE`] is `true`, [`Key::Secondary`] will contain
|
||||
/// the "subkey", or secondary key needed to access the actual value.
|
||||
///
|
||||
/// If [`Key::DUPLICATE`] is `false`, [`Key::Secondary`]
|
||||
/// will just be the same type as [`Key::Primary`].
|
||||
const DUPLICATE: bool;
|
||||
|
||||
/// The primary key type.
|
||||
type Primary;
|
||||
|
||||
/// The secondary key type.
|
||||
///
|
||||
/// Only needs to be different than [`Key::Primary`]
|
||||
/// if [`Key::DUPLICATE`] is `true`.
|
||||
type Secondary;
|
||||
|
||||
/// Acquire [`Key::Primary`].
|
||||
fn primary(self) -> Self::Primary;
|
||||
|
||||
/// Acquire [`Self::Primary`] & [`Self::Secondary`].
|
||||
///
|
||||
/// This only needs to be implemented on types that are [`Self::DUPLICATE`].
|
||||
///
|
||||
/// It is `unreachable!()` on non-duplicate key tables.
|
||||
fn primary_secondary(self) -> (Self::Primary, Self::Secondary);
|
||||
}
|
||||
|
||||
/// Duplicate key container.
|
||||
///
|
||||
/// This is a generic container to use alongside [`Key`] to support
|
||||
/// tables that require more than 1 key to access the value.
|
||||
#[derive(Copy, Clone, Debug, PartialEq, Eq, PartialOrd, Ord, Hash)]
|
||||
#[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))]
|
||||
#[cfg_attr(
|
||||
feature = "borsh",
|
||||
derive(borsh::BorshSerialize, borsh::BorshDeserialize)
|
||||
)]
|
||||
pub struct DupKey<P, S> {
|
||||
/// Primary key type.
|
||||
pub primary: P,
|
||||
/// Secondary key type.
|
||||
pub secondary: S,
|
||||
}
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Impl
|
||||
/// Implement `Key` on most primitive types.
|
||||
///
|
||||
/// `Key::DUPLICATE` is always `false`.
|
||||
macro_rules! impl_key {
|
||||
(
|
||||
$(
|
||||
$t:ident // Key type.
|
||||
),* $(,)?
|
||||
) => {
|
||||
$(
|
||||
impl Key for $t {
|
||||
const DUPLICATE: bool = false;
|
||||
type Primary = $t;
|
||||
type Secondary = $t;
|
||||
|
||||
#[inline(always)]
|
||||
fn primary(self) -> Self::Primary {
|
||||
self
|
||||
}
|
||||
|
||||
#[cold] #[inline(never)]
|
||||
fn primary_secondary(self) -> (Self::Primary, Self::Secondary) {
|
||||
unreachable!();
|
||||
}
|
||||
}
|
||||
)*
|
||||
};
|
||||
}
|
||||
|
||||
// Implement `Key` for primitives.
|
||||
impl_key! {
|
||||
u8,
|
||||
u16,
|
||||
u32,
|
||||
u64,
|
||||
i8,
|
||||
i16,
|
||||
i32,
|
||||
i64,
|
||||
}
|
||||
|
||||
// Implement `Key` for any [`DupKey`] using [`Copy`] types.
|
||||
impl<P, S> Key for DupKey<P, S>
|
||||
where
|
||||
P: Key + Copy,
|
||||
S: Key + Copy,
|
||||
{
|
||||
const DUPLICATE: bool = true;
|
||||
|
||||
type Primary = Self;
|
||||
|
||||
type Secondary = S;
|
||||
|
||||
#[inline]
|
||||
fn primary(self) -> Self::Primary {
|
||||
self
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn primary_secondary(self) -> (Self::Primary, Self::Secondary) {
|
||||
(self, self.secondary)
|
||||
}
|
||||
}
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Tests
|
||||
#[cfg(test)]
|
||||
mod test {
|
||||
// use super::*;
|
||||
}
|
|
@ -1,221 +1,274 @@
|
|||
// Copyright (C) 2023 Cuprate Contributors
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful,
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <https://www.gnu.org/licenses/>.
|
||||
|
||||
//! The cuprate-db crate implement (as its name suggests) the relations between the blockchain/txpool objects and their databases.
|
||||
//! `lib.rs` contains all the generics, trait and specification for interfaces between blockchain and a backend-agnostic database
|
||||
//! Every other files in this folder are implementation of these traits/methods to real storage engine.
|
||||
//! Database abstraction and utilities.
|
||||
//!
|
||||
//! At the moment, the only storage engine available is MDBX.
|
||||
//! The next storage engine planned is HSE (Heteregeonous Storage Engine) from Micron.
|
||||
//! This crate does 3 things:
|
||||
//! 1. Abstracts various databases with the [`Env`], [`Database`], [`Table`], [`Key`], [`RoTx`], and [`RwTx`] traits
|
||||
//! 2. Implements various `Monero` related [functions](ops) & [`tables`]
|
||||
//! 3. Exposes a [`tower::Service`] backed by a thread-pool
|
||||
//!
|
||||
//! For more informations, please consult this docs:
|
||||
//! # Terminology
|
||||
//! To be more clear on some terms used in this crate:
|
||||
//!
|
||||
//! | Term | Meaning |
|
||||
//! |------------|--------------------------------------|
|
||||
//! | `Env` | The 1 database environment, the "whole" thing
|
||||
//! | `Database` | A `key/value` store
|
||||
//! | `Table` | Solely the metadata of a `Database` (the `key` and `value` types, and the name)
|
||||
//! | `RoTx` | Read only transaction
|
||||
//! | `RwTx` | Read/write transaction
|
||||
//!
|
||||
//! The dataflow is `Env` -> `Tx` -> `Database`
|
||||
//!
|
||||
//! Which reads as:
|
||||
//! 1. You have a database `Environment`
|
||||
//! 2. You open up a `Transaction`
|
||||
//! 2. You get a particular `Database` from that `Environment`
|
||||
//! 3. You can now read/write data from/to that `Database`
|
||||
//!
|
||||
//! # `ConcreteEnv`
|
||||
//! This crate exposes [`ConcreteEnv`], which is a non-generic/non-dynamic,
|
||||
//! concrete object representing a database [`Env`]ironment.
|
||||
//!
|
||||
//! The actual backend for this type is determined via feature flags.
|
||||
//!
|
||||
//! This object existing means `E: Env` doesn't need to be spread all through the codebase,
|
||||
//! however, it also means some small invariants should be kept in mind.
|
||||
//!
|
||||
//! As `ConcreteEnv` is just a re-exposed type which has varying inner types,
|
||||
//! it means some properties will change depending on the backend used.
|
||||
//!
|
||||
//! For example:
|
||||
//! - [`std::mem::size_of::<ConcreteEnv>`]
|
||||
//! - [`std::mem::align_of::<ConcreteEnv>`]
|
||||
//! - [`Drop::<ConcreteEnv>::drop`]
|
||||
//!
|
||||
//! Things like these functions are affected by the backend and inner data,
|
||||
//! and should not be relied upon. This extends to any `struct/enum` that contains `ConcreteEnv`.
|
||||
//!
|
||||
//! The only thing about `ConcreteEnv` that should
|
||||
//! be relied upon is that it implements [`Env`].
|
||||
//!
|
||||
//! TODO: we could also expose `ConcreteDatabase` if we're
|
||||
//! going to be storing any databases in structs, to lessen
|
||||
//! the generic `<D: Database>` pain.
|
||||
//!
|
||||
//! # Feature flags
|
||||
//! The `service` module requires the `service` feature to be enabled.
|
||||
//! See the module for more documentation.
|
||||
//!
|
||||
//! Different database backends are enabled by the feature flags:
|
||||
//! - `heed`
|
||||
//! - `sanakirja`
|
||||
//!
|
||||
//! The default is `heed`.
|
||||
|
||||
#![deny(unused_attributes)]
|
||||
#![forbid(unsafe_code)]
|
||||
#![allow(non_camel_case_types)]
|
||||
#![deny(clippy::expect_used, clippy::panic)]
|
||||
#![allow(dead_code, unused_macros)] // temporary
|
||||
//---------------------------------------------------------------------------------------------------- Lints
|
||||
// Forbid lints.
|
||||
// Our code, and code generated (e.g macros) cannot overrule these.
|
||||
#![forbid(
|
||||
// Never.
|
||||
unsafe_code,
|
||||
unused_unsafe,
|
||||
redundant_semicolons,
|
||||
unused_allocation,
|
||||
coherence_leak_check,
|
||||
single_use_lifetimes,
|
||||
while_true,
|
||||
clippy::missing_docs_in_private_items,
|
||||
|
||||
use monero::{util::ringct::RctSig, Block, BlockHeader, Hash};
|
||||
use std::ops::Range;
|
||||
use thiserror::Error;
|
||||
// Maybe can be put into `#[deny]`.
|
||||
unconditional_recursion,
|
||||
for_loops_over_fallibles,
|
||||
unused_braces,
|
||||
unused_doc_comments,
|
||||
unused_labels,
|
||||
keyword_idents,
|
||||
non_ascii_idents,
|
||||
variant_size_differences,
|
||||
unused_mut, // Annoying when debugging, maybe put in allow.
|
||||
|
||||
#[cfg(feature = "mdbx")]
|
||||
pub mod mdbx;
|
||||
//#[cfg(feature = "hse")]
|
||||
//pub mod hse;
|
||||
// Probably can be put into `#[deny]`.
|
||||
future_incompatible,
|
||||
let_underscore,
|
||||
break_with_label_and_loop,
|
||||
duplicate_macro_attributes,
|
||||
exported_private_dependencies,
|
||||
large_assignments,
|
||||
overlapping_range_endpoints,
|
||||
semicolon_in_expressions_from_macros,
|
||||
noop_method_call,
|
||||
unreachable_pub,
|
||||
)]
|
||||
// Deny lints.
|
||||
// Some of these are `#[allow]`'ed on a per-case basis.
|
||||
#![deny(
|
||||
clippy::all,
|
||||
clippy::correctness,
|
||||
clippy::suspicious,
|
||||
clippy::style,
|
||||
clippy::complexity,
|
||||
clippy::perf,
|
||||
clippy::pedantic,
|
||||
clippy::nursery,
|
||||
clippy::cargo,
|
||||
missing_docs,
|
||||
deprecated,
|
||||
unused_comparisons,
|
||||
nonstandard_style
|
||||
)]
|
||||
#![allow(unreachable_code, unused_variables, dead_code)] // TODO: remove
|
||||
#![allow(
|
||||
// FIXME: this lint affects crates outside of
|
||||
// `database/` for some reason, allow for now.
|
||||
clippy::cargo_common_metadata,
|
||||
|
||||
pub mod encoding;
|
||||
pub mod error;
|
||||
pub mod interface;
|
||||
pub mod table;
|
||||
pub mod types;
|
||||
// FIXME: adding `#[must_use]` onto everything
|
||||
// might just be more annoying than useful...
|
||||
// although it is sometimes nice.
|
||||
clippy::must_use_candidate,
|
||||
|
||||
const DEFAULT_BLOCKCHAIN_DATABASE_DIRECTORY: &str = "blockchain";
|
||||
const DEFAULT_TXPOOL_DATABASE_DIRECTORY: &str = "txpool_mem";
|
||||
const BINCODE_CONFIG: bincode::config::Configuration<
|
||||
bincode::config::LittleEndian,
|
||||
bincode::config::Fixint,
|
||||
> = bincode::config::standard().with_fixed_int_encoding();
|
||||
// TODO: should be removed after all `todo!()`'s are gone.
|
||||
clippy::diverging_sub_expression,
|
||||
|
||||
// ------------------------------------------| Database |------------------------------------------
|
||||
// FIXME:
|
||||
// If #[deny(clippy::restriction)] is used, it
|
||||
// enables a whole bunch of very subjective lints.
|
||||
// The below disables most of the ones that are
|
||||
// a bit too unwieldy.
|
||||
//
|
||||
// Figure out if if `clippy::restriction` should be
|
||||
// used (it enables a bunch of good lints but has
|
||||
// many false positives).
|
||||
|
||||
pub mod database {
|
||||
//! This module contains the Database abstraction trait. Any key/value storage engine implemented need
|
||||
//! to fullfil these associated types and functions, in order to be usable. This module also contains the
|
||||
//! Interface struct which is used by the DB Reactor to interact with the database.
|
||||
// clippy::single_char_lifetime_names,
|
||||
// clippy::implicit_return,
|
||||
// clippy::std_instead_of_alloc,
|
||||
// clippy::std_instead_of_core,
|
||||
// clippy::unwrap_used,
|
||||
// clippy::min_ident_chars,
|
||||
// clippy::absolute_paths,
|
||||
// clippy::missing_inline_in_public_items,
|
||||
// clippy::shadow_reuse,
|
||||
// clippy::shadow_unrelated,
|
||||
// clippy::missing_trait_methods,
|
||||
// clippy::pub_use,
|
||||
// clippy::pub_with_shorthand,
|
||||
// clippy::blanket_clippy_restriction_lints,
|
||||
// clippy::exhaustive_structs,
|
||||
// clippy::exhaustive_enums,
|
||||
// clippy::unsafe_derive_deserialize,
|
||||
// clippy::multiple_inherent_impl,
|
||||
// clippy::unreadable_literal,
|
||||
// clippy::indexing_slicing,
|
||||
// clippy::float_arithmetic,
|
||||
// clippy::cast_possible_truncation,
|
||||
// clippy::as_conversions,
|
||||
// clippy::cast_precision_loss,
|
||||
// clippy::cast_sign_loss,
|
||||
// clippy::missing_asserts_for_indexing,
|
||||
// clippy::default_numeric_fallback,
|
||||
// clippy::module_inception,
|
||||
// clippy::mod_module_files,
|
||||
// clippy::multiple_unsafe_ops_per_block,
|
||||
// clippy::too_many_lines,
|
||||
// clippy::missing_assert_message,
|
||||
// clippy::len_zero,
|
||||
// clippy::separated_literal_suffix,
|
||||
// clippy::single_call_fn,
|
||||
// clippy::unreachable,
|
||||
// clippy::many_single_char_names,
|
||||
// clippy::redundant_pub_crate,
|
||||
// clippy::decimal_literal_representation,
|
||||
// clippy::option_if_let_else,
|
||||
// clippy::lossy_float_literal,
|
||||
// clippy::modulo_arithmetic,
|
||||
// clippy::print_stdout,
|
||||
// clippy::module_name_repetitions,
|
||||
// clippy::no_effect,
|
||||
// clippy::semicolon_outside_block,
|
||||
// clippy::panic,
|
||||
// clippy::question_mark_used,
|
||||
// clippy::expect_used,
|
||||
// clippy::integer_division,
|
||||
// clippy::type_complexity,
|
||||
// clippy::pattern_type_mismatch,
|
||||
// clippy::arithmetic_side_effects,
|
||||
// clippy::default_trait_access,
|
||||
// clippy::similar_names,
|
||||
// clippy::needless_pass_by_value,
|
||||
// clippy::inline_always,
|
||||
// clippy::if_then_some_else_none,
|
||||
// clippy::arithmetic_side_effects,
|
||||
// clippy::float_cmp,
|
||||
// clippy::items_after_statements,
|
||||
// clippy::use_debug,
|
||||
// clippy::mem_forget,
|
||||
// clippy::else_if_without_else,
|
||||
// clippy::str_to_string,
|
||||
// clippy::branches_sharing_code,
|
||||
// clippy::impl_trait_in_params,
|
||||
// clippy::struct_excessive_bools,
|
||||
// clippy::exit,
|
||||
// // This lint is actually good but
|
||||
// // it sometimes hits false positive.
|
||||
// clippy::self_named_module_files
|
||||
|
||||
use crate::{
|
||||
error::DB_FAILURES,
|
||||
transaction::{Transaction, WriteTransaction},
|
||||
};
|
||||
use std::{ops::Deref, path::PathBuf, sync::Arc};
|
||||
clippy::module_name_repetitions,
|
||||
clippy::module_inception,
|
||||
clippy::redundant_pub_crate,
|
||||
)]
|
||||
// Allow some lints when running in debug mode.
|
||||
#![cfg_attr(debug_assertions, allow(clippy::todo, clippy::multiple_crate_versions))]
|
||||
|
||||
/// `Database` Trait implement all the methods necessary to generate transactions as well as execute specific functions. It also implement generic associated types to identify the
|
||||
/// different transaction modes (read & write) and it's native errors.
|
||||
pub trait Database<'a> {
|
||||
type TX: Transaction<'a>;
|
||||
type TXMut: WriteTransaction<'a>;
|
||||
type Error: Into<DB_FAILURES>;
|
||||
// Only allow building 64-bit targets.
|
||||
//
|
||||
// This allows us to assume 64-bit
|
||||
// invariants in code, e.g. `usize as u64`.
|
||||
#[cfg(not(target_pointer_width = "64"))]
|
||||
compile_error!("Cuprate is only compatible with 64-bit CPUs");
|
||||
|
||||
// Create a transaction from the database
|
||||
fn tx(&'a self) -> Result<Self::TX, Self::Error>;
|
||||
//---------------------------------------------------------------------------------------------------- Public API
|
||||
// Import private modules, export public types.
|
||||
//
|
||||
// Documentation for each module is
|
||||
// located in the respective file.
|
||||
|
||||
// Create a mutable transaction from the database
|
||||
fn tx_mut(&'a self) -> Result<Self::TXMut, Self::Error>;
|
||||
mod backend;
|
||||
pub use backend::ConcreteEnv;
|
||||
|
||||
// Open a database from the specified path
|
||||
fn open(path: PathBuf) -> Result<Self, Self::Error>
|
||||
where
|
||||
Self: std::marker::Sized;
|
||||
mod constants;
|
||||
pub use constants::{CUPRATE_DATABASE_DIR, CUPRATE_DATABASE_FILE, DATABASE_BACKEND};
|
||||
|
||||
// Check if the database is built.
|
||||
fn check_all_tables_exist(&'a self) -> Result<(), Self::Error>;
|
||||
mod database;
|
||||
pub use database::Database;
|
||||
|
||||
// Build the database
|
||||
fn build(&'a self) -> Result<(), Self::Error>;
|
||||
}
|
||||
mod env;
|
||||
pub use env::Env;
|
||||
|
||||
/// `Interface` is a struct containing a shared pointer to the database and transaction's to be used for the implemented method of Interface.
|
||||
pub struct Interface<'a, D: Database<'a>> {
|
||||
pub db: Arc<D>,
|
||||
pub tx: Option<<D as Database<'a>>::TXMut>,
|
||||
}
|
||||
mod error;
|
||||
pub use error::{InitError, RuntimeError};
|
||||
|
||||
// Convenient implementations for database
|
||||
impl<'service, D: Database<'service>> Interface<'service, D> {
|
||||
fn from(db: Arc<D>) -> Result<Self, DB_FAILURES> {
|
||||
Ok(Self { db, tx: None })
|
||||
}
|
||||
mod free;
|
||||
|
||||
fn open(&'service mut self) -> Result<(), DB_FAILURES> {
|
||||
let tx = self.db.tx_mut().map_err(Into::into)?;
|
||||
self.tx = Some(tx);
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
mod key;
|
||||
pub use key::{DupKey, Key};
|
||||
|
||||
impl<'service, D: Database<'service>> Deref for Interface<'service, D> {
|
||||
type Target = <D as Database<'service>>::TXMut;
|
||||
mod macros;
|
||||
|
||||
fn deref(&self) -> &Self::Target {
|
||||
return self.tx.as_ref().unwrap();
|
||||
}
|
||||
}
|
||||
}
|
||||
pub mod ops;
|
||||
|
||||
// ------------------------------------------| DatabaseTx |------------------------------------------
|
||||
mod pod;
|
||||
pub use pod::Pod;
|
||||
|
||||
pub mod transaction {
|
||||
//! This module contains the abstractions of Transactional Key/Value database functions.
|
||||
//! Any key/value database/storage engine can be implemented easily for Cuprate as long as
|
||||
//! these functions or equivalent logic exist for it.
|
||||
mod table;
|
||||
pub use table::Table;
|
||||
|
||||
use crate::{
|
||||
error::DB_FAILURES,
|
||||
table::{DupTable, Table},
|
||||
};
|
||||
pub mod tables;
|
||||
|
||||
// Abstraction of a read-only cursor, for simple tables
|
||||
#[allow(clippy::type_complexity)]
|
||||
pub trait Cursor<'t, T: Table> {
|
||||
fn first(&mut self) -> Result<Option<(T::Key, T::Value)>, DB_FAILURES>;
|
||||
mod transaction;
|
||||
pub use transaction::{RoTx, RwTx};
|
||||
|
||||
fn get_cursor(&mut self) -> Result<Option<(T::Key, T::Value)>, DB_FAILURES>;
|
||||
//---------------------------------------------------------------------------------------------------- Feature-gated
|
||||
#[cfg(feature = "service")]
|
||||
pub mod service;
|
||||
|
||||
fn last(&mut self) -> Result<Option<(T::Key, T::Value)>, DB_FAILURES>;
|
||||
|
||||
fn next(&mut self) -> Result<Option<(T::Key, T::Value)>, DB_FAILURES>;
|
||||
|
||||
fn prev(&mut self) -> Result<Option<(T::Key, T::Value)>, DB_FAILURES>;
|
||||
|
||||
fn set(&mut self, key: &T::Key) -> Result<Option<T::Value>, DB_FAILURES>;
|
||||
}
|
||||
|
||||
// Abstraction of a read-only cursor with support for duplicated tables. DupCursor inherit Cursor methods as
|
||||
// a duplicated table can be treated as a simple table.
|
||||
#[allow(clippy::type_complexity)]
|
||||
pub trait DupCursor<'t, T: DupTable>: Cursor<'t, T> {
|
||||
fn first_dup(&mut self) -> Result<Option<(T::SubKey, T::Value)>, DB_FAILURES>;
|
||||
|
||||
fn get_dup(
|
||||
&mut self,
|
||||
key: &T::Key,
|
||||
subkey: &T::SubKey,
|
||||
) -> Result<Option<T::Value>, DB_FAILURES>;
|
||||
|
||||
fn last_dup(&mut self) -> Result<Option<(T::SubKey, T::Value)>, DB_FAILURES>;
|
||||
|
||||
fn next_dup(&mut self) -> Result<Option<(T::Key, (T::SubKey, T::Value))>, DB_FAILURES>;
|
||||
|
||||
fn prev_dup(&mut self) -> Result<Option<(T::Key, (T::SubKey, T::Value))>, DB_FAILURES>;
|
||||
}
|
||||
|
||||
// Abstraction of a read-write cursor, for simple tables. WriteCursor inherit Cursor methods.
|
||||
pub trait WriteCursor<'t, T: Table>: Cursor<'t, T> {
|
||||
fn put_cursor(&mut self, key: &T::Key, value: &T::Value) -> Result<(), DB_FAILURES>;
|
||||
|
||||
fn del(&mut self) -> Result<(), DB_FAILURES>;
|
||||
}
|
||||
|
||||
// Abstraction of a read-write cursor with support for duplicated tables. DupWriteCursor inherit DupCursor and WriteCursor methods.
|
||||
pub trait DupWriteCursor<'t, T: DupTable>: WriteCursor<'t, T> {
|
||||
fn put_cursor_dup(
|
||||
&mut self,
|
||||
key: &T::Key,
|
||||
subkey: &T::SubKey,
|
||||
value: &T::Value,
|
||||
) -> Result<(), DB_FAILURES>;
|
||||
|
||||
/// Delete all data under associated to its key
|
||||
fn del_nodup(&mut self) -> Result<(), DB_FAILURES>;
|
||||
}
|
||||
|
||||
// Abstraction of a read-only transaction.
|
||||
pub trait Transaction<'a>: Send + Sync {
|
||||
type Cursor<T: Table>: Cursor<'a, T>;
|
||||
type DupCursor<T: DupTable>: DupCursor<'a, T> + Cursor<'a, T>;
|
||||
|
||||
fn get<T: Table>(&self, key: &T::Key) -> Result<Option<T::Value>, DB_FAILURES>;
|
||||
|
||||
fn commit(self) -> Result<(), DB_FAILURES>;
|
||||
|
||||
fn cursor<T: Table>(&self) -> Result<Self::Cursor<T>, DB_FAILURES>;
|
||||
|
||||
fn cursor_dup<T: DupTable>(&self) -> Result<Self::DupCursor<T>, DB_FAILURES>;
|
||||
|
||||
fn num_entries<T: Table>(&self) -> Result<usize, DB_FAILURES>;
|
||||
}
|
||||
|
||||
// Abstraction of a read-write transaction. WriteTransaction inherits Transaction methods.
|
||||
pub trait WriteTransaction<'a>: Transaction<'a> {
|
||||
type WriteCursor<T: Table>: WriteCursor<'a, T>;
|
||||
type DupWriteCursor<T: DupTable>: DupWriteCursor<'a, T> + DupCursor<'a, T>;
|
||||
|
||||
fn put<T: Table>(&self, key: &T::Key, value: &T::Value) -> Result<(), DB_FAILURES>;
|
||||
|
||||
fn delete<T: Table>(
|
||||
&self,
|
||||
key: &T::Key,
|
||||
value: &Option<T::Value>,
|
||||
) -> Result<(), DB_FAILURES>;
|
||||
|
||||
fn clear<T: Table>(&self) -> Result<(), DB_FAILURES>;
|
||||
|
||||
fn write_cursor<T: Table>(&self) -> Result<Self::WriteCursor<T>, DB_FAILURES>;
|
||||
|
||||
fn write_cursor_dup<T: DupTable>(&self) -> Result<Self::DupWriteCursor<T>, DB_FAILURES>;
|
||||
}
|
||||
}
|
||||
//---------------------------------------------------------------------------------------------------- Private
|
||||
|
|
17
database/src/macros.rs
Normal file
17
database/src/macros.rs
Normal file
|
@ -0,0 +1,17 @@
|
|||
//! General macros used throughout `cuprate-database`.
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Import
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Constants
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- TYPE
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- IMPL
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Trait Impl
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Tests
|
||||
#[cfg(test)]
|
||||
mod test {
|
||||
// use super::*;
|
||||
}
|
37
database/src/ops/alt_block.rs
Normal file
37
database/src/ops/alt_block.rs
Normal file
|
@ -0,0 +1,37 @@
|
|||
//! Alternative blocks.
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Import
|
||||
#[allow(unused_imports)] // FIXME: these traits will be eventually in the function impls.
|
||||
use crate::{
|
||||
database::Database,
|
||||
env::Env,
|
||||
table::Table,
|
||||
transaction::{RoTx, RwTx},
|
||||
ConcreteEnv,
|
||||
};
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Free Functions
|
||||
/// TODO
|
||||
pub fn add_alt_block() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn get_alt_block() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn remove_alt_block() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn get_alt_block_count() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn drop_alt_blocks() {
|
||||
todo!()
|
||||
}
|
97
database/src/ops/block.rs
Normal file
97
database/src/ops/block.rs
Normal file
|
@ -0,0 +1,97 @@
|
|||
//! Blocks.
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Import
|
||||
#[allow(unused_imports)] // FIXME: these traits will be eventually in the function impls.
|
||||
use crate::{
|
||||
database::Database,
|
||||
env::Env,
|
||||
table::Table,
|
||||
transaction::{RoTx, RwTx},
|
||||
ConcreteEnv,
|
||||
};
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Free Functions
|
||||
/// TODO
|
||||
pub fn add_block() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn add_block_data() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn pop_block() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn block_exists() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn get_block_hash() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn get_block_height() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn get_block_weight() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn get_block_already_generated_coins() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn get_block_long_term_weight() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn get_block_timestamp() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn get_block_cumulative_rct_outputs() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn get_block() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn get_block_from_height() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn get_block_header() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn get_block_header_from_height() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn get_top_block() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn get_top_block_hash() {
|
||||
todo!()
|
||||
}
|
17
database/src/ops/blockchain.rs
Normal file
17
database/src/ops/blockchain.rs
Normal file
|
@ -0,0 +1,17 @@
|
|||
//! Blockchain.
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Import
|
||||
#[allow(unused_imports)] // FIXME: these traits will be eventually in the function impls.
|
||||
use crate::{
|
||||
database::Database,
|
||||
env::Env,
|
||||
table::Table,
|
||||
transaction::{RoTx, RwTx},
|
||||
ConcreteEnv,
|
||||
};
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Free Functions
|
||||
/// TODO
|
||||
pub fn height() {
|
||||
todo!()
|
||||
}
|
21
database/src/ops/mod.rs
Normal file
21
database/src/ops/mod.rs
Normal file
|
@ -0,0 +1,21 @@
|
|||
//! Abstracted Monero database operations.
|
||||
//!
|
||||
//! This module contains many free functions that use the
|
||||
//! traits in this crate to generically call Monero-related
|
||||
//! database operations.
|
||||
//!
|
||||
//! # TODO
|
||||
//! TODO: These functions should pretty much map 1-1 to the `Request` enum.
|
||||
//!
|
||||
//! TODO: These are function names from `old_database/` for now.
|
||||
//! The actual underlying functions (e.g `get()`) aren't implemented.
|
||||
//!
|
||||
//! TODO: All of these functions need to take in generic
|
||||
//! database trait parameters (and their actual inputs).
|
||||
|
||||
pub mod alt_block;
|
||||
pub mod block;
|
||||
pub mod output;
|
||||
pub mod property;
|
||||
pub mod spent_key;
|
||||
pub mod tx;
|
42
database/src/ops/output.rs
Normal file
42
database/src/ops/output.rs
Normal file
|
@ -0,0 +1,42 @@
|
|||
//! Outputs.
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Import
|
||||
#[allow(unused_imports)] // FIXME: these traits will be eventually in the function impls.
|
||||
use crate::{
|
||||
database::Database,
|
||||
env::Env,
|
||||
table::Table,
|
||||
transaction::{RoTx, RwTx},
|
||||
ConcreteEnv,
|
||||
};
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Free Functions
|
||||
/// TODO
|
||||
pub fn add_output() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn remove_output() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn get_output() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn get_output_list() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn get_rct_num_outputs() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn get_pre_rct_num_outputs() {
|
||||
todo!()
|
||||
}
|
17
database/src/ops/property.rs
Normal file
17
database/src/ops/property.rs
Normal file
|
@ -0,0 +1,17 @@
|
|||
//! Properties.
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Import
|
||||
#[allow(unused_imports)] // FIXME: these traits will be eventually in the function impls.
|
||||
use crate::{
|
||||
database::Database,
|
||||
env::Env,
|
||||
table::Table,
|
||||
transaction::{RoTx, RwTx},
|
||||
ConcreteEnv,
|
||||
};
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Free Functions
|
||||
/// TODO
|
||||
pub fn get_blockchain_pruning_seed() {
|
||||
todo!()
|
||||
}
|
27
database/src/ops/spent_key.rs
Normal file
27
database/src/ops/spent_key.rs
Normal file
|
@ -0,0 +1,27 @@
|
|||
//! Spent keys.
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Import
|
||||
#[allow(unused_imports)] // FIXME: these traits will be eventually in the function impls.
|
||||
use crate::{
|
||||
database::Database,
|
||||
env::Env,
|
||||
table::Table,
|
||||
transaction::{RoTx, RwTx},
|
||||
ConcreteEnv,
|
||||
};
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Free Functions
|
||||
/// TODO
|
||||
pub fn add_spent_key() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn remove_spent_key() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn is_spent_key_recorded() {
|
||||
todo!()
|
||||
}
|
72
database/src/ops/tx.rs
Normal file
72
database/src/ops/tx.rs
Normal file
|
@ -0,0 +1,72 @@
|
|||
//! Transactions.
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Import
|
||||
#[allow(unused_imports)] // FIXME: these traits will be eventually in the function impls.
|
||||
use crate::{
|
||||
database::Database,
|
||||
env::Env,
|
||||
table::Table,
|
||||
transaction::{RoTx, RwTx},
|
||||
ConcreteEnv,
|
||||
};
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Free Functions
|
||||
/// TODO
|
||||
pub fn add_transaction() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn add_transaction_data() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn remove_transaction() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn remove_transaction_data() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn remove_tx_outputs() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn get_num_tx() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn tx_exists() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn get_tx_unlock_time() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn get_tx() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn get_tx_list() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn get_pruned_tx() {
|
||||
todo!()
|
||||
}
|
||||
|
||||
/// TODO
|
||||
pub fn get_tx_block_height() {
|
||||
todo!()
|
||||
}
|
481
database/src/pod.rs
Normal file
481
database/src/pod.rs
Normal file
|
@ -0,0 +1,481 @@
|
|||
//! (De)serialization for table keys & values.
|
||||
//!
|
||||
//! All keys and values in database tables must be able
|
||||
//! to be (de)serialized into/from raw bytes ([u8]).
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Import
|
||||
use std::{
|
||||
borrow::Cow,
|
||||
io::{Read, Write},
|
||||
sync::Arc,
|
||||
};
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Pod
|
||||
/// Plain Old Data.
|
||||
///
|
||||
/// Trait representing very simple types that can be
|
||||
/// (de)serialized into/from bytes.
|
||||
///
|
||||
/// Reference: <https://docs.rs/bytemuck/latest/bytemuck/trait.Pod.html>
|
||||
///
|
||||
/// ## Endianness
|
||||
/// As `bytemuck` provides everything needed here + more, it could be used,
|
||||
/// _but_, its `Pod` is endian dependant. We need to ensure bytes are the
|
||||
/// exact same such that the database stores the same bytes on different machines;
|
||||
/// so we use little endian functions instead, e.g. [`u8::to_le_bytes`].
|
||||
///
|
||||
/// This also means an `INVARIANT` of this trait is that
|
||||
/// implementors must use little endian bytes when applicable.
|
||||
///
|
||||
/// Slice types (just raw `[u8]` bytes) are (de)serialized as-is.
|
||||
///
|
||||
/// ## Sealed
|
||||
/// This trait is [`Sealed`](https://rust-lang.github.io/api-guidelines/future-proofing.html#sealed-traits-protect-against-downstream-implementations-c-sealed).
|
||||
///
|
||||
/// It cannot be implemented outside this crate,
|
||||
/// and is only implemented on specific types.
|
||||
///
|
||||
/// # TODO
|
||||
/// This could be implemented on `bytes::Bytes` if needed.
|
||||
///
|
||||
/// Maybe under a `bytes` feature flag.
|
||||
pub trait Pod: Sized + private::Sealed {
|
||||
/// Return `self` in byte form.
|
||||
///
|
||||
/// The returned bytes can be any form of array,
|
||||
/// - `[u8]`
|
||||
/// - `[u8; N]`
|
||||
/// - [`Vec<u8>`]
|
||||
///
|
||||
/// ..etc.
|
||||
///
|
||||
/// This is used on slice types (`Vec<u8>`, `[u8; N]`, etc) for cheap conversions.
|
||||
///
|
||||
/// Integer types ([`u8`], [`f32`], [`i8`], etc) return a fixed-sized array.
|
||||
fn as_bytes(&self) -> impl AsRef<[u8]>;
|
||||
|
||||
/// TODO
|
||||
fn into_bytes(self) -> Cow<'static, [u8]>;
|
||||
|
||||
/// Create [`Self`] from bytes.
|
||||
///
|
||||
/// # Panics
|
||||
/// This function should be infallible.
|
||||
///
|
||||
/// If `bytes` is invalid, this should panic.
|
||||
fn from_bytes(bytes: &[u8]) -> Self;
|
||||
|
||||
/// Convert [`Self`] into bytes, and write those bytes into a [`Write`]r.
|
||||
///
|
||||
/// The `usize` returned should be how many bytes were written.
|
||||
///
|
||||
/// TODO: do we ever actually need how many bytes were written?
|
||||
///
|
||||
/// # Panics
|
||||
/// This function should be infallible.
|
||||
///
|
||||
/// If the `writer` errors, this should panic.
|
||||
fn to_writer<W: Write>(self, writer: &mut W) -> usize;
|
||||
|
||||
/// Create [`Self`] by reading bytes from a [`Read`]er.
|
||||
///
|
||||
/// # Panics
|
||||
/// This function should be infallible.
|
||||
///
|
||||
/// If the `reader` errors, this should panic.
|
||||
fn from_reader<R: Read>(reader: &mut R) -> Self;
|
||||
}
|
||||
|
||||
/// Private module, should not be accessible outside this crate.
|
||||
///
|
||||
/// Used to block outsiders implementing [`Pod`].
|
||||
/// All [`Pod`] types must also implement [`Sealed`].
|
||||
mod private {
|
||||
/// Private sealed trait.
|
||||
///
|
||||
/// Cannot be implemented outside this crate.
|
||||
pub trait Sealed {}
|
||||
|
||||
/// Implement `Sealed`.
|
||||
macro_rules! impl_sealed {
|
||||
($(
|
||||
$t:ty // The type to implement for.
|
||||
),* $(,)?) => {
|
||||
$(
|
||||
impl Sealed for $t {}
|
||||
)*
|
||||
};
|
||||
}
|
||||
|
||||
// Special case cause of generic.
|
||||
impl<const N: usize> Sealed for [u8; N] {}
|
||||
|
||||
impl_sealed! {
|
||||
Vec<u8>,
|
||||
Box<[u8]>,
|
||||
std::sync::Arc<[u8]>,
|
||||
f32,
|
||||
f64,
|
||||
u8,
|
||||
u16,
|
||||
u32,
|
||||
u64,
|
||||
u128,
|
||||
usize,
|
||||
i8,
|
||||
i16,
|
||||
i32,
|
||||
i64,
|
||||
i128,
|
||||
isize,
|
||||
}
|
||||
}
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Pod Impl (bytes)
|
||||
// Implement for owned `Vec` bytes.
|
||||
impl Pod for Vec<u8> {
|
||||
#[inline]
|
||||
fn as_bytes(&self) -> impl AsRef<[u8]> {
|
||||
self
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn into_bytes(self) -> Cow<'static, [u8]> {
|
||||
Cow::Owned(self)
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn from_bytes(bytes: &[u8]) -> Self {
|
||||
bytes.to_vec()
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn from_reader<R: Read>(reader: &mut R) -> Self {
|
||||
// FIXME: Could be `Vec::with_capacity(likely_size)`?
|
||||
let mut vec = vec![];
|
||||
|
||||
reader
|
||||
.read_to_end(&mut vec)
|
||||
.expect("Pod::<Vec<u8>>::read_to_end() failed");
|
||||
|
||||
vec
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn to_writer<W: Write>(self, writer: &mut W) -> usize {
|
||||
writer
|
||||
.write_all(&self)
|
||||
.expect("Pod::<Vec<u8>>::write_all() failed");
|
||||
|
||||
self.len()
|
||||
}
|
||||
}
|
||||
|
||||
// Implement for any sized stack array.
|
||||
impl<const N: usize> Pod for [u8; N] {
|
||||
#[inline]
|
||||
fn as_bytes(&self) -> impl AsRef<[u8]> {
|
||||
self
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn into_bytes(self) -> Cow<'static, [u8]> {
|
||||
Cow::Owned(self.to_vec())
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn from_bytes(bytes: &[u8]) -> Self {
|
||||
// Return if the bytes are too short/long.
|
||||
let bytes_len = bytes.len();
|
||||
assert_eq!(
|
||||
bytes_len, N,
|
||||
"Pod::<[u8; {N}]>::from_bytes() failed, expected_len: {N}, found_len: {bytes_len}",
|
||||
);
|
||||
|
||||
let mut array = [0_u8; N];
|
||||
// INVARIANT: we checked the length is valid above.
|
||||
array.copy_from_slice(bytes);
|
||||
|
||||
array
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn from_reader<R: Read>(reader: &mut R) -> Self {
|
||||
let mut bytes = [0_u8; N];
|
||||
reader
|
||||
.read_exact(&mut bytes)
|
||||
.expect("Pod::<[u8; {N}]>::read_exact() failed");
|
||||
|
||||
bytes
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn to_writer<W: Write>(self, writer: &mut W) -> usize {
|
||||
writer
|
||||
.write_all(&self)
|
||||
.expect("Pod::<[u8; {N}]>::write_all() failed");
|
||||
self.len()
|
||||
}
|
||||
}
|
||||
|
||||
// Implement for any sized boxed array.
|
||||
//
|
||||
// In-case `[u8; N]` is too big and would
|
||||
// overflow the stack, this can be used.
|
||||
//
|
||||
// The benefit over `Vec<u8>` is that the capacity & length are static.
|
||||
//
|
||||
// The weird constructions of `Box` below are on purpose to avoid this:
|
||||
// <https://github.com/rust-lang/rust/issues/53827>
|
||||
impl Pod for Box<[u8]> {
|
||||
#[inline]
|
||||
fn as_bytes(&self) -> impl AsRef<[u8]> {
|
||||
self
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn into_bytes(self) -> Cow<'static, [u8]> {
|
||||
Cow::Owned(self.into())
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn from_bytes(bytes: &[u8]) -> Self {
|
||||
Self::from(bytes)
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn from_reader<R: Read>(reader: &mut R) -> Self {
|
||||
let mut bytes = vec![];
|
||||
reader
|
||||
.read_to_end(bytes.as_mut())
|
||||
.expect("Pod::<Box<[u8]>>::read_to_end() failed");
|
||||
bytes.into_boxed_slice()
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn to_writer<W: Write>(self, writer: &mut W) -> usize {
|
||||
writer
|
||||
.write_all(&self)
|
||||
.expect("Pod::<Box<[u8]>>::write_all() failed");
|
||||
self.len()
|
||||
}
|
||||
}
|
||||
|
||||
// Implement for any Arc bytes.
|
||||
impl Pod for Arc<[u8]> {
|
||||
#[inline]
|
||||
fn as_bytes(&self) -> impl AsRef<[u8]> {
|
||||
self
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn into_bytes(self) -> Cow<'static, [u8]> {
|
||||
Cow::Owned(self.to_vec())
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn from_bytes(bytes: &[u8]) -> Self {
|
||||
Self::from(bytes)
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn from_reader<R: Read>(reader: &mut R) -> Self {
|
||||
let mut bytes = vec![];
|
||||
reader
|
||||
.read_to_end(bytes.as_mut())
|
||||
.expect("Pod::<Arc<[u8]>>::read_to_end() failed");
|
||||
Self::from(bytes)
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn to_writer<W: Write>(self, writer: &mut W) -> usize {
|
||||
writer
|
||||
.write_all(&self)
|
||||
.expect("Pod::<Arc<[u8]>>::write_all() failed");
|
||||
self.len()
|
||||
}
|
||||
}
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Pod Impl (numbers)
|
||||
/// Implement `Pod` on primitive numbers.
|
||||
///
|
||||
/// This will always use little endian representations.
|
||||
macro_rules! impl_pod_le_bytes {
|
||||
($(
|
||||
$number:ident => // The number type.
|
||||
$length:literal // The length of `u8`'s this type takes up.
|
||||
),* $(,)?) => {
|
||||
$(
|
||||
impl Pod for $number {
|
||||
#[inline]
|
||||
fn as_bytes(&self) -> impl AsRef<[u8]> {
|
||||
$number::to_le_bytes(*self)
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn into_bytes(self) -> Cow<'static, [u8]> {
|
||||
Cow::Owned(self.as_bytes().as_ref().to_vec())
|
||||
}
|
||||
|
||||
#[inline]
|
||||
/// This function returns [`Err`] if `bytes`'s length is not
|
||||
#[doc = concat!(" ", stringify!($length), ".")]
|
||||
fn from_bytes(bytes: &[u8]) -> Self {
|
||||
// Return if the bytes are too short/long.
|
||||
let bytes_len = bytes.len();
|
||||
assert_eq!(
|
||||
bytes_len, $length,
|
||||
"Pod::<[u8; {0}]>::from_bytes() failed, expected_len: {0}, found_len: {bytes_len}",
|
||||
$length,
|
||||
);
|
||||
|
||||
let mut array = [0_u8; $length];
|
||||
// INVARIANT: we checked the length is valid above.
|
||||
array.copy_from_slice(bytes);
|
||||
|
||||
$number::from_le_bytes(array)
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn to_writer<W: Write>(self, writer: &mut W) -> usize {
|
||||
writer.write_all(self.as_bytes().as_ref()).expect(concat!(
|
||||
"Pod::<",
|
||||
stringify!($number),
|
||||
">::write_all() failed",
|
||||
));
|
||||
$length
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn from_reader<R: Read>(reader: &mut R) -> Self {
|
||||
let mut bytes = [0_u8; $length];
|
||||
|
||||
// Read exactly the bytes required.
|
||||
reader.read_exact(&mut bytes).expect(concat!(
|
||||
"Pod::<",
|
||||
stringify!($number),
|
||||
">::react_exact() failed",
|
||||
));
|
||||
|
||||
// INVARIANT: we checked the length is valid above.
|
||||
$number::from_le_bytes(bytes)
|
||||
}
|
||||
}
|
||||
)*
|
||||
};
|
||||
}
|
||||
|
||||
impl_pod_le_bytes! {
|
||||
f32 => 4,
|
||||
f64 => 8,
|
||||
|
||||
u8 => 1,
|
||||
u16 => 2,
|
||||
u32 => 4,
|
||||
u64 => 8,
|
||||
usize => 8,
|
||||
u128 => 16,
|
||||
|
||||
i8 => 1,
|
||||
i16 => 2,
|
||||
i32 => 4,
|
||||
i64 => 8,
|
||||
isize => 8,
|
||||
i128 => 16,
|
||||
}
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Tests
|
||||
#[cfg(test)]
|
||||
mod test {
|
||||
use super::*;
|
||||
|
||||
/// Serialize, deserialize, and compare that
|
||||
/// the intermediate/end results are correct.
|
||||
fn test_serde<const LEN: usize, T: Pod + Copy + PartialEq + std::fmt::Debug>(
|
||||
// The primitive number function that
|
||||
// converts the number into little endian bytes,
|
||||
// e.g `u8::to_le_bytes`.
|
||||
to_le_bytes: fn(T) -> [u8; LEN],
|
||||
// A `Vec` of the numbers to test.
|
||||
t: Vec<T>,
|
||||
) {
|
||||
for t in t {
|
||||
let expected_bytes = to_le_bytes(t);
|
||||
|
||||
println!("testing: {t:?}, expected_bytes: {expected_bytes:?}");
|
||||
|
||||
let mut bytes = vec![];
|
||||
|
||||
// (De)serialize.
|
||||
let se: usize = t.to_writer::<Vec<u8>>(bytes.as_mut());
|
||||
let de: T = T::from_reader::<&[u8]>(&mut bytes.as_slice());
|
||||
|
||||
println!("written: {se}, deserialize_t: {de:?}, bytes: {bytes:?}\n");
|
||||
|
||||
// Assert we wrote correct amount of bytes
|
||||
// and deserialized correctly.
|
||||
assert_eq!(se, expected_bytes.len());
|
||||
assert_eq!(de, t);
|
||||
}
|
||||
}
|
||||
|
||||
/// Create all the float tests.
|
||||
macro_rules! test_float {
|
||||
($(
|
||||
$float:ident // The float type.
|
||||
),* $(,)?) => {
|
||||
$(
|
||||
#[test]
|
||||
fn $float() {
|
||||
test_serde(
|
||||
$float::to_le_bytes,
|
||||
vec![
|
||||
-1.0,
|
||||
0.0,
|
||||
1.0,
|
||||
$float::MIN,
|
||||
$float::MAX,
|
||||
$float::INFINITY,
|
||||
$float::NEG_INFINITY,
|
||||
],
|
||||
);
|
||||
}
|
||||
)*
|
||||
};
|
||||
}
|
||||
|
||||
test_float! {
|
||||
f32,
|
||||
f64,
|
||||
}
|
||||
|
||||
/// Create all the (un)signed number tests.
|
||||
/// u8 -> u128, i8 -> i128.
|
||||
macro_rules! test_unsigned {
|
||||
($(
|
||||
$number:ident // The integer type.
|
||||
),* $(,)?) => {
|
||||
$(
|
||||
#[test]
|
||||
fn $number() {
|
||||
test_serde($number::to_le_bytes, vec![$number::MIN, 0, 1, $number::MAX]);
|
||||
}
|
||||
)*
|
||||
};
|
||||
}
|
||||
|
||||
test_unsigned! {
|
||||
u8,
|
||||
u16,
|
||||
u32,
|
||||
u64,
|
||||
u128,
|
||||
usize,
|
||||
i8,
|
||||
i16,
|
||||
i32,
|
||||
i64,
|
||||
i128,
|
||||
isize,
|
||||
}
|
||||
}
|
79
database/src/service/free.rs
Normal file
79
database/src/service/free.rs
Normal file
|
@ -0,0 +1,79 @@
|
|||
//! General free functions used (related to `cuprate_database::service`).
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Import
|
||||
use std::sync::Arc;
|
||||
|
||||
use crate::{
|
||||
service::read::DatabaseReader,
|
||||
service::write::DatabaseWriter,
|
||||
service::{DatabaseReadHandle, DatabaseWriteHandle},
|
||||
ConcreteEnv,
|
||||
};
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Init
|
||||
#[cold]
|
||||
#[inline(never)] // Only called once.
|
||||
/// Initialize a database & thread-pool, and return a read/write handle to it.
|
||||
///
|
||||
/// The returned handles are cheaply [`Clone`]able.
|
||||
///
|
||||
/// TODO: add blocking behavior docs.
|
||||
pub fn init() -> (DatabaseReadHandle, DatabaseWriteHandle) {
|
||||
// TODO:
|
||||
// This should only ever be called once?
|
||||
// We could `panic!()` if called twice.
|
||||
|
||||
// Initialize the database itself.
|
||||
// TODO: there's probably shutdown code we have to run.
|
||||
let db: Arc<ConcreteEnv> = Arc::new(todo!());
|
||||
|
||||
// Spawn the `Reader/Writer` thread pools.
|
||||
let readers = DatabaseReader::init(&db);
|
||||
let writers = DatabaseWriter::init(&db);
|
||||
|
||||
// Return the handles to those pools.
|
||||
(readers, writers)
|
||||
}
|
||||
|
||||
#[cold]
|
||||
#[inline(never)] // Only called once.
|
||||
/// Sync/flush all data, and shutdown the database thread-pool.
|
||||
///
|
||||
/// This function **blocks**, waiting until:
|
||||
/// 1. All database transactions are complete
|
||||
/// 2. All data has been flushed to disk
|
||||
/// 3. All database threads have exited
|
||||
///
|
||||
/// The database being shutdown is the one started in [`init()`],
|
||||
/// aka, the single program global database.
|
||||
///
|
||||
/// # TODO
|
||||
/// Maybe the visibility/access of this function should somehow be
|
||||
/// limited such that only certain parts of `cuprate` can actually
|
||||
/// call this function.
|
||||
///
|
||||
/// Anyone/everyone being able to shutdown the database seems dangerous.
|
||||
///
|
||||
/// Counter-argument: we can just CTRL+F to see who calls this i guess.
|
||||
pub fn shutdown(db: Arc<ConcreteEnv>) {
|
||||
// Not sure how this function is going
|
||||
// to work on a `&'static` database, but:
|
||||
|
||||
// 1. Send a shutdown message to all database threads, maybe `Request::Shutdown`
|
||||
// 2. Wait on barrier until all threads are "ready" (all tx's are done)
|
||||
// 3. Writer thread will flush all data to disk
|
||||
// 4. All threads exit, 1 of them sends us back an OK
|
||||
// 5. We don't need to reclaim ownership of `&'static ConcreteEnv` because...
|
||||
// 5a) a bunch of threads have a `&` to it, so this is hard (impossible?)
|
||||
// 5b) as along as data is flushed, we can just `std::process::exit`
|
||||
// and there's no need to (manually) drop the actual database
|
||||
|
||||
drop(db);
|
||||
todo!();
|
||||
}
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Tests
|
||||
#[cfg(test)]
|
||||
mod test {
|
||||
// use super::*;
|
||||
}
|
61
database/src/service/mod.rs
Normal file
61
database/src/service/mod.rs
Normal file
|
@ -0,0 +1,61 @@
|
|||
//! [`tower::Service`] integeration + thread-pool.
|
||||
//!
|
||||
//! ## `service`
|
||||
//! The `service` module implements the [`tower`] integration,
|
||||
//! along with the reader/writer thread-pool system.
|
||||
//!
|
||||
//! The thread-pool allows outside crates to communicate with it by
|
||||
//! sending database [`Request`](ReadRequest)s and receiving [`Response`]s `async`hronously -
|
||||
//! without having to actually worry and handle the database themselves.
|
||||
//!
|
||||
//! The system is managed by this crate, and only
|
||||
//! requires [`init`] and [`shutdown`] by the user.
|
||||
//!
|
||||
//! This module must be enabled with the `service` feature.
|
||||
//!
|
||||
//! ## Initialization
|
||||
//! The database & thread-pool system can be initialized with [`init()`].
|
||||
//!
|
||||
//! This causes the underlying database/threads to be setup
|
||||
//! and returns a read/write handle to that database.
|
||||
//!
|
||||
//! ## Handles
|
||||
//! The 2 handles to the database are:
|
||||
//! - [`DatabaseReadHandle`]
|
||||
//! - [`DatabaseWriteHandle`]
|
||||
//!
|
||||
//! The 1st allows any caller to send [`ReadRequest`]s.
|
||||
//!
|
||||
//! The 2nd allows any caller to send [`WriteRequest`]s.
|
||||
//!
|
||||
//! The `DatabaseReadHandle` can be shared as it is cheaply [`Clone`]able, however,
|
||||
//! the `DatabaseWriteHandle` cannot be cloned. There is only 1 place in Cuprate that
|
||||
//! writes, so it is passed there and used.
|
||||
//!
|
||||
//! ## Request and Response
|
||||
//! To interact with the database (whether reading or writing data),
|
||||
//! a `Request` can be sent using one of the above handles.
|
||||
//!
|
||||
//! Both the handles implement `tower::Service`, so they can be [`tower::Service::call`]ed.
|
||||
//!
|
||||
//! An `async`hronous channel will be returned from the call.
|
||||
//! This channel can be `.await`ed upon to (eventually) receive
|
||||
//! corresponding `Response` to your `Request`.
|
||||
|
||||
mod read;
|
||||
pub use read::DatabaseReadHandle;
|
||||
|
||||
mod write;
|
||||
pub use write::DatabaseWriteHandle;
|
||||
|
||||
mod free;
|
||||
pub use free::{init, shutdown};
|
||||
|
||||
mod request;
|
||||
pub use request::{ReadRequest, WriteRequest};
|
||||
|
||||
mod response;
|
||||
pub use response::Response;
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests;
|
191
database/src/service/read.rs
Normal file
191
database/src/service/read.rs
Normal file
|
@ -0,0 +1,191 @@
|
|||
//! Database read thread-pool definitions and logic.
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Import
|
||||
use std::{
|
||||
sync::Arc,
|
||||
task::{Context, Poll},
|
||||
};
|
||||
|
||||
use cuprate_helper::asynch::InfallibleOneshotReceiver;
|
||||
|
||||
use crate::{
|
||||
error::RuntimeError,
|
||||
service::{request::ReadRequest, response::Response},
|
||||
ConcreteEnv,
|
||||
};
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Types
|
||||
/// The actual type of the response.
|
||||
///
|
||||
/// Either our [Response], or a database error occurred.
|
||||
type ResponseResult = Result<Response, RuntimeError>;
|
||||
|
||||
/// The `Receiver` channel that receives the read response.
|
||||
///
|
||||
/// This is owned by the caller (the reader)
|
||||
/// who `.await`'s for the response.
|
||||
///
|
||||
/// The channel itself should never fail,
|
||||
/// but the actual database operation might.
|
||||
type ResponseRecv = InfallibleOneshotReceiver<ResponseResult>;
|
||||
|
||||
/// The `Sender` channel for the response.
|
||||
///
|
||||
/// The database reader thread uses this to send
|
||||
/// the database result to the caller.
|
||||
type ResponseSend = tokio::sync::oneshot::Sender<ResponseResult>;
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- DatabaseReadHandle
|
||||
/// Read handle to the database.
|
||||
///
|
||||
/// This is cheaply [`Clone`]able handle that
|
||||
/// allows `async`hronously reading from the database.
|
||||
///
|
||||
/// Calling [`tower::Service::call`] with a [`DatabaseReadHandle`] & [`ReadRequest`]
|
||||
/// will return an `async`hronous channel that can be `.await`ed upon
|
||||
/// to receive the corresponding [`Response`].
|
||||
#[derive(Clone, Debug)]
|
||||
pub struct DatabaseReadHandle {
|
||||
/// Sender channel to the database read thread-pool.
|
||||
///
|
||||
/// We provide the response channel for the thread-pool.
|
||||
pub(super) sender: crossbeam::channel::Sender<(ReadRequest, ResponseSend)>,
|
||||
}
|
||||
|
||||
impl tower::Service<ReadRequest> for DatabaseReadHandle {
|
||||
type Response = Response;
|
||||
type Error = RuntimeError;
|
||||
type Future = ResponseRecv;
|
||||
|
||||
#[inline]
|
||||
fn poll_ready(&mut self, _cx: &mut Context<'_>) -> Poll<Result<(), Self::Error>> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn call(&mut self, _req: ReadRequest) -> Self::Future {
|
||||
todo!()
|
||||
}
|
||||
}
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- DatabaseReader Impl
|
||||
/// Database reader thread.
|
||||
///
|
||||
/// This struct essentially represents a thread.
|
||||
///
|
||||
/// Each reader thread is spawned with access to this struct (self).
|
||||
pub(super) struct DatabaseReader {
|
||||
/// Receiver side of the database request channel.
|
||||
///
|
||||
/// Any caller can send some requests to this channel.
|
||||
/// They send them alongside another `Response` channel,
|
||||
/// which we will eventually send to.
|
||||
///
|
||||
/// We (the database reader thread) are not responsible
|
||||
/// for creating this channel, the caller provides it.
|
||||
///
|
||||
/// SOMEDAY: this struct itself could cache a return channel
|
||||
/// instead of creating a new `oneshot` each request.
|
||||
receiver: crossbeam::channel::Receiver<(ReadRequest, ResponseSend)>,
|
||||
|
||||
/// Access to the database.
|
||||
db: Arc<ConcreteEnv>,
|
||||
}
|
||||
|
||||
impl DatabaseReader {
|
||||
/// Initialize the `DatabaseReader` thread-pool.
|
||||
///
|
||||
/// This spawns `N` amount of `DatabaseReader`'s
|
||||
/// attached to `db` and returns a handle to the pool.
|
||||
///
|
||||
/// Should be called _once_ per actual database.
|
||||
#[cold]
|
||||
#[inline(never)] // Only called once.
|
||||
pub(super) fn init(db: &Arc<ConcreteEnv>) -> DatabaseReadHandle {
|
||||
// Initialize `Request/Response` channels.
|
||||
let (sender, receiver) = crossbeam::channel::unbounded();
|
||||
|
||||
// TODO: slightly _less_ readers per thread may be more ideal.
|
||||
// We could account for the writer count as well such that
|
||||
// readers + writers == total_thread_count
|
||||
//
|
||||
// TODO: take in a config option that allows
|
||||
// manually adjusting this thread-count.
|
||||
let readers = cuprate_helper::thread::threads().get();
|
||||
|
||||
// Spawn pool of readers.
|
||||
for _ in 0..readers {
|
||||
let receiver = receiver.clone();
|
||||
let db = db.clone();
|
||||
|
||||
std::thread::spawn(move || {
|
||||
let this = Self { receiver, db };
|
||||
|
||||
Self::main(this);
|
||||
});
|
||||
}
|
||||
|
||||
// Return a handle to the pool.
|
||||
DatabaseReadHandle { sender }
|
||||
}
|
||||
|
||||
/// The `DatabaseReader`'s main function.
|
||||
///
|
||||
/// Each thread just loops in this function.
|
||||
#[cold]
|
||||
#[inline(never)] // Only called once.
|
||||
fn main(mut self) {
|
||||
loop {
|
||||
// 1. Hang on request channel
|
||||
// 2. Map request to some database function
|
||||
// 3. Execute that function, get the result
|
||||
// 4. Return the result via channel
|
||||
let (request, response_send) = match self.receiver.recv() {
|
||||
Ok((r, c)) => (r, c),
|
||||
|
||||
// Shutdown on error.
|
||||
Err(e) => {
|
||||
Self::shutdown(self);
|
||||
return;
|
||||
}
|
||||
};
|
||||
|
||||
// Map [`Request`]'s to specific database functions.
|
||||
match request {
|
||||
ReadRequest::Example1 => self.example_handler_1(response_send),
|
||||
ReadRequest::Example2(_x) => self.example_handler_2(response_send),
|
||||
ReadRequest::Example3(_x) => self.example_handler_3(response_send),
|
||||
ReadRequest::Shutdown => {
|
||||
/* TODO: run shutdown code */
|
||||
Self::shutdown(self);
|
||||
|
||||
// Return, exiting the thread.
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// TODO
|
||||
fn example_handler_1(&mut self, response_send: ResponseSend) {
|
||||
let db_result = todo!();
|
||||
response_send.send(db_result).unwrap();
|
||||
}
|
||||
|
||||
/// TODO
|
||||
fn example_handler_2(&mut self, response_send: ResponseSend) {
|
||||
let db_result = todo!();
|
||||
response_send.send(db_result).unwrap();
|
||||
}
|
||||
|
||||
/// TODO
|
||||
fn example_handler_3(&mut self, response_send: ResponseSend) {
|
||||
let db_result = todo!();
|
||||
response_send.send(db_result).unwrap();
|
||||
}
|
||||
|
||||
/// TODO
|
||||
fn shutdown(self) {
|
||||
todo!()
|
||||
}
|
||||
}
|
45
database/src/service/request.rs
Normal file
45
database/src/service/request.rs
Normal file
|
@ -0,0 +1,45 @@
|
|||
//! Read/write `Request`s to the database.
|
||||
//!
|
||||
//! TODO: could add `strum` derives.
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Import
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Constants
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- ReadRequest
|
||||
#[derive(Debug)]
|
||||
/// A read request to the database.
|
||||
pub enum ReadRequest {
|
||||
/// TODO
|
||||
Example1,
|
||||
/// TODO
|
||||
Example2(usize),
|
||||
/// TODO
|
||||
Example3(String),
|
||||
/// TODO
|
||||
Shutdown,
|
||||
}
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- WriteRequest
|
||||
#[derive(Debug)]
|
||||
/// A write request to the database.
|
||||
pub enum WriteRequest {
|
||||
/// TODO
|
||||
Example1,
|
||||
/// TODO
|
||||
Example2(usize),
|
||||
/// TODO
|
||||
Example3(String),
|
||||
///
|
||||
Shutdown,
|
||||
}
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- IMPL
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Trait Impl
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Tests
|
||||
#[cfg(test)]
|
||||
mod test {
|
||||
// use super::*;
|
||||
}
|
38
database/src/service/response.rs
Normal file
38
database/src/service/response.rs
Normal file
|
@ -0,0 +1,38 @@
|
|||
//! Read/write `Response`'s from the database.
|
||||
//!
|
||||
//! TODO: could add `strum` derives.
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Import
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Constants
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Response
|
||||
#[derive(Debug)]
|
||||
/// A response from the database.
|
||||
///
|
||||
/// TODO
|
||||
pub enum Response {
|
||||
//-------------------------------------------------------- Read responses
|
||||
/// TODO
|
||||
Example1,
|
||||
/// TODO
|
||||
Example2(usize),
|
||||
/// TODO
|
||||
Example3(String),
|
||||
|
||||
//-------------------------------------------------------- Write responses
|
||||
/// The response
|
||||
///
|
||||
/// TODO
|
||||
ExampleWriteResponse, // Probably will be just `Ok`
|
||||
}
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- IMPL
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Trait Impl
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Tests
|
||||
#[cfg(test)]
|
||||
mod test {
|
||||
// use super::*;
|
||||
}
|
14
database/src/service/tests.rs
Normal file
14
database/src/service/tests.rs
Normal file
|
@ -0,0 +1,14 @@
|
|||
//! `crate::service` tests.
|
||||
//!
|
||||
//! This module contains general tests for the `service` implementation.
|
||||
//!
|
||||
//! Testing a thread-pool is slightly more complicated,
|
||||
//! so this file provides TODO.
|
||||
|
||||
// This is only imported on `#[cfg(test)]` in `mod.rs`.
|
||||
|
||||
#[test]
|
||||
const fn test() {
|
||||
// TODO: remove me.
|
||||
// Just to see if the module gets imported correctly on test mode.
|
||||
}
|
157
database/src/service/write.rs
Normal file
157
database/src/service/write.rs
Normal file
|
@ -0,0 +1,157 @@
|
|||
//! Database write thread-pool definitions and logic.
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Import
|
||||
use std::{
|
||||
sync::Arc,
|
||||
task::{Context, Poll},
|
||||
};
|
||||
|
||||
use cuprate_helper::asynch::InfallibleOneshotReceiver;
|
||||
|
||||
use crate::{
|
||||
error::RuntimeError,
|
||||
service::{request::WriteRequest, response::Response},
|
||||
ConcreteEnv,
|
||||
};
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Types
|
||||
/// The actual type of the response.
|
||||
///
|
||||
/// Either our [Response], or a database error occurred.
|
||||
type ResponseResult = Result<Response, RuntimeError>;
|
||||
|
||||
/// The `Receiver` channel that receives the write response.
|
||||
///
|
||||
/// The channel itself should never fail,
|
||||
/// but the actual database operation might.
|
||||
type ResponseRecv = InfallibleOneshotReceiver<ResponseResult>;
|
||||
|
||||
/// The `Sender` channel for the response.
|
||||
type ResponseSend = tokio::sync::oneshot::Sender<ResponseResult>;
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- DatabaseWriteHandle
|
||||
/// Write handle to the database.
|
||||
///
|
||||
/// This is handle that allows `async`hronously writing to the database,
|
||||
/// it is not [`Clone`]able as there is only ever 1 place within Cuprate
|
||||
/// that writes.
|
||||
///
|
||||
/// Calling [`tower::Service::call`] with a [`DatabaseWriteHandle`] & [`WriteRequest`]
|
||||
/// will return an `async`hronous channel that can be `.await`ed upon
|
||||
/// to receive the corresponding [`Response`].
|
||||
#[derive(Debug)]
|
||||
pub struct DatabaseWriteHandle {
|
||||
/// Sender channel to the database write thread-pool.
|
||||
///
|
||||
/// We provide the response channel for the thread-pool.
|
||||
pub(super) sender: crossbeam::channel::Sender<(WriteRequest, ResponseSend)>,
|
||||
}
|
||||
|
||||
impl tower::Service<WriteRequest> for DatabaseWriteHandle {
|
||||
type Response = Response;
|
||||
type Error = RuntimeError;
|
||||
type Future = ResponseRecv;
|
||||
|
||||
#[inline]
|
||||
fn poll_ready(&mut self, _cx: &mut Context<'_>) -> Poll<Result<(), Self::Error>> {
|
||||
todo!()
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn call(&mut self, _req: WriteRequest) -> Self::Future {
|
||||
todo!()
|
||||
}
|
||||
}
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- DatabaseWriter
|
||||
/// The single database writer thread.
|
||||
pub(super) struct DatabaseWriter {
|
||||
/// Receiver side of the database request channel.
|
||||
///
|
||||
/// Any caller can send some requests to this channel.
|
||||
/// They send them alongside another `Response` channel,
|
||||
/// which we will eventually send to.
|
||||
receiver: crossbeam::channel::Receiver<(WriteRequest, ResponseSend)>,
|
||||
|
||||
/// Access to the database.
|
||||
db: Arc<ConcreteEnv>,
|
||||
}
|
||||
|
||||
impl DatabaseWriter {
|
||||
/// Initialize the single `DatabaseWriter` thread.
|
||||
#[cold]
|
||||
#[inline(never)] // Only called once.
|
||||
pub(super) fn init(db: &Arc<ConcreteEnv>) -> DatabaseWriteHandle {
|
||||
// Initialize `Request/Response` channels.
|
||||
let (sender, receiver) = crossbeam::channel::unbounded();
|
||||
|
||||
// Spawn the writer.
|
||||
let db = Arc::clone(db);
|
||||
std::thread::spawn(move || {
|
||||
let this = Self { receiver, db };
|
||||
|
||||
Self::main(this);
|
||||
});
|
||||
|
||||
// Return a handle to the pool.
|
||||
DatabaseWriteHandle { sender }
|
||||
}
|
||||
|
||||
/// The `DatabaseWriter`'s main function.
|
||||
///
|
||||
/// The writer just loops in this function.
|
||||
#[cold]
|
||||
#[inline(never)] // Only called once.
|
||||
fn main(mut self) {
|
||||
loop {
|
||||
// 1. Hang on request channel
|
||||
// 2. Map request to some database function
|
||||
// 3. Execute that function, get the result
|
||||
// 4. Return the result via channel
|
||||
let (request, response_send) = match self.receiver.recv() {
|
||||
Ok(tuple) => tuple,
|
||||
Err(e) => {
|
||||
// TODO: what to do with this channel error?
|
||||
todo!();
|
||||
}
|
||||
};
|
||||
|
||||
// Map [`Request`]'s to specific database functions.
|
||||
match request {
|
||||
WriteRequest::Example1 => self.example_handler_1(response_send),
|
||||
WriteRequest::Example2(_x) => self.example_handler_2(response_send),
|
||||
WriteRequest::Example3(_x) => self.example_handler_3(response_send),
|
||||
WriteRequest::Shutdown => {
|
||||
/* TODO: run shutdown code */
|
||||
Self::shutdown(self);
|
||||
|
||||
// Return, exiting the thread.
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// TODO
|
||||
fn example_handler_1(&mut self, response_send: ResponseSend) {
|
||||
let db_result = todo!();
|
||||
response_send.send(db_result).unwrap();
|
||||
}
|
||||
|
||||
/// TODO
|
||||
fn example_handler_2(&mut self, response_send: ResponseSend) {
|
||||
let db_result = todo!();
|
||||
response_send.send(db_result).unwrap();
|
||||
}
|
||||
|
||||
/// TODO
|
||||
fn example_handler_3(&mut self, response_send: ResponseSend) {
|
||||
let db_result = todo!();
|
||||
response_send.send(db_result).unwrap();
|
||||
}
|
||||
|
||||
/// TODO
|
||||
fn shutdown(self) {
|
||||
todo!()
|
||||
}
|
||||
}
|
|
@ -1,181 +1,49 @@
|
|||
//! ### Table module
|
||||
//! This module contains the definition of the [`Table`] and [`DupTable`] trait, and the actual tables used in the database.
|
||||
//! [`DupTable`] are just a trait used to define that they support DUPSORT|DUPFIXED operation (as of now we don't know the equivalent for HSE).
|
||||
//! All tables are defined with docs explaining its purpose, what types are the key and data.
|
||||
//! For more details please look at Cuprate's book : <link to cuprate book>
|
||||
//! Database table abstraction; `trait Table`.
|
||||
|
||||
use crate::{
|
||||
encoding::Compat,
|
||||
types::{
|
||||
/*OutTx,*/ AltBlock, BlockMetadata, /*RctOutkey,*/ OutputMetadata,
|
||||
TransactionPruned, TxIndex, /*OutAmountIdx,*/ /*KeyImage,*/ TxOutputIdx,
|
||||
},
|
||||
};
|
||||
use bincode::{de::Decode, enc::Encode};
|
||||
use monero::{blockdata::transaction::KeyImage, Block, Hash};
|
||||
//---------------------------------------------------------------------------------------------------- Import
|
||||
use crate::{key::Key, pod::Pod};
|
||||
|
||||
/// A trait implementing a table interaction for the database. It is implemented to an empty struct to specify the name and table's associated types. These associated
|
||||
/// types are used to simplify deserialization process.
|
||||
pub trait Table: Send + Sync + 'static + Clone {
|
||||
// name of the table
|
||||
const TABLE_NAME: &'static str;
|
||||
//---------------------------------------------------------------------------------------------------- Table
|
||||
/// Database table metadata.
|
||||
///
|
||||
/// Purely compile time information for database tables.
|
||||
/// Not really an accurate name for `K/V` database but
|
||||
/// this represents the metadata of a `K/V` storing object.
|
||||
pub trait Table {
|
||||
// TODO:
|
||||
//
|
||||
// Add K/V comparison `type`s that define
|
||||
// how this table will be stored.
|
||||
//
|
||||
// type KeyComparator: fn(&Self::Key, &Self::Key) -> Ordering;
|
||||
// type ValueComparator: fn(&Self::Value, &Self::Value) -> Ordering;
|
||||
|
||||
// Definition of a key & value types of the database
|
||||
type Key: Encode + Decode;
|
||||
type Value: Encode + Decode;
|
||||
}
|
||||
/// Name of the database table.
|
||||
const NAME: &'static str;
|
||||
|
||||
/// A trait implementing a table with duplicated data support.
|
||||
pub trait DupTable: Table {
|
||||
// Subkey of the table (prefix of the data)
|
||||
type SubKey: Encode + Decode;
|
||||
}
|
||||
/// Whether the table's values are all the same size or not.
|
||||
const CONSTANT_SIZE: bool;
|
||||
|
||||
/// This declarative macro declare a new empty struct and impl the specified name, and corresponding types.
|
||||
macro_rules! impl_table {
|
||||
( $(#[$docs:meta])* $table:ident , $key:ty , $value:ty ) => {
|
||||
#[derive(Clone)]
|
||||
$(#[$docs])*
|
||||
pub(crate) struct $table;
|
||||
// TODO: fix this sanakirja bound.
|
||||
cfg_if::cfg_if! {
|
||||
if #[cfg(all(feature = "sanakirja", not(feature = "heed")))] {
|
||||
/// Primary key type.
|
||||
type Key: Key + Pod + sanakirja::Storable;
|
||||
|
||||
impl Table for $table {
|
||||
const TABLE_NAME: &'static str = stringify!($table);
|
||||
type Key = $key;
|
||||
type Value = $value;
|
||||
/// Value type.
|
||||
type Value: Pod + sanakirja::Storable;
|
||||
} else {
|
||||
/// Primary key type.
|
||||
type Key: Key + Pod;
|
||||
|
||||
/// Value type.
|
||||
type Value: Pod;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
/// This declarative macro declare extend the original impl_table! macro by implementy DupTable trait.
|
||||
macro_rules! impl_duptable {
|
||||
($(#[$docs:meta])* $table:ident, $key:ty, $subkey:ty, $value:ty) => {
|
||||
impl_table!($(#[$docs])* $table, $key, $value);
|
||||
|
||||
impl DupTable for $table {
|
||||
type SubKey = $subkey;
|
||||
}
|
||||
};
|
||||
//---------------------------------------------------------------------------------------------------- Tests
|
||||
#[cfg(test)]
|
||||
mod test {
|
||||
// use super::*;
|
||||
}
|
||||
|
||||
// ------------------------------------------| Tables definition |------------------------------------------
|
||||
|
||||
// ----- BLOCKS -----
|
||||
|
||||
impl_duptable!(
|
||||
/// `blockhash` is table defining a relation between the hash of a block and its height. Its primary use is to quickly find block's hash by its height.
|
||||
blockhash,
|
||||
(),
|
||||
Compat<Hash>,
|
||||
u64
|
||||
);
|
||||
|
||||
impl_duptable!(
|
||||
/// `blockmetadata` store block metadata alongside their corresponding Hash. The blocks metadata can contains the total_coins_generated, weight, long_term_block_weight & cumulative RingCT
|
||||
blockmetadata,
|
||||
(),
|
||||
u64,
|
||||
BlockMetadata
|
||||
);
|
||||
|
||||
impl_table!(
|
||||
/// `blockbody` store blocks' bodies along their Hash. The blocks body contains the coinbase transaction and its corresponding mined transactions' hashes.
|
||||
blocks,
|
||||
u64,
|
||||
Compat<Block>
|
||||
);
|
||||
|
||||
/*
|
||||
impl_table!(
|
||||
/// `blockhfversion` keep track of block's hard fork version. If an outdated node continue to run after a hard fork, it needs to know, after updating, what blocks needs to be update.
|
||||
blockhfversion, u64, u8);
|
||||
*/
|
||||
|
||||
impl_table!(
|
||||
/// `altblock` is a table that permits the storage of blocks from an alternative chain, which may cause a re-org. These blocks can be fetch by their corresponding hash.
|
||||
altblock,
|
||||
Compat<Hash>,
|
||||
AltBlock
|
||||
);
|
||||
|
||||
// ------- TXNs -------
|
||||
|
||||
impl_table!(
|
||||
/// `txspruned` is table storing TransactionPruned (or Pruned Tx). These can be fetch by the corresponding Transaction ID.
|
||||
txspruned,
|
||||
u64,
|
||||
TransactionPruned
|
||||
);
|
||||
|
||||
impl_table!(
|
||||
/// `txsprunable` is a table storing the Prunable part of transactions (Signatures and RctSig), stored as raw bytes. These can be fetch by the corresponding Transaction ID.
|
||||
txsprunable,
|
||||
u64,
|
||||
Vec<u8>
|
||||
);
|
||||
|
||||
impl_duptable!(
|
||||
/// `txsprunablehash` is a table storing hashes of prunable part of transactions. These hash can be fetch by the corresponding Transaction ID.
|
||||
txsprunablehash,
|
||||
u64,
|
||||
(),
|
||||
Compat<Hash>
|
||||
);
|
||||
|
||||
impl_table!(
|
||||
/// `txsprunabletip` is a table used for optimization purpose. It defines at which block's height this transaction belong as long as the block is with Tip blocks. These can be fetch by the corresponding Transaction ID.
|
||||
txsprunabletip,
|
||||
u64,
|
||||
u64
|
||||
);
|
||||
|
||||
impl_duptable!(
|
||||
/// `txsoutputs` is a table storing output indices used in a transaction. These can be fetch by the corresponding Transaction ID.
|
||||
txsoutputs,
|
||||
u64,
|
||||
(),
|
||||
TxOutputIdx
|
||||
);
|
||||
|
||||
impl_duptable!(
|
||||
/// `txsidentifier` is a table defining a relation between the hash of a transaction and its transaction Indexes. Its primarly used to quickly find tx's ID by its hash.
|
||||
txsidentifier,
|
||||
Compat<Hash>,
|
||||
(),
|
||||
TxIndex
|
||||
);
|
||||
|
||||
// ---- OUTPUTS ----
|
||||
|
||||
impl_duptable!(
|
||||
/// `prerctoutputmetadata` is a duplicated table storing Pre-RingCT output's metadata. The key is the amount of this output, and the subkey is its amount idx.
|
||||
prerctoutputmetadata,
|
||||
u64,
|
||||
u64,
|
||||
OutputMetadata
|
||||
);
|
||||
impl_duptable!(
|
||||
/// `prerctoutputmetadata` is a table storing RingCT output's metadata. The key is the amount idx of this output since amount is always 0 for RingCT outputs.
|
||||
outputmetadata,
|
||||
(),
|
||||
u64,
|
||||
OutputMetadata
|
||||
);
|
||||
|
||||
// ---- SPT KEYS ----
|
||||
|
||||
impl_duptable!(
|
||||
/// `spentkeys`is a table storing every KeyImage that have been used to create decoys input. As these KeyImage can't be re used they need to marked.
|
||||
spentkeys,
|
||||
(),
|
||||
Compat<KeyImage>,
|
||||
()
|
||||
);
|
||||
|
||||
// ---- PROPERTIES ----
|
||||
|
||||
impl_table!(
|
||||
/// `spentkeys`is a table storing every KeyImage that have been used to create decoys input. As these KeyImage can't be re used they need to marked.
|
||||
properties,
|
||||
u32,
|
||||
u32
|
||||
);
|
||||
|
|
128
database/src/tables.rs
Normal file
128
database/src/tables.rs
Normal file
|
@ -0,0 +1,128 @@
|
|||
//! Database tables.
|
||||
//!
|
||||
//! This module contains all the table definitions used by `cuprate-database`
|
||||
//! and [`Tables`], an `enum` containing all [`Table`]s.
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Import
|
||||
use crate::table::Table;
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Tables
|
||||
/// An enumeration of _all_ database tables.
|
||||
///
|
||||
/// TODO: I don't think we need this.
|
||||
#[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))]
|
||||
#[cfg_attr(
|
||||
feature = "borsh",
|
||||
derive(borsh::BorshSerialize, borsh::BorshDeserialize)
|
||||
)]
|
||||
#[derive(Copy, Clone, Debug, PartialEq, PartialOrd, Eq, Ord, Hash)]
|
||||
#[allow(missing_docs)]
|
||||
pub enum Tables {
|
||||
TestTable(TestTable),
|
||||
TestTable2(TestTable2),
|
||||
}
|
||||
|
||||
impl Tables {
|
||||
/// Get the [`Table::NAME`].
|
||||
pub const fn name(&self) -> &'static str {
|
||||
/// Hack to access associated trait constant via a variable.
|
||||
const fn get<T: Table>(t: &T) -> &'static str {
|
||||
T::NAME
|
||||
}
|
||||
|
||||
match self {
|
||||
Self::TestTable(t) => get(t),
|
||||
Self::TestTable2(t) => get(t),
|
||||
}
|
||||
}
|
||||
|
||||
/// Get the [`Table::CONSTANT_SIZE`].
|
||||
pub const fn constant_size(&self) -> bool {
|
||||
/// Hack to access associated trait constant via a variable.
|
||||
const fn get<T: Table>(t: &T) -> bool {
|
||||
T::CONSTANT_SIZE
|
||||
}
|
||||
|
||||
match self {
|
||||
Self::TestTable(t) => get(t),
|
||||
Self::TestTable2(t) => get(t),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Table macro
|
||||
/// Create all tables, should be used _once_.
|
||||
///
|
||||
/// Generating this macro once and using `$()*` is probably
|
||||
/// faster for compile times than calling the macro _per_ table.
|
||||
///
|
||||
/// All tables are zero-sized table structs, and implement the `Table` trait.
|
||||
///
|
||||
/// Table structs are automatically `CamelCase`,
|
||||
/// and their static string names are automatically `snake_case`.
|
||||
macro_rules! tables {
|
||||
(
|
||||
$(
|
||||
$(#[$attr:meta])* // Documentation and any `derive`'s.
|
||||
$table:ident, // The table name + doubles as the table struct name.
|
||||
$size:literal, // Are the table's values all the same size?
|
||||
$key:ty => // Key type.
|
||||
$value:ty // Value type.
|
||||
),* $(,)?
|
||||
) => {
|
||||
paste::paste! { $(
|
||||
// Table struct.
|
||||
$(#[$attr])*
|
||||
// The below test show the `snake_case` table name in cargo docs.
|
||||
/// ## Table Name
|
||||
/// ```rust
|
||||
/// # use cuprate_database::{*,tables::*};
|
||||
#[doc = concat!(
|
||||
"assert_eq!(",
|
||||
stringify!([<$table:camel>]),
|
||||
"::NAME, \"",
|
||||
stringify!([<$table:snake>]),
|
||||
"\");",
|
||||
)]
|
||||
/// ```
|
||||
#[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))]
|
||||
#[cfg_attr(feature = "borsh", derive(borsh::BorshSerialize, borsh::BorshDeserialize))]
|
||||
#[derive(Copy,Clone,Debug,PartialEq,PartialOrd,Eq,Ord,Hash)]
|
||||
pub struct [<$table:camel>];
|
||||
|
||||
// Table trait impl.
|
||||
impl Table for [<$table:camel>] {
|
||||
const NAME: &'static str = stringify!([<$table:snake>]);
|
||||
const CONSTANT_SIZE: bool = $size;
|
||||
type Key = $key;
|
||||
type Value = $value;
|
||||
}
|
||||
|
||||
// Table enum.
|
||||
impl From<[<$table:camel>]> for Tables {
|
||||
fn from(table: [<$table:camel>]) -> Self {
|
||||
Self::[<$table:camel>](table)
|
||||
}
|
||||
}
|
||||
)* }
|
||||
};
|
||||
}
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Tables
|
||||
tables! {
|
||||
/// Test documentation.
|
||||
TestTable,
|
||||
true,
|
||||
i64 => u64,
|
||||
|
||||
/// Test documentation 2.
|
||||
TestTable2,
|
||||
true,
|
||||
u8 => i8,
|
||||
}
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Tests
|
||||
#[cfg(test)]
|
||||
mod test {
|
||||
// use super::*;
|
||||
}
|
29
database/src/transaction.rs
Normal file
29
database/src/transaction.rs
Normal file
|
@ -0,0 +1,29 @@
|
|||
//! Database transaction abstraction; `trait RoTx`, `trait RwTx`.
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- Import
|
||||
use crate::error::RuntimeError;
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- RoTx
|
||||
/// Read-only database transaction.
|
||||
///
|
||||
/// TODO
|
||||
pub trait RoTx<'db> {
|
||||
/// TODO
|
||||
/// # Errors
|
||||
/// TODO
|
||||
fn commit(self) -> Result<(), RuntimeError>;
|
||||
}
|
||||
|
||||
//---------------------------------------------------------------------------------------------------- RwTx
|
||||
/// Read/write database transaction.
|
||||
///
|
||||
/// TODO
|
||||
pub trait RwTx<'db> {
|
||||
/// TODO
|
||||
/// # Errors
|
||||
/// TODO
|
||||
fn commit(self) -> Result<(), RuntimeError>;
|
||||
|
||||
/// TODO
|
||||
fn abort(self);
|
||||
}
|
33
old_database/Cargo.toml
Normal file
33
old_database/Cargo.toml
Normal file
|
@ -0,0 +1,33 @@
|
|||
[package]
|
||||
name = "cuprate-database"
|
||||
version = "0.0.1"
|
||||
edition = "2021"
|
||||
license = "AGPL-3.0-only"
|
||||
|
||||
# All Contributors on github
|
||||
authors=[
|
||||
"SyntheticBird45 <@someoneelse495495:matrix.org>",
|
||||
"Boog900"
|
||||
]
|
||||
|
||||
[features]
|
||||
mdbx = ["dep:libmdbx"]
|
||||
hse = []
|
||||
|
||||
[dependencies]
|
||||
monero = {workspace = true, features = ["serde"]}
|
||||
tiny-keccak = { version = "2.0", features = ["sha3"] }
|
||||
serde = { workspace = true}
|
||||
thiserror = {workspace = true }
|
||||
bincode = { workspace = true }
|
||||
libmdbx = { version = "0.3.1", optional = true }
|
||||
|
||||
[build]
|
||||
linker="clang"
|
||||
rustflags=[
|
||||
"-Clink-arg=-fuse-ld=mold",
|
||||
"-Zcf-protection=full",
|
||||
"-Zsanitizer=cfi",
|
||||
"-Crelocation-model=pie",
|
||||
"-Cstack-protector=all",
|
||||
]
|
53
old_database/src/error.rs
Normal file
53
old_database/src/error.rs
Normal file
|
@ -0,0 +1,53 @@
|
|||
//! ### Error module
|
||||
//! This module contains all errors abstraction used by the database crate. By implementing [`From<E>`] to the specific errors of storage engine crates, it let us
|
||||
//! handle more easily any type of error that can happen. This module does **NOT** contain interpretation of these errors, as these are defined for Blockchain abstraction. This is another difference
|
||||
//! from monerod which interpret these errors directly in its database functions:
|
||||
//! ```cpp
|
||||
//! /**
|
||||
//! * @brief A base class for BlockchainDB exceptions
|
||||
//! */
|
||||
//! class DB_EXCEPTION : public std::exception
|
||||
//! ```
|
||||
//! see `blockchain_db/blockchain_db.h` in monerod `src/` folder for more details.
|
||||
|
||||
#[derive(thiserror::Error, Debug)]
|
||||
/// `DB_FAILURES` is an enum for backend-agnostic, internal database errors. The `From` Trait must be implemented to the specific backend errors to match DB_FAILURES.
|
||||
pub enum DB_FAILURES {
|
||||
#[error("MDBX returned an error {0}")]
|
||||
MDBX_Error(#[from] libmdbx::Error),
|
||||
|
||||
#[error("\n<DB_FAILURES::EncodingError> Failed to encode some data : `{0}`")]
|
||||
SerializeIssue(DB_SERIAL),
|
||||
|
||||
#[error("\nObject already exist in the database : {0}")]
|
||||
AlreadyExist(&'static str),
|
||||
|
||||
#[error("NotFound? {0}")]
|
||||
NotFound(&'static str),
|
||||
|
||||
#[error("\n<DB_FAILURES::Other> `{0}`")]
|
||||
Other(&'static str),
|
||||
|
||||
#[error(
|
||||
"\n<DB_FAILURES::FailedToCommit> A transaction tried to commit to the db, but failed."
|
||||
)]
|
||||
FailedToCommit,
|
||||
}
|
||||
|
||||
#[derive(thiserror::Error, Debug)]
|
||||
pub enum DB_SERIAL {
|
||||
#[error("An object failed to be serialized into bytes. It is likely an issue from monero-rs library. Please report this error on cuprate's github : https://github.com/Cuprate/cuprate/issues")]
|
||||
ConsensusEncode,
|
||||
|
||||
#[error("Bytes failed to be deserialized into the requested object. It is likely an issue from monero-rs library. Please report this error on cuprate's github : https://github.com/Cuprate/cuprate/issues")]
|
||||
ConsensusDecode(Vec<u8>),
|
||||
|
||||
#[error("monero-rs encoding|decoding logic failed : {0}")]
|
||||
MoneroEncode(#[from] monero::consensus::encode::Error),
|
||||
|
||||
#[error("Bincode failed to decode a type from the database : {0}")]
|
||||
BincodeDecode(#[from] bincode::error::DecodeError),
|
||||
|
||||
#[error("Bincode failed to encode a type for the database : {0}")]
|
||||
BincodeEncode(#[from] bincode::error::EncodeError),
|
||||
}
|
221
old_database/src/lib.rs
Normal file
221
old_database/src/lib.rs
Normal file
|
@ -0,0 +1,221 @@
|
|||
// Copyright (C) 2023 Cuprate Contributors
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful,
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <https://www.gnu.org/licenses/>.
|
||||
|
||||
//! The cuprate-db crate implement (as its name suggests) the relations between the blockchain/txpool objects and their databases.
|
||||
//! `lib.rs` contains all the generics, trait and specification for interfaces between blockchain and a backend-agnostic database
|
||||
//! Every other files in this folder are implementation of these traits/methods to real storage engine.
|
||||
//!
|
||||
//! At the moment, the only storage engine available is MDBX.
|
||||
//! The next storage engine planned is HSE (Heteregeonous Storage Engine) from Micron.
|
||||
//!
|
||||
//! For more informations, please consult this docs:
|
||||
|
||||
#![deny(unused_attributes)]
|
||||
#![forbid(unsafe_code)]
|
||||
#![allow(non_camel_case_types)]
|
||||
#![deny(clippy::expect_used, clippy::panic)]
|
||||
#![allow(dead_code, unused_macros)] // temporary
|
||||
|
||||
use monero::{util::ringct::RctSig, Block, BlockHeader, Hash};
|
||||
use std::ops::Range;
|
||||
use thiserror::Error;
|
||||
|
||||
#[cfg(feature = "mdbx")]
|
||||
pub mod mdbx;
|
||||
//#[cfg(feature = "hse")]
|
||||
//pub mod hse;
|
||||
|
||||
pub mod encoding;
|
||||
pub mod error;
|
||||
pub mod interface;
|
||||
pub mod table;
|
||||
pub mod types;
|
||||
|
||||
const DEFAULT_BLOCKCHAIN_DATABASE_DIRECTORY: &str = "blockchain";
|
||||
const DEFAULT_TXPOOL_DATABASE_DIRECTORY: &str = "txpool_mem";
|
||||
const BINCODE_CONFIG: bincode::config::Configuration<
|
||||
bincode::config::LittleEndian,
|
||||
bincode::config::Fixint,
|
||||
> = bincode::config::standard().with_fixed_int_encoding();
|
||||
|
||||
// ------------------------------------------| Database |------------------------------------------
|
||||
|
||||
pub mod database {
|
||||
//! This module contains the Database abstraction trait. Any key/value storage engine implemented need
|
||||
//! to fullfil these associated types and functions, in order to be usable. This module also contains the
|
||||
//! Interface struct which is used by the DB Reactor to interact with the database.
|
||||
|
||||
use crate::{
|
||||
error::DB_FAILURES,
|
||||
transaction::{Transaction, WriteTransaction},
|
||||
};
|
||||
use std::{ops::Deref, path::PathBuf, sync::Arc};
|
||||
|
||||
/// `Database` Trait implement all the methods necessary to generate transactions as well as execute specific functions. It also implement generic associated types to identify the
|
||||
/// different transaction modes (read & write) and it's native errors.
|
||||
pub trait Database<'a> {
|
||||
type TX: Transaction<'a>;
|
||||
type TXMut: WriteTransaction<'a>;
|
||||
type Error: Into<DB_FAILURES>;
|
||||
|
||||
// Create a transaction from the database
|
||||
fn tx(&'a self) -> Result<Self::TX, Self::Error>;
|
||||
|
||||
// Create a mutable transaction from the database
|
||||
fn tx_mut(&'a self) -> Result<Self::TXMut, Self::Error>;
|
||||
|
||||
// Open a database from the specified path
|
||||
fn open(path: PathBuf) -> Result<Self, Self::Error>
|
||||
where
|
||||
Self: std::marker::Sized;
|
||||
|
||||
// Check if the database is built.
|
||||
fn check_all_tables_exist(&'a self) -> Result<(), Self::Error>;
|
||||
|
||||
// Build the database
|
||||
fn build(&'a self) -> Result<(), Self::Error>;
|
||||
}
|
||||
|
||||
/// `Interface` is a struct containing a shared pointer to the database and transaction's to be used for the implemented method of Interface.
|
||||
pub struct Interface<'a, D: Database<'a>> {
|
||||
pub db: Arc<D>,
|
||||
pub tx: Option<<D as Database<'a>>::TXMut>,
|
||||
}
|
||||
|
||||
// Convenient implementations for database
|
||||
impl<'service, D: Database<'service>> Interface<'service, D> {
|
||||
fn from(db: Arc<D>) -> Result<Self, DB_FAILURES> {
|
||||
Ok(Self { db, tx: None })
|
||||
}
|
||||
|
||||
fn open(&'service mut self) -> Result<(), DB_FAILURES> {
|
||||
let tx = self.db.tx_mut().map_err(Into::into)?;
|
||||
self.tx = Some(tx);
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
impl<'service, D: Database<'service>> Deref for Interface<'service, D> {
|
||||
type Target = <D as Database<'service>>::TXMut;
|
||||
|
||||
fn deref(&self) -> &Self::Target {
|
||||
return self.tx.as_ref().unwrap();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// ------------------------------------------| DatabaseTx |------------------------------------------
|
||||
|
||||
pub mod transaction {
|
||||
//! This module contains the abstractions of Transactional Key/Value database functions.
|
||||
//! Any key/value database/storage engine can be implemented easily for Cuprate as long as
|
||||
//! these functions or equivalent logic exist for it.
|
||||
|
||||
use crate::{
|
||||
error::DB_FAILURES,
|
||||
table::{DupTable, Table},
|
||||
};
|
||||
|
||||
// Abstraction of a read-only cursor, for simple tables
|
||||
#[allow(clippy::type_complexity)]
|
||||
pub trait Cursor<'t, T: Table> {
|
||||
fn first(&mut self) -> Result<Option<(T::Key, T::Value)>, DB_FAILURES>;
|
||||
|
||||
fn get_cursor(&mut self) -> Result<Option<(T::Key, T::Value)>, DB_FAILURES>;
|
||||
|
||||
fn last(&mut self) -> Result<Option<(T::Key, T::Value)>, DB_FAILURES>;
|
||||
|
||||
fn next(&mut self) -> Result<Option<(T::Key, T::Value)>, DB_FAILURES>;
|
||||
|
||||
fn prev(&mut self) -> Result<Option<(T::Key, T::Value)>, DB_FAILURES>;
|
||||
|
||||
fn set(&mut self, key: &T::Key) -> Result<Option<T::Value>, DB_FAILURES>;
|
||||
}
|
||||
|
||||
// Abstraction of a read-only cursor with support for duplicated tables. DupCursor inherit Cursor methods as
|
||||
// a duplicated table can be treated as a simple table.
|
||||
#[allow(clippy::type_complexity)]
|
||||
pub trait DupCursor<'t, T: DupTable>: Cursor<'t, T> {
|
||||
fn first_dup(&mut self) -> Result<Option<(T::SubKey, T::Value)>, DB_FAILURES>;
|
||||
|
||||
fn get_dup(
|
||||
&mut self,
|
||||
key: &T::Key,
|
||||
subkey: &T::SubKey,
|
||||
) -> Result<Option<T::Value>, DB_FAILURES>;
|
||||
|
||||
fn last_dup(&mut self) -> Result<Option<(T::SubKey, T::Value)>, DB_FAILURES>;
|
||||
|
||||
fn next_dup(&mut self) -> Result<Option<(T::Key, (T::SubKey, T::Value))>, DB_FAILURES>;
|
||||
|
||||
fn prev_dup(&mut self) -> Result<Option<(T::Key, (T::SubKey, T::Value))>, DB_FAILURES>;
|
||||
}
|
||||
|
||||
// Abstraction of a read-write cursor, for simple tables. WriteCursor inherit Cursor methods.
|
||||
pub trait WriteCursor<'t, T: Table>: Cursor<'t, T> {
|
||||
fn put_cursor(&mut self, key: &T::Key, value: &T::Value) -> Result<(), DB_FAILURES>;
|
||||
|
||||
fn del(&mut self) -> Result<(), DB_FAILURES>;
|
||||
}
|
||||
|
||||
// Abstraction of a read-write cursor with support for duplicated tables. DupWriteCursor inherit DupCursor and WriteCursor methods.
|
||||
pub trait DupWriteCursor<'t, T: DupTable>: WriteCursor<'t, T> {
|
||||
fn put_cursor_dup(
|
||||
&mut self,
|
||||
key: &T::Key,
|
||||
subkey: &T::SubKey,
|
||||
value: &T::Value,
|
||||
) -> Result<(), DB_FAILURES>;
|
||||
|
||||
/// Delete all data under associated to its key
|
||||
fn del_nodup(&mut self) -> Result<(), DB_FAILURES>;
|
||||
}
|
||||
|
||||
// Abstraction of a read-only transaction.
|
||||
pub trait Transaction<'a>: Send + Sync {
|
||||
type Cursor<T: Table>: Cursor<'a, T>;
|
||||
type DupCursor<T: DupTable>: DupCursor<'a, T> + Cursor<'a, T>;
|
||||
|
||||
fn get<T: Table>(&self, key: &T::Key) -> Result<Option<T::Value>, DB_FAILURES>;
|
||||
|
||||
fn commit(self) -> Result<(), DB_FAILURES>;
|
||||
|
||||
fn cursor<T: Table>(&self) -> Result<Self::Cursor<T>, DB_FAILURES>;
|
||||
|
||||
fn cursor_dup<T: DupTable>(&self) -> Result<Self::DupCursor<T>, DB_FAILURES>;
|
||||
|
||||
fn num_entries<T: Table>(&self) -> Result<usize, DB_FAILURES>;
|
||||
}
|
||||
|
||||
// Abstraction of a read-write transaction. WriteTransaction inherits Transaction methods.
|
||||
pub trait WriteTransaction<'a>: Transaction<'a> {
|
||||
type WriteCursor<T: Table>: WriteCursor<'a, T>;
|
||||
type DupWriteCursor<T: DupTable>: DupWriteCursor<'a, T> + DupCursor<'a, T>;
|
||||
|
||||
fn put<T: Table>(&self, key: &T::Key, value: &T::Value) -> Result<(), DB_FAILURES>;
|
||||
|
||||
fn delete<T: Table>(
|
||||
&self,
|
||||
key: &T::Key,
|
||||
value: &Option<T::Value>,
|
||||
) -> Result<(), DB_FAILURES>;
|
||||
|
||||
fn clear<T: Table>(&self) -> Result<(), DB_FAILURES>;
|
||||
|
||||
fn write_cursor<T: Table>(&self) -> Result<Self::WriteCursor<T>, DB_FAILURES>;
|
||||
|
||||
fn write_cursor_dup<T: DupTable>(&self) -> Result<Self::DupWriteCursor<T>, DB_FAILURES>;
|
||||
}
|
||||
}
|
181
old_database/src/table.rs
Normal file
181
old_database/src/table.rs
Normal file
|
@ -0,0 +1,181 @@
|
|||
//! ### Table module
|
||||
//! This module contains the definition of the [`Table`] and [`DupTable`] trait, and the actual tables used in the database.
|
||||
//! [`DupTable`] are just a trait used to define that they support DUPSORT|DUPFIXED operation (as of now we don't know the equivalent for HSE).
|
||||
//! All tables are defined with docs explaining its purpose, what types are the key and data.
|
||||
//! For more details please look at Cuprate's book : <link to cuprate book>
|
||||
|
||||
use crate::{
|
||||
encoding::Compat,
|
||||
types::{
|
||||
/*OutTx,*/ AltBlock, BlockMetadata, /*RctOutkey,*/ OutputMetadata,
|
||||
TransactionPruned, TxIndex, /*OutAmountIdx,*/ /*KeyImage,*/ TxOutputIdx,
|
||||
},
|
||||
};
|
||||
use bincode::{de::Decode, enc::Encode};
|
||||
use monero::{blockdata::transaction::KeyImage, Block, Hash};
|
||||
|
||||
/// A trait implementing a table interaction for the database. It is implemented to an empty struct to specify the name and table's associated types. These associated
|
||||
/// types are used to simplify deserialization process.
|
||||
pub trait Table: Send + Sync + 'static + Clone {
|
||||
// name of the table
|
||||
const TABLE_NAME: &'static str;
|
||||
|
||||
// Definition of a key & value types of the database
|
||||
type Key: Encode + Decode;
|
||||
type Value: Encode + Decode;
|
||||
}
|
||||
|
||||
/// A trait implementing a table with duplicated data support.
|
||||
pub trait DupTable: Table {
|
||||
// Subkey of the table (prefix of the data)
|
||||
type SubKey: Encode + Decode;
|
||||
}
|
||||
|
||||
/// This declarative macro declare a new empty struct and impl the specified name, and corresponding types.
|
||||
macro_rules! impl_table {
|
||||
( $(#[$docs:meta])* $table:ident , $key:ty , $value:ty ) => {
|
||||
#[derive(Clone)]
|
||||
$(#[$docs])*
|
||||
pub(crate) struct $table;
|
||||
|
||||
impl Table for $table {
|
||||
const TABLE_NAME: &'static str = stringify!($table);
|
||||
type Key = $key;
|
||||
type Value = $value;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/// This declarative macro declare extend the original impl_table! macro by implementy DupTable trait.
|
||||
macro_rules! impl_duptable {
|
||||
($(#[$docs:meta])* $table:ident, $key:ty, $subkey:ty, $value:ty) => {
|
||||
impl_table!($(#[$docs])* $table, $key, $value);
|
||||
|
||||
impl DupTable for $table {
|
||||
type SubKey = $subkey;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
// ------------------------------------------| Tables definition |------------------------------------------
|
||||
|
||||
// ----- BLOCKS -----
|
||||
|
||||
impl_duptable!(
|
||||
/// `blockhash` is table defining a relation between the hash of a block and its height. Its primary use is to quickly find block's hash by its height.
|
||||
blockhash,
|
||||
(),
|
||||
Compat<Hash>,
|
||||
u64
|
||||
);
|
||||
|
||||
impl_duptable!(
|
||||
/// `blockmetadata` store block metadata alongside their corresponding Hash. The blocks metadata can contains the total_coins_generated, weight, long_term_block_weight & cumulative RingCT
|
||||
blockmetadata,
|
||||
(),
|
||||
u64,
|
||||
BlockMetadata
|
||||
);
|
||||
|
||||
impl_table!(
|
||||
/// `blockbody` store blocks' bodies along their Hash. The blocks body contains the coinbase transaction and its corresponding mined transactions' hashes.
|
||||
blocks,
|
||||
u64,
|
||||
Compat<Block>
|
||||
);
|
||||
|
||||
/*
|
||||
impl_table!(
|
||||
/// `blockhfversion` keep track of block's hard fork version. If an outdated node continue to run after a hard fork, it needs to know, after updating, what blocks needs to be update.
|
||||
blockhfversion, u64, u8);
|
||||
*/
|
||||
|
||||
impl_table!(
|
||||
/// `altblock` is a table that permits the storage of blocks from an alternative chain, which may cause a re-org. These blocks can be fetch by their corresponding hash.
|
||||
altblock,
|
||||
Compat<Hash>,
|
||||
AltBlock
|
||||
);
|
||||
|
||||
// ------- TXNs -------
|
||||
|
||||
impl_table!(
|
||||
/// `txspruned` is table storing TransactionPruned (or Pruned Tx). These can be fetch by the corresponding Transaction ID.
|
||||
txspruned,
|
||||
u64,
|
||||
TransactionPruned
|
||||
);
|
||||
|
||||
impl_table!(
|
||||
/// `txsprunable` is a table storing the Prunable part of transactions (Signatures and RctSig), stored as raw bytes. These can be fetch by the corresponding Transaction ID.
|
||||
txsprunable,
|
||||
u64,
|
||||
Vec<u8>
|
||||
);
|
||||
|
||||
impl_duptable!(
|
||||
/// `txsprunablehash` is a table storing hashes of prunable part of transactions. These hash can be fetch by the corresponding Transaction ID.
|
||||
txsprunablehash,
|
||||
u64,
|
||||
(),
|
||||
Compat<Hash>
|
||||
);
|
||||
|
||||
impl_table!(
|
||||
/// `txsprunabletip` is a table used for optimization purpose. It defines at which block's height this transaction belong as long as the block is with Tip blocks. These can be fetch by the corresponding Transaction ID.
|
||||
txsprunabletip,
|
||||
u64,
|
||||
u64
|
||||
);
|
||||
|
||||
impl_duptable!(
|
||||
/// `txsoutputs` is a table storing output indices used in a transaction. These can be fetch by the corresponding Transaction ID.
|
||||
txsoutputs,
|
||||
u64,
|
||||
(),
|
||||
TxOutputIdx
|
||||
);
|
||||
|
||||
impl_duptable!(
|
||||
/// `txsidentifier` is a table defining a relation between the hash of a transaction and its transaction Indexes. Its primarly used to quickly find tx's ID by its hash.
|
||||
txsidentifier,
|
||||
Compat<Hash>,
|
||||
(),
|
||||
TxIndex
|
||||
);
|
||||
|
||||
// ---- OUTPUTS ----
|
||||
|
||||
impl_duptable!(
|
||||
/// `prerctoutputmetadata` is a duplicated table storing Pre-RingCT output's metadata. The key is the amount of this output, and the subkey is its amount idx.
|
||||
prerctoutputmetadata,
|
||||
u64,
|
||||
u64,
|
||||
OutputMetadata
|
||||
);
|
||||
impl_duptable!(
|
||||
/// `prerctoutputmetadata` is a table storing RingCT output's metadata. The key is the amount idx of this output since amount is always 0 for RingCT outputs.
|
||||
outputmetadata,
|
||||
(),
|
||||
u64,
|
||||
OutputMetadata
|
||||
);
|
||||
|
||||
// ---- SPT KEYS ----
|
||||
|
||||
impl_duptable!(
|
||||
/// `spentkeys`is a table storing every KeyImage that have been used to create decoys input. As these KeyImage can't be re used they need to marked.
|
||||
spentkeys,
|
||||
(),
|
||||
Compat<KeyImage>,
|
||||
()
|
||||
);
|
||||
|
||||
// ---- PROPERTIES ----
|
||||
|
||||
impl_table!(
|
||||
/// `spentkeys`is a table storing every KeyImage that have been used to create decoys input. As these KeyImage can't be re used they need to marked.
|
||||
properties,
|
||||
u32,
|
||||
u32
|
||||
);
|
Loading…
Reference in a new issue