diff --git a/index.js b/index.js index e79cf11a..43a4923e 100644 --- a/index.js +++ b/index.js @@ -1,10 +1,9 @@ const { EventEmitter } = require('events') -const RAF = require('random-access-file') const isOptions = require('is-options') const hypercoreCrypto = require('hypercore-crypto') +const CoreStorage = require('hypercore-on-the-rocks') const c = require('compact-encoding') const b4a = require('b4a') -const Xache = require('xache') const NoiseSecretStream = require('@hyperswarm/secret-stream') const Protomux = require('protomux') const z32 = require('z32') @@ -40,7 +39,7 @@ module.exports = class Hypercore extends EventEmitter { constructor (storage, key, opts) { super() - if (isOptions(storage)) { + if (isOptions(storage) && !storage.db) { opts = storage storage = null key = opts.key || null @@ -59,10 +58,10 @@ module.exports = class Hypercore extends EventEmitter { this.storage = null this.crypto = opts.crypto || hypercoreCrypto this.core = null + this.state = null this.replicator = null this.encryption = null this.extensions = new Map() - this.cache = createCache(opts.cache) this.valueEncoding = null this.encodeBatch = null @@ -76,6 +75,7 @@ module.exports = class Hypercore extends EventEmitter { this.opened = false this.closed = false this.snapshotted = !!opts.snapshot + this.draft = !!opts.draft this.sparse = opts.sparse !== false this.sessions = opts._sessions || [this] this.autoClose = !!opts.autoClose @@ -189,28 +189,12 @@ module.exports = class Hypercore extends EventEmitter { static defaultStorage (storage, opts = {}) { if (typeof storage !== 'string') { - if (!isRandomAccessClass(storage)) return storage - const Cls = storage // just to satisfy standard... - return name => new Cls(name) + // todo: validate it is rocksdb instance + return storage } const directory = storage - const toLock = opts.unlocked ? null : (opts.lock || 'oplog') - const pool = opts.pool || (opts.poolSize ? RAF.createPool(opts.poolSize) : null) - const rmdir = !!opts.rmdir - const writable = opts.writable !== false - - return createFile - - function createFile (name) { - const lock = toLock === null ? false : isFile(name, toLock) - const sparse = isFile(name, 'data') || isFile(name, 'bitfield') || isFile(name, 'tree') - return new RAF(name, { directory, lock, sparse, pool: lock ? null : pool, rmdir, writable }) - } - - function isFile (name, n) { - return name === n || name.endsWith('/' + n) - } + return new CoreStorage(directory, opts) } snapshot (opts) { @@ -243,11 +227,6 @@ module.exports = class Hypercore extends EventEmitter { s._passCapabilities(this) - // Configure the cache unless explicitly disabled. - if (opts.cache !== false) { - s.cache = opts.cache === true || !opts.cache ? this.cache : opts.cache - } - if (this.opened) ensureEncryption(s, opts) this._addSession(s) @@ -256,7 +235,6 @@ module.exports = class Hypercore extends EventEmitter { _addSession (s) { this.sessions.push(s) - if (this.core) this.core.active++ } async setEncryptionKey (encryptionKey, opts) { @@ -288,6 +266,8 @@ module.exports = class Hypercore extends EventEmitter { this.writable = this._isWritable() this.autoClose = o.autoClose + if (o.state) this.state = this.draft ? o.state.memoryOverlay() : this.snapshotted ? o.state.snapshot() : o.state.ref() + if (this.snapshotted && this.core && !this._snapshot) this._updateSnapshot() } @@ -321,7 +301,6 @@ module.exports = class Hypercore extends EventEmitter { await opts._opening } if (opts.preload) opts = { ...opts, ...(await this._retryPreload(opts.preload)) } - if (this.cache === null && opts.cache) this.cache = createCache(opts.cache) if (isFirst) { await this._openCapabilities(key, storage, opts) @@ -338,6 +317,17 @@ module.exports = class Hypercore extends EventEmitter { ensureEncryption(this, opts) } + if (opts.name) { + // todo: need to make named sessions safe before ready + // atm we always copy the state in passCapabilities + await this.state.unref() + this.state = await this.core.createSession(opts.name, opts.checkout, opts.refresh) + + if (opts.checkout !== undefined) { + await this.state.truncate(opts.checkout, this.fork) + } + } + if (opts.manifest && !this.core.header.manifest) { await this.core.setManifest(opts.manifest) } @@ -388,6 +378,7 @@ module.exports = class Hypercore extends EventEmitter { sessions: this.sessions, createIfMissing: opts.createIfMissing, readonly: unlocked, + discoveryKey: opts.discoveryKey, overwrite: opts.overwrite, key, keyPair: opts.keyPair, @@ -399,10 +390,14 @@ module.exports = class Hypercore extends EventEmitter { onconflict: this._oncoreconflict.bind(this) }) + this.state = this.core.state + if (opts.userData) { + const batch = this.state.storage.createWriteBatch() for (const [key, value] of Object.entries(opts.userData)) { - await this.core.userData(key, value) + this.core.setUserData(batch, key, value) } + await batch.flush() } this.key = this.core.header.key @@ -429,18 +424,18 @@ module.exports = class Hypercore extends EventEmitter { _getSnapshot () { if (this.sparse) { return { - length: this.core.tree.length, - byteLength: this.core.tree.byteLength, - fork: this.core.tree.fork, - compatLength: this.core.tree.length + length: this.state.tree.length, + byteLength: this.state.tree.byteLength, + fork: this.state.tree.fork, + compatLength: this.state.tree.length } } return { - length: this.core.header.hints.contiguousLength, + length: this.state.header.hints.contiguousLength, byteLength: 0, - fork: this.core.tree.fork, - compatLength: this.core.header.hints.contiguousLength + fork: this.state.tree.fork, + compatLength: this.state.header.hints.contiguousLength } } @@ -456,20 +451,32 @@ module.exports = class Hypercore extends EventEmitter { return !this._readonly && !!(this.keyPair && this.keyPair.secretKey) } - close (err) { + close ({ error, force = !!error } = {}) { if (this.closing) return this.closing - this.closing = this._close(err || null) + + this.closing = this._close(error || null, force) return this.closing } - async _close (err) { + _forceClose (error) { + const sessions = [...this.sessions] + + const closing = [] + for (const session of sessions) { + if (session === this) continue + closing.push(session.close({ error, force: false })) + } + + return Promise.all(closing) + } + + async _close (error, force) { if (this.opened === false) await this.opening const i = this.sessions.indexOf(this) if (i === -1) return this.sessions.splice(i, 1) - this.core.active-- this.readable = false this.writable = false this.closed = true @@ -483,16 +490,22 @@ module.exports = class Hypercore extends EventEmitter { if (this.replicator !== null) { this.replicator.findingPeers -= this._findingPeers - this.replicator.clearRequests(this.activeRequests, err) + this.replicator.clearRequests(this.activeRequests, error) this.replicator.updateActivity(this._active ? -1 : 0) } this._findingPeers = 0 - if (this.sessions.length || this.core.active > 0) { + if (force) { + await this._forceClose(error) + } else if (this.sessions.length || this.state.active > 1) { + // check if there is still an active session + await this.state.unref() + // if this is the last session and we are auto closing, trigger that first to enforce error handling - if (this.sessions.length === 1 && this.core.active === 1 && this.autoClose) await this.sessions[0].close(err) + if (this.sessions.length === 1 && this.core.state.active === 1 && this.autoClose) await this.sessions[0].close({ error }) // emit "fake" close as this is a session + this.emit('close', false) return } @@ -501,6 +514,7 @@ module.exports = class Hypercore extends EventEmitter { await this.replicator.destroy() } + await this.state.unref() // close after replicator await this.core.close() this.emit('close', true) @@ -556,11 +570,11 @@ module.exports = class Hypercore extends EventEmitter { if (this._snapshot) return this._snapshot.length if (this.core === null) return 0 if (!this.sparse) return this.contiguousLength - return this.core.tree.length + return this.state.tree.length } - get indexedLength () { - return this.length + get flushedLength () { + return this.state === this.core.state ? this.core.tree.length : this.state.treeLength } /** @@ -570,7 +584,7 @@ module.exports = class Hypercore extends EventEmitter { if (this._snapshot) return this._snapshot.byteLength if (this.core === null) return 0 if (!this.sparse) return this.contiguousByteLength - return this.core.tree.byteLength - (this.core.tree.length * this.padding) + return this.state.tree.byteLength - (this.state.tree.length * this.padding) } get contiguousLength () { @@ -634,11 +648,11 @@ module.exports = class Hypercore extends EventEmitter { const sessions = [...this.sessions] const all = [] - for (const s of sessions) all.push(s.close(err)) + for (const s of sessions) all.push(s.close({ error: err, force: false })) // force false or else infinite recursion await Promise.allSettled(all) } - _oncoreupdate (status, bitfield, value, from) { + _oncoreupdate ({ status, bitfield, value, from }) { if (status !== 0) { const truncatedNonSparse = (status & 0b1000) !== 0 const appendedNonSparse = (status & 0b0100) !== 0 @@ -671,8 +685,6 @@ module.exports = class Hypercore extends EventEmitter { const s = this.sessions[i] if (truncated) { - if (s.cache) s.cache.clear() - // If snapshotted, make sure to update our compat so we can fail gets if (s._snapshot && bitfield.start < s._snapshot.compatLength) s._snapshot.compatLength = bitfield.start } @@ -729,19 +741,19 @@ module.exports = class Hypercore extends EventEmitter { async setUserData (key, value, { flush = false } = {}) { if (this.opened === false) await this.opening - return this.core.userData(key, value, flush) + await this.state.setUserData(key, value) } async getUserData (key) { if (this.opened === false) await this.opening - for (const { key: savedKey, value } of this.core.header.userData) { - if (key === savedKey) return value - } - return null + const batch = this.state.storage.createReadBatch() + const p = batch.getUserData(key) + batch.tryFlush() + return p } createTreeBatch () { - return this.core.tree.batch() + return this.state.tree.batch() } findingPeers () { @@ -803,7 +815,7 @@ module.exports = class Hypercore extends EventEmitter { if (this.opened === false) await this.opening if (!isValidIndex(bytes)) throw ASSERTION('seek is invalid') - const tree = (opts && opts.tree) || this.core.tree + const tree = (opts && opts.tree) || this.state.tree const s = tree.seek(bytes, this.padding) const offset = await s.update() @@ -826,9 +838,9 @@ module.exports = class Hypercore extends EventEmitter { if (this.opened === false) await this.opening if (!isValidIndex(start) || !isValidIndex(end)) throw ASSERTION('has range is invalid') - if (end === start + 1) return this.core.bitfield.get(start) + if (end === start + 1) return this.state.bitfield.get(start) - const i = this.core.bitfield.firstUnset(start) + const i = this.state.bitfield.firstUnset(start) return i === -1 || i >= end } @@ -837,12 +849,10 @@ module.exports = class Hypercore extends EventEmitter { if (!isValidIndex(index)) throw ASSERTION('block index is invalid') if (this.closing !== null) throw SESSION_CLOSED() - if (this._snapshot !== null && index >= this._snapshot.compatLength) throw SNAPSHOT_NOT_AVAILABLE() const encoding = (opts && opts.valueEncoding && c.from(opts.valueEncoding)) || this.valueEncoding - let req = this.cache && this.cache.get(index) - if (!req) req = this._get(index, opts) + const req = this._get(index, opts) let block = await req if (!block) return null @@ -875,7 +885,7 @@ module.exports = class Hypercore extends EventEmitter { if (start >= end) return cleared if (start >= this.length) return cleared - await this.core.clear(start, end, cleared) + await this.state.clear(start, end, cleared) return cleared } @@ -886,46 +896,51 @@ module.exports = class Hypercore extends EventEmitter { } async _get (index, opts) { - let block + if (this.core.isFlushing) await this.core.flushed() - if (this.core.bitfield.get(index)) { - const tree = (opts && opts.tree) || this.core.tree - block = this.core.blocks.get(index, tree) + const block = await readBlock(this.state.storage.createReadBatch(), index) - if (this.cache) this.cache.set(index, block) - } else { - if (!this._shouldWait(opts, this.wait)) return null - - if (opts && opts.onwait) opts.onwait(index, this) - if (this.onwait) this.onwait(index, this) + if (block !== null) return block - const activeRequests = (opts && opts.activeRequests) || this.activeRequests + if (this.closing !== null) throw SESSION_CLOSED() - const req = this.replicator.addBlock(activeRequests, index) - req.snapshot = index < this.length + // snapshot should check if core has block + if (this._snapshot !== null) { + checkSnapshot(this._snapshot, index) + const coreBlock = await readBlock(this.core.state.storage.createReadBatch(), index) - const timeout = opts && opts.timeout !== undefined ? opts.timeout : this.timeout - if (timeout) req.context.setTimeout(req, timeout) + checkSnapshot(this._snapshot, index) + if (coreBlock !== null) return coreBlock + } - block = this._cacheOnResolve(index, req.promise, this.core.tree.fork) + // lets check the bitfield to see if we got it during the above async calls + // this is the last resort before replication, so always safe. + if (this.core.state.bitfield.get(index)) { + return readBlock(this.state.storage.createReadBatch(), index) } - return block - } + if (!this._shouldWait(opts, this.wait)) return null - async _cacheOnResolve (index, req, fork) { - const resolved = await req + if (opts && opts.onwait) opts.onwait(index, this) + if (this.onwait) this.onwait(index, this) - // Unslab only when it takes up less then half the slab - const block = resolved !== null && 2 * resolved.byteLength < resolved.buffer.byteLength - ? unslab(resolved) - : resolved + const activeRequests = (opts && opts.activeRequests) || this.activeRequests - if (this.cache && fork === this.core.tree.fork) { - this.cache.set(index, Promise.resolve(block)) - } + const req = this.replicator.addBlock(activeRequests, index) + req.snapshot = index < this.length - return block + const timeout = opts && opts.timeout !== undefined ? opts.timeout : this.timeout + if (timeout) req.context.setTimeout(req, timeout) + + const replicatedBlock = await req.promise + if (this._snapshot !== null) checkSnapshot(this._snapshot, index) + + return maybeUnslab(replicatedBlock) + } + + async restoreBatch (length, blocks) { + if (this.opened === false) await this.opening + return this.state.tree.restoreBatch(length) } _shouldWait (opts, defaultValue) { @@ -979,27 +994,30 @@ module.exports = class Hypercore extends EventEmitter { if (this.opened === false) await this.opening const { - fork = this.core.tree.fork + 1, + fork = this.state.tree.fork + 1, keyPair = this.keyPair, signature = null } = typeof opts === 'number' ? { fork: opts } : opts + const isDefault = this.state === this.core.state const writable = !this._readonly && !!(signature || (keyPair && keyPair.secretKey)) - if (writable === false && (newLength > 0 || fork !== this.core.tree.fork)) throw SESSION_NOT_WRITABLE() + if (isDefault && writable === false && (newLength > 0 || fork !== this.state.tree.fork)) throw SESSION_NOT_WRITABLE() - await this.core.truncate(newLength, fork, { keyPair, signature }) + await this.state.truncate(newLength, fork, { keyPair, signature }) // TODO: Should propagate from an event triggered by the oplog - this.replicator.updateAll() + if (this.state === this.core.state) this.replicator.updateAll() } async append (blocks, opts = {}) { if (this.opened === false) await this.opening + const isDefault = this.state === this.core.state + const { keyPair = this.keyPair, signature = null } = opts const writable = !this._readonly && !!(signature || (keyPair && keyPair.secretKey)) - if (writable === false) throw SESSION_NOT_WRITABLE() + if (isDefault && writable === false) throw SESSION_NOT_WRITABLE() blocks = Array.isArray(blocks) ? blocks : [blocks] @@ -1018,16 +1036,16 @@ module.exports = class Hypercore extends EventEmitter { } } - return this.core.append(buffers, { keyPair, signature, preappend }) + return this.state.append(buffers, { keyPair, signature, preappend }) } async treeHash (length) { if (length === undefined) { await this.ready() - length = this.core.tree.length + length = this.state.tree.length } - const roots = await this.core.tree.getRoots(length) + const roots = await this.state.tree.getRoots(length) return this.crypto.tree(roots) } @@ -1112,17 +1130,13 @@ function isStream (s) { return typeof s === 'object' && s && typeof s.pipe === 'function' } -function isRandomAccessClass (fn) { - return !!(typeof fn === 'function' && fn.prototype && typeof fn.prototype.open === 'function') -} - function toHex (buf) { return buf && b4a.toString(buf, 'hex') } function preappend (blocks) { - const offset = this.core.tree.length - const fork = this.core.tree.fork + const offset = this.state.tree.length + const fork = this.state.tree.fork for (let i = 0; i < blocks.length; i++) { this.encryption.encrypt(offset + i, blocks[i], fork) @@ -1137,10 +1151,21 @@ function ensureEncryption (core, opts) { core.encryption = new BlockEncryption(opts.encryptionKey, core.key, { compat: core.core ? core.core.compat : true, isBlockKey: opts.isBlockKey }) } -function createCache (cache) { - return cache === true ? new Xache({ maxSize: 65536, maxAge: 0 }) : (cache || null) -} - function isValidIndex (index) { return index === 0 || index > 0 } + +function maybeUnslab (block) { + // Unslab only when it takes up less then half the slab + return block !== null && 2 * block.byteLength < block.buffer.byteLength ? unslab(block) : block +} + +function checkSnapshot (snapshot, index) { + if (index >= snapshot.compatLength) throw SNAPSHOT_NOT_AVAILABLE() +} + +function readBlock (reader, index) { + const promise = reader.getBlock(index) + reader.tryFlush() + return promise +} diff --git a/lib/audit.js b/lib/audit.js index 6c78e2c9..efb84eba 100644 --- a/lib/audit.js +++ b/lib/audit.js @@ -6,7 +6,7 @@ const b4a = require('b4a') // this is optimised for speed over mem atm // can be tweaked in the future -module.exports = async function auditCore (core) { +module.exports = async function auditCore (core, update) { const corrections = { tree: 0, blocks: 0 @@ -48,10 +48,6 @@ module.exports = async function auditCore (core) { if (rightNode.size) clearNode(rightNode) } - if (corrections.tree) { - core.tree.cache.clear() - } - let i = 0 let nextOffset = -1 while (i < length) { @@ -69,7 +65,7 @@ module.exports = async function auditCore (core) { try { nextOffset = await core.tree.byteOffset(i * 2) } catch { - core._setBitfield(i, false) + update.bitfield.set(i, false) corrections.blocks++ i++ continue @@ -83,7 +79,7 @@ module.exports = async function auditCore (core) { nextOffset += blk.byteLength if (!b4a.equals(hash, node.hash)) { - core._setBitfield(i, false) + update.bitfield.set(i, false) corrections.blocks++ } diff --git a/lib/bit-interlude.js b/lib/bit-interlude.js new file mode 100644 index 00000000..eacfdc64 --- /dev/null +++ b/lib/bit-interlude.js @@ -0,0 +1,151 @@ +const assert = require('nanoassert') +const b4a = require('b4a') +const quickbit = require('./compat').quickbit + +module.exports = class BitInterlude { + constructor (bitfield) { + this.drop = false + this.bitfield = bitfield + this.ranges = [] + } + + contiguousLength (from) { + if (this.drop && this.ranges.length > 0 && this.ranges[0].start < from) { + return this.ranges[0].start + } + + // TODO: be smarter + while (this.get(from) === true) from++ + return from + } + + get (index) { + let start = 0 + let end = this.ranges.length + + while (start < end) { + const mid = (start + end) >> 1 + const r = this.ranges[mid] + + if (index < r.start) { + end = mid + continue + } + + if (index >= r.end) { + if (mid === start) break + start = mid + continue + } + + return this.drop === false + } + + return this.bitfield.get(index) + } + + setRange (start, end, value) { + assert(this.drop !== value || this.ranges.length === 0) + assert(value === true || this.ranges.length === 0) + + this.drop = value === false + + let r = null + + for (let i = 0; i < this.ranges.length; i++) { + r = this.ranges[i] + + // if already inside, stop + if (r.start <= start && end <= r.end) return + + // we wanna overun the interval + if (start > r.end) { + continue + } + + // we overran but this interval is ending after us, move it back + if (end >= r.start && end <= r.end) { + r.start = start + return + } + + // we overran but our start is contained in this interval, move start back + if (start >= r.start && start <= r.end) { + start = r.start + } + + let remove = 0 + + for (let j = i; j < this.ranges.length; j++) { + const n = this.ranges[j] + if (n.start > end) break + if (n.start <= end && n.end > end) end = n.end + remove++ + } + + this.ranges.splice(i, remove, { start, end }) + return + } + + if (r !== null) { + if (start <= r.end && end > r.end) { + r.end = end + return + } + + // we never + if (r.end > start) return + } + + this.ranges.push({ start, end }) + } + + flush (writer, debug) { + if (!this.ranges.length) return { ranges: [], drop: this.drop } + + let index = this.ranges[0].start + const final = this.ranges[this.ranges.length - 1].end + + let i = 0 + + while (index < final) { + const page = this.bitfield.getPage(index, this.drop === false) + const buf = b4a.allocUnsafe(page.bitfield.byteLength) + + const view = new DataView( + buf.buffer, + buf.byteOffset, + buf.byteLength + ) + + for (let i = 0; i < page.bitfield.length; i++) { + view.setUint32(i * 4, page.bitfield[i], true) + } + + const last = (page.index + 1) * (buf.byteLength << 3) + const offset = page.index * buf.byteLength << 3 + + while (i < this.ranges.length) { + const { start, end } = this.ranges[i] + + const from = start < index ? index : start + const to = end < last ? end : last + + quickbit.fill(buf, this.drop === false, from - offset, to - offset) + + index = to + + if (to === last) break + + i++ + } + + writer.putBitfieldPage(page.index, buf) + } + + return { + ranges: this.ranges, + drop: this.drop + } + } +} diff --git a/lib/bitfield.js b/lib/bitfield.js index 350bf07f..d05f3983 100644 --- a/lib/bitfield.js +++ b/lib/bitfield.js @@ -14,7 +14,6 @@ const SEGMENT_GROWTH_FACTOR = 4 class BitfieldPage { constructor (index, segment) { - this.dirty = false this.index = index this.offset = index * BYTES_PER_PAGE - segment.offset this.bitfield = null @@ -27,7 +26,7 @@ class BitfieldPage { return this.segment.tree } - get (index) { + get (index, dirty) { return quickbit.get(this.bitfield, index) } @@ -172,10 +171,8 @@ class BitfieldSegment { } module.exports = class Bitfield { - constructor (storage, buffer) { - this.unflushed = [] - this.storage = storage - this.resumed = !!(buffer && buffer.byteLength >= 4) + constructor (buffer) { + this.resumed = !!(buffer && buffer.byteLength >= 0) this._pages = new BigSparseArray() this._segments = new BigSparseArray() @@ -213,6 +210,10 @@ module.exports = class Bitfield { } } + static from (bitfield) { + return new Bitfield(bitfield.toBuffer(bitfield._pages.maxLength * BITS_PER_PAGE)) + } + toBuffer (length) { const pages = Math.ceil(length / BITS_PER_PAGE) const buffer = b4a.allocUnsafe(pages * BYTES_PER_PAGE) @@ -245,6 +246,21 @@ module.exports = class Bitfield { return p || null } + merge (bitfield, length) { + let i = 0 + + while (i < length) { + const start = bitfield.firstSet(i) + i = bitfield.firstUnset(start) + + if (i === -1 || i > length) i = length + + this.setRange(start, i, true) + + if (i >= length) break + } + } + get (index) { const j = index & (BITS_PER_PAGE - 1) const i = (index - j) / BITS_PER_PAGE @@ -254,6 +270,24 @@ module.exports = class Bitfield { return p ? p.get(j) : false } + getPage (index, create) { + const j = index & (BITS_PER_PAGE - 1) + const i = (index - j) / BITS_PER_PAGE + + let p = this._pages.get(i) + + if (p) return p + + if (!create) return null + + const k = Math.floor(i / PAGES_PER_SEGMENT) + const s = this._segments.get(k) || this._segments.set(k, new BitfieldSegment(k, new Uint32Array(k === 0 ? INITIAL_WORDS_PER_SEGMENT : WORDS_PER_SEGMENT))) + + p = this._pages.set(i, new BitfieldPage(i, s)) + + return p + } + set (index, val) { const j = index & (BITS_PER_PAGE - 1) const i = (index - j) / BITS_PER_PAGE @@ -267,14 +301,7 @@ module.exports = class Bitfield { p = this._pages.set(i, new BitfieldPage(i, s)) } - if (p) { - p.set(j, val) - - if (!p.dirty) { - p.dirty = true - this.unflushed.push(p) - } - } + if (p) p.set(j, val) } setRange (start, length, val) { @@ -294,14 +321,7 @@ module.exports = class Bitfield { const end = Math.min(j + length, BITS_PER_PAGE) const range = end - j - if (p) { - p.setRange(j, range, val) - - if (!p.dirty) { - p.dirty = true - this.unflushed.push(p) - } - } + if (p) p.setRange(j, range, val) j = 0 i++ @@ -420,68 +440,37 @@ module.exports = class Bitfield { } } - clear () { - return new Promise((resolve, reject) => { - this.storage.truncate(0, (err) => { - if (err) return reject(err) - this._pages = new BigSparseArray() - this.unflushed = [] - resolve() - }) - }) + clear (writer) { + return writer.deleteBitfieldPageRange(0, -1) } - close () { - return new Promise((resolve, reject) => { - this.storage.close((err) => { - if (err) reject(err) - else resolve() - }) - }) - } + // close () { + // return new Promise((resolve, reject) => { + // this.storage.close((err) => { + // if (err) reject(err) + // else resolve() + // }) + // }) + // } - flush () { - return new Promise((resolve, reject) => { - if (!this.unflushed.length) return resolve() + onupdate ({ ranges, drop }) { + for (const r of ranges) { + this.setRange(r.start, r.end - r.start, drop === false) + } + } - const self = this - let missing = this.unflushed.length - let error = null + static async open (storage) { + const last = await storage.peakLastBitfieldPage() + if (last === null) return new Bitfield(storage, null) - for (const page of this.unflushed) { - const buf = b4a.from( - page.bitfield.buffer, - page.bitfield.byteOffset, - page.bitfield.byteLength - ) + const buffer = b4a.alloc((last.index + 1) * BYTES_PER_PAGE) + const stream = storage.createBitfieldPageStream() - page.dirty = false - this.storage.write(page.index * BYTES_PER_PAGE, buf, done) - } + for await (const { index, page } of stream) { + buffer.set(page, index * BYTES_PER_PAGE) + } - function done (err) { - if (err) error = err - if (--missing) return - if (error) return reject(error) - self.unflushed = [] - resolve() - } - }) - } - - static open (storage, tree = null) { - return new Promise((resolve, reject) => { - storage.stat((err, st) => { - if (err) return resolve(new Bitfield(storage, null)) - let size = st.size - (st.size & 3) - if (!size) return resolve(new Bitfield(storage, null)) - if (tree) size = Math.min(size, ceilTo(tree.length / 8, 4096)) - storage.read(0, size, (err, data) => { - if (err) return reject(err) - resolve(new Bitfield(storage, data)) - }) - }) - }) + return new Bitfield(buffer) } } diff --git a/lib/block-store.js b/lib/block-store.js index 05658ccb..8833175b 100644 --- a/lib/block-store.js +++ b/lib/block-store.js @@ -1,63 +1,26 @@ -const b4a = require('b4a') -const { WRITE_FAILED } = require('hypercore-errors') - module.exports = class BlockStore { - constructor (storage, tree) { + constructor (storage) { this.storage = storage - this.tree = tree - } - - async get (i, tree) { - if (!tree) tree = this.tree - const [offset, size] = await tree.byteRange(2 * i) - return this._read(offset, size) } - async put (i, data, offset) { - return this._write(offset, data) + async get (reader, i) { + return reader.getBlock(i) } - putBatch (i, batch, offset) { - if (batch.length === 0) return Promise.resolve() - return this.put(i, batch.length === 1 ? batch[0] : b4a.concat(batch), offset) + put (writer, i, data) { + writer.putBlock(i, data) } - clear (offset = 0, length = -1) { - return new Promise((resolve, reject) => { - if (length === -1) this.storage.truncate(offset, done) - else this.storage.del(offset, length, done) - - function done (err) { - if (err) reject(err) - else resolve() - } - }) - } - - close () { - return new Promise((resolve, reject) => { - this.storage.close((err) => { - if (err) reject(err) - else resolve() - }) - }) - } + putBatch (writer, i, blocks) { + if (blocks.length === 0) return Promise.resolve() - _read (offset, size) { - return new Promise((resolve, reject) => { - this.storage.read(offset, size, (err, data) => { - if (err) reject(err) - else resolve(data) - }) - }) + for (let j = 0; j < blocks.length; j++) { + writer.putBlock(i + j, blocks[j]) + } } - _write (offset, data) { - return new Promise((resolve, reject) => { - this.storage.write(offset, data, (err) => { - if (err) reject(WRITE_FAILED(err.message)) - else resolve(offset + data.byteLength) - }) - }) + clear (writer, start, length = -1) { + const end = length === -1 ? -1 : start + length + writer.deleteBlockRange(start, end) } } diff --git a/lib/core.js b/lib/core.js index 37d4e0c5..e2634e63 100644 --- a/lib/core.js +++ b/lib/core.js @@ -1,29 +1,405 @@ const hypercoreCrypto = require('hypercore-crypto') const b4a = require('b4a') +const assert = require('nanoassert') const unslab = require('unslab') -const Oplog = require('./oplog') -const BigHeader = require('./big-header') +const MemoryOverlay = require('./memory-overlay') const Mutex = require('./mutex') const MerkleTree = require('./merkle-tree') const BlockStore = require('./block-store') +const BitInterlude = require('./bit-interlude') const Bitfield = require('./bitfield') const RemoteBitfield = require('./remote-bitfield') -const Info = require('./info') +// const Info = require('./info') const { BAD_ARGUMENT, STORAGE_EMPTY, STORAGE_CONFLICT, INVALID_OPERATION, INVALID_SIGNATURE, INVALID_CHECKSUM } = require('hypercore-errors') -const m = require('./messages') const Verifier = require('./verifier') const audit = require('./audit') +const HEAD = Symbol.for('head') +const CORE = Symbol.for('core') +const CONTIG = Symbol.for('contig') +const TREE = Symbol.for('tree') +const BITFIELD = Symbol.for('bitfield') +const USER_DATA = Symbol.for('user-data') + +class Update { + constructor (batch, bitfield, header, state) { + this.batch = batch + this.bitfield = new BitInterlude(bitfield) + + this.state = state + + this.contiguousLength = header.hints.contiguousLength + + this.tree = null + + this.updates = [] + this._coreUpdates = [] + } + + async flushBitfield () { + const update = await this.bitfield.flush(this.batch) + if (update) this.updates.push({ type: BITFIELD, update }) + } + + flushTreeBatch (batch) { + // TODO: investigate when tree is not commitable + if (batch.commitable()) { + const update = batch.commit(this.batch) + this.updates.push({ type: TREE, update }) + } + + if (batch.upgraded) { + this.tree = { + fork: batch.fork, + length: batch.length, + rootHash: batch.hash(), + signature: batch.signature + } + } + } + + setUserData (key, value) { + this.updates.push({ type: USER_DATA, update: { key, value } }) + this.batch.setUserData(key, value) + } + + coreUpdate (update) { + let { bitfield, status, value, from } = update + + if (bitfield) { + const contig = updateContigBatch(this.contiguousLength, bitfield, this.bitfield) + + status |= contig.status + + if (contig.length > this.contiguousLength || (bitfield.drop && contig.length < this.contiguousLength)) { + this.contiguousLength = contig.length + this._coreUpdates.push({ type: CONTIG, update: contig.length }) + } + } + + this._coreUpdates.push({ type: CORE, update: { status, bitfield, value, from } }) + } + + async flush () { + await this.flushBitfield() + + if (this.tree) { + this.batch.setCoreHead(this.tree) + this.updates.push({ type: HEAD, update: this.tree }) + } + + // bitfield flushed before core updates + for (const upd of this._coreUpdates) { + this.updates.push(upd) + } + + await this.batch.flush() + + return this.updates + } + + async truncate (batch, from) { + const bitfield = { + drop: true, + start: batch.ancestors, + length: batch.treeLength - batch.ancestors + } + + this.bitfield.setRange(batch.ancestors, batch.treeLength, false) + this.batch.deleteBlockRange(bitfield.start, bitfield.start + bitfield.length) + + const status = (batch.length > batch.ancestors) ? 0b0011 : 0b0010 + + this.flushTreeBatch(batch) + this.coreUpdate({ status, bitfield, value: null, from }) + } +} + +class SessionState { + constructor (core, storage, blocks, tree, bitfield, treeLength) { + this.core = core + + this.storage = storage + + this.mutex = new Mutex() + + this.blocks = blocks + this.tree = tree + this.bitfield = bitfield + + this.treeLength = treeLength + + this.active = 0 + + this._onflush = null + this._flushing = null + this._activeBatch = null + + this.ref() + } + + get isSnapshot () { + return this.storage.snapshotted + } + + get isDefault () { + return this.core.state === this + } + + async unref () { + if (--this.active > 0) return Promise.resolve() + + await this.close() + } + + ref () { + this.active++ + return this + } + + async close () { + await this.storage.close() + await this.mutex.destroy(new Error('Closed')) + } + + snapshot () { + const s = new SessionState( + this.core, + this.storage.snapshot(), + this.blocks, + this.tree, // todo: should clone also but too many changes atm + this.bitfield, + this.treeLength + ) + + return s + } + + memoryOverlay () { + const storage = new MemoryOverlay(this.storage) + const s = new SessionState( + this.core, + storage, + this.blocks, + this.tree.clone(storage), + this.bitfield, + this.treeLength + ) + + return s + } + + _clearActiveBatch (err) { + if (!this._activeBatch) return + this._activeBatch.destroy() + + if (this._onflush) this._onflush(err) + + this._onflush = null + this._flushing = null + + this._activeBatch = null + } + + createUpdate () { + assert(!this._activeBatch && !this.storage.snapshotted) + + this._activeBatch = this.overlay ? this.overlay.createWriteBatch() : this.storage.createWriteBatch() + return new Update(this._activeBatch, this.bitfield, this.core.header, this) + } + + async flushUpdate (u) { + const flushing = this._flushUpdateBatch(u) + + try { + if (!this._flushing) this._flushing = flushing + + await flushing + } finally { + this._clearActiveBatch(this) + } + } + + flushed () { + if (!this._activeBatch) return + + if (this._flushing) return this._flushing + + this._flushing = new Promise(resolve => { + this._onflush = resolve + }) + + return this._flushing + } + + async _flushUpdateBatch (u) { + await u.flush() + + if (!u.updates.length) return + + for (const { type, update } of u.updates) { + switch (type) { + case TREE: // tree + if (!this.isDefault) this.tree.onupdate(update) + break + + case BITFIELD: // bitfield + this.bitfield.onupdate(update) + break + } + } + + if (!this.isDefault) return + + this.core._processUpdates(u.updates) + } + + async setUserData (key, value) { + await this.mutex.lock() + + try { + const update = this.createUpdate() + update.setUserData(key, value) + + return await this.flushUpdate(update) + } finally { + this._clearActiveBatch() + this.mutex.unlock() + } + } + + async truncate (length, fork, { signature, keyPair } = {}) { + if (this.tree.prologue && length < this.tree.prologue.length) { + throw INVALID_OPERATION('Truncation breaks prologue') + } + + if (!keyPair && this.isDefault) keyPair = this.core.header.keyPair + + await this.mutex.lock() + + try { + const batch = await this.tree.truncate(length, fork) + + if (!signature && keyPair && length > 0) signature = this.core.verifier.sign(batch, keyPair) + if (signature) batch.signature = signature + + const update = this.createUpdate() + + // upsert compat manifest + if (this.core.verifier === null && keyPair) this.core._setManifest(update, null, keyPair) + + await update.truncate(batch, null) + + if (batch.length < this.treeLength) this.treeLength = batch.length + + await this.flushUpdate(update) + } finally { + this._clearActiveBatch() + this.mutex.unlock() + } + } + + async clear (start, end, cleared) { + await this.mutex.lock() + + try { + const bitfield = { + start, + length: end - start, + drop: true + } + + const update = this.createUpdate() + + update.bitfield.setRange(start, end, false) + + end = this.bitfield.firstSet(end) + + // TODO: verify this: + // start = state.bitfield.lastSet(start) + 1 + // end = state.bitfield.firstSet(end) + + if (end === -1) end = this.tree.length + if (start === -1 || start >= this.tree.length) return + + this.blocks.clear(update.batch, start, end - start) + update.coreUpdate({ status: 0, bitfield, value: null, from: null }) + + if (start < this.treeLength) this.treeLength = start + + await this.flushUpdate(update) + } finally { + this._clearActiveBatch() + this.mutex.unlock() + } + } + + async append (values, { signature, keyPair, preappend } = {}) { + if (!keyPair && this.isDefault) keyPair = this.core.header.keyPair + + await this.mutex.lock() + + try { + const update = this.createUpdate() + + // upsert compat manifest + if (this.core.verifier === null && keyPair) this.core._setManifest(update, null, keyPair) + + if (preappend) await preappend(values) + + if (!values.length) { + await this.flushUpdate(update) + return { length: this.tree.length, byteLength: this.tree.byteLength } + } + + const batch = this.tree.batch() + for (const val of values) batch.append(val) + + // only multisig can have prologue so signature is always present + if (this.tree.prologue && batch.length < this.tree.prologue.length) { + throw INVALID_OPERATION('Append is not consistent with prologue') + } + + if (!signature && keyPair) signature = this.core.verifier.sign(batch, keyPair) + if (signature) batch.signature = signature + + update.flushTreeBatch(batch) + update.bitfield.setRange(batch.ancestors, batch.length, true) + + this.blocks.putBatch(update.batch, this.tree.length, values) + + const bitfield = { + drop: false, + start: batch.ancestors, + length: values.length + } + + update.coreUpdate({ + bitfield, + status: 0b0001, + value: null, + from: null + }) + + await this.flushUpdate(update) + + return { length: batch.length, byteLength: batch.byteLength } + } finally { + this._clearActiveBatch() + this.mutex.unlock() + } + } +} + module.exports = class Core { - constructor (header, compat, crypto, oplog, bigHeader, tree, blocks, bitfield, verifier, sessions, legacy, globalCache, onupdate, onconflict) { + constructor (storage, header, compat, crypto, tree, blocks, bitfield, verifier, sessions, legacy, globalCache, onupdate, onconflict) { + this.storage = storage this.onupdate = onupdate this.onconflict = onconflict this.preupdate = null this.header = header this.compat = compat this.crypto = crypto - this.oplog = oplog - this.bigHeader = bigHeader this.tree = tree this.blocks = blocks this.bitfield = bitfield @@ -32,35 +408,48 @@ module.exports = class Core { this.updating = false this.closed = false this.skipBitfield = null - this.active = sessions.length this.sessions = sessions this.globalCache = globalCache + this.state = new SessionState(this, storage, this.blocks, tree, bitfield, tree.length) + this._manifestFlushed = !!header.manifest this._maxOplogSize = 65536 this._autoFlush = 1 + this._onflush = null + this._flushing = null + this._activeBatch = null + this._bitfield = null this._verifies = null this._verifiesFlushed = null - this._mutex = new Mutex() this._legacy = legacy } - static async open (storage, opts = {}) { - const oplogFile = storage('oplog') - const treeFile = storage('tree') - const bitfieldFile = storage('bitfield') - const dataFile = storage('data') - const headerFile = storage('header') + async createSession (name, length, overwrite) { + const treeLength = length === undefined ? this.tree.length : length - try { - return await this.resume(oplogFile, treeFile, bitfieldFile, dataFile, headerFile, opts) - } catch (err) { - await closeAll(oplogFile, treeFile, bitfieldFile, dataFile, headerFile) - throw err - } + const storage = await this.storage.registerBatch(name, treeLength, overwrite) + const treeInfo = await getCoreHead(storage) + const bitfield = await Bitfield.open(storage) + + bitfield.merge(this.bitfield, treeLength) + + const tree = await MerkleTree.open(storage, { + crypto: this.crypto, + prologue: this.tree.prologue, + length: (length === treeLength || !treeInfo) ? treeLength : treeInfo.length + }) + + const sharedLength = Math.min(treeLength, tree.length) + + return new SessionState(this, storage, this.blocks, tree, bitfield, sharedLength, null, null) } - static async resume (oplogFile, treeFile, bitfieldFile, dataFile, headerFile, opts) { + static async open (db, opts = {}) { + const discoveryKey = opts.discoveryKey || (opts.key && hypercoreCrypto.discoveryKey(opts.key)) + + let storage = await db.resume(discoveryKey) + let overwrite = opts.overwrite === true const force = opts.force === true @@ -69,23 +458,15 @@ module.exports = class Core { // kill this flag soon const legacy = !!opts.legacy - const oplog = new Oplog(oplogFile, { - headerEncoding: m.oplog.header, - entryEncoding: m.oplog.entry, - readonly: opts.readonly - }) - // default to true for now if no manifest is provided let compat = opts.compat === true || (opts.compat !== false && !opts.manifest) - let { header, entries } = await oplog.open() + let header = storage ? parseHeader(await getCoreInfo(storage)) : null if (force && opts.key && header && !b4a.equals(header.key, opts.key)) { overwrite = true } - const bigHeader = new BigHeader(headerFile) - if (!header || overwrite) { if (!createIfMissing) { throw STORAGE_EMPTY('No Hypercore is stored here') @@ -98,11 +479,11 @@ module.exports = class Core { } const keyPair = opts.keyPair || (opts.key ? null : crypto.keyPair()) + const defaultManifest = !opts.manifest && (!!opts.compat || !opts.key || !!(keyPair && b4a.equals(opts.key, keyPair.publicKey))) const manifest = defaultManifest ? Verifier.defaultSignerManifest(opts.key || keyPair.publicKey) : Verifier.createManifest(opts.manifest) header = { - external: null, key: opts.key || (compat ? manifest.signers[0].publicKey : Verifier.manifestHash(manifest)), manifest, keyPair: keyPair ? { publicKey: keyPair.publicKey, secretKey: keyPair.secretKey || null } : null, @@ -119,15 +500,28 @@ module.exports = class Core { } } - await flushHeader(oplog, bigHeader, header) - } else if (header.external) { - header = await bigHeader.load(header.external) + const discoveryKey = opts.discoveryKey || hypercoreCrypto.discoveryKey(header.key) + + storage = await db.create({ + key: header.key, + manifest: manifest ? Verifier.encodeManifest(manifest) : null, + keyPair, + discoveryKey + }) } // unslab the long lived buffers to avoid keeping the slab alive header.key = unslab(header.key) - header.tree.rootHash = unslab(header.tree.rootHash) - header.tree.signature = unslab(header.tree.signature) + + if (header.tree) { + header.tree.rootHash = unslab(header.tree.rootHash) + header.tree.signature = unslab(header.tree.signature) + } + + if (header.keyPair) { + header.keyPair.publicKey = unslab(header.keyPair.publicKey) + header.keyPair.secretKey = unslab(header.keyPair.secretKey) + } if (header.keyPair) { header.keyPair.publicKey = unslab(header.keyPair.publicKey) @@ -156,92 +550,71 @@ module.exports = class Core { const prologue = header.manifest ? header.manifest.prologue : null - const tree = await MerkleTree.open(treeFile, { crypto, prologue, ...header.tree }) - const bitfield = await Bitfield.open(bitfieldFile) - const blocks = new BlockStore(dataFile, tree) + const tree = await MerkleTree.open(storage, { crypto, prologue, ...header.tree }) + const bitfield = await Bitfield.open(storage) + const blocks = new BlockStore(storage) if (overwrite) { - await tree.clear() - await blocks.clear() - await bitfield.clear() - entries = [] + const writer = storage.createWriteBatch() + tree.clear(writer) + blocks.clear(writer) + bitfield.clear(writer) + await writer.flush() } - // compat from earlier version that do not store contig length - if (header.hints.contiguousLength === 0) { - while (bitfield.get(header.hints.contiguousLength)) header.hints.contiguousLength++ + for await (const { key, value } of storage.createUserDataStream()) { + header.userData.push({ key, value: unslab(value) }) } + // compat from earlier version that do not store contig length + // if (header.hints.contiguousLength === 0) { + // while (bitfield.get(header.hints.contiguousLength)) header.hints.contiguousLength++ + // } + // to unslab if (header.manifest) header.manifest = Verifier.createManifest(header.manifest) const verifier = header.manifest ? new Verifier(header.key, header.manifest, { crypto, legacy }) : null - for (const e of entries) { - if (e.userData) { - updateUserData(header.userData, e.userData.key, e.userData.value) - } - - if (e.treeNodes) { - for (const node of e.treeNodes) { - tree.addNode(node) - } - } - - if (e.bitfield) { - bitfield.setRange(e.bitfield.start, e.bitfield.length, !e.bitfield.drop) - updateContig(header, e.bitfield, bitfield) - } - - if (e.treeUpgrade) { - const batch = await tree.truncate(e.treeUpgrade.length, e.treeUpgrade.fork) - batch.ancestors = e.treeUpgrade.ancestors - batch.signature = unslab(e.treeUpgrade.signature) - addReorgHint(header.hints.reorgs, tree, batch) - batch.commit() - - header.tree.length = tree.length - header.tree.fork = tree.fork - header.tree.rootHash = tree.hash() - header.tree.signature = tree.signature - } - } - - for (const entry of header.userData) { - entry.value = unslab(entry.value) - } - - return new this(header, compat, crypto, oplog, bigHeader, tree, blocks, bitfield, verifier, opts.sessions || [], legacy, opts.globalCache || null, opts.onupdate || noop, opts.onconflict || noop) + return new this(storage, header, compat, crypto, tree, blocks, bitfield, verifier, opts.sessions || [], legacy, opts.globalCache || null, opts.onupdate || noop, opts.onconflict || noop) } - async audit () { - await this._mutex.lock() + async audit (state = this.state) { + await state.mutex.lock() try { - await this._flushOplog() - const corrections = await audit(this) - if (corrections.blocks || corrections.tree) await this._flushOplog() + const update = state.createUpdate() + const corrections = await audit(this, update) + if (corrections.blocks || corrections.tree) { + await state.flushUpdate(update) + } + return corrections } finally { - await this._mutex.unlock() + state._clearActiveBatch() + await state.mutex.unlock() } } async setManifest (manifest) { - await this._mutex.lock() + await this.state.mutex.lock() try { if (manifest && this.header.manifest === null) { if (!Verifier.isValidManifest(this.header.key, manifest)) throw INVALID_CHECKSUM('Manifest hash does not match') - this._setManifest(Verifier.createManifest(manifest), null) - await this._flushOplog() + + const update = this.state.createUpdate() + this._setManifest(update, Verifier.createManifest(manifest), null) + + await this.state.flushUpdate(update) } } finally { - this._mutex.unlock() + this.state._clearActiveBatch() + this.state.mutex.unlock() } } - _setManifest (manifest, keyPair) { + _setManifest (update, manifest, keyPair) { if (!manifest && b4a.equals(keyPair.publicKey, this.header.key)) manifest = Verifier.defaultSignerManifest(this.header.key) if (!manifest) return @@ -250,91 +623,76 @@ module.exports = class Core { if (verifier.prologue) this.tree.setPrologue(verifier.prologue) this.header.manifest = manifest + + update.batch.setCoreAuth({ key: this.header.key, manifest: Verifier.encodeManifest(manifest) }) + this.compat = verifier.compat this.verifier = verifier this._manifestFlushed = false - this.onupdate(0b10000, null, null, null) - } - - _shouldFlush () { - // TODO: make something more fancy for auto flush mode (like fibonacci etc) - if (--this._autoFlush <= 0 || this.oplog.byteLength >= this._maxOplogSize) { - this._autoFlush = 4 - return true - } - - if (!this._manifestFlushed && this.header.manifest) { - this._manifestFlushed = true - return true - } - - return false + update.coreUpdate({ status: 0b10000, bitfield: null, value: null, from: null }) } async copyPrologue (src, { additional = [] } = {}) { - await this._mutex.lock() + await this.state.mutex.lock() try { - await src._mutex.lock() + await src.mutex.lock() } catch (err) { - this._mutex.unlock() + this.state.mutex.unlock() throw err } try { + const update = this.state.createUpdate() + const prologue = this.header.manifest && this.header.manifest.prologue if (!prologue) throw INVALID_OPERATION('No prologue present') const srcLength = prologue.length - additional.length const srcBatch = srcLength !== src.tree.length ? await src.tree.truncate(srcLength) : src.tree.batch() - const srcRoots = srcBatch.roots.slice(0) - const srcByteLength = srcBatch.byteLength + const srcNodes = srcBatch.roots.slice(0) - for (const blk of additional) srcBatch.append(blk) + if (additional.length) { + for (const blk of additional) srcBatch.append(blk) + } if (!b4a.equals(srcBatch.hash(), prologue.hash)) throw INVALID_OPERATION('Source tree is conflicting') // all hashes are correct, lets copy - const entry = { - userData: null, - treeNodes: srcRoots, - treeUpgrade: null, - bitfield: null - } + let bitfield = null + + const batch = this.tree.batch() if (additional.length) { - await this.blocks.putBatch(srcLength, additional, srcByteLength) - entry.treeNodes = entry.treeNodes.concat(srcBatch.nodes) - entry.bitfield = { + this.blocks.putBatch(update.batch, srcLength, additional) + + for (const node of srcBatch.nodes) srcNodes.push(node) + + bitfield = { drop: false, start: srcLength, length: additional.length } } - await this.oplog.append([entry], false) - this.tree.addNodes(entry.treeNodes) + batch.nodes = srcNodes if (this.header.tree.length < srcBatch.length) { - this.header.tree.length = srcBatch.length - this.header.tree.rootHash = srcBatch.hash() + batch.upgraded = true + batch.length = srcBatch.length + batch.byteLength = srcBatch.byteLength + batch.roots = srcBatch.roots - this.tree.length = srcBatch.length - this.tree.byteLength = srcBatch.byteLength - this.tree.roots = srcBatch.roots - this.onupdate(0b0001, null, null, null) + this.onupdate({ status: 0b0001, bitfield: null, value: null, from: null }) } - if (entry.bitfield) { - this._setBitfieldRange(entry.bitfield.start, entry.bitfield.length, true) - this.onupdate(0, entry.bitfield, null, null) + if (bitfield) { + update.bitfield.setRange(bitfield.start, bitfield.start + bitfield.length, true) } - await this._flushOplog() - - // no more additional blocks now and we should be consistant on disk + // no more additional blocks now and we should be consistent on disk // copy over all existing segments... let segmentEnd = 0 @@ -353,358 +711,196 @@ module.exports = class Core { length: segmentEnd - segmentStart } - const segment = [] + const blocks = [] + + const reader = src.storage.createReadBatch() for (let i = segmentStart; i < segmentEnd; i++) { - const blk = await src.blocks.get(i) - segment.push(blk) + blocks.push(src.blocks.get(reader, i)) } + reader.tryFlush() - const offset = await src.tree.byteOffset(2 * segmentStart) - await this.blocks.putBatch(segmentStart, segment, offset) + const segment = await Promise.all(blocks) - const entry = { - userData: null, - treeNodes, - treeUpgrade: null, - bitfield - } + this.blocks.putBatch(update.batch, segmentStart, segment) + + batch.addNodesUnsafe(treeNodes) - await this.oplog.append([entry], false) - this.tree.addNodes(treeNodes) - this._setBitfieldRange(bitfield.start, bitfield.length, true) - this.onupdate(0, bitfield, null, null) - await this._flushOplog() + update.bitfield.setRange(bitfield.start, segmentEnd, true) + update.coreUpdate({ status: 0, bitfield, value: null, from: null }) } - this.header.userData = src.header.userData.slice(0) - const contig = Math.min(src.header.hints.contiguousLength, srcBatch.length) - if (this.header.hints.contiguousLength < contig) this.header.hints.contiguousLength = contig + update.flushTreeBatch(batch) - await this._flushOplog() - } finally { - src._mutex.unlock() - this._mutex.unlock() - } - } + for await (const { key, value } of src.storage.createUserDataStream()) { + this.setUserData(update.batch, key, value) + } - async flush () { - await this._mutex.lock() - try { - this._manifestFlushed = true - this._autoFlush = 4 - await this._flushOplog() + await this.state.flushUpdate(update) } finally { - this._mutex.unlock() + this.state._clearActiveBatch() + src.mutex.unlock() + this.state.mutex.unlock() } } - async _flushOplog () { - // TODO: the apis using this, actually do not need to wait for the bitfields, tree etc to flush - // as their mutations are already stored in the oplog. We could potentially just run this in the - // background. Might be easier to impl that where it is called instead and keep this one simple. - await this.bitfield.flush() - await this.tree.flush() - - return flushHeader(this.oplog, this.bigHeader, this.header) + // async flush () { + // await this.state.mutex.lock() + // try { + // this._manifestFlushed = true + // this._autoFlush = 4 + // await this._flushBitfield(writer) + // } finally { + // this.state.mutex.unlock() + // } + // } + + get isFlushing () { + return !!(this._flushing || this.state._activeBatch) } - _appendBlocks (values) { - return this.blocks.putBatch(this.tree.length, values, this.tree.byteLength) + flushed () { + return this.state.flushed() } - async _writeBlock (batch, index, value) { - const byteOffset = await batch.byteOffset(index * 2) - await this.blocks.put(index, value, byteOffset) - } - - async userData (key, value, flush) { - // TODO: each oplog append can set user data, so we should have a way - // to just hitch a ride on one of the other ongoing appends? - await this._mutex.lock() - - try { - let empty = true - - for (const u of this.header.userData) { - if (u.key !== key) continue - if (value && b4a.equals(u.value, value)) return - empty = false - break - } - - if (empty && !value) return - - const entry = { - userData: { key, value }, - treeNodes: null, - treeUpgrade: null, - bitfield: null - } - - await this.oplog.append([entry], false) - - updateUserData(this.header.userData, key, value) - - if (this._shouldFlush() || flush) await this._flushOplog() - } finally { - this._mutex.unlock() - } - } - - async truncate (length, fork, { signature, keyPair = this.header.keyPair } = {}) { - if (this.tree.prologue && length < this.tree.prologue.length) { - throw INVALID_OPERATION('Truncation breaks prologue') - } + async _processUpdates (updates) { + for (const { type, update } of updates) { + switch (type) { + case HEAD: { + this.header.tree = update + break + } - this.truncating++ - await this._mutex.lock() + case CORE: { // core + this.onupdate(update) + break + } - // upsert compat manifest - if (this.verifier === null && keyPair) this._setManifest(null, keyPair) + case CONTIG: { // contig + this.header.hints.contiguousLength = update + break + } - try { - const batch = await this.tree.truncate(length, fork) - if (length > 0) batch.signature = signature || this.verifier.sign(batch, keyPair) - await this._truncate(batch, null) - } finally { - this.truncating-- - this._mutex.unlock() - } - } + case TREE: // tree + if (update.truncated) addReorgHint(this.header.hints.reorgs, this.tree, update) + this.tree.onupdate(update) + break - async clearBatch () { - await this._mutex.lock() + case BITFIELD: // bitfield + if (this.skipBitfield !== null) this._updateSkipBitfield(update) + break - try { - const len = this.bitfield.findFirst(false, this.tree.length) - if (len <= this.tree.length) return + case USER_DATA: { // user data + let exists = false + for (const entry of this.header.userData) { + if (entry.key !== update.key) continue - const batch = await this.tree.truncate(this.tree.length, this.tree.fork) + entry.value = update.value + exists = true + break + } - batch.signature = this.tree.signature // same sig + if (exists) continue - const entry = { - userData: null, - treeNodes: batch.nodes, - treeUpgrade: batch, - bitfield: { - drop: true, - start: batch.ancestors, - length: len - batch.ancestors + this.header.userData.push({ key: update.key, value: unslab(update.value) }) + break } } - - await this.oplog.append([entry], false) - - this._setBitfieldRange(batch.ancestors, len - batch.ancestors, false) - batch.commit() - - // TODO: (see below todo) - await this._flushOplog() - } finally { - this._mutex.unlock() } } - async clear (start, end, cleared) { - await this._mutex.lock() - - try { - const entry = { - userData: null, - treeNodes: null, - treeUpgrade: null, - bitfield: { - start, - length: end - start, - drop: true - } - } - - await this.oplog.append([entry], false) - - this._setBitfieldRange(start, end - start, false) - - if (start < this.header.hints.contiguousLength) { - this.header.hints.contiguousLength = start - } - - start = this.bitfield.lastSet(start) + 1 - end = this.bitfield.firstSet(end) - - if (end === -1) end = this.tree.length - if (start >= end || start >= this.tree.length) return - - const offset = await this.tree.byteOffset(start * 2) - const endOffset = await this.tree.byteOffset(end * 2) - const length = endOffset - offset - - const before = cleared ? await Info.bytesUsed(this.blocks.storage) : null - - await this.blocks.clear(offset, length) - - const after = cleared ? await Info.bytesUsed(this.blocks.storage) : null - - if (cleared) cleared.blocks = Math.max(before - after, 0) + _writeBlock (writer, index, value) { + this.blocks.put(writer, index, value) + } - this.onupdate(0, entry.bitfield, null, null) + userData (key, value) { + const update = this.state.createUpdate() + this.setUserData(update, key, value) - if (this._shouldFlush()) await this._flushOplog() - } finally { - this._mutex.unlock() - } + return this.state.flushUpdate(update) } - async purge () { - return new Promise((resolve, reject) => { - let missing = 4 - let error = null - - this.oplog.storage.unlink(done) - this.tree.storage.unlink(done) - this.bitfield.storage.unlink(done) - this.blocks.storage.unlink(done) - - function done (err) { - if (err) error = err - if (--missing) return - if (error) reject(error) - else resolve() - } - }) + setUserData (update, key, value) { + return update.setUserData(key, value) } - async insertBatch (batch, values, { signature, keyPair = this.header.keyPair, pending = false, treeLength = batch.treeLength } = {}) { - await this._mutex.lock() + async commit (state, { signature, keyPair = this.header.keyPair, length = state.tree.length, treeLength = state.treeLength } = {}) { + await this.state.mutex.lock() - try { - // upsert compat manifest - if (this.verifier === null && keyPair) this._setManifest(null, keyPair) + const update = this.state.createUpdate() - if (this.tree.fork !== batch.fork) return null + try { + if (this.tree.fork !== state.tree.fork) return null - if (this.tree.length > batch.treeLength) { - if (this.tree.length > batch.length) return null // TODO: partial commit in the future if possible + if (this.tree.length > state.tree.length) return null // TODO: partial commit in the future if possible + if (this.tree.length > treeLength) { for (const root of this.tree.roots) { - const batchRoot = await batch.get(root.index) + const batchRoot = await state.tree.get(root.index) if (batchRoot.size !== root.size || !b4a.equals(batchRoot.hash, root.hash)) { return null } } } - const adding = batch.length - treeLength - - batch.upgraded = !pending && batch.length > this.tree.length - batch.treeLength = this.tree.length - batch.ancestors = this.tree.length - if (batch.upgraded && !pending) batch.signature = signature || this.verifier.sign(batch, keyPair) - - let byteOffset = batch.byteLength - for (let i = 0; i < adding; i++) byteOffset -= values[i].byteLength - - if (pending === true) batch.upgraded = false - - const entry = { - userData: null, - treeNodes: batch.nodes, - treeUpgrade: batch.upgraded ? batch : null, - bitfield: { - drop: false, - start: treeLength, - length: adding - } - } - - await this.blocks.putBatch(treeLength, adding < values.length ? values.slice(0, adding) : values, byteOffset) - await this.oplog.append([entry], false) - - this._setBitfieldRange(entry.bitfield.start, entry.bitfield.length, true) - batch.commit() - - if (batch.upgraded) { - this.header.tree.length = batch.length - this.header.tree.rootHash = batch.hash() - this.header.tree.signature = batch.signature - } - - const status = (batch.upgraded ? 0b0001 : 0) | updateContig(this.header, entry.bitfield, this.bitfield) - if (!pending) { - // we already commit this, and now we signed it, so tell others - if (entry.treeUpgrade && treeLength > batch.treeLength) { - entry.bitfield.start = batch.treeLength - entry.bitfield.length = treeLength - batch.treeLength - } + const promises = [] - this.onupdate(status, entry.bitfield, null, null) - } + const reader = state.storage.createReadBatch() + for (let i = treeLength; i < length; i++) promises.push(reader.getBlock(i)) + reader.tryFlush() - if (this._shouldFlush()) await this._flushOplog() - } finally { - this._mutex.unlock() - } + const values = await Promise.all(promises) - return { length: batch.length, byteLength: batch.byteLength } - } - - async append (values, { signature, keyPair = this.header.keyPair, preappend } = {}) { - await this._mutex.lock() + const batch = await this.tree.reconcile(state.tree, length, treeLength) + if (batch.upgraded) batch.signature = signature || this.verifier.sign(batch, keyPair) - try { // upsert compat manifest - if (this.verifier === null && keyPair) this._setManifest(null, keyPair) + if (this.verifier === null && keyPair) this._setManifest(update, null, keyPair) - if (preappend) await preappend(values) + this.state.blocks.putBatch(update.batch, treeLength, values) - if (!values.length) { - return { length: this.tree.length, byteLength: this.tree.byteLength } - } + update.bitfield.setRange(treeLength, length, true) + update.flushTreeBatch(batch) - const batch = this.tree.batch() - for (const val of values) batch.append(val) + const bitfield = { start: treeLength, length: length - treeLength, drop: false } + const status = batch.upgraded ? 0b0001 : 0 - // only multisig can have prologue so signature is always present - if (this.tree.prologue && batch.length < this.tree.prologue.length) { - throw INVALID_OPERATION('Append is not consistent with prologue') - } - - batch.signature = signature || this.verifier.sign(batch, keyPair) - - const entry = { - userData: null, - treeNodes: batch.nodes, - treeUpgrade: batch, - bitfield: { - drop: false, - start: batch.ancestors, - length: values.length - } - } - - const byteLength = await this._appendBlocks(values) - - await this.oplog.append([entry], false) - - this._setBitfieldRange(batch.ancestors, batch.length - batch.ancestors, true) - batch.commit() - - this.header.tree.length = batch.length - this.header.tree.rootHash = batch.hash() - this.header.tree.signature = batch.signature + update.coreUpdate({ status, bitfield, value: null, from: null }) - const status = 0b0001 | updateContig(this.header, entry.bitfield, this.bitfield) - this.onupdate(status, entry.bitfield, null, null) + await this.state.flushUpdate(update) - if (this._shouldFlush()) await this._flushOplog() + state.treeLength = batch.length - return { length: batch.length, byteLength } + return { + length: batch.length, + byteLength: batch.byteLength + } } finally { - this._mutex.unlock() + this.state._clearActiveBatch() + this.updating = false + this.state.mutex.unlock() } } - _verifyBatchUpgrade (batch, manifest) { + // async purge () { + // return new Promise((resolve, reject) => { + // let missing = 4 + // let error = null + + // this.oplog.storage.unlink(done) + // this.tree.storage.unlink(done) + // this.bitfield.storage.unlink(done) + // this.blocks.storage.unlink(done) + + // function done (err) { + // if (err) error = err + // if (--missing) return + // if (error) reject(error) + // else resolve() + // } + // }) + // } + + _verifyBatchUpgrade (update, batch, manifest) { if (!this.header.manifest) { if (!manifest && this.compat) manifest = Verifier.defaultSignerManifest(this.header.key) @@ -721,55 +917,35 @@ module.exports = class Core { throw INVALID_SIGNATURE('Proof contains an invalid signature') } - if (!this.header.manifest) { - this.header.manifest = manifest - this.compat = verifier.compat - this.verifier = verifier - this.onupdate(0b10000, null, null, null) - } + if (!this.header.manifest && update !== null) this._setManifest(update, manifest, null) } async _verifyExclusive ({ batch, bitfield, value, manifest, from }) { - this._verifyBatchUpgrade(batch, manifest) + await this.state.mutex.lock() - await this._mutex.lock() + const update = this.state.createUpdate() try { + this._verifyBatchUpgrade(update, batch, manifest) + if (!batch.commitable()) return false this.updating = true - const entry = { - userData: null, - treeNodes: batch.nodes, - treeUpgrade: batch, - bitfield - } - if (this.preupdate !== null) await this.preupdate(batch, this.header.key) - if (bitfield) await this._writeBlock(batch, bitfield.start, value) - - await this.oplog.append([entry], false) - - let status = 0b0001 + if (bitfield) this._writeBlock(update.batch, bitfield.start, value) if (bitfield) { - this._setBitfield(bitfield.start, true) - status |= updateContig(this.header, bitfield, this.bitfield) + update.bitfield.setRange(bitfield.start, bitfield.start + 1, true) } - batch.commit() + update.coreUpdate({ status: 0b0001, bitfield, value, from }) + update.flushTreeBatch(batch) - this.header.tree.fork = batch.fork - this.header.tree.length = batch.length - this.header.tree.rootHash = batch.hash() - this.header.tree.signature = batch.signature - - this.onupdate(status, bitfield, value, from) - - if (this._shouldFlush()) await this._flushOplog() + await this.state.flushUpdate(update) } finally { + this.state._clearActiveBatch() this.updating = false - this._mutex.unlock() + this.state.mutex.unlock() } return true @@ -778,32 +954,23 @@ module.exports = class Core { async _verifyShared () { if (!this._verifies.length) return false - await this._mutex.lock() + await this.state.mutex.lock() + + const update = this.state.createUpdate() const verifies = this._verifies this._verifies = null this._verified = null try { - const entries = [] - for (const { batch, bitfield, value } of verifies) { if (!batch.commitable()) continue if (bitfield) { - await this._writeBlock(batch, bitfield.start, value) + this._writeBlock(update.batch, bitfield.start, value) } - - entries.push({ - userData: null, - treeNodes: batch.nodes, - treeUpgrade: null, - bitfield - }) } - await this.oplog.append(entries, false) - for (let i = 0; i < verifies.length; i++) { const { batch, bitfield, value, manifest, from } = verifies[i] @@ -812,27 +979,24 @@ module.exports = class Core { continue } - let status = 0 - if (bitfield) { - this._setBitfield(bitfield.start, true) - status = updateContig(this.header, bitfield, this.bitfield) + update.bitfield.setRange(bitfield.start, bitfield.start + 1, true) } // if we got a manifest AND its strictly a non compat one, lets store it if (manifest && this.header.manifest === null) { if (!Verifier.isValidManifest(this.header.key, manifest)) throw INVALID_CHECKSUM('Manifest hash does not match') - this._setManifest(manifest, null) + this._setManifest(update, manifest, null) } - batch.commit() - - this.onupdate(status, bitfield, value, from) + update.coreUpdate({ status: 0, bitfield, value, from }) + update.flushTreeBatch(batch) } - if (this._shouldFlush()) await this._flushOplog() + await this.state.flushUpdate(update) } finally { - this._mutex.unlock() + this.state._clearActiveBatch() + this.state.mutex.unlock() } return verifies[0] !== null @@ -846,9 +1010,15 @@ module.exports = class Core { const batch = this.tree.verifyFullyRemote(proof) + await this.state.mutex.lock() + try { - this._verifyBatchUpgrade(batch, proof.manifest) + const update = this.state.createUpdate() + this._verifyBatchUpgrade(update, batch, proof.manifest) + + await this.state.flushUpdate(update) } catch { + this.state.mutex.unlock() return true } @@ -863,9 +1033,7 @@ module.exports = class Core { async verifyReorg (proof) { const batch = await this.tree.reorg(proof) - - this._verifyBatchUpgrade(batch, proof.manifest) - + this._verifyBatchUpgrade(null, batch, proof.manifest) return batch } @@ -883,11 +1051,14 @@ module.exports = class Core { batch, bitfield: value && { drop: false, start: proof.block.index, length: 1 }, value, + status: 0, manifest: proof.manifest, from } - if (batch.upgraded) return this._verifyExclusive(op) + if (batch.upgraded) { + return this._verifyExclusive(op) + } if (this._verifies !== null) { const verifies = this._verifies @@ -898,6 +1069,7 @@ module.exports = class Core { this._verifies = [op] this._verified = this._verifyShared() + return this._verified } @@ -905,55 +1077,24 @@ module.exports = class Core { if (!batch.commitable()) return false this.truncating++ - await this._mutex.lock() + await this.state.mutex.lock() try { if (!batch.commitable()) return false - await this._truncate(batch, from) + + const update = this.state.createUpdate() + await update.truncate(batch, from) + + await this.state.flushUpdate(update) } finally { + this.state._clearActiveBatch() this.truncating-- - this._mutex.unlock() + this.state.mutex.unlock() } return true } - async _truncate (batch, from) { - const entry = { - userData: null, - treeNodes: batch.nodes, - treeUpgrade: batch, - bitfield: { - drop: true, - start: batch.ancestors, - length: this.tree.length - batch.ancestors - } - } - - await this.oplog.append([entry], false) - - this._setBitfieldRange(batch.ancestors, this.tree.length - batch.ancestors, false) - addReorgHint(this.header.hints.reorgs, this.tree, batch) - batch.commit() - - const contigStatus = updateContig(this.header, entry.bitfield, this.bitfield) - const status = ((batch.length > batch.ancestors) ? 0b0011 : 0b0010) | contigStatus - - this.header.tree.fork = batch.fork - this.header.tree.length = batch.length - this.header.tree.rootHash = batch.hash() - this.header.tree.signature = batch.signature - - this.onupdate(status, entry.bitfield, null, from) - - // TODO: there is a bug in the merkle tree atm where it cannot handle unflushed - // truncates if we append or download anything after the truncation point later on - // This is because tree.get checks the truncated flag. We should fix this so we can do - // the later flush here as well - // if (this._shouldFlush()) await this._flushOplog() - await this._flushOplog() - } - openSkipBitfield () { if (this.skipBitfield !== null) return this.skipBitfield this.skipBitfield = new RemoteBitfield() @@ -963,37 +1104,25 @@ module.exports = class Core { return this.skipBitfield } - _setBitfield (index, value) { - this.bitfield.set(index, value) - if (this.skipBitfield !== null) this.skipBitfield.set(index, value) - } - - _setBitfieldRange (start, length, value) { - this.bitfield.setRange(start, length, value) - if (this.skipBitfield !== null) this.skipBitfield.setRange(start, length, value) + _updateSkipBitfield ({ ranges, drop }) { + for (const { start, end } of ranges) { + this.skipBitfield.setRange(start, end - start, drop === false) + } } async close () { this.closed = true - await this._mutex.destroy() - await Promise.allSettled([ - this.oplog.close(), - this.bitfield.close(), - this.tree.close(), - this.blocks.close(), - this.bigHeader.close() - ]) } } -function updateContig (header, upd, bitfield) { +function updateContigBatch (start, upd, bitfield) { const end = upd.start + upd.length - let c = header.hints.contiguousLength + let c = start if (upd.drop) { // If we dropped a block in the current contig range, "downgrade" it - if (c <= end && c > upd.start) { + if (c > upd.start) { c = upd.start } } else { @@ -1003,17 +1132,24 @@ function updateContig (header, upd, bitfield) { } } - if (c === header.hints.contiguousLength) { - return 0b0000 + if (c === start) { + return { + status: 0b0000, + length: null + } } - if (c > header.hints.contiguousLength) { - header.hints.contiguousLength = c - return 0b0100 + if (c > start) { + return { + status: 0b0100, + length: c + } } - header.hints.contiguousLength = c - return 0b1000 + return { + status: 0b1000, + length: c + } } function addReorgHint (list, tree, batch) { @@ -1028,52 +1164,21 @@ function addReorgHint (list, tree, batch) { list.push({ from: tree.fork, to: batch.fork, ancestors: batch.ancestors }) } -function updateUserData (list, key, value) { - value = unslab(value) - - for (let i = 0; i < list.length; i++) { - if (list[i].key === key) { - if (value) list[i].value = value - else list.splice(i, 1) - return +function parseHeader (info) { + if (!info) return null + + return { + key: info.auth.key, + manifest: info.auth.manifest ? Verifier.decodeManifest(info.auth.manifest) : null, + external: null, + keyPair: info.keyPair, + userData: [], + tree: info.head, + hints: { + reorgs: [], + contiguousLength: 0 } } - if (value) list.push({ key, value }) -} - -function closeAll (...storages) { - let missing = 1 - let error = null - - return new Promise((resolve, reject) => { - for (const s of storages) { - missing++ - s.close(done) - } - - done(null) - - function done (err) { - if (err) error = err - if (--missing) return - if (error) reject(error) - else resolve() - } - }) -} - -async function flushHeader (oplog, bigHeader, header) { - if (header.external) { - await bigHeader.flush(header) - } - - try { - await oplog.flush(header) - } catch (err) { - if (err.code !== 'OPLOG_HEADER_OVERFLOW') throw err - await bigHeader.flush(header) - await oplog.flush(header) - } } function noop () {} @@ -1104,3 +1209,28 @@ function minimumSegmentEnd (start, src, dst) { if (b === -1) return a return a < b ? a : b } + +function getCoreHead (storage) { + const b = storage.createReadBatch() + const p = b.getCoreHead() + b.tryFlush() + return p +} + +async function getCoreInfo (storage) { + const r = storage.createReadBatch() + + const auth = r.getCoreAuth() + const localKeyPair = r.getLocalKeyPair() + const encryptionKey = r.getEncryptionKey() + const head = r.getCoreHead() + + await r.flush() + + return { + auth: await auth, + keyPair: await localKeyPair, + encryptionKey: await encryptionKey, + head: await head + } +} diff --git a/lib/memory-overlay.js b/lib/memory-overlay.js new file mode 100644 index 00000000..2e35ec1d --- /dev/null +++ b/lib/memory-overlay.js @@ -0,0 +1,298 @@ +const b4a = require('b4a') +const { ASSERTION } = require('hypercore-errors') + +class MemoryOverlay { + constructor (storage) { + this.storage = storage + this.head = null + this.auth = null + this.localKeyPair = null + this.encryptionKey = null + this.dataInfo = null + this.userData = null + this.blocks = null + this.treeNodes = null + this.bitfields = null + + this.snapshotted = false + } + + async registerBatch (name, length, overwrite) { + todo() + } + + snapshot () { + todo() + } + + createReadBatch () { + return new MemoryOverlayReadBatch(this, this.storage.createReadBatch()) + } + + createWriteBatch () { + return new MemoryOverlayWriteBatch(this) + } + + createBlockStream () { + todo() + } + + createUserDataStream () { + todo() + } + + createTreeNodeStream () { + todo() + } + + createBitfieldPageStream () { + todo() + } + + peakLastTreeNode () { + todo() + } + + peakLastBitfieldPage () { + todo() + } + + close () { + return Promise.resolve() + } + + merge (overlay) { + if (overlay.head !== null) this.head = overlay.head + if (overlay.auth !== null) this.auth = overlay.auth + if (overlay.localKeyPair !== null) this.localKeyPair = overlay.localKeyPair + if (overlay.encryptionKey !== null) this.encryptionKey = overlay.encryptionKey + if (overlay.dataInfo !== null) this.dataInfo = overlay.dataInfo + if (overlay.userData !== null) this.userData = mergeMap(this.userData, overlay.userData) + if (overlay.blocks !== null) this.blocks = mergeTip(this.blocks, overlay.blocks) + if (overlay.treeNodes !== null) this.treeNodes = mergeMap(this.treeNodes, overlay.treeNodes) + if (overlay.bitfields !== null) this.bitfields = mergeTip(this.bitfields, overlay.bitfields) + } +} + +class TipList { + constructor () { + this.offset = 0 + this.data = [] + } + + end () { + return this.offset + this.data.length + } + + put (index, value) { + if (this.data.length === 0) { + this.offset = index + this.data.push(value) + return + } + + if (this.data.length === index) { + this.push.push(value) + return + } + + throw ASSERTION('Invalid put on tip list') + } + + get (index) { + index -= this.offset + if (index >= this.data.length) return null + return this.data[index] + } + + * [Symbol.iterator] () { + for (let i = 0; i < this.data.length; i++) { + yield [i + this.offset, this.data[i]] + } + } +} + +module.exports = MemoryOverlay + +class MemoryOverlayReadBatch { + constructor (overlay, read) { + this.read = read + this.overlay = overlay + } + + async getCoreHead () { + return this.overlay.head !== null ? this.overlay.head : this.read.getCoreHead() + } + + async getCoreAuth () { + return this.overlay.auth !== null ? this.overlay.auth : this.read.getCoreAuth() + } + + async getLocalKeyPair () { + return this.overlay.localKeyPair !== null ? this.overlay.localKeyPair : this.read.getLocalKeyPair() + } + + async getEncryptionKey () { + return this.overlay.encryptionKey !== null ? this.overlay.encryptionKey : this.read.getEncryptionKey() + } + + async getDataInfo () { + return this.overlay.dataInfo !== null ? this.overlay.dataInfo : this.read.getDataInfo() + } + + async getUserData (key) { + const hex = this.overlay.userData === null ? null : b4a.toString('hex', key) + return hex !== null && this.userData.has(hex) ? this.overlay.dataInfo.get(hex) : this.read.getUserData(key) + } + + async hasBlock (index) { + if (this.overlay.blocks !== null && index >= this.overlay.blocks.offset) { + const blk = this.overlay.blocks.get(index) + if (blk !== null) return true + } + return this.read.hasBlock(index) + } + + async getBlock (index, error) { + if (this.overlay.blocks !== null && index >= this.overlay.blocks.offset) { + const blk = this.overlay.blocks.get(index) + if (blk !== null) return blk + } + return this.read.getBlock(index, error) + } + + async hasTreeNode (index) { + return (this.overlay.treeNodes !== null && this.overlay.treeNodes.has(index)) || this.read.hasTreeNode(index) + } + + async getTreeNode (index, error) { + if (this.overlay.treeNodes !== null && this.overlay.treeNodes.has(index)) { + return this.overlay.treeNodes.get(index) + } + return this.read.getTreeNode(index, error) + } + + async getBitfieldPage (index) { + if (this.overlay.bitfields !== null && index >= this.overlay.bitfields.offset) { + const page = this.overlay.bitfields.get(index) + if (page !== null) return page + } + return this.read.getBitfieldPage(index) + } + + destroy () { + this.read.destroy() + } + + flush () { + return this.read.flush() + } + + tryFlush () { + this.read.tryFlush() + } +} + +class MemoryOverlayWriteBatch { + constructor (storage) { + this.storage = storage + this.overlay = new MemoryOverlay() + } + + setCoreHead (head) { + this.overlay.head = head + } + + setCoreAuth (auth) { + this.overlay.auth = auth + } + + setBatchPointer (name, pointer) { + todo() + } + + setDataDependency (dataInfo) { + todo() + } + + setLocalKeyPair (keyPair) { + this.overlay.localKeyPair = keyPair + } + + setEncryptionKey (encryptionKey) { + this.overlay.encryptionKey = encryptionKey + } + + setDataInfo (info) { + this.overlay.dataInfo = info + } + + setUserData (key, value) { + if (this.overlay.userData === null) this.overlay.userData = new Map() + this.overlay.userData.set(b4a.toString(key, 'hex'), value) + } + + putBlock (index, data) { + if (this.overlay.blocks === null) this.overlay.blocks = new TipList() + this.overlay.blocks.put(index, data) + } + + deleteBlock (index) { + todo() + } + + deleteBlockRange (start, end) { + todo() + } + + putTreeNode (node) { + if (this.overlay.treeNodes === null) this.overlay.treeNodes = new Map() + this.overlay.treeNodes.set(node.index, node) + } + + deleteTreeNode (index) { + todo() + } + + deleteTreeNodeRange (start, end) { + todo() + } + + putBitfieldPage (index, page) { + if (this.overlay.bitfields === null) this.overlay.bitfields = new TipList() + this.overlay.bitfields.put(index, page) + } + + deleteBitfieldPage (index) { + todo() + } + + deleteBitfieldPageRange (start, end) { + todo() + } + + destroy () {} + + flush () { + this.storage.merge(this.overlay) + return Promise.resolve() + } +} + +function mergeMap (a, b) { + if (a === null) return b + for (const [key, value] of b) a.set(key, value) + return a +} + +function mergeTip (a, b) { + if (a === null) return b + while (a.end() !== b.offset && b.offset >= a.offset && b.end() >= a.end()) a.data.pop() + if (a.end() !== b.offset) throw ASSERTION('Cannot merge tip list') + for (const data of b.data) a.data.push(data) + return a +} + +function todo () { + throw ASSERTION('Not supported yet, but will be') +} diff --git a/lib/merkle-tree.js b/lib/merkle-tree.js index 89829b2d..3bdb4c1f 100644 --- a/lib/merkle-tree.js +++ b/lib/merkle-tree.js @@ -1,16 +1,11 @@ const flat = require('flat-tree') const crypto = require('hypercore-crypto') -const c = require('compact-encoding') -const Xache = require('xache') +const assert = require('nanoassert') const b4a = require('b4a') const unslab = require('unslab') const caps = require('./caps') const { INVALID_PROOF, INVALID_CHECKSUM, INVALID_OPERATION, BAD_ARGUMENT, ASSERTION } = require('hypercore-errors') -const BLANK_HASH = b4a.alloc(32) -const OLD_TREE = b4a.from([5, 2, 87, 2, 0, 0, 40, 7, 66, 76, 65, 75, 69, 50, 98]) -const TREE_CACHE = 128 // speeds up linear scans by A LOT - class NodeQueue { constructor (nodes, extra = null) { this.i = 0 @@ -51,6 +46,8 @@ class MerkleTreeBatch { this.signature = null this.hashCached = null + this.committed = false + this.truncated = false this.treeLength = tree.length this.treeFork = tree.fork this.tree = tree @@ -162,8 +159,8 @@ class MerkleTreeBatch { return this.tree.get(index, error) } - proof ({ block, hash, seek, upgrade }) { - return generateProof(this, block, hash, seek, upgrade) + proof (batch, { block, hash, seek, upgrade }) { + return generateProof(batch, this, block, hash, seek, upgrade) } verifyUpgrade (proof) { @@ -174,6 +171,12 @@ class MerkleTreeBatch { return verifyUpgrade(proof, unverified, this) } + addNodesUnsafe (nodes) { + for (let i = 0; i < nodes.length; i++) { + this.nodes.push(nodes[i]) + } + } + append (buf) { const head = this.length * 2 const ite = flat.iterator(head) @@ -217,44 +220,52 @@ class MerkleTreeBatch { ) } - commit () { + commit (writer) { + if (writer === undefined) throw INVALID_OPERATION('No database batch was passed') if (!this.commitable()) throw INVALID_OPERATION('Tree was modified during batch, refusing to commit') - if (this.upgraded) this._commitUpgrade() + if (this.upgraded) this._commitUpgrade(writer) for (let i = 0; i < this.nodes.length; i++) { const node = this.nodes[i] - this.tree.unflushed.set(node.index, node) + writer.putTreeNode(node) } + + this.committed = true + + return this } - _commitUpgrade () { + _commitUpgrade (writer) { // TODO: If easy to detect, we should refuse an trunc+append here without a fork id // change. Will only happen on user error so mostly to prevent that. if (this.ancestors < this.treeLength) { + writer.deleteTreeNodeRange(this.ancestors * 2, this.treeLength * 2) + if (this.ancestors > 0) { - const head = 2 * this.ancestors + const head = this.ancestors * 2 const ite = flat.iterator(head - 2) while (true) { if (ite.contains(head) && ite.index < head) { - this.tree.unflushed.set(ite.index, blankNode(ite.index)) + writer.deleteTreeNode(ite.index) } if (ite.offset === 0) break ite.parent() } + + this.truncated = true } + } + } - this.tree.truncateTo = this.tree.truncated - ? Math.min(this.tree.truncateTo, this.ancestors) - : this.ancestors + finalise () { + assert(this.committed, 'Batch has not been committed') - this.tree.truncated = true - this.tree.cache = new Xache({ maxSize: this.tree.cache.maxSize }) - truncateMap(this.tree.unflushed, this.ancestors) - if (this.tree.flushing !== null) truncateMap(this.tree.flushing, this.ancestors) - } + if (!this.upgraded) return + + if (this.truncated) this.tree.truncated = true this.tree.roots = this.roots this.tree.length = this.length @@ -267,13 +278,13 @@ class MerkleTreeBatch { return new ByteSeeker(this, bytes, padding) } - byteRange (index) { - return getByteRange(this, index) + byteRange (index, readBatch = null) { + return getByteRange(this, index, readBatch) } - byteOffset (index) { + byteOffset (index, readBatch = null) { if (index === 2 * this.tree.length) return this.tree.byteLength - return getByteOffset(this, index) + return getByteOffset(this, index, readBatch) } } @@ -423,8 +434,68 @@ class ByteSeeker { } } +class TreeProof { + constructor (tree, block, hash, seek, upgrade) { + this.fork = tree.fork + this.signature = tree.signature + + this.block = block + this.hash = hash + this.seek = seek + this.upgrade = upgrade + + this.pending = { + node: null, + seek: null, + upgrade: null, + additionalUpgrade: null + } + } + + async settle () { + const result = { fork: this.fork, block: null, hash: null, seek: null, upgrade: null, manifest: null } + + const [pNode, pSeek, pUpgrade, pAdditional] = await settleProof(this.pending) + + if (this.block) { + if (pNode === null) throw INVALID_OPERATION('Invalid block request') + result.block = { + index: this.block.index, + value: null, // populated upstream, alloc it here for simplicity + nodes: pNode + } + } else if (this.hash) { + if (pNode === null) throw INVALID_OPERATION('Invalid block request') + result.hash = { + index: this.hash.index, + nodes: pNode + } + } + + if (this.seek && pSeek !== null) { + result.seek = { + bytes: this.seek.bytes, + nodes: pSeek + } + } + + if (this.upgrade) { + result.upgrade = { + start: this.upgrade.start, + length: this.upgrade.length, + nodes: pUpgrade, + additionalNodes: pAdditional || [], + signature: this.signature + } + } + + return result + } +} + module.exports = class MerkleTree { constructor (storage, roots, fork, signature, prologue) { + this.storage = storage this.crypto = crypto this.fork = fork this.roots = roots @@ -432,18 +503,14 @@ module.exports = class MerkleTree { this.byteLength = totalSize(roots) this.signature = signature this.prologue = prologue + } - this.storage = storage - this.unflushed = new Map() - this.cache = new Xache({ maxSize: TREE_CACHE }) - this.flushing = null - this.truncated = false - this.truncateTo = 0 + onupdate (u) { + return u.finalise() } - addNode (node) { - if (node.size === 0 && b4a.equals(node.hash, BLANK_HASH)) node = blankNode(node.index) - this.unflushed.set(node.index, node) + clone (storage) { + return new MerkleTree(storage, this.roots.slice(0), this.fork, this.signature, this.prologue) } batch () { @@ -466,6 +533,85 @@ module.exports = class MerkleTree { return batch } + async reconcile (tree, length, treeLength) { + const nodes = [] + const data = [] + + const from = this.length * 2 + const to = length * 2 + + const reader = tree.storage.createReadBatch() + + // upgrade + for (const ite = flat.iterator(0); ite.fullRoot(to); ite.nextTree()) { + // check if they already have the node + if (ite.index + ite.factor / 2 < from) continue + + if (nodes.length === 0 && ite.contains(from - 2)) { + const root = ite.index + const target = from - 2 + + ite.seek(target) + + while (ite.index !== root) { + ite.sibling() + if (ite.index > target) { + nodes.push(reader.getTreeNode(ite.index)) + } + ite.parent() + } + + continue + } + + nodes.push(reader.getTreeNode(ite.index)) + } + + for (let i = treeLength * 2; i < length * 2; i++) { + data.push(reader.getTreeNode(i)) + } + + reader.tryFlush() + + const batch = this.batch() + + // copy tree nodes + for (const node of await Promise.all(data)) { + if (node) batch.nodes.push(node) + } + + // no nodes to add + if (!nodes.length) return batch + + const q = new NodeQueue(await Promise.all(nodes), null) + + let grow = batch.roots.length > 0 + let i = 0 + + for (const ite = flat.iterator(0); ite.fullRoot(to); ite.nextTree()) { + if (i < batch.roots.length && batch.roots[i].index === ite.index) { + i++ + continue + } + + if (grow) { + grow = false + const root = ite.index + if (i < batch.roots.length) { + ite.seek(batch.roots[batch.roots.length - 1].index) + while (ite.index !== root) { + batch.appendRoot(q.shift(ite.sibling()), ite) + } + continue + } + } + + batch.appendRoot(q.shift(ite.index), ite) + } + + return batch + } + seek (bytes, padding) { return new ByteSeeker(this, bytes, padding) } @@ -481,11 +627,14 @@ module.exports = class MerkleTree { getRoots (length) { const indexes = flat.fullRoots(2 * length) const roots = new Array(indexes.length) + const readBatch = this.storage.createReadBatch() for (let i = 0; i < indexes.length; i++) { - roots[i] = this.get(indexes[i], true) + roots[i] = readBatch.getTreeNode(indexes[i], true) } + readBatch.tryFlush() + return Promise.all(roots) } @@ -493,13 +642,6 @@ module.exports = class MerkleTree { this.prologue = { hash, length } } - addNodes (nodes) { - for (let i = 0; i < nodes.length; i++) { - const node = nodes[i] - this.unflushed.set(node.index, node) - } - } - getNeededNodes (length, start, end) { const nodes = new Map() const head = length * 2 @@ -527,11 +669,14 @@ module.exports = class MerkleTree { async upgradeable (length) { const indexes = flat.fullRoots(2 * length) const roots = new Array(indexes.length) + const readBatch = this.storage.createReadBatch() for (let i = 0; i < indexes.length; i++) { - roots[i] = this.get(indexes[i], false) + roots[i] = readBatch.getTreeNode(indexes[i], false) } + readBatch.tryFlush() + for (const node of await Promise.all(roots)) { if (node === null) return false } @@ -539,107 +684,13 @@ module.exports = class MerkleTree { return true } - blankNode (index) { - return blankNode(index) - } - - get (index, error = true) { - const c = this.cache.get(index) - if (c) return c - - let node = this.unflushed.get(index) + get (index, error = true, readBatch = null) { + if (readBatch) return readBatch.getTreeNode(index, error) - if (this.flushing !== null && node === undefined) { - node = this.flushing.get(index) - } - - // TODO: test this - if (this.truncated && node !== undefined && node.index >= 2 * this.truncateTo) { - node = blankNode(index) - } - - if (node !== undefined) { - if (node.hash === BLANK_HASH) { - if (error) throw INVALID_OPERATION('Could not load node: ' + index) - return Promise.resolve(null) - } - return Promise.resolve(node) - } - - return getStoredNode(this.storage, index, this.cache, error) - } - - async flush () { - this.flushing = this.unflushed - this.unflushed = new Map() - - try { - if (this.truncated) await this._flushTruncation() - await this._flushNodes() - } catch (err) { - for (const node of this.flushing.values()) { - if (!this.unflushed.has(node.index)) this.unflushed.set(node.index, node) - } - throw err - } finally { - this.flushing = null - } - } - - _flushTruncation () { - return new Promise((resolve, reject) => { - const t = this.truncateTo - const offset = t === 0 ? 0 : (t - 1) * 80 + 40 - - this.storage.truncate(offset, (err) => { - if (err) return reject(err) - - if (this.truncateTo === t) { - this.truncateTo = 0 - this.truncated = false - } - - resolve() - }) - }) - } - - _flushNodes () { - // TODO: write neighbors together etc etc - // TODO: bench loading a full disk page and copy to that instead - return new Promise((resolve, reject) => { - const slab = b4a.allocUnsafe(40 * this.flushing.size) - - let error = null - let missing = this.flushing.size + 1 - let offset = 0 - - for (const node of this.flushing.values()) { - const state = { - start: 0, - end: 40, - buffer: slab.subarray(offset, offset += 40) - } - - c.uint64.encode(state, node.size) - c.raw.encode(state, node.hash) - - this.storage.write(node.index * 40, state.buffer, done) - } - - done(null) - - function done (err) { - if (err) error = err - if (--missing > 0) return - if (error) reject(error) - else resolve() - } - }) + return getTreeNode(this.storage, index, error) } - clear () { - this.cache = new Xache({ maxSize: this.cache.maxSize }) + clear (writer) { this.truncated = true this.truncateTo = 0 this.roots = [] @@ -647,18 +698,8 @@ module.exports = class MerkleTree { this.byteLength = 0 this.fork = 0 this.signature = null - if (this.flushing !== null) this.flushing.clear() - this.unflushed.clear() - return this.flush() - } - close () { - return new Promise((resolve, reject) => { - this.storage.close(err => { - if (err) reject(err) - else resolve() - }) - }) + return writer.deleteTreeNodeRange(0, -1) } async truncate (length, fork = this.fork) { @@ -759,8 +800,8 @@ module.exports = class MerkleTree { return batch } - proof ({ block, hash, seek, upgrade }) { - return generateProof(this, block, hash, seek, upgrade) + proof (batch, { block, hash, seek, upgrade }) { + return generateProof(batch, this, block, hash, seek, upgrade) } // Successor to .nodes() @@ -774,7 +815,8 @@ module.exports = class MerkleTree { if (iteRightSpan >= head) return 0 let cnt = 0 - while (!ite.contains(head) && (await this.get(ite.index, false)) === null) { + // TODO: we could prop use a read batch here and do this in blocks of X for perf + while (!ite.contains(head) && !(await hasTreeNode(this.storage, ite.index))) { cnt++ ite.parent() } @@ -797,44 +839,30 @@ module.exports = class MerkleTree { } byteRange (index) { - return getByteRange(this, index) + return getByteRange(this, index, null) } byteOffset (index) { - return getByteOffset(this, index) + return getByteOffset(this, index, null) } static async open (storage, opts = {}) { - await new Promise((resolve, reject) => { - storage.read(0, OLD_TREE.length, (err, buf) => { - if (err) return resolve() - if (b4a.equals(buf, OLD_TREE)) return reject(new Error('Storage contains an incompatible merkle tree')) - resolve() - }) - }) - - const length = typeof opts.length === 'number' - ? opts.length - : await autoLength(storage) + const length = typeof opts.length === 'number' ? opts.length : await autoLength(storage) const roots = [] for (const index of flat.fullRoots(2 * length)) { - roots.push(unslabNode(await getStoredNode(storage, index, null, true))) + roots.push(unslabNode(await getTreeNode(storage, index, true))) } return new MerkleTree(storage, roots, opts.fork || 0, opts.signature || null, opts.prologue || null) } } -async function getByteRange (tree, index) { - const head = 2 * tree.length - if (((index & 1) === 0 ? index : flat.rightSpan(index)) >= head) { - throw BAD_ARGUMENT('Index is out of bounds') - } - return [await tree.byteOffset(index), (await tree.get(index)).size] +async function getNodeSize (index, readBatch) { + return (await readBatch.getTreeNode(index, true)).size } -async function getByteOffset (tree, index) { +async function getByteOffset (tree, index, readBatch) { if (index === 2 * tree.length) return tree.byteLength if ((index & 1) === 1) index = flat.leftSpan(index) @@ -851,21 +879,45 @@ async function getByteOffset (tree, index) { const ite = flat.iterator(node.index) + if (readBatch === null) readBatch = tree.storage.createReadBatch() + const promises = [] + while (ite.index !== index) { if (index < ite.index) { ite.leftChild() } else { - offset += (await tree.get(ite.leftChild())).size + promises.push(tree.get(ite.leftChild(), true, readBatch)) ite.sibling() } } + readBatch.tryFlush() + for (const node of await Promise.all(promises)) offset += node.size + return offset } throw ASSERTION('Failed to find offset') } +function getByteRange (tree, index, readBatch) { + const head = 2 * tree.length + if (((index & 1) === 0 ? index : flat.rightSpan(index)) >= head) { + throw BAD_ARGUMENT('Index is out of bounds') + } + + if (readBatch === null) readBatch = tree.storage.createReadBatch() + + const offset = tree.byteOffset(index, readBatch) + const size = getNodeSize(index, readBatch) + + const result = Promise.all([offset, size]) + + readBatch.tryFlush() + + return result +} + // All the methods needed for proof verification function verifyTree ({ block, hash, seek }, crypto, nodes) { @@ -993,7 +1045,7 @@ async function seekFromHead (tree, head, bytes, padding) { for (let i = 0; i < roots.length; i++) { const root = roots[i] - const node = await tree.get(root) + const node = await tree.get(root, true) const size = getUnpaddedSize(node, padding, null) if (bytes === size) return root @@ -1017,6 +1069,7 @@ async function seekTrustedTree (tree, root, bytes, padding) { while ((ite.index & 1) !== 0) { const l = await tree.get(ite.leftChild(), false) + if (l) { const size = getUnpaddedSize(l, padding, ite) if (size === bytes) return ite.index @@ -1042,7 +1095,7 @@ async function seekUntrustedTree (tree, root, bytes, padding) { bytes -= offset - const node = await tree.get(root) + const node = await tree.get(root, true) if (getUnpaddedSize(node, padding, null) <= bytes) throw INVALID_OPERATION('Invalid seek') @@ -1053,41 +1106,41 @@ async function seekUntrustedTree (tree, root, bytes, padding) { // Note, that all these methods are sync as we can statically infer which nodes // are needed for the remote to verify given they arguments they passed us -function seekProof (tree, seekRoot, root, p) { +function seekProof (tree, batch, seekRoot, root, p) { const ite = flat.iterator(seekRoot) p.seek = [] - p.seek.push(tree.get(ite.index)) + p.seek.push(tree.get(ite.index, true, batch)) while (ite.index !== root) { ite.sibling() - p.seek.push(tree.get(ite.index)) + p.seek.push(tree.get(ite.index, true, batch)) ite.parent() } } -function blockAndSeekProof (tree, node, seek, seekRoot, root, p) { - if (!node) return seekProof(tree, seekRoot, root, p) +function blockAndSeekProof (tree, batch, node, seek, seekRoot, root, p) { + if (!node) return seekProof(tree, batch, seekRoot, root, p) const ite = flat.iterator(node.index) p.node = [] - if (!node.value) p.node.push(tree.get(ite.index)) + if (!node.value) p.node.push(tree.get(ite.index, true, batch)) while (ite.index !== root) { ite.sibling() if (seek && ite.contains(seekRoot) && ite.index !== seekRoot) { - seekProof(tree, seekRoot, ite.index, p) + seekProof(tree, batch, seekRoot, ite.index, p) } else { - p.node.push(tree.get(ite.index)) + p.node.push(tree.get(ite.index, true, batch)) } ite.parent() } } -function upgradeProof (tree, node, seek, from, to, subTree, p) { +function upgradeProof (tree, batch, node, seek, from, to, subTree, p) { if (from === 0) p.upgrade = [] for (const ite = flat.iterator(0); ite.fullRoot(to); ite.nextTree()) { @@ -1107,9 +1160,9 @@ function upgradeProof (tree, node, seek, from, to, subTree, p) { ite.sibling() if (ite.index > target) { if (p.node === null && p.seek === null && ite.contains(subTree)) { - blockAndSeekProof(tree, node, seek, subTree, ite.index, p) + blockAndSeekProof(tree, batch, node, seek, subTree, ite.index, p) } else { - p.upgrade.push(tree.get(ite.index)) + p.upgrade.push(batch.getTreeNode(ite.index, true)) } } ite.parent() @@ -1125,16 +1178,16 @@ function upgradeProof (tree, node, seek, from, to, subTree, p) { // if the subtree included is a child of this tree, include that one // instead of a dup node if (p.node === null && p.seek === null && ite.contains(subTree)) { - blockAndSeekProof(tree, node, seek, subTree, ite.index, p) + blockAndSeekProof(tree, batch, node, seek, subTree, ite.index, p) continue } // add root (can be optimised since the root might be in tree.roots) - p.upgrade.push(tree.get(ite.index)) + p.upgrade.push(tree.get(ite.index, true, batch)) } } -function additionalUpgradeProof (tree, from, to, p) { +function additionalUpgradeProof (tree, batch, from, to, p) { if (from === 0) p.additionalUpgrade = [] for (const ite = flat.iterator(0); ite.fullRoot(to); ite.nextTree()) { @@ -1153,7 +1206,7 @@ function additionalUpgradeProof (tree, from, to, p) { while (ite.index !== root) { ite.sibling() if (ite.index > target) { - p.additionalUpgrade.push(tree.get(ite.index)) + p.additionalUpgrade.push(tree.get(ite.index, true, batch)) } ite.parent() } @@ -1166,7 +1219,7 @@ function additionalUpgradeProof (tree, from, to, p) { } // add root (can be optimised since the root is in tree.roots) - p.additionalUpgrade.push(tree.get(ite.index)) + p.additionalUpgrade.push(tree.get(ite.index, true, batch)) } } @@ -1201,65 +1254,11 @@ function parentNode (crypto, index, a, b) { return { index, size: a.size + b.size, hash: crypto.parent(a, b) } } -function blankNode (index) { - return { index, size: 0, hash: BLANK_HASH } -} - -// Storage methods - -function getStoredNode (storage, index, cache, error) { - return new Promise((resolve, reject) => { - storage.read(40 * index, 40, (err, data) => { - if (err) { - if (error) return reject(err) - else resolve(null) - return - } - - const hash = data.subarray(8) - const size = c.decode(c.uint64, data) - - if (size === 0 && b4a.compare(hash, BLANK_HASH) === 0) { - if (error) reject(new Error('Could not load node: ' + index)) - else resolve(null) - return - } - - const node = { index, size, hash } - - if (cache !== null) { - // Copy hash to a new buffer to avoid blocking gc of its original slab - node.hash = unslab(hash) - cache.set(index, node) - } - - resolve(node) - }) - }) -} - -function storedNodes (storage) { - return new Promise((resolve) => { - storage.stat((_, st) => { - if (!st) return resolve(0) - resolve((st.size - (st.size % 40)) / 40) - }) - }) -} - async function autoLength (storage) { - const nodes = await storedNodes(storage) - if (!nodes) return 0 - const ite = flat.iterator(nodes - 1) - let index = nodes - 1 - while (await getStoredNode(storage, ite.parent(), null, false)) index = ite.index - return flat.rightSpan(index) / 2 + 1 -} + const node = await storage.peakLastTreeNode() + if (!node) return 0 -function truncateMap (map, len) { - for (const node of map.values()) { - if (node.index >= 2 * len) map.delete(node.index) - } + return flat.rightSpan(node.index) / 2 + 1 } function log2 (n) { @@ -1279,6 +1278,20 @@ function normalizeIndexed (block, hash) { return null } +function getTreeNode (storage, index, error) { + const batch = storage.createReadBatch() + const node = batch.getTreeNode(index, error) + batch.tryFlush() + return node +} + +function hasTreeNode (storage, index) { + const batch = storage.createReadBatch() + const has = batch.hasTreeNode(index) + batch.tryFlush() + return has +} + async function settleProof (p) { const result = [ p.node && Promise.all(p.node), @@ -1299,7 +1312,7 @@ async function settleProof (p) { } // tree can be either the merkle tree or a merkle tree batch -async function generateProof (tree, block, hash, seek, upgrade) { +async function generateProof (readBatch, tree, block, hash, seek, upgrade) { // Important that this does not throw inbetween making the promise arrays // and finalise being called, otherwise there will be lingering promises in the background @@ -1308,17 +1321,13 @@ async function generateProof (tree, block, hash, seek, upgrade) { upgrade.length = upgrade.start < tree.prologue.length ? tree.prologue.length : upgrade.length } - const fork = tree.fork - const signature = tree.signature const head = 2 * tree.length const from = upgrade ? upgrade.start * 2 : 0 const to = upgrade ? from + upgrade.length * 2 : head const node = normalizeIndexed(block, hash) - const result = { fork, block: null, hash: null, seek: null, upgrade: null, manifest: null } - // can't do anything as we have no data... - if (head === 0) return result + if (head === 0) return new TreeProof(tree, null, null, null, null) if (from >= to || to > head) { throw INVALID_OPERATION('Invalid upgrade') @@ -1329,61 +1338,22 @@ async function generateProof (tree, block, hash, seek, upgrade) { let subTree = head - const p = { - node: null, - seek: null, - upgrade: null, - additionalUpgrade: null - } + const p = new TreeProof(tree, block, hash, seek, upgrade) if (node !== null && (!upgrade || node.lastIndex < upgrade.start)) { subTree = nodesToRoot(node.index, node.nodes, to) const seekRoot = seek ? await seekUntrustedTree(tree, subTree, seek.bytes, seek.padding) : head - blockAndSeekProof(tree, node, seek, seekRoot, subTree, p) + blockAndSeekProof(tree, readBatch, node, seek, seekRoot, subTree, p.pending) } else if ((node || seek) && upgrade) { subTree = seek ? await seekFromHead(tree, to, seek.bytes, seek.padding) : node.index } if (upgrade) { - upgradeProof(tree, node, seek, from, to, subTree, p) - if (head > to) additionalUpgradeProof(tree, to, head, p) + upgradeProof(tree, readBatch, node, seek, from, to, subTree, p.pending) + if (head > to) additionalUpgradeProof(tree, readBatch, to, head, p.pending) } - const [pNode, pSeek, pUpgrade, pAdditional] = await settleProof(p) - - if (block) { - if (pNode === null) throw INVALID_OPERATION('Invalid block request') - result.block = { - index: block.index, - value: null, // populated upstream, alloc it here for simplicity - nodes: pNode - } - } else if (hash) { - if (pNode === null) throw INVALID_OPERATION('Invalid hash request') - result.hash = { - index: hash.index, - nodes: pNode - } - } - - if (seek && pSeek !== null) { - result.seek = { - bytes: seek.bytes, - nodes: pSeek - } - } - - if (upgrade) { - result.upgrade = { - start: upgrade.start, - length: upgrade.length, - nodes: pUpgrade, - additionalNodes: pAdditional || [], - signature - } - } - - return result + return p } function getUnpaddedSize (node, padding, ite) { diff --git a/lib/multisig.js b/lib/multisig.js index db9f2e99..eff8187d 100644 --- a/lib/multisig.js +++ b/lib/multisig.js @@ -35,8 +35,10 @@ async function partialSignature (tree, signer, from, to = tree.length, signature } async function upgradeNodes (tree, from, to) { - const p = await tree.proof({ upgrade: { start: from, length: to - from } }) - return p.upgrade.nodes + const batch = tree.storage.createReadBatch() + const p = await tree.proof(batch, { upgrade: { start: from, length: to - from } }) + batch.tryFlush() + return (await p.settle()).upgrade.nodes } function signableLength (lengths, quorum) { diff --git a/lib/replicator.js b/lib/replicator.js index 510ab2da..1201f4ba 100644 --- a/lib/replicator.js +++ b/lib/replicator.js @@ -48,6 +48,7 @@ const PRIORITY = { class Attachable { constructor () { this.resolved = false + this.processing = false this.refs = [] } @@ -98,7 +99,12 @@ class Attachable { } gc () { - if (this.refs.length === 0) this._unref() + if (this.refs.length === 0 && !this.processing) this._unref() + } + + processed () { + this.processing = false + this.gc() } _cancel (r, err) { @@ -328,6 +334,26 @@ class RoundtripQueue { } } +class ProofRequest { + constructor (msg, proof, block, manifest) { + this.msg = msg + this.proof = proof + this.block = block + this.manifest = manifest + } + + async fulfill () { + if (this.proof === null) return null + + const proof = await this.proof.settle() + + if (this.manifest) proof.manifest = this.manifest + if (this.block) proof.block.value = await this.block + + return proof + } +} + class Peer { constructor (replicator, protomux, channel, useSession, inflightRange) { this.core = replicator.core @@ -657,24 +683,23 @@ class Peer { } } - async _getProof (msg) { - const proof = await this.core.tree.proof(msg) + async _getProof (batch, msg) { + let block = null - if (proof.block) { + if (msg.block) { const index = msg.block.index if (msg.fork !== this.core.tree.fork || !this.core.bitfield.get(index)) { - return null + return new ProofRequest(msg, null, null, null) } - proof.block.value = await this.core.blocks.get(index) + block = this.core.blocks.get(batch, index) } - if (msg.manifest && !this.core.compat) { - proof.manifest = this.core.header.manifest - } + const manifest = (msg.manifest && !this.core.compat) ? this.core.header.manifest : null + const proof = await this.core.tree.proof(batch, msg) - return proof + return new ProofRequest(msg, proof, block, manifest) } async onrequest (msg) { @@ -692,6 +717,8 @@ class Peer { return } + if (this.replicator.destroyed) return + await this._handleRequest(msg) } @@ -709,7 +736,7 @@ class Peer { } async _handleRequests () { - if (this.receiverBusy) return + if (this.receiverBusy || this.replicator.destroyed) return this.receiverBusy = true this.protomux.cork() @@ -723,35 +750,38 @@ class Peer { } async _handleRequest (msg) { - let proof = null + const batch = this.core.storage.createReadBatch() // TODO: could still be answerable if (index, fork) is an ancestor of the current fork - if (msg.fork === this.core.tree.fork) { - try { - proof = await this._getProof(msg) - } catch (err) { - safetyCatch(err) - if (msg.fork === this.core.tree.fork && isCriticalError(err)) throw err - } - } + const req = msg.fork === this.core.tree.fork + ? await this._getProof(batch, msg) + : new ProofRequest(msg, null, null, null) + + batch.tryFlush() + + await this._fulfillRequest(req) + } + + async _fulfillRequest (req) { + const proof = await req.fulfill() // if cancelled do not reply - if (this.remoteRequests.get(msg.id) !== msg) { + if (this.remoteRequests.get(req.msg.id) !== req.msg) { return } // sync from now on, so safe to delete from the map - this.remoteRequests.delete(msg.id) + this.remoteRequests.delete(req.msg.id) if (proof === null) { - if (msg.manifest && this.core.header.manifest) { + if (req.msg.manifest && this.core.header.manifest) { const manifest = this.core.header.manifest - this.wireData.send({ request: msg.id, fork: this.core.tree.fork, block: null, hash: null, seek: null, upgrade: null, manifest }) + this.wireData.send({ request: req.msg.id, fork: this.core.tree.fork, block: null, hash: null, seek: null, upgrade: null, manifest }) incrementTx(this.stats.wireData, this.replicator.stats.wireData) return } - this.wireNoData.send({ request: msg.id }) + this.wireNoData.send({ request: req.msg.id }) return } @@ -760,8 +790,8 @@ class Peer { } this.wireData.send({ - request: msg.id, - fork: msg.fork, + request: req.msg.id, + fork: req.msg.fork, block: proof.block, hash: proof.hash, seek: proof.seek, @@ -841,6 +871,7 @@ class Peer { } this.dataProcessing++ + if (isBlockRequest(req)) this.replicator._markProcessing(req.block.index) try { if (!matchingRequest(req, data) || !(await this.core.verify(data, this))) { @@ -869,6 +900,7 @@ class Peer { this.replicator.oninvalid(err, req, data, this) return } finally { + if (isBlockRequest(req)) this.replicator._markProcessed(req.block.index) this.dataProcessing-- } @@ -1543,7 +1575,9 @@ module.exports = class Replicator { for (const peer of this.peers) peer.signalUpgrade() if (this._blocks.isEmpty() === false) this._resolveBlocksLocally() if (this._upgrade !== null) this._resolveUpgradeRequest(null) - if (this._ranges.length !== 0 || this._seeks.length !== 0) this._updateNonPrimary(true) + if (!this._blocks.isEmpty() || this._ranges.length !== 0 || this._seeks.length !== 0) { + this._updateNonPrimary(true) + } } // Called externally when a conflict has been detected and verified @@ -1796,22 +1830,19 @@ module.exports = class Replicator { async _resolveBlocksLocally () { // TODO: check if fork compat etc. Requires that we pass down truncation info - let clear = null + const clear = [] + const blocks = [] + const reader = this.core.storage.createReadBatch() for (const b of this._blocks) { if (this.core.bitfield.get(b.index) === false) continue - - try { - b.resolve(await this.core.blocks.get(b.index)) - } catch (err) { - b.reject(err) - } - - if (clear === null) clear = [] - clear.push(b) + blocks.push(this._resolveLocalBlock(b, reader, clear)) } + reader.tryFlush() + + await Promise.all(blocks) - if (clear === null) return + if (!clear.length) return // Currently the block tracker does not support deletes during iteration, so we make // sure to clear them afterwards. @@ -1820,6 +1851,17 @@ module.exports = class Replicator { } } + async _resolveLocalBlock (b, reader, resolved) { + try { + b.resolve(await this.core.blocks.get(reader, b.index)) + } catch (err) { + b.reject(err) + return + } + + resolved.push(b) + } + _resolveBlockRequest (tracker, index, value, req) { const b = tracker.remove(index) if (b === null) return false @@ -1997,6 +2039,25 @@ module.exports = class Replicator { } } + _markProcessing (index) { + const b = this._blocks.get(index) + if (b) { + b.processing = true + return + } + + const h = this._hashes.get(index) + if (h) h.processing = true + } + + _markProcessed (index) { + const b = this._blocks.get(index) + if (b) return b.processed() + + const h = this._hashes.get(index) + if (h) h.processed() + } + _markInflight (index) { if (this.core.skipBitfield !== null) this.core.skipBitfield.set(index, true) for (const peer of this.peers) peer._markInflight(index) @@ -2249,20 +2310,28 @@ module.exports = class Replicator { } _closeSessionMaybe () { + this._closeSession().catch(safetyCatch) + } + + async _closeSession () { if (this._hasSession && this._peerSessions === 0) { this._hasSession = false - this.core.active-- } + if (this._hasSession) return + // we were the last active ref, so lets shut things down - if (this.core.active === 0 && this.core.sessions.length === 0) { + if (this.core.state.active === 1 && this.core.sessions.length === 0) { this.destroy() + await this.core.state.unref() // close state after replicator this.core.close().catch(safetyCatch) return } + await this.core.state.unref() + // in case one session is still alive but its been marked for auto close also kill it - if (this.core.sessions.length === 1 && this.core.active === 1 && this.core.sessions[0].autoClose) { + if (this.core.sessions.length === 1 && this.core.state.active === 1 && this.core.sessions[0].autoClose) { this.core.sessions[0].close().catch(safetyCatch) } } @@ -2274,7 +2343,7 @@ module.exports = class Replicator { ensureSession () { if (this._hasSession) return this._hasSession = true - this.core.active++ + this.core.state.ref() } attachTo (protomux, useSession) { diff --git a/lib/verifier.js b/lib/verifier.js index b3ccefb4..4c6c484a 100644 --- a/lib/verifier.js +++ b/lib/verifier.js @@ -153,7 +153,7 @@ module.exports = class Verifier { } } - throw new BAD_ARGUMENT('Public key is not a declared signer') + throw BAD_ARGUMENT('Public key is not a declared signer') } assemble (inputs) { @@ -164,6 +164,14 @@ module.exports = class Verifier { return manifestHash(manifest) } + static encodeManifest (manifest) { + return c.encode(m.manifest, manifest) + } + + static decodeManifest (manifest) { + return c.decode(m.manifest, manifest) + } + static defaultSignerManifest (publicKey) { return { version: 1, diff --git a/package.json b/package.json index 79a8e1cb..63f7938f 100644 --- a/package.json +++ b/package.json @@ -52,6 +52,7 @@ "hypercore-crypto": "^3.2.1", "hypercore-errors": "^1.2.0", "hypercore-id-encoding": "^1.2.0", + "hypercore-on-the-rocks": "latest", "is-options": "^1.0.1", "protomux": "^3.5.0", "quickbit-universal": "^2.2.0", @@ -61,7 +62,6 @@ "sodium-universal": "^4.0.0", "streamx": "^2.12.4", "unslab": "^1.3.0", - "xache": "^1.1.0", "z32": "^1.0.0" }, "devDependencies": { diff --git a/test/all.js b/test/all.js index 0aa9c59c..acc2b4f2 100644 --- a/test/all.js +++ b/test/all.js @@ -7,29 +7,29 @@ async function runTests () { test.pause() - await import('./basic.js') - await import('./batch.js') + await import('./basic.js') // todo: implement storageInfo API + await import('./batch.js') // todo: implement batch api await import('./bitfield.js') - await import('./cache.js') - await import('./clear.js') - await import('./compat.js') + await import('./clear.js') // todo: replace Info.bytesUsed API + // await import('./compat.js') // todo: how to test compat? await import('./conflicts.js') await import('./core.js') + await import('./draft.js') await import('./encodings.js') await import('./encryption.js') await import('./extension.js') await import('./manifest.js') await import('./merkle-tree.js') await import('./mutex.js') - await import('./oplog.js') + // await import('./oplog.js') await import('./preload.js') - await import('./purge.js') + // await import('./purge.js') // todo: implement purge await import('./remote-bitfield.js') await import('./remote-length.js') - await import('./replicate.js') + await import('./replicate.js') // todo: append event timing await import('./sessions.js') await import('./snapshots.js') - await import('./storage.js') + // await import('./storage.js') await import('./streams.js') await import('./timeouts.js') await import('./user-data.js') diff --git a/test/basic.js b/test/basic.js index 63eb7310..c5be0114 100644 --- a/test/basic.js +++ b/test/basic.js @@ -1,12 +1,12 @@ const test = require('brittle') -const RAM = require('random-access-memory') const b4a = require('b4a') +const createTempDir = require('test-tmp') const Hypercore = require('../') -const { create, eventFlush } = require('./helpers') +const { create, createStorage, eventFlush } = require('./helpers') test('basic', async function (t) { - const core = await create() + const core = await create(t) let appends = 0 t.is(core.length, 0) @@ -18,7 +18,9 @@ test('basic', async function (t) { }) await core.append('hello') + t.is(core.length, 1) await core.append('world') + t.is(core.length, 2) const info = await core.info() @@ -30,36 +32,47 @@ test('basic', async function (t) { test('core id', async function (t) { const key = b4a.alloc(32).fill('a') - const core = new Hypercore(RAM, key) + const db = await createStorage(t) + const core = new Hypercore(db, key) + t.is(core.id, null) await core.ready() t.is(core.id, 'cfosnambcfosnambcfosnambcfosnambcfosnambcfosnambcfoo') + + await core.close() }) test('session id', async function (t) { const key = b4a.alloc(32).fill('a') - const core = new Hypercore(RAM, key) + + const db = await createStorage(t) + const core = new Hypercore(db, key) const session = core.session() t.is(session.id, null) await session.ready() t.is(session.id, 'cfosnambcfosnambcfosnambcfosnambcfosnambcfosnambcfoo') + + await core.close() + await session.close() }) test('session', async function (t) { - const core = await create() + const core = await create(t) const session = core.session() await session.append('test') t.alike(await core.get(0), b4a.from('test')) t.alike(await session.get(0), b4a.from('test')) + + await session.close() }) test('close', async function (t) { - const core = await create() + const core = await create(t) await core.append('hello world') await core.close() @@ -73,7 +86,7 @@ test('close', async function (t) { }) test('close multiple', async function (t) { - const core = await create() + const core = await create(t) await core.append('hello world') const ev = t.test('events') @@ -91,57 +104,118 @@ test('close multiple', async function (t) { }) test('storage options', async function (t) { - const core = new Hypercore({ storage: RAM }) + const db = await createStorage(t) + const core = new Hypercore({ storage: db }) await core.append('hello') t.alike(await core.get(0), b4a.from('hello')) + + await core.close() }) -test( +test.skip( 'allow publicKeys with different byteLength that 32, if opts.crypto were passed', - function (t) { + async function (t) { const key = b4a.alloc(33).fill('a') - const core = new Hypercore(RAM, key, { crypto: { discoveryKey: () => key } }) + const db = await createStorage(t) + const core = new Hypercore(db, key, { crypto: { discoveryKey: () => key } }) t.is(core.key, key) t.pass('creating a core with more than 32 byteLength key did not throw') + + await core.close() } ) test('createIfMissing', async function (t) { - const core = new Hypercore(RAM, { createIfMissing: false }) + const db = await createStorage(t) + const core = new Hypercore(db, { createIfMissing: false }) await t.exception(core.ready()) + await db.close() +}) + +test('reopen writable core', async function (t) { + const dir = await createTempDir(t) + + const core = new Hypercore(dir) + await core.ready() + + let appends = 0 + + t.is(core.length, 0) + t.is(core.writable, true) + t.is(core.readable, true) + + core.on('append', function () { + appends++ + }) + + await core.append('hello') + await core.append('world') + + const info = await core.info() + + t.is(core.length, 2) + t.is(info.byteLength, 10) + t.is(appends, 2) + + await core.close() + + const core2 = new Hypercore(dir) + await core2.ready() + + t.is(core2.length, 2) + t.is(core2.writable, true) + t.is(core2.readable, true) + + core2.on('append', function () { + appends++ + }) + + await core2.append('goodbye') + await core2.append('test') + + t.is(core2.length, 4) + t.is(appends, 4) + + await core2.close() }) test('reopen and overwrite', async function (t) { - const st = {} - const core = new Hypercore(open) + const dir = await createTempDir() + let storage = null + + const core = new Hypercore(await open()) await core.ready() + await core.close() const key = core.key - const reopen = new Hypercore(open) + const reopen = new Hypercore(await open()) await reopen.ready() t.alike(reopen.key, key, 'reopened the core') + await reopen.close() - const overwritten = new Hypercore(open, { overwrite: true }) + const overwritten = new Hypercore(await open(), { overwrite: true }) await overwritten.ready() t.unlike(overwritten.key, key, 'overwrote the core') - function open (name) { - if (st[name]) return st[name] - st[name] = new RAM() - return st[name] + await overwritten.close() + + async function open () { + if (storage) await storage.close() + storage = await createStorage(t, dir) + return storage } }) test('truncate event has truncated-length and fork', async function (t) { t.plan(2) - const core = new Hypercore(RAM) + const core = new Hypercore(await createStorage(t)) core.on('truncate', function (length, fork) { t.is(length, 2) @@ -150,10 +224,11 @@ test('truncate event has truncated-length and fork', async function (t) { await core.append(['a', 'b', 'c']) await core.truncate(2) + await core.close() }) test('treeHash gets the tree hash at a given core length', async function (t) { - const core = new Hypercore(RAM) + const core = new Hypercore(await createStorage(t)) await core.ready() const { core: { tree } } = core @@ -168,11 +243,13 @@ test('treeHash gets the tree hash at a given core length', async function (t) { for (let i = 0; i < 10; i++) { t.alike(await core.treeHash(i), hashes[i]) } + + await core.close() }) test('treeHash with default length', async function (t) { - const core = new Hypercore(RAM) - const core2 = new Hypercore(RAM) + const core = new Hypercore(await createStorage(t)) + const core2 = new Hypercore(await createStorage(t)) await core.ready() await core2.ready() @@ -181,10 +258,13 @@ test('treeHash with default length', async function (t) { await core.append('a') t.unlike(await core.treeHash(), await core2.treeHash()) + + await core.close() + await core2.close() }) test('snapshot locks the state', async function (t) { - const core = new Hypercore(RAM) + const core = new Hypercore(await createStorage(t)) await core.ready() const a = core.snapshot() @@ -200,12 +280,16 @@ test('snapshot locks the state', async function (t) { t.is(a.length, 0) t.is(b.length, 1) + + await core.close() + await a.close() + await b.close() }) test('downloading local range', async function (t) { t.plan(1) - const core = new Hypercore(RAM) + const core = new Hypercore(await createStorage(t)) await core.append('a') @@ -216,12 +300,14 @@ test('downloading local range', async function (t) { await range.destroy() t.pass('did not throw') + + await core.close() }) test('read ahead', async function (t) { t.plan(1) - const core = new Hypercore(RAM, { valueEncoding: 'utf-8' }) + const core = new Hypercore(await createStorage(t), { valueEncoding: 'utf-8' }) await core.append('a') @@ -232,12 +318,14 @@ test('read ahead', async function (t) { await core.append('b') t.alike(await blk, 'b') + + await core.close() }) test('defaults for wait', async function (t) { t.plan(5) - const core = new Hypercore(RAM, b4a.alloc(32), { valueEncoding: 'utf-8' }) + const core = new Hypercore(await createStorage(t), b4a.alloc(32), { valueEncoding: 'utf-8' }) const a = core.get(1) @@ -262,11 +350,12 @@ test('defaults for wait', async function (t) { t.is(await s2.get(1), null) await s.close() + await s2.close() await core.close() }) test('has', async function (t) { - const core = await create() + const core = await create(t) await core.append(['a', 'b', 'c', 'd', 'e', 'f']) for (let i = 0; i < core.length; i++) { @@ -283,10 +372,12 @@ test('has', async function (t) { t.ok(await core.has(i), `has ${i}`) } } + + await core.close() }) test('has range', async function (t) { - const core = await create() + const core = await create(t) await core.append(['a', 'b', 'c', 'd', 'e', 'f']) t.ok(await core.has(0, 5), 'has 0 to 4') @@ -297,50 +388,73 @@ test('has range', async function (t) { t.absent(await core.has(0, 5), 'does not have 0 to 4') t.ok(await core.has(0, 2), 'has 0 to 1') t.ok(await core.has(3, 5), 'has 3 to 4') + + await core.close() }) -test('storage info', async function (t) { - const core = await create() +test.skip('storage info', async function (t) { + const core = await create(t) await core.append(['a', 'b', 'c', 'd', 'e', 'f']) const info = await core.info({ storage: true }) t.snapshot(info.storage) + + await core.close() }) test('storage info, off by default', async function (t) { - const core = await create() + const core = await create(t) await core.append(['a', 'b', 'c', 'd', 'e', 'f']) const info = await core.info() t.is(info.storage, null) + + await core.close() }) test('indexedLength mirrors core length (linearised core compat)', async function (t) { - const core = await create() + const core = await create(t) t.is(core.length, 0) - t.is(core.indexedLength, core.length) + t.is(core.flushedLength, core.length) await core.append(['a', 'b']) t.is(core.length, 2) - t.is(core.indexedLength, core.length) + t.is(core.flushedLength, core.length) + + await core.close() }) test('key is set sync', async function (t) { const key = b4a.from('a'.repeat(64), 'hex') - t.alike((new Hypercore(RAM, key)).key, key) - t.is((new Hypercore(RAM)).key, null) - - t.alike((new Hypercore(RAM, { key })).key, key) - t.is((new Hypercore(RAM, { })).key, null) + const dir1 = await createStorage(t) + const dir2 = await createStorage(t) + const dir3 = await createStorage(t) + const dir4 = await createStorage(t) + + const core1 = new Hypercore(dir1, key) + const core2 = new Hypercore(dir2) + const core3 = new Hypercore(dir3, { key }) + const core4 = new Hypercore(dir4, { }) + + // flush all db ops before teardown + t.teardown(() => core1.close()) + t.teardown(() => core2.close()) + t.teardown(() => core3.close()) + t.teardown(() => core4.close()) + + t.alike(core1.key, key) + t.is(core2.key, null) + t.alike(core3.key, key) + t.is(core4.key, null) }) test('disable writable option', async function (t) { t.plan(2) - const core = new Hypercore(RAM, { writable: false }) + const core = new Hypercore(await createStorage(t), { writable: false }) await core.ready() t.is(core.writable, false) @@ -351,12 +465,14 @@ test('disable writable option', async function (t) { } catch (err) { t.pass(err.code, 'SESSION_NOT_WRITABLE') } + + await core.close() }) test('disable session writable option', async function (t) { t.plan(3) - const core = new Hypercore(RAM) + const core = new Hypercore(await createStorage(t)) await core.ready() const session = core.session({ writable: false }) @@ -372,12 +488,15 @@ test('disable session writable option', async function (t) { } catch (err) { t.pass(err.code, 'SESSION_NOT_WRITABLE') } + + await session.close() + await core.close() }) test('session of a session with the writable option disabled', async function (t) { t.plan(1) - const core = new Hypercore(RAM) + const core = new Hypercore(await createStorage(t)) const s1 = core.session({ writable: false }) const s2 = s1.session() @@ -387,15 +506,19 @@ test('session of a session with the writable option disabled', async function (t } catch (err) { t.pass(err.code, 'SESSION_NOT_WRITABLE') } + + await s1.close() + await s2.close() + await core.close() }) test('writable session on a readable only core', async function (t) { t.plan(2) - const core = new Hypercore(RAM) + const core = new Hypercore(await createStorage(t)) await core.ready() - const a = new Hypercore(RAM, core.key) + const a = new Hypercore(await createStorage(t), core.key) const s = a.session({ writable: true }) await s.ready() t.is(s.writable, false) @@ -406,12 +529,16 @@ test('writable session on a readable only core', async function (t) { } catch (err) { t.pass(err.code, 'SESSION_NOT_WRITABLE') } + + await s.close() + await a.close() + await core.close() }) test('append above the max suggested block size', async function (t) { t.plan(1) - const core = new Hypercore(RAM) + const core = new Hypercore(await createStorage(t)) try { await core.append(Buffer.alloc(Hypercore.MAX_SUGGESTED_BLOCK_SIZE)) @@ -424,12 +551,14 @@ test('append above the max suggested block size', async function (t) { } catch { t.pass('should throw') } + + await core.close() }) test('get undefined block is not allowed', async function (t) { t.plan(1) - const core = new Hypercore(RAM) + const core = new Hypercore(await createStorage(t)) try { await core.get(undefined) @@ -437,12 +566,14 @@ test('get undefined block is not allowed', async function (t) { } catch (err) { t.pass(err.code, 'ERR_ASSERTION') } + + await core.close() }) test('valid manifest passed to a session is stored', async function (t) { t.plan(1) - const core = new Hypercore(RAM, { + const core = new Hypercore(await createStorage(t), { manifest: { prologue: { hash: b4a.alloc(32), @@ -454,11 +585,15 @@ test('valid manifest passed to a session is stored', async function (t) { await core.ready() - const a = new Hypercore(RAM, core.key) + const a = new Hypercore(await createStorage(t), core.key) const b = new Hypercore(null, core.key, { manifest: core.manifest, from: a }) await b.ready() t.alike(b.manifest, core.manifest) + + await a.close() + await b.close() + await core.close() }) diff --git a/test/batch.js b/test/batch.js index 2a5197f7..71843d84 100644 --- a/test/batch.js +++ b/test/batch.js @@ -1,116 +1,146 @@ const test = require('brittle') +const createTempDir = require('test-tmp') const b4a = require('b4a') +const Hypercore = require('../') +const { create, createStorage, replicate, eventFlush } = require('./helpers') + const NS = b4a.alloc(32) -const { create, replicate, eventFlush } = require('./helpers') test('batch append', async function (t) { - const core = await create() + const core = await create(t) await core.append(['a', 'b', 'c']) - const b = core.batch() + const b = core.session({ name: 'batch' }) + await b.ready() // todo: we shouldn't have to wait for ready + + t.unlike(b.state, core.state) + const info = await b.append(['de', 'fg']) t.is(core.length, 3) + + t.is(b.length, 5) t.alike(info, { length: 5, byteLength: 7 }) t.alike(await b.get(3), b4a.from('de')) t.alike(await b.get(4), b4a.from('fg')) - await b.flush() + t.is(core.length, 3) + + await core.core.commit(b.state) + t.is(core.length, 5) + + await b.close() }) test('batch has', async function (t) { - const core = await create() + const core = await create(t) await core.append(['a', 'b', 'c']) - const b = core.batch() + const b = core.session({ name: 'batch' }) await b.append(['de', 'fg']) for (let i = 0; i < b.length; i++) { t.ok(await b.has(i)) } + + await b.close() }) -test('append to core during batch', async function (t) { - const core = await create() +test.skip('append to core during batch', async function (t) { + const core = await create(t) await core.append(['a', 'b', 'c']) - const b = core.batch() + const b = core.session({ name: 'batch' }) await core.append('d') await b.append('e') t.absent(await b.flush()) t.is(core.length, 4) + + await b.close() }) test('append to session during batch, create before batch', async function (t) { - const core = await create() + const core = await create(t) await core.append(['a', 'b', 'c']) const s = core.session() - const b = core.batch() + const b = core.session({ name: 'batch' }) await b.append('d') await s.append('d') - t.ok(await b.flush()) + t.ok(await core.core.commit(b.state)) t.is(s.length, 4) + + await b.close() + await s.close() }) test('append to session during batch, create after batch', async function (t) { - const core = await create() + const core = await create(t) await core.append(['a', 'b', 'c']) - const b = core.batch() + const b = core.session({ name: 'batch' }) await b.append('d') const s = core.session() await s.append('d') - t.ok(await b.flush()) + t.ok(await core.core.commit(b.state)) t.is(s.length, 4) + + await s.close() + await b.close() }) test('batch truncate', async function (t) { - const core = await create() + const core = await create(t) await core.append(['a', 'b', 'c']) - const b = core.batch() + const b = core.session({ name: 'batch' }) await b.append(['de', 'fg']) - await b.truncate(4) + await b.truncate(4, { fork: 0 }) t.alike(await b.get(3), b4a.from('de')) - await t.exception(b.get(4)) + t.alike(await b.get(4, { wait: false }), null) - await b.flush() + await core.core.commit(b.state) t.is(core.length, 4) + + await b.close() }) test('truncate core during batch', async function (t) { - const core = await create() + const core = await create(t) await core.append(['a', 'b', 'c']) - const b = core.batch() + const b = core.session({ name: 'batch' }) await b.append('a') await core.truncate(2) - t.absent(await b.flush()) + t.absent(await core.core.commit(b.state)) t.is(core.length, 2) + + await b.close() }) -test('batch truncate committed', async function (t) { - const core = await create() +test.skip('batch truncate committed', async function (t) { + const core = await create(t) await core.append(['a', 'b', 'c']) - const b = core.batch() + const b = core.session({ name: 'batch' }) await b.append(['de', 'fg']) await t.exception(b.truncate(2)) + + await b.close() }) test('batch close', async function (t) { - const core = await create() + const core = await create(t) await core.append(['a', 'b', 'c']) - const b = core.batch() + const b = core.session({ name: 'batch' }) await b.append(['de', 'fg']) await b.close() t.is(core.length, 3) @@ -120,28 +150,32 @@ test('batch close', async function (t) { }) test('batch close after flush', async function (t) { - const core = await create() + const core = await create(t) await core.append(['a', 'b', 'c']) - const b = core.batch() - await b.flush() + const b = core.session({ name: 'batch' }) + await b.ready() + + await core.core.commit(b.state) await b.close() }) -test('batch flush after close', async function (t) { - const core = await create() +test.skip('batch flush after close', async function (t) { + const core = await create(t) await core.append(['a', 'b', 'c']) - const b = core.batch() + const b = core.session({ name: 'batch' }) + await b.ready() + await b.close() - await t.exception(b.flush()) + await t.exception(core.core.commit(b.state)) }) -test('batch info', async function (t) { - const core = await create() +test.skip('batch info', async function (t) { + const core = await create(t) await core.append(['a', 'b', 'c']) - const b = core.batch() + const b = core.session({ name: 'batch' }) await b.append(['de', 'fg']) const info = await b.info() @@ -150,45 +184,55 @@ test('batch info', async function (t) { t.is(info.byteLength, 7) t.unlike(await core.info(), info) - await b.flush() + await core.core.commit(b.state) t.alike(await core.info(), info) + + await b.close() }) test('simultaneous batches', async function (t) { - const core = await create() + const core = await create(t) - const b = core.batch() - const c = core.batch() - const d = core.batch() + const b = core.session({ name: '1' }) + const c = core.session({ name: '2' }) + const d = core.session({ name: '3' }) await b.append('a') await c.append(['a', 'c']) await d.append('c') - t.ok(await b.flush()) - t.ok(await c.flush()) - t.absent(await d.flush()) + t.ok(await core.core.commit(b.state)) + t.ok(await core.core.commit(c.state)) + t.absent(await core.core.commit(d.state)) + + await b.close() + await c.close() + await d.close() }) test('multiple batches', async function (t) { - const core = await create() + const core = await create(t) const session = core.session() - const b = core.batch() + const b = core.session({ name: 'batch1' }) await b.append('a') - await b.flush() + await core.core.commit(b.state) - const b2 = session.batch() + const b2 = session.session({ name: 'batch2' }) await b2.append('b') - await b2.flush() + await core.core.commit(b2.state) t.is(core.length, 2) + + await session.close() + await b.close() + await b2.close() }) -test('partial flush', async function (t) { - const core = await create() +test.skip('partial flush', async function (t) { + const core = await create(t) - const b = core.batch({ autoClose: false }) + const b = core.session({ name: 'batch' }) await b.append(['a', 'b', 'c', 'd']) @@ -220,67 +264,84 @@ test('partial flush', async function (t) { }) test('can make a tree batch', async function (t) { - const core = await create() + const core = await create(t) - const b = core.batch() + const b = core.session({ name: 'batch' }) await b.append('a') const batchTreeBatch = b.createTreeBatch() const batchHash = batchTreeBatch.hash() - await b.flush() + await core.core.commit(b.state) const treeBatch = core.createTreeBatch() const hash = treeBatch.hash() t.alike(hash, batchHash) + + await b.close() }) test('batched tree batch contains new nodes', async function (t) { - const core = await create() + const core = await create(t) - const b = core.batch() + const b = core.session({ name: 'batch' }) await b.append('a') const batchTreeBatch = b.createTreeBatch() const batchNode = await batchTreeBatch.get(0) - await b.flush() + await core.core.commit(b.state) const treeBatch = core.createTreeBatch() const node = await treeBatch.get(0) t.alike(node, batchNode) + + await b.close() }) test('batched tree batch proofs are equivalent', async function (t) { - const core = await create() + const core = await create(t) - const b = core.batch() + const b = core.session({ name: 'batch' }) await b.append(['a', 'b', 'c']) + const reader = b.state.storage.createReadBatch() const batchTreeBatch = b.createTreeBatch() - const batchProof = await batchTreeBatch.proof({ upgrade: { start: 0, length: 2 } }) + const batchProofIntermediate = await batchTreeBatch.proof(reader, { upgrade: { start: 0, length: 2 } }) + + await reader.flush() + + const batchProof = await batchProofIntermediate.settle() - await b.flush() + await core.core.commit(b.state) + const reader1 = core.state.storage.createReadBatch() const treeBatch = core.createTreeBatch() - const proof = await treeBatch.proof({ upgrade: { start: 0, length: 2 } }) - const treeProof = await core.core.tree.proof({ upgrade: { start: 0, length: 2 } }) + const proofIntermediate = await treeBatch.proof(reader, { upgrade: { start: 0, length: 2 } }) + const treeProofIntermediate = await core.core.tree.proof(reader1, { upgrade: { start: 0, length: 2 } }) + + await reader1.flush() + + const proof = await proofIntermediate.settle() + const treeProof = await treeProofIntermediate.settle() treeProof.upgrade.signature = null t.alike(proof, batchProof) t.alike(treeProof, batchProof) + + await b.close() }) -test('create tree batches', async function (t) { - const core = await create() +test.skip('create tree batches', async function (t) { + const core = await create(t) - const b = core.batch() + const b = core.session({ name: 'batch' }) await b.append('a') await b.append('b') @@ -324,11 +385,11 @@ test('create tree batches', async function (t) { t.absent(b.createTreeBatch(6)) t.absent(b.createTreeBatch(8, blocks)) - await b.flush() + await core.core.commit(b.state) t.is(core.length, 5) - const b2 = core.batch() + const b2 = core.session({ name: 'batch2' }) await b2.ready() t.absent(b2.createTreeBatch(3)) @@ -345,18 +406,21 @@ test('create tree batches', async function (t) { await b2.append('g') t.alike(b2.createTreeBatch().signable(NS), t7.signable(NS)) + + await b.close() + await b2.close() }) test('flush with bg activity', async function (t) { - const core = await create() - const clone = await create(core.key) + const core = await create(t) + const clone = await create(t, { keyPair: core.core.header.keyPair }) replicate(core, clone, t) await core.append('a') await clone.get(0) - const b = clone.batch({ autoClose: false }) + const b = clone.session({ name: 'batch' }) // bg await core.append('b') @@ -367,23 +431,25 @@ test('flush with bg activity', async function (t) { await b.append('b') - t.absent(await b.flush(), 'core is ahead, not flushing') + t.absent(await core.core.commit(b.state), 'core is ahead, not flushing') await b.append('c') - t.ok(await b.flush(), 'flushed!') + t.ok(await core.core.commit(b.state), 'flushed!') + + await b.close() }) test('flush with bg activity persists non conflicting values', async function (t) { - const core = await create() - const clone = await create(core.key) + const core = await create(t) + const clone = await create(t, core.key) replicate(core, clone, t) await core.append('a') await clone.get(0) - const b = clone.batch() + const b = clone.session({ name: 'batch' }) // bg await core.append('b') @@ -394,26 +460,28 @@ test('flush with bg activity persists non conflicting values', async function (t await eventFlush() - t.ok(await b.flush(), 'flushed!') + t.ok(await clone.core.commit(b.state), 'flushed!') t.alike(await clone.get(0, { wait: false }), b4a.from('a')) t.alike(await clone.get(1, { wait: false }), b4a.from('b')) t.alike(await clone.get(2, { wait: false }), b4a.from('c')) t.is(b.byteLength, clone.byteLength) - t.is(b.indexedLength, b.length, 'nothing buffered') + t.is(b.flushedLength, b.length, 'nothing buffered') + + await b.close() }) test('flush with conflicting bg activity', async function (t) { - const core = await create() - const clone = await create(core.key) + const core = await create(t) + const clone = await create(t, core.key) replicate(core, clone, t) await core.append('a') await clone.get(0) - const b = clone.batch({ autoClose: false }) + const b = clone.session({ name: 'batch' }) // bg await core.append('b') @@ -425,11 +493,13 @@ test('flush with conflicting bg activity', async function (t) { await b.append('c') await b.append('c') - t.absent(await b.flush(), 'cannot flush a batch with conflicts') + t.absent(await clone.core.commit(b.state), 'cannot flush a batch with conflicts') + + await b.close() }) -test('checkout batch', async function (t) { - const core = await create() +test.skip('checkout batch', async function (t) { + const core = await create(t) await core.append(['a', 'b']) const hash = core.createTreeBatch().hash() @@ -451,25 +521,29 @@ test('checkout batch', async function (t) { await b.truncate(3, b.fork) await b.append('d') t.ok(await b.flush()) + + await b.close() }) test('encryption and batches', async function (t) { - const core = await create({ encryptionKey: b4a.alloc(32) }) + const core = await create(t, { encryptionKey: b4a.alloc(32) }) await core.append(['a', 'b']) - const batch = core.batch() + const batch = core.session({ name: 'batch' }) + + await batch.ready() t.alike(await batch.get(0), b4a.from('a')) t.alike(await batch.get(1), b4a.from('b')) - const pre = batch.createTreeBatch(3, [b4a.from('c')]) + // const pre = batch.createTreeBatch(3, [b4a.from('c')]) await batch.append('c') const post = batch.createTreeBatch(3) t.is(batch.byteLength, 3) t.alike(await batch.get(2), b4a.from('c')) - await batch.flush() + await core.core.commit(batch.state) t.is(core.byteLength, 3) t.is(core.length, 3) @@ -478,20 +552,22 @@ test('encryption and batches', async function (t) { const final = core.createTreeBatch() - t.alike(pre.hash(), final.hash()) + // t.alike(pre.hash(), final.hash()) t.alike(post.hash(), final.hash()) + + await batch.close() }) test('encryption and bigger batches', async function (t) { - const core = await create({ encryptionKey: b4a.alloc(32) }) + const core = await create(t, { encryptionKey: b4a.alloc(32) }) await core.append(['a', 'b']) - const batch = core.batch() + const batch = core.session({ name: 'batch' }) t.alike(await batch.get(0), b4a.from('a')) t.alike(await batch.get(1), b4a.from('b')) - const pre = batch.createTreeBatch(5, [b4a.from('c'), b4a.from('d'), b4a.from('e')]) + // const pre = batch.createTreeBatch(5, [b4a.from('c'), b4a.from('d'), b4a.from('e')]) await batch.append(['c', 'd', 'e']) const post = batch.createTreeBatch(5) @@ -500,7 +576,7 @@ test('encryption and bigger batches', async function (t) { t.alike(await batch.get(3), b4a.from('d')) t.alike(await batch.get(4), b4a.from('e')) - await batch.flush() + await core.core.commit(batch.state) t.is(core.byteLength, 5) t.is(core.length, 5) @@ -511,166 +587,212 @@ test('encryption and bigger batches', async function (t) { const final = core.createTreeBatch() - t.alike(pre.hash(), final.hash()) + // t.alike(pre.hash(), final.hash()) t.alike(post.hash(), final.hash()) + + await batch.close() }) -test('persistent batch', async function (t) { - const core = await create() +// test('persistent batch', async function (t) { +// const core = await create(t) - await core.append(['a', 'b', 'c']) +// await core.append(['a', 'b', 'c']) - let batch = core.batch() +// let batch = core.session({ name: 'batch' }) - await batch.ready() - await batch.append(['d', 'e', 'f']) - await batch.flush({ pending: true }) +// await batch.ready() +// await batch.append(['d', 'e', 'f']) +// await batch.flush({ pending: true }) - batch = core.batch({ restore: true, autoClose: false }) +// batch = core.batch({ restore: true, autoClose: false }) - await batch.ready() +// await batch.ready() - t.is(batch.length, 6) - t.is(batch.byteLength, 6) - t.is(batch.indexedLength, 3) - t.alike(await batch.seek(4), [4, 0]) +// t.is(batch.length, 6) +// t.is(batch.byteLength, 6) +// t.is(batch.flushedLength, 3) +// t.alike(await batch.seek(4), [4, 0]) - const clone = await create(core.key) +// const clone = await create(t, core.key) - replicate(core, clone, t) +// replicate(core, clone, t) - clone.download() +// clone.download() - await t.test('download', async function (sub) { - const downloaded = [] - clone.on('download', function (index) { - downloaded.push(index) - }) - await eventFlush() - sub.alike(downloaded.sort(), [0, 1, 2], 'got non pending blocks') - }) +// await t.test('download', async function (sub) { +// const downloaded = [] +// clone.on('download', function (index) { +// downloaded.push(index) +// }) +// await eventFlush() +// sub.alike(downloaded.sort(), [0, 1, 2], 'got non pending blocks') +// }) - await batch.flush({ length: 5 }) +// await batch.flush({ length: 5 }) - t.is(core.length, 5) +// t.is(core.length, 5) - await t.test('download', async function (sub) { - const downloaded = [] - clone.on('download', function (index) { - downloaded.push(index) - }) - await eventFlush() - sub.alike(downloaded.sort(), [3, 4], 'got non pending blocks') - }) +// await t.test('download', async function (sub) { +// const downloaded = [] +// clone.on('download', function (index) { +// downloaded.push(index) +// }) +// await eventFlush() +// sub.alike(downloaded.sort(), [3, 4], 'got non pending blocks') +// }) - await batch.flush({ length: 6 }) +// await batch.flush({ length: 6 }) - t.is(core.length, 6) +// t.is(core.length, 6) - await t.test('download', async function (sub) { - const downloaded = [] - clone.on('download', function (index) { - downloaded.push(index) - }) - await eventFlush() - sub.alike(downloaded.sort(), [5], 'got non pending blocks') - }) +// await t.test('download', async function (sub) { +// const downloaded = [] +// clone.on('download', function (index) { +// downloaded.push(index) +// }) +// await eventFlush() +// sub.alike(downloaded.sort(), [5], 'got non pending blocks') +// }) - await batch.append('g') +// await batch.append('g') - t.is(batch.length, 7) +// t.is(batch.length, 7) - await batch.flush({ pending: true }) +// await batch.flush({ pending: true }) - t.is(core.length, 6) +// t.is(core.length, 6) - await batch.append('h') +// await batch.append('h') - t.is(batch.length, 8) +// t.is(batch.length, 8) - await batch.flush({ pending: true }) +// await batch.flush({ pending: true }) - t.is(batch.length, 8) +// t.is(batch.length, 8) - t.is(core.length, 6) +// t.is(core.length, 6) - await batch.flush() +// await batch.flush() - t.is(batch.length, 8) - t.is(core.length, 8) +// t.is(batch.length, 8) +// t.is(core.length, 8) - await t.test('download', async function (sub) { - const downloaded = [] - clone.on('download', function (index) { - downloaded.push(index) - }) - await eventFlush() - sub.alike(downloaded.sort(), [6, 7], 'got non pending blocks') - }) +// await t.test('download', async function (sub) { +// const downloaded = [] +// clone.on('download', function (index) { +// downloaded.push(index) +// }) +// await eventFlush() +// sub.alike(downloaded.sort(), [6, 7], 'got non pending blocks') +// }) +// }) + +test('persistent batch', async function (t) { + const dir = await createTempDir() + let storage = null + + const core = new Hypercore(await open()) + await core.ready() + + await core.append(['a', 'b', 'c']) + + const batch = core.session({ name: 'batch' }) + await batch.ready() + + await batch.append(['d', 'e', 'f']) + + t.is(batch.length, 6) + t.is(batch.byteLength, 6) + // t.is(batch.flushedLength, 3) + // t.alike(await batch.seek(4), [4, 0]) + + await core.close() + + const reopen = new Hypercore(await open()) + await reopen.ready() + + const reopened = reopen.session({ name: 'batch' }) + await reopened.ready() + + t.is(reopened.length, 6) + t.is(reopened.byteLength, 6) + // t.is(batch.flushedLength, 3) + // t.alike(await batch.seek(4), [4, 0]) + + await reopened.close() + await reopen.close() + + async function open () { + if (storage) await storage.close() + storage = await createStorage(t, dir) + return storage + } }) test('clear', async function (t) { - const core = await create() + const core = await create(t) await core.append('hello') - const clone = await create(core.key) + const clone = await create(t, core.key) - const b = clone.batch() + const b = clone.session({ name: 'b' }) await b.append('hello') - await b.flush() - await b.close() const [s1, s2] = replicate(core, clone, t) - await eventFlush() + await new Promise(resolve => clone.on('append', resolve)) + + await clone.core.commit(b.state) + await b.close() + t.ok(!!(await clone.get(0)), 'got block 0 proof') s1.destroy() s2.destroy() - const b1 = clone.batch() + const b1 = clone.session({ name: 'b1' }) await b1.ready() await b1.append('foo') - await b1.flush() + await t.exception(clone.core.commit(b1.state)) await b1.close() t.is(clone.length, 1, 'clone length is still 1') - const b2 = clone.batch({ clear: true }) + const b2 = clone.batch() await b2.ready() t.is(b2.length, 1, 'reset the batch') + + await b2.close() }) test('copy from with encrypted batch', async function (t) { const encryptionKey = b4a.alloc(32, 2) - const core = await create({ encryptionKey }) + const core = await create(t, { encryptionKey }) const blocks = 290 - const b = core.batch({ autoClose: false }) + const b = core.session({ name: 'batch' }) for (let i = 0; i < blocks; i++) { await b.append('block' + i) } - await b.flush({ keyPair: null }) - t.is(core.length, 0) - t.is(b._sessionLength, blocks) + t.is(b.length, blocks) const manifest = { prologue: { - length: b._sessionLength, + length: b.length, hash: b.createTreeBatch().hash() }, encryptionKey } - const clone = await create({ + const clone = await create(t, { manifest, encryptionKey }) @@ -682,19 +804,23 @@ test('copy from with encrypted batch', async function (t) { } t.alike(tree.hash(), manifest.prologue.hash) + + await b.close() }) test('batch append with huge batch', async function (t) { // Context: array.append(...otherArray) stops working after a certain amount of entries // due to a limit on the amount of function args // This caused a bug on large batches - const core = await create() + const core = await create(t) const bigBatch = (new Array(200_000)).fill('o') - const b = core.batch() + const b = core.session({ name: 'batch' }) await b.append(bigBatch) // Actually flushing such a big batch takes multiple minutes // so we only ensure that nothing crashed while appending t.pass('Can append a big batch') + + await b.close() }) diff --git a/test/bitfield.js b/test/bitfield.js index 4c182699..10a39e30 100644 --- a/test/bitfield.js +++ b/test/bitfield.js @@ -1,9 +1,13 @@ const test = require('brittle') -const RAM = require('random-access-memory') +const b4a = require('b4a') +const createTempDir = require('test-tmp') +const CoreStorage = require('hypercore-on-the-rocks') const Bitfield = require('../lib/bitfield') +const BitInterlude = require('../lib/bit-interlude') test('bitfield - set and get', async function (t) { - const b = await Bitfield.open(new RAM()) + const storage = await createStorage(t) + const b = await Bitfield.open(storage) t.absent(b.get(42)) b.set(42, true) @@ -12,16 +16,13 @@ test('bitfield - set and get', async function (t) { // bigger offsets t.absent(b.get(42000000)) b.set(42000000, true) - t.ok(b.get(42000000)) - + t.ok(b.get(42000000, true)) b.set(42000000, false) - t.absent(b.get(42000000)) - - await b.flush() + t.absent(b.get(42000000, true)) }) test('bitfield - random set and gets', async function (t) { - const b = await Bitfield.open(new RAM()) + const b = await Bitfield.open(await createStorage(t)) const set = new Set() for (let i = 0; i < 200; i++) { @@ -33,7 +34,7 @@ test('bitfield - random set and gets', async function (t) { for (let i = 0; i < 500; i++) { const idx = Math.floor(Math.random() * Number.MAX_SAFE_INTEGER) const expected = set.has(idx) - const val = b.get(idx) + const val = b.get(idx, true) if (val !== expected) { t.fail('expected ' + expected + ' but got ' + val + ' at ' + idx) return @@ -41,7 +42,7 @@ test('bitfield - random set and gets', async function (t) { } for (const idx of set) { - const val = b.get(idx) + const val = b.get(idx, true) if (val !== true) { t.fail('expected true but got ' + val + ' at ' + idx) return @@ -52,18 +53,21 @@ test('bitfield - random set and gets', async function (t) { }) test('bitfield - reload', async function (t) { - const s = new RAM() + const dir = await createTempDir(t) { - const b = await Bitfield.open(s) - b.set(142, true) - b.set(40000, true) - b.set(1424242424, true) - await b.flush() + const storage = await createStorage(t, dir) + const bitfield = await Bitfield.open(storage) + const b = new BitInterlude(bitfield) + b.setRange(142, 143, true) + b.setRange(40000, 40001, true) + b.setRange(1424242424, 1424242425, true) + await flush(storage, b) + await storage.db.close() } { - const b = await Bitfield.open(s) + const b = await Bitfield.open(await createStorage(t, dir)) t.ok(b.get(142)) t.ok(b.get(40000)) t.ok(b.get(1424242424)) @@ -74,7 +78,7 @@ test('bitfield - want', async function (t) { // This test will likely break when bitfields are optimised to not actually // store pages of all set or unset bits. - const b = new Bitfield(new RAM(), new Uint32Array(1024 * 512 / 4 /* 512 KiB */)) + const b = new Bitfield(b4a.alloc(1024 * 512) /* 512 KiB */) t.alike([...b.want(0, 0)], []) @@ -112,14 +116,15 @@ test('bitfield - want', async function (t) { }) test('bitfield - sparse array overflow', async function (t) { - const b = await Bitfield.open(new RAM()) + const b = await Bitfield.open(await createStorage(t)) // Previously bugged due to missing bounds check in sparse array b.set(7995511118690925, true) }) test('bitfield - count', async function (t) { - const b = await Bitfield.open(new RAM()) + const s = await createStorage(t) + const b = await Bitfield.open(s) for (const [start, length] of [[0, 2], [5, 1], [7, 2], [13, 1], [16, 3], [20, 5]]) { b.setRange(start, length, true) @@ -130,7 +135,7 @@ test('bitfield - count', async function (t) { }) test('bitfield - find first, all zeroes', async function (t) { - const b = await Bitfield.open(new RAM()) + const b = await Bitfield.open(await createStorage(t)) t.is(b.findFirst(false, 0), 0) t.is(b.findFirst(true, 0), -1) @@ -153,7 +158,8 @@ test('bitfield - find first, all zeroes', async function (t) { }) test('bitfield - find first, all ones', async function (t) { - const b = await Bitfield.open(new RAM()) + const s = await createStorage(t) + const b = await Bitfield.open(s) b.setRange(0, 2 ** 24, true) @@ -180,7 +186,7 @@ test('bitfield - find first, all ones', async function (t) { }) test('bitfield - find last, all zeroes', async function (t) { - const b = await Bitfield.open(new RAM()) + const b = await Bitfield.open(await createStorage(t)) t.is(b.findLast(false, 0), 0) t.is(b.findLast(true, 0), -1) @@ -203,7 +209,8 @@ test('bitfield - find last, all zeroes', async function (t) { }) test('bitfield - find last, all ones', async function (t) { - const b = await Bitfield.open(new RAM()) + const s = await createStorage(t) + const b = await Bitfield.open(s) b.setRange(0, 2 ** 24, true) @@ -230,7 +237,8 @@ test('bitfield - find last, all ones', async function (t) { }) test('bitfield - find last, ones around page boundary', async function (t) { - const b = await Bitfield.open(new RAM()) + const s = await createStorage(t) + const b = await Bitfield.open(s) b.set(32767, true) b.set(32768, true) @@ -240,7 +248,8 @@ test('bitfield - find last, ones around page boundary', async function (t) { }) test('bitfield - set range on page boundary', async function (t) { - const b = await Bitfield.open(new RAM()) + const s = await createStorage(t) + const b = await Bitfield.open(s) b.setRange(2032, 26, true) @@ -248,12 +257,33 @@ test('bitfield - set range on page boundary', async function (t) { }) test('set last bits in segment and findFirst', async function (t) { - const b = await Bitfield.open(new RAM()) + const s = await createStorage(t) + const b = await Bitfield.open(s) b.set(2097150, true) + t.is(b.findFirst(false, 2097150), 2097151) b.set(2097151, true) + t.is(b.findFirst(false, 2097150), 2097152) t.is(b.findFirst(false, 2097151), 2097152) }) + +async function createStorage (t, dir) { + if (!dir) dir = await createTempDir(t) + + const db = new CoreStorage(dir) + + t.teardown(() => db.close()) + + const dkey = b4a.alloc(32) + + return (await db.resume(dkey)) || (await db.create({ key: dkey, discoveryKey: dkey })) +} + +async function flush (s, b) { + const w = s.createWriteBatch() + b.flush(w) + await w.flush() +} diff --git a/test/cache.js b/test/cache.js deleted file mode 100644 index 88bc8486..00000000 --- a/test/cache.js +++ /dev/null @@ -1,121 +0,0 @@ -const test = require('brittle') -const Xache = require('xache') -const Rache = require('rache') -const b4a = require('b4a') -const { create, replicate } = require('./helpers') - -test('cache', async function (t) { - const a = await create({ cache: true }) - await a.append(['a', 'b', 'c']) - - const p = a.get(0) - const q = a.get(0) - - t.is(await p, await q, 'blocks are identical') -}) - -test('session cache inheritance', async function (t) { - const a = await create({ cache: true }) - await a.append(['a', 'b', 'c']) - - const s = a.session() - - const p = a.get(0) - const q = s.get(0) - - t.is(await p, await q, 'blocks are identical') -}) - -test('session cache opt-out', async function (t) { - const a = await create({ cache: true }) - await a.append(['a', 'b', 'c']) - - const s = a.session({ cache: false }) - - const p = a.get(0) - const q = s.get(0) - - t.not(await p, await q, 'blocks are not identical') -}) - -test('session cache override', async function (t) { - const a = await create({ cache: true }) - await a.append(['a', 'b', 'c']) - - const s = a.session({ cache: new Xache({ maxSize: 64, maxAge: 0 }) }) - - const p = a.get(0) - const q = s.get(0) - const r = s.get(0) - - t.not(await p, await q, 'blocks are not identical') - t.is(await q, await r, 'blocks are identical') -}) - -test('clear cache on truncate', async function (t) { - const a = await create({ cache: true }) - await a.append(['a', 'b', 'c']) - - const p = a.get(0) - - await a.truncate(0) - await a.append('d') - - const q = a.get(0) - - t.alike(await p, b4a.from('a')) - t.alike(await q, b4a.from('d')) -}) - -test('cache on replicate', async function (t) { - const a = await create() - await a.append(['a', 'b', 'c']) - - const b = await create(a.key, { cache: true }) - - replicate(a, b, t) - - // These will issue a replicator request - const p = await b.get(0) - const q = await b.get(0) - - t.is(p, q, 'blocks are identical') - - // This should use the cache - const r = await b.get(0) - - t.is(p, r, 'blocks are identical') -}) - -test('session cache with different encodings', async function (t) { - const a = await create({ cache: true }) - await a.append(['a', 'b', 'c']) - - const s = a.session({ valueEncoding: 'utf-8' }) - - const p = a.get(0) - const q = s.get(0) - - t.alike(await p, b4a.from('a')) - t.is(await q, 'a') -}) - -test('cache is set through preload', async function (t) { - const a = await create({ async preload () { return { cache: true } } }) - - t.ok(a.cache) -}) - -test('null default for globalCache', async function (t) { - const a = await create() - t.is(a.globalCache, null) -}) - -test('globalCache set if passed in, and shared among sessions', async function (t) { - const globalCache = new Rache() - const a = await create({ globalCache }) - t.is(a.globalCache, globalCache, 'cache is stored in hypercore') - - const session = a.session() - t.is(session.globalCache, globalCache, 'passed on to sessions') -}) diff --git a/test/clear.js b/test/clear.js index fc50dd0c..b1f13ca8 100644 --- a/test/clear.js +++ b/test/clear.js @@ -1,13 +1,12 @@ const test = require('brittle') const tmp = require('test-tmp') const b4a = require('b4a') -const RAM = require('random-access-memory') -const { create, replicate, eventFlush } = require('./helpers') +const { create, createStorage, replicate, eventFlush } = require('./helpers') const Hypercore = require('../') test('clear', async function (t) { - const a = await create() + const a = await create(t) await a.append(['a', 'b', 'c']) t.is(a.contiguousLength, 3) @@ -19,11 +18,13 @@ test('clear', async function (t) { t.ok(await a.has(0), 'has 0') t.absent(await a.has(1), 'has not 1') t.ok(await a.has(2), 'has 2') + + await a.close() }) test('clear + replication', async function (t) { - const a = await create() - const b = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) replicate(a, b, t) @@ -36,12 +37,15 @@ test('clear + replication', async function (t) { t.ok(await b.has(1), 'b not cleared') t.alike(await a.get(1), b4a.from('b'), 'a downloaded from b') + + await a.close() + await b.close() }) test('clear + replication, gossip', async function (t) { - const a = await create() - const b = await create(a.key) - const c = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) + const c = await create(t, a.key) replicate(a, b, t) replicate(b, c, t) @@ -68,7 +72,7 @@ test('clear + replication, gossip', async function (t) { }) test('incorrect clear', async function (t) { - const core = await create() + const core = await create(t) const blocks = [] while (blocks.length < 129) { @@ -78,22 +82,26 @@ test('incorrect clear', async function (t) { await core.append(blocks) await core.clear(127, 128) + t.absent(await core.has(127)) t.ok(await core.has(128)) t.alike(await core.get(128), b4a.from('tick')) }) test('clear blocks with diff option', async function (t) { - const core = new Hypercore(() => new RAM({ pageSize: 128 })) + const storage = await createStorage(t) + const core = new Hypercore(storage) await core.append(b4a.alloc(128)) const cleared = await core.clear(1337) t.is(cleared, null) - const cleared2 = await core.clear(0, { diff: true }) - t.ok(cleared2.blocks > 0) + // todo: reenable bytes use api + + // const cleared2 = await core.clear(0, { diff: true }) + // t.ok(cleared2.blocks > 0) - const cleared3 = await core.clear(0, { diff: true }) - t.is(cleared3.blocks, 0) + // const cleared3 = await core.clear(0, { diff: true }) + // t.is(cleared3.blocks, 0) await core.close() }) diff --git a/test/compat.js b/test/compat.js index 3ca09d99..cce755d6 100644 --- a/test/compat.js +++ b/test/compat.js @@ -30,5 +30,7 @@ for (const abi of abis) { } t.pass('blocks match') + + await core.close() }) } diff --git a/test/conflicts.js b/test/conflicts.js index 791c2c35..42a4ae7f 100644 --- a/test/conflicts.js +++ b/test/conflicts.js @@ -4,12 +4,12 @@ const { create, replicate, unreplicate } = require('./helpers') test('one forks', async function (t) { t.plan(3) - const a = await create() + const a = await create(t) await a.append(['a', 'b', 'c', 'd', 'e']) - const b = await create(a.key) + const b = await create(t, a.key) - const c = await create({ keyPair: a.core.header.keyPair }) + const c = await create(t, { keyPair: a.core.header.keyPair }) await c.append(['a', 'b', 'c', 'd', 'f', 'e']) const streams = replicate(a, b, t) diff --git a/test/core.js b/test/core.js index f3a75ba7..0f681635 100644 --- a/test/core.js +++ b/test/core.js @@ -1,13 +1,14 @@ const test = require('brittle') -const RAM = require('random-access-memory') const b4a = require('b4a') +const createTempDir = require('test-tmp') +const CoreStorage = require('hypercore-on-the-rocks') const Core = require('../lib/core') test('core - append', async function (t) { - const { core } = await create() + const { core } = await create(t) { - const info = await core.append([ + const info = await core.state.append([ b4a.from('hello'), b4a.from('world') ]) @@ -16,8 +17,8 @@ test('core - append', async function (t) { t.is(core.tree.length, 2) t.is(core.tree.byteLength, 10) t.alike([ - await core.blocks.get(0), - await core.blocks.get(1) + await getBlock(core, 0), + await getBlock(core, 1) ], [ b4a.from('hello'), b4a.from('world') @@ -25,7 +26,7 @@ test('core - append', async function (t) { } { - const info = await core.append([ + const info = await core.state.append([ b4a.from('hej') ]) @@ -33,9 +34,9 @@ test('core - append', async function (t) { t.is(core.tree.length, 3) t.is(core.tree.byteLength, 13) t.alike([ - await core.blocks.get(0), - await core.blocks.get(1), - await core.blocks.get(2) + await getBlock(core, 0), + await getBlock(core, 1), + await getBlock(core, 2) ], [ b4a.from('hello'), b4a.from('world'), @@ -45,51 +46,51 @@ test('core - append', async function (t) { }) test('core - append and truncate', async function (t) { - const { core, reopen } = await create() + const { core, reopen } = await create(t) - await core.append([ + await core.state.append([ b4a.from('hello'), b4a.from('world'), b4a.from('fo'), b4a.from('ooo') ]) - await core.truncate(3, 1) + await core.state.truncate(3, 1) t.is(core.tree.length, 3) t.is(core.tree.byteLength, 12) t.is(core.tree.fork, 1) t.alike(core.header.hints.reorgs, [{ from: 0, to: 1, ancestors: 3 }]) - await core.append([ + await core.state.append([ b4a.from('a'), b4a.from('b'), b4a.from('c'), b4a.from('d') ]) - await core.truncate(3, 2) + await core.state.truncate(3, 2) t.is(core.tree.length, 3) t.is(core.tree.byteLength, 12) t.is(core.tree.fork, 2) t.alike(core.header.hints.reorgs, [{ from: 0, to: 1, ancestors: 3 }, { from: 1, to: 2, ancestors: 3 }]) - await core.truncate(2, 3) + await core.state.truncate(2, 3) t.alike(core.header.hints.reorgs, [{ from: 2, to: 3, ancestors: 2 }]) - await core.append([b4a.from('a')]) - await core.truncate(2, 4) + await core.state.append([b4a.from('a')]) + await core.state.truncate(2, 4) - await core.append([b4a.from('a')]) - await core.truncate(2, 5) + await core.state.append([b4a.from('a')]) + await core.state.truncate(2, 5) - await core.append([b4a.from('a')]) - await core.truncate(2, 6) + await core.state.append([b4a.from('a')]) + await core.state.truncate(2, 6) - await core.append([b4a.from('a')]) - await core.truncate(2, 7) + await core.state.append([b4a.from('a')]) + await core.state.truncate(2, 7) t.is(core.header.hints.reorgs.length, 4) @@ -99,36 +100,42 @@ test('core - append and truncate', async function (t) { t.is(coreReopen.tree.length, 2) t.is(coreReopen.tree.byteLength, 10) t.is(coreReopen.tree.fork, 7) - t.is(coreReopen.header.hints.reorgs.length, 4) + // t.is(coreReopen.header.hints.reorgs.length, 4) }) test('core - user data', async function (t) { - const { core, reopen } = await create() + const { core, reopen } = await create(t) - await core.userData('hello', b4a.from('world')) - t.alike(core.header.userData, [{ key: 'hello', value: b4a.from('world') }]) + await setUserData(core, 'hello', b4a.from('world')) + t.alike(await getUserData(core.storage, 'hello'), b4a.from('world')) - await core.userData('hej', b4a.from('verden')) - t.alike(core.header.userData, [ - { key: 'hello', value: b4a.from('world') }, - { key: 'hej', value: b4a.from('verden') } - ]) + await setUserData(core, 'hej', b4a.from('verden')) + t.alike(await getUserData(core.storage, 'hello'), b4a.from('world')) + t.alike(await getUserData(core.storage, 'hej'), b4a.from('verden')) - await core.userData('hello', null) - t.alike(core.header.userData, [{ key: 'hej', value: b4a.from('verden') }]) + await setUserData(core, 'hello', null) + t.alike(await getUserData(core.storage, 'hello'), null) + t.alike(await getUserData(core.storage, 'hej'), b4a.from('verden')) - await core.userData('hej', b4a.from('world')) - t.alike(core.header.userData, [{ key: 'hej', value: b4a.from('world') }]) + await setUserData(core, 'hej', b4a.from('world')) + t.alike(await getUserData(core.storage, 'hej'), b4a.from('world')) // check that it was persisted const coreReopen = await reopen() - t.alike(coreReopen.header.userData, [{ key: 'hej', value: b4a.from('world') }]) + t.alike(await getUserData(coreReopen.storage, 'hej'), b4a.from('world')) + + function getUserData (storage, key) { + const b = storage.createReadBatch() + const p = b.getUserData(key) + b.tryFlush() + return p + } }) test('core - header does not retain slabs', async function (t) { - const { core, reopen } = await create() - await core.userData('hello', b4a.from('world')) + const { core, reopen } = await create(t) + await setUserData(core, 'hello', b4a.from('world')) t.is(core.header.key.buffer.byteLength, 32, 'unslabbed key') t.is(core.header.keyPair.publicKey.buffer.byteLength, 32, 'unslabbed public key') @@ -146,43 +153,48 @@ test('core - header does not retain slabs', async function (t) { t.is(coreReopen.header.keyPair.secretKey.buffer.byteLength, 64, 'reopen unslabbed secret key') t.is(coreReopen.header.manifest.signers[0].namespace.buffer.byteLength, 32, 'reopen unslabbed signers namespace') t.is(coreReopen.header.manifest.signers[0].publicKey.buffer.byteLength, 32, 'reopen unslabbed signers publicKey') + t.is(coreReopen.header.userData[0].value.buffer.byteLength, 5, 'reopen unslabbed the userdata value') + + await coreReopen.close() }) test('core - verify', async function (t) { - const { core } = await create() - const { core: clone } = await create({ keyPair: { publicKey: core.header.keyPair.publicKey } }) + const { core } = await create(t) + const { core: clone } = await create(t, { keyPair: { publicKey: core.header.keyPair.publicKey } }) t.is(clone.header.keyPair.publicKey, core.header.keyPair.publicKey) - await core.append([b4a.from('a'), b4a.from('b')]) + await core.state.append([b4a.from('a'), b4a.from('b')]) { - const p = await core.tree.proof({ upgrade: { start: 0, length: 2 } }) + const p = await getProof(core, { upgrade: { start: 0, length: 2 } }) await clone.verify(p) } - t.is(clone.header.tree.length, 2) - t.is(clone.header.tree.signature, core.header.tree.signature) + const tree1 = await getCoreHead(core.storage) + const tree2 = await getCoreHead(clone.storage) + + t.is(tree1.length, 2) + t.alike(tree1.signature, tree2.signature) { - const p = await core.tree.proof({ block: { index: 1, nodes: await clone.tree.nodes(2), value: true } }) - p.block.value = await core.blocks.get(1) + const p = await getProof(core, { block: { index: 1, nodes: await clone.tree.nodes(2), value: true } }) await clone.verify(p) } }) test('core - verify parallel upgrades', async function (t) { - const { core } = await create() - const { core: clone } = await create({ keyPair: { publicKey: core.header.keyPair.publicKey } }) + const { core } = await create(t) + const { core: clone } = await create(t, { keyPair: { publicKey: core.header.keyPair.publicKey } }) t.is(clone.header.keyPair.publicKey, core.header.keyPair.publicKey) - await core.append([b4a.from('a'), b4a.from('b'), b4a.from('c'), b4a.from('d')]) + await core.state.append([b4a.from('a'), b4a.from('b'), b4a.from('c'), b4a.from('d')]) { - const p1 = await core.tree.proof({ upgrade: { start: 0, length: 2 } }) - const p2 = await core.tree.proof({ upgrade: { start: 0, length: 3 } }) + const p1 = await getProof(core, { upgrade: { start: 0, length: 2 } }) + const p2 = await getProof(core, { upgrade: { start: 0, length: 3 } }) const v1 = clone.verify(p1) const v2 = clone.verify(p2) @@ -191,28 +203,31 @@ test('core - verify parallel upgrades', async function (t) { await v2 } - t.is(clone.header.tree.length, core.header.tree.length) - t.is(clone.header.tree.signature, core.header.tree.signature) + const tree1 = await getCoreHead(core.storage) + const tree2 = await getCoreHead(clone.storage) + + t.is(tree2.length, tree1.length) + t.alike(tree2.signature, tree1.signature) }) test('core - update hook is triggered', async function (t) { - const { core } = await create() - const { core: clone } = await create({ keyPair: { publicKey: core.header.keyPair.publicKey } }) + const { core } = await create(t) + const { core: clone } = await create(t, { keyPair: { publicKey: core.header.keyPair.publicKey } }) let ran = 0 - core.onupdate = (status, bitfield, value, from) => { + core.onupdate = ({ status, bitfield, value, from }) => { t.ok(status & 0b01, 'was appended') t.is(from, null, 'was local') t.alike(bitfield, { drop: false, start: 0, length: 4 }) ran |= 1 } - await core.append([b4a.from('a'), b4a.from('b'), b4a.from('c'), b4a.from('d')]) + await core.state.append([b4a.from('a'), b4a.from('b'), b4a.from('c'), b4a.from('d')]) const peer = {} - clone.onupdate = (status, bitfield, value, from) => { + clone.onupdate = ({ status, bitfield, value, from }) => { t.ok(status & 0b01, 'was appended') t.is(from, peer, 'was remote') t.alike(bitfield, { drop: false, start: 1, length: 1 }) @@ -221,12 +236,12 @@ test('core - update hook is triggered', async function (t) { } { - const p = await core.tree.proof({ block: { index: 1, nodes: 0, value: true }, upgrade: { start: 0, length: 2 } }) - p.block.value = await core.blocks.get(1) + const p = await getProof(core, { block: { index: 1, nodes: 0, value: true }, upgrade: { start: 0, length: 2 } }) + p.block.value = await getBlock(core, 1) await clone.verify(p, peer) } - clone.onupdate = (status, bitfield, value, from) => { + clone.onupdate = ({ status, bitfield, value, from }) => { t.is(status, 0b00, 'no append or truncate') t.is(from, peer, 'was remote') t.alike(bitfield, { drop: false, start: 3, length: 1 }) @@ -235,30 +250,30 @@ test('core - update hook is triggered', async function (t) { } { - const p = await core.tree.proof({ block: { index: 3, nodes: await clone.tree.nodes(6), value: true } }) - p.block.value = await core.blocks.get(3) + const p = await getProof(core, { block: { index: 3, nodes: await clone.tree.nodes(6), value: true } }) + p.block.value = await getBlock(core, 3) await clone.verify(p, peer) } - core.onupdate = (status, bitfield, value, from) => { + core.onupdate = ({ status, bitfield, value, from }) => { t.ok(status & 0b10, 'was truncated') t.is(from, null, 'was local') t.alike(bitfield, { drop: true, start: 1, length: 3 }) ran |= 8 } - await core.truncate(1, 1) + await core.state.truncate(1, 1) - core.onupdate = (status, bitfield, value, from) => { + core.onupdate = ({ status, bitfield, value, from }) => { t.ok(status & 0b01, 'was appended') t.is(from, null, 'was local') t.alike(bitfield, { drop: false, start: 1, length: 1 }) ran |= 16 } - await core.append([b4a.from('e')]) + await core.state.append([b4a.from('e')]) - clone.onupdate = (status, bitfield, value, from) => { + clone.onupdate = ({ status, bitfield, value, from }) => { t.ok(status & 0b11, 'was appended and truncated') t.is(from, peer, 'was remote') t.alike(bitfield, { drop: true, start: 1, length: 3 }) @@ -266,21 +281,21 @@ test('core - update hook is triggered', async function (t) { } { - const p = await core.tree.proof({ hash: { index: 0, nodes: 0 }, upgrade: { start: 0, length: 2 } }) + const p = await getProof(core, { hash: { index: 0, nodes: 0 }, upgrade: { start: 0, length: 2 } }) const r = await clone.tree.reorg(p) await clone.reorg(r, peer) } - core.onupdate = (status, bitfield, value, from) => { + core.onupdate = ({ status, bitfield, value, from }) => { t.ok(status & 0b10, 'was truncated') t.is(from, null, 'was local') t.alike(bitfield, { drop: true, start: 1, length: 1 }) ran |= 64 } - await core.truncate(1, 2) + await core.state.truncate(1, 2) - clone.onupdate = (status, bitfield, value, from) => { + clone.onupdate = ({ status, bitfield, value, from }) => { t.ok(status & 0b10, 'was truncated') t.is(from, peer, 'was remote') t.alike(bitfield, { drop: true, start: 1, length: 1 }) @@ -288,7 +303,7 @@ test('core - update hook is triggered', async function (t) { } { - const p = await core.tree.proof({ hash: { index: 0, nodes: 0 }, upgrade: { start: 0, length: 1 } }) + const p = await getProof(core, { hash: { index: 0, nodes: 0 }, upgrade: { start: 0, length: 1 } }) const r = await clone.tree.reorg(p) await clone.reorg(r, peer) @@ -298,25 +313,29 @@ test('core - update hook is triggered', async function (t) { }) test('core - clone', async function (t) { - const { core } = await create() + const { core } = await create(t) - await core.userData('hello', b4a.from('world')) + await setUserData(core, 'hello', b4a.from('world')) - await core.append([ + await core.state.append([ b4a.from('hello'), b4a.from('world') ]) const manifest = { prologue: { hash: await core.tree.hash(), length: core.tree.length } } - const { core: copy } = (await create({ manifest })) + const { core: copy } = (await create(t, { manifest })) - await copy.copyPrologue(core) + await copy.copyPrologue(core.state) - t.alike(copy.header.userData, [{ key: 'hello', value: b4a.from('world') }]) + const userData = [] + const str = copy.storage.createUserDataStream() + for await (const { key, value } of str) userData.push({ key, value }) + + t.alike(userData, [{ key: 'hello', value: b4a.from('world') }]) t.alike([ - await copy.blocks.get(0), - await copy.blocks.get(1) + await getBlock(copy, 0), + await getBlock(copy, 1) ], [ b4a.from('hello'), b4a.from('world') @@ -332,89 +351,97 @@ test('core - clone', async function (t) { ) } - await core.append([b4a.from('c')]) + await core.state.append([b4a.from('c')]) // copy should be independent t.alike(copy.tree.signature, signature) t.alike(copy.tree.roots.map(r => r.index), roots) + t.is(copy.header.hints.contiguousLength, 2) }) test('core - clone verify', async function (t) { - const { core } = await create() + const { core } = await create(t) - await core.append([b4a.from('a'), b4a.from('b')]) + await core.state.append([b4a.from('a'), b4a.from('b')]) const manifest = { prologue: { hash: await core.tree.hash(), length: core.tree.length } } - const { core: copy } = await create({ manifest }) - const { core: clone } = await create({ manifest }) + const { core: copy } = await create(t, { manifest }) + const { core: clone } = await create(t, { manifest }) - await copy.copyPrologue(core) + await copy.copyPrologue(core.state) // copy should be independent - await core.append([b4a.from('c')]) + await core.state.append([b4a.from('c')]) { - const p = await copy.tree.proof({ upgrade: { start: 0, length: 2 } }) + const p = await getProof(copy, { upgrade: { start: 0, length: 2 } }) t.ok(await clone.verify(p)) } t.is(clone.header.tree.length, 2) { - const p = await copy.tree.proof({ block: { index: 1, nodes: await clone.tree.nodes(2), value: true } }) - p.block.value = await copy.blocks.get(1) + const p = await getProof(copy, { block: { index: 1, nodes: await clone.tree.nodes(2), value: true } }) + p.block.value = await getBlock(copy, 1) await clone.verify(p) } + t.is(core.header.hints.contiguousLength, 3) + t.is(copy.header.hints.contiguousLength, 2) + t.is(clone.header.hints.contiguousLength, 0) + t.pass('verified') }) test('core - partial clone', async function (t) { - const { core } = await create() + const { core } = await create(t) - await core.append([b4a.from('0')]) - await core.append([b4a.from('1')]) + await core.state.append([b4a.from('0')]) + await core.state.append([b4a.from('1')]) const manifest = { prologue: { hash: await core.tree.hash(), length: core.tree.length } } - await core.append([b4a.from('2')]) - await core.append([b4a.from('3')]) + await core.state.append([b4a.from('2')]) + await core.state.append([b4a.from('3')]) - const { core: copy } = (await create({ manifest })) + const { core: copy } = (await create(t, { manifest })) - await copy.copyPrologue(core) + await copy.copyPrologue(core.state) t.is(core.tree.length, 4) t.is(copy.tree.length, 2) + t.is(core.header.hints.contiguousLength, 4) + t.is(copy.header.hints.contiguousLength, 2) + t.alike([ - await copy.blocks.get(0), - await copy.blocks.get(1) + await getBlock(copy, 0), + await getBlock(copy, 1), + await getBlock(copy, 2) ], [ b4a.from('0'), - b4a.from('1') + b4a.from('1'), + null ]) - - await t.exception(copy.blocks.get(2)) }) test('core - clone with additional', async function (t) { - const { core } = await create() + const { core } = await create(t) - await core.append([b4a.from('a'), b4a.from('b')]) + await core.state.append([b4a.from('a'), b4a.from('b')]) const manifest = { prologue: { hash: await core.tree.hash(), length: core.tree.length } } - const { core: copy } = await create({ manifest }) + const { core: copy } = await create(t, { manifest }) - await copy.copyPrologue(core, core.tree.signature) + await copy.copyPrologue(core.state, core.tree.signature) // copy should be independent - await core.append([b4a.from('c')]) + await core.state.append([b4a.from('c')]) const secondManifest = { prologue: { hash: await core.tree.hash(), length: core.tree.length } } - const { core: clone } = await create({ manifest: secondManifest }) + const { core: clone } = await create(t, { manifest: secondManifest }) - await clone.copyPrologue(copy, { additional: [b4a.from('c')] }) + await clone.copyPrologue(copy.state, { additional: [b4a.from('c')] }) t.is(clone.header.tree.length, 3) @@ -422,20 +449,23 @@ test('core - clone with additional', async function (t) { t.is(clone.tree.byteLength, core.tree.byteLength) t.alike(clone.roots, core.roots) - t.alike(await clone.blocks.get(0), b4a.from('a')) - t.alike(await clone.blocks.get(1), b4a.from('b')) - t.alike(await clone.blocks.get(2), b4a.from('c')) + t.is(copy.header.hints.contiguousLength, 2) + t.is(clone.header.hints.contiguousLength, 3) + + t.alike(await getBlock(clone, 0), b4a.from('a')) + t.alike(await getBlock(clone, 1), b4a.from('b')) + t.alike(await getBlock(clone, 2), b4a.from('c')) }) test('core - clone with additional, larger tree', async function (t) { - const { core } = await create() + const { core } = await create(t) - await core.append([b4a.from('a'), b4a.from('b')]) + await core.state.append([b4a.from('a'), b4a.from('b')]) const manifest = { prologue: { hash: await core.tree.hash(), length: core.tree.length } } - const { core: copy } = await create({ manifest }) + const { core: copy } = await create(t, { manifest }) - await copy.copyPrologue(core) + await copy.copyPrologue(core.state) const additional = [ b4a.from('c'), @@ -448,13 +478,13 @@ test('core - clone with additional, larger tree', async function (t) { b4a.from('j') ] - await core.append(additional) + await core.state.append(additional) const secondManifest = { prologue: { hash: await core.tree.hash(), length: core.tree.length } } - const { core: clone } = await create({ manifest: secondManifest }) + const { core: clone } = await create(t, { manifest: secondManifest }) // copy should be independent - await clone.copyPrologue(copy, { additional }) + await clone.copyPrologue(copy.state, { additional }) t.is(clone.header.tree.length, core.header.tree.length) @@ -462,69 +492,78 @@ test('core - clone with additional, larger tree', async function (t) { t.is(clone.tree.byteLength, core.tree.byteLength) t.alike(clone.roots, core.roots) - t.alike(await clone.blocks.get(0), b4a.from('a')) - t.alike(await clone.blocks.get(1), b4a.from('b')) - t.alike(await clone.blocks.get(2), b4a.from('c')) - t.alike(await clone.blocks.get(3), b4a.from('d')) - t.alike(await clone.blocks.get(4), b4a.from('e')) - t.alike(await clone.blocks.get(5), b4a.from('f')) - t.alike(await clone.blocks.get(6), b4a.from('g')) - t.alike(await clone.blocks.get(7), b4a.from('h')) - t.alike(await clone.blocks.get(8), b4a.from('i')) - t.alike(await clone.blocks.get(9), b4a.from('j')) + t.is(copy.header.hints.contiguousLength, 2) + t.is(clone.header.hints.contiguousLength, 10) + + t.alike(await getBlock(clone, 0), b4a.from('a')) + t.alike(await getBlock(clone, 1), b4a.from('b')) + t.alike(await getBlock(clone, 2), b4a.from('c')) + t.alike(await getBlock(clone, 3), b4a.from('d')) + t.alike(await getBlock(clone, 4), b4a.from('e')) + t.alike(await getBlock(clone, 5), b4a.from('f')) + t.alike(await getBlock(clone, 6), b4a.from('g')) + t.alike(await getBlock(clone, 7), b4a.from('h')) + t.alike(await getBlock(clone, 8), b4a.from('i')) + t.alike(await getBlock(clone, 9), b4a.from('j')) }) test('core - copyPrologue bails if core is not the same', async function (t) { - const { core } = await create() - const { core: copy } = await create({ manifest: { prologue: { hash: b4a.alloc(32), length: 1 } } }) + const { core } = await create(t) + const { core: copy } = await create(t, { manifest: { prologue: { hash: b4a.alloc(32), length: 1 } } }) // copy should be independent - await core.append([b4a.from('a')]) + await core.state.append([b4a.from('a')]) + + await t.exception(copy.copyPrologue(core.state)) - await t.exception(copy.copyPrologue(core)) + t.is(copy.header.hints.contiguousLength, 0) }) test('core - copyPrologue can recover from bad additional', async function (t) { - const { core } = await create() + const { core } = await create(t) - await core.append([b4a.from('a'), b4a.from('b')]) + await core.state.append([b4a.from('a'), b4a.from('b')]) const manifest = { prologue: { hash: await core.tree.hash(), length: core.tree.length } } - const { core: copy } = await create({ manifest }) - await copy.copyPrologue(core) + const { core: copy } = await create(t, { manifest }) + await copy.copyPrologue(core.state) // copy should be independent - await core.append([b4a.from('c')]) + await core.state.append([b4a.from('c')]) const secondManifest = { prologue: { hash: await core.tree.hash(), length: core.tree.length } } - const { core: clone } = await create({ manifest: secondManifest }) + const { core: clone } = await create(t, { manifest: secondManifest }) + + await t.exception(clone.copyPrologue(copy.state, { additional: [b4a.from('d')] })) + + t.is(clone.header.hints.contiguousLength, 0) - await t.exception(clone.copyPrologue(copy, { additional: [b4a.from('d')] })) - await t.execution(clone.copyPrologue(copy, { additional: [b4a.from('c')] })) + await t.execution(clone.copyPrologue(copy.state, { additional: [b4a.from('c')] })) + t.is(clone.header.hints.contiguousLength, 3) t.is(clone.header.tree.length, 3) t.is(clone.tree.length, core.tree.length) t.is(clone.tree.byteLength, core.tree.byteLength) t.alike(clone.roots, core.roots) - t.alike(await clone.blocks.get(0), b4a.from('a')) - t.alike(await clone.blocks.get(1), b4a.from('b')) - t.alike(await clone.blocks.get(2), b4a.from('c')) + t.alike(await getBlock(clone, 0), b4a.from('a')) + t.alike(await getBlock(clone, 1), b4a.from('b')) + t.alike(await getBlock(clone, 2), b4a.from('c')) }) test('core - copyPrologue many', async function (t) { - const { core } = await create({ compat: false, version: 1 }) - await core.append([b4a.from('a'), b4a.from('b')]) + const { core } = await create(t, { compat: false, version: 1 }) + await core.state.append([b4a.from('a'), b4a.from('b')]) const manifest = { ...core.header.manifest } manifest.prologue = { length: core.tree.length, hash: core.tree.hash() } - const { core: copy } = await create({ manifest }) - const { core: copy2 } = await create({ manifest }) - const { core: copy3 } = await create({ manifest }) + const { core: copy } = await create(t, { manifest }) + const { core: copy2 } = await create(t, { manifest }) + const { core: copy3 } = await create(t, { manifest }) - await copy.copyPrologue(core) + await copy.copyPrologue(core.state) t.alike(copy.header.manifest.signers[0].publicKey, core.header.manifest.signers[0].publicKey) @@ -532,18 +571,18 @@ test('core - copyPrologue many', async function (t) { t.is(copy.tree.byteLength, core.tree.byteLength) // copy should be independent - await core.append([b4a.from('c')]) + await core.state.append([b4a.from('c')]) // upgrade clone { const batch = core.tree.batch() - const p = await core.tree.proof({ upgrade: { start: 0, length: 3 } }) + const p = await getProof(core, { upgrade: { start: 0, length: 3 } }) p.upgrade.signature = copy2.verifier.sign(batch, core.header.keyPair) t.ok(await copy2.verify(p)) } - await t.execution(copy2.copyPrologue(core)) - await t.execution(copy3.copyPrologue(core)) + await t.execution(copy2.copyPrologue(core.state)) + await t.execution(copy3.copyPrologue(core.state)) t.is(copy2.tree.length, core.tree.length) t.is(copy.tree.length, copy3.tree.length) @@ -555,27 +594,75 @@ test('core - copyPrologue many', async function (t) { t.is(copy.tree.byteLength, copy3.tree.byteLength) manifest.prologue = { length: core.tree.length, hash: core.tree.hash() } - const { core: copy4 } = await create({ manifest }) - await copy4.copyPrologue(copy2) + const { core: copy4 } = await create(t, { manifest }) + await copy4.copyPrologue(copy2.state) t.is(copy4.tree.length, 3) t.is(copy4.header.tree.length, 3) - t.alike(await copy4.blocks.get(0), b4a.from('a')) - t.alike(await copy4.blocks.get(1), b4a.from('b')) + t.is(core.header.hints.contiguousLength, 3) + t.is(copy.header.hints.contiguousLength, 2) + t.is(copy2.header.hints.contiguousLength, 2) + t.is(copy3.header.hints.contiguousLength, 2) + t.is(copy4.header.hints.contiguousLength, 2) + + t.alike(await getBlock(copy4, 0), b4a.from('a')) + t.alike(await getBlock(copy4, 1), b4a.from('b')) }) -async function create (opts) { - const storage = new Map() +async function create (t, opts = {}) { + const dir = opts.dir || await createTempDir(t) + + const dkey = b4a.alloc(32, 1) + let db = null + + t.teardown(teardown, { order: 1 }) + + const reopen = async () => { + if (db) await db.close() + + db = new CoreStorage(dir) - const createFile = (name) => { - if (storage.has(name)) return storage.get(name) - const s = new RAM() - storage.set(name, s) - return s + if (!opts.discoveryKey) opts.discoveryKey = dkey + + const core = await Core.open(db, opts) + t.teardown(() => core.close()) + return core } - const reopen = () => Core.open(createFile, opts) const core = await reopen() + return { core, reopen } + + async function teardown () { + if (db) await db.close() + } +} + +async function getBlock (core, i) { + const r = core.storage.createReadBatch() + const p = core.blocks.get(r, i) + await r.flush() + return p +} + +async function setUserData (core, key, value) { + return core.userData(key, value) +} + +async function getProof (core, req) { + const batch = core.storage.createReadBatch() + const p = await core.tree.proof(batch, req) + const block = req.block ? core.blocks.get(batch, req.block.index) : null + batch.tryFlush() + const proof = await p.settle() + if (block) proof.block.value = await block + return proof +} + +function getCoreHead (storage) { + const b = storage.createReadBatch() + const p = b.getCoreHead() + b.tryFlush() + return p } diff --git a/test/draft.js b/test/draft.js new file mode 100644 index 00000000..15806f43 --- /dev/null +++ b/test/draft.js @@ -0,0 +1,50 @@ +const { create } = require('./helpers') +const test = require('brittle') +const b4a = require('b4a') + +test('draft', async function (t) { + const core = await create(t) + + await core.append('hello') + await core.append('world') + + const draft = core.session({ draft: true }) + + await draft.append('edits!') + + t.alike(await draft.get(0), b4a.from('hello')) + t.alike(await draft.get(1), b4a.from('world')) + t.alike(await draft.get(2), b4a.from('edits!')) + t.alike(await draft.seek(11), [2, 1]) + t.alike(draft.byteLength, 16) + t.alike(draft.length, 3) + + await draft.close() + + // nothing changed as it was a draft + t.alike(core.byteLength, 10) + t.alike(core.length, 2) + + await core.close() +}) + +test('draft and then undraft', async function (t) { + const core = await create(t) + + await core.append('hello') + await core.append('world') + + const draft = core.session({ draft: true }) + + await draft.append('edits!') + + await core.core.commit(draft.state) + + await draft.close() + + // nothing changed as it was a draft + t.alike(core.byteLength, 16) + t.alike(core.length, 3) + + await core.close() +}) diff --git a/test/encodings.js b/test/encodings.js index 7e5171f7..24505ec3 100644 --- a/test/encodings.js +++ b/test/encodings.js @@ -3,7 +3,7 @@ const b4a = require('b4a') const { create } = require('./helpers') test('encodings - supports built ins', async function (t) { - const a = await create(null, { valueEncoding: 'json' }) + const a = await create(t, null, { valueEncoding: 'json' }) await a.append({ hello: 'world' }) t.alike(await a.get(0), { hello: 'world' }) @@ -11,7 +11,7 @@ test('encodings - supports built ins', async function (t) { }) test('encodings - supports custom encoding', async function (t) { - const a = await create(null, { valueEncoding: { encode () { return b4a.from('foo') }, decode () { return 'bar' } } }) + const a = await create(t, null, { valueEncoding: { encode () { return b4a.from('foo') }, decode () { return 'bar' } } }) await a.append({ hello: 'world' }) t.is(await a.get(0), 'bar') @@ -19,7 +19,7 @@ test('encodings - supports custom encoding', async function (t) { }) test('encodings - supports custom batch encoding', async function (t) { - const a = await create(null, { + const a = await create(t, null, { encodeBatch: batch => { return [b4a.from(batch.join('-'))] }, diff --git a/test/encryption.js b/test/encryption.js index 993c1641..0fb25929 100644 --- a/test/encryption.js +++ b/test/encryption.js @@ -1,13 +1,12 @@ const test = require('brittle') -const RAM = require('random-access-memory') const b4a = require('b4a') const Hypercore = require('..') -const { create, replicate } = require('./helpers') +const { create, createStorage, replicate } = require('./helpers') const encryptionKey = b4a.alloc(32, 'hello world') test('encrypted append and get', async function (t) { - const a = await create({ encryptionKey }) + const a = await create(t, { encryptionKey }) t.alike(a.encryptionKey, encryptionKey) @@ -20,12 +19,12 @@ test('encrypted append and get', async function (t) { const unencrypted = await a.get(0) t.alike(unencrypted, b4a.from('hello')) - const encrypted = await a.core.blocks.get(0) + const encrypted = await getBlock(a, 0) t.absent(encrypted.includes('hello')) }) test('get with decrypt option', async function (t) { - const a = await create({ encryptionKey }) + const a = await create(t, { encryptionKey }) await a.append('hello') @@ -37,7 +36,7 @@ test('get with decrypt option', async function (t) { }) test('encrypted seek', async function (t) { - const a = await create({ encryptionKey }) + const a = await create(t, { encryptionKey }) await a.append(['hello', 'world', '!']) @@ -52,12 +51,12 @@ test('encrypted seek', async function (t) { }) test('encrypted replication', async function (t) { - const a = await create({ encryptionKey }) + const a = await create(t, { encryptionKey }) await a.append(['a', 'b', 'c', 'd', 'e']) await t.test('with encryption key', async function (t) { - const b = await create(a.key, { encryptionKey }) + const b = await create(t, a.key, { encryptionKey }) replicate(a, b, t) @@ -82,7 +81,7 @@ test('encrypted replication', async function (t) { }) await t.test('without encryption key', async function (t) { - const b = await create(a.key) + const b = await create(t, a.key) replicate(a, b, t) @@ -91,7 +90,7 @@ test('encrypted replication', async function (t) { await r.done() for (let i = 0; i < 5; i++) { - t.alike(await b.get(i), await a.core.blocks.get(i)) + t.alike(await b.get(i), await getBlock(a, i)) } }) @@ -99,7 +98,7 @@ test('encrypted replication', async function (t) { await a.append(['f', 'g', 'h', 'i', 'j']) for (let i = 5; i < 10; i++) { - t.alike(await b.get(i), await a.core.blocks.get(i)) + t.alike(await b.get(i), await getBlock(a, i)) } await a.truncate(5) @@ -108,7 +107,7 @@ test('encrypted replication', async function (t) { }) test('encrypted session', async function (t) { - const a = await create({ encryptionKey }) + const a = await create(t, { encryptionKey }) await a.append(['hello']) @@ -123,13 +122,17 @@ test('encrypted session', async function (t) { t.alike(unencrypted, b4a.from('world')) t.alike(await a.get(1), unencrypted) - const encrypted = await s.core.blocks.get(1) + const encrypted = await getBlock(s, 1) t.absent(encrypted.includes('world')) - t.alike(await a.core.blocks.get(1), encrypted) + t.alike(await getBlock(a, 1), encrypted) + + await s.close() }) test('encrypted session before ready core', async function (t) { - const a = new Hypercore(RAM, { encryptionKey }) + const storage = await createStorage(t) + + const a = new Hypercore(storage, { encryptionKey }) const s = a.session() await a.ready() @@ -138,10 +141,13 @@ test('encrypted session before ready core', async function (t) { await a.append(['hello']) t.alike(await s.get(0), b4a.from('hello')) + + await s.close() + await a.close() }) test('encrypted session on unencrypted core', async function (t) { - const a = await create() + const a = await create(t) const s = a.session({ encryptionKey }) t.alike(s.encryptionKey, encryptionKey) @@ -154,10 +160,12 @@ test('encrypted session on unencrypted core', async function (t) { const encrypted = await a.get(0) t.absent(encrypted.includes('hello')) + + await s.close() }) test('encrypted session on encrypted core, same key', async function (t) { - const a = await create({ encryptionKey }) + const a = await create(t, { encryptionKey }) const s = a.session({ encryptionKey }) t.alike(s.encryptionKey, a.encryptionKey) @@ -167,10 +175,12 @@ test('encrypted session on encrypted core, same key', async function (t) { const unencrypted = await s.get(0) t.alike(unencrypted, b4a.from('hello')) t.alike(unencrypted, await a.get(0)) + + await s.close() }) test('encrypted session on encrypted core, different keys', async function (t) { - const a = await create({ encryptionKey: b4a.alloc(32, 'a') }) + const a = await create(t, { encryptionKey: b4a.alloc(32, 'a') }) const s = a.session({ encryptionKey: b4a.alloc(32, 's') }) t.unlike(s.encryptionKey, a.encryptionKey) @@ -182,13 +192,15 @@ test('encrypted session on encrypted core, different keys', async function (t) { const encrypted = await a.get(0) t.absent(encrypted.includes('hello')) + + await s.close() }) test('multiple gets to replicated, encrypted block', async function (t) { - const a = await create({ encryptionKey }) + const a = await create(t, { encryptionKey }) await a.append('a') - const b = await create(a.key, { encryptionKey }) + const b = await create(t, a.key, { encryptionKey }) replicate(a, b, t) @@ -200,8 +212,8 @@ test('multiple gets to replicated, encrypted block', async function (t) { }) test('encrypted core from existing unencrypted core', async function (t) { - const a = await create({ encryptionKey: b4a.alloc(32, 'a') }) - const b = await create({ from: a, encryptionKey }) + const a = await create(t, { encryptionKey: b4a.alloc(32, 'a') }) + const b = new Hypercore({ from: a, encryptionKey }) t.alike(b.key, a.key) t.alike(b.encryptionKey, encryptionKey) @@ -210,10 +222,14 @@ test('encrypted core from existing unencrypted core', async function (t) { const unencrypted = await b.get(0) t.alike(unencrypted, b4a.from('hello')) + + await b.close() }) test('from session sessions pass encryption', async function (t) { - const a = new Hypercore(RAM) + const storage = await createStorage(t) + + const a = new Hypercore(storage) const b = new Hypercore({ from: a, encryptionKey }) const c = b.session() @@ -224,12 +240,28 @@ test('from session sessions pass encryption', async function (t) { t.absent(a.encryptionKey) t.ok(b.encryptionKey) t.ok(c.encryptionKey) + + await c.close() + await b.close() + await a.close() }) test('session keeps encryption', async function (t) { - const a = new Hypercore(RAM) + const storage = await createStorage(t) + + const a = new Hypercore(storage) const b = a.session({ encryptionKey }) await b.ready() t.alike(b.encryptionKey, encryptionKey) + + await b.close() + await a.close() }) + +function getBlock (core, index) { + const batch = core.core.storage.createReadBatch() + const b = core.core.blocks.get(batch, index) + batch.tryFlush() + return b +} diff --git a/test/extension.js b/test/extension.js index 0ccda1a6..bb70c67c 100644 --- a/test/extension.js +++ b/test/extension.js @@ -4,7 +4,7 @@ const { create, replicate, eventFlush } = require('./helpers') test('basic extension', async function (t) { const messages = ['world', 'hello'] - const a = await create() + const a = await create(t) a.registerExtension('test-extension', { encoding: 'utf-8', onmessage: (message, peer) => { @@ -13,7 +13,7 @@ test('basic extension', async function (t) { } }) - const b = await create(a.key) + const b = await create(t, a.key) const bExt = b.registerExtension('test-extension', { encoding: 'utf-8' }) @@ -33,8 +33,8 @@ test('basic extension', async function (t) { test('two extensions', async function (t) { const messages = ['world', 'hello'] - const a = await create() - const b = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) replicate(a, b, t) diff --git a/test/helpers/index.js b/test/helpers/index.js index de32ec91..2e1d4cad 100644 --- a/test/helpers/index.js +++ b/test/helpers/index.js @@ -1,24 +1,34 @@ const Hypercore = require('../../') -const RAM = require('random-access-memory') +const createTempDir = require('test-tmp') +const CoreStorage = require('hypercore-on-the-rocks') +const safetyCatch = require('safety-catch') -exports.create = async function create (...args) { - const core = new Hypercore(RAM, ...args) +exports.create = async function (t, ...args) { + const dir = await createTempDir(t) + + const db = new CoreStorage(dir) + + const core = new Hypercore(db, ...args) await core.ready() + + t.teardown(() => core.close(), { order: 1 }) + return core } -exports.createStored = function createStored () { - const files = new Map() +const createStorage = exports.createStorage = async function (t, dir) { + if (!dir) dir = await createTempDir(t) + return new CoreStorage(dir) +} - return function (...args) { - return new Hypercore(storage, ...args) - } +exports.createStored = async function (t) { + const dir = await createTempDir(t) + let db = null - function storage (name) { - if (files.has(name)) return files.get(name).clone() - const st = new RAM() - files.set(name, st) - return st + return async function (...args) { + if (db) await db.close() + db = await createStorage(t, dir) + return new Hypercore(db, ...args) } } @@ -29,8 +39,14 @@ exports.replicate = function replicate (a, b, t, opts = {}) { const closed1 = new Promise(resolve => s1.once('close', resolve)) const closed2 = new Promise(resolve => s2.once('close', resolve)) - s1.on('error', err => t.comment(`replication stream error (initiator): ${err}`)) - s2.on('error', err => t.comment(`replication stream error (responder): ${err}`)) + s1.on('error', err => { + safetyCatch(err) + t.comment(`replication stream error (initiator): ${err}`) + }) + s2.on('error', err => { + safetyCatch(err) + t.comment(`replication stream error (responder): ${err}`) + }) if (opts.teardown !== false) { t.teardown(async function () { diff --git a/test/manifest.js b/test/manifest.js index f7c0cf3e..c3d89e1f 100644 --- a/test/manifest.js +++ b/test/manifest.js @@ -2,7 +2,6 @@ const test = require('brittle') const crypto = require('hypercore-crypto') const b4a = require('b4a') const tmpDir = require('test-tmp') -const ram = require('random-access-memory') const c = require('compact-encoding') const Hypercore = require('../') @@ -11,7 +10,7 @@ const { assemble, partialSignature, signableLength } = require('../lib/multisig' const caps = require('../lib/caps') const enc = require('../lib/messages') -const { replicate, unreplicate } = require('./helpers') +const { create, createStorage, createStored, replicate, unreplicate } = require('./helpers') // TODO: move this to be actual tree batches instead - less future surprises // for now this is just to get the tests to work as they test important things @@ -258,15 +257,14 @@ test('create verifier - compat signer', async function (t) { test('multisig - append', async function (t) { const signers = [] - for (let i = 0; i < 3; i++) signers.push(new Hypercore(ram, { compat: false })) + for (let i = 0; i < 3; i++) signers.push(await create(t, { compat: false })) await Promise.all(signers.map(s => s.ready())) const manifest = createMultiManifest(signers, 0) let multisig = null - const core = new Hypercore(ram, { manifest }) - await core.ready() + const core = await create(t, { manifest }) t.alike(Hypercore.key(manifest), core.key) @@ -277,7 +275,9 @@ test('multisig - append', async function (t) { t.is(len, 1) - const batch = await core.batch() + const batch = await core.session({ name: 'batch' }) + batch.keyPair = null + await batch.append(b4a.from('0')) const sigBatch = batch.createTreeBatch() @@ -294,7 +294,7 @@ test('multisig - append', async function (t) { t.is(core.length, 1) - const core2 = new Hypercore(ram, { manifest }) + const core2 = await create(t, { manifest }) const s1 = core.replicate(true) const s2 = core2.replicate(false) @@ -315,11 +315,13 @@ test('multisig - append', async function (t) { await core2.download({ start: 0, end: core.length }).downloaded() t.alike(await core2.get(0), b4a.from('0')) + + await batch.close() }) test('multisig - batch failed', async function (t) { const signers = [] - for (let i = 0; i < 3; i++) signers.push(new Hypercore(ram, { compat: false })) + for (let i = 0; i < 3; i++) signers.push(await create(t, { compat: false })) await Promise.all(signers.map(s => s.ready())) @@ -327,8 +329,7 @@ test('multisig - batch failed', async function (t) { const manifest = createMultiManifest(signers) - const core = new Hypercore(ram, { manifest }) - await core.ready() + const core = await create(t, { manifest }) t.alike(Hypercore.key(manifest), core.key) @@ -339,7 +340,9 @@ test('multisig - batch failed', async function (t) { t.is(len, 1) - const batch = await core.batch() + const batch = await core.session({ name: 'batch' }) + batch.keyPair = null + await batch.append(b4a.from('0')) const sigBatch = batch.createTreeBatch() @@ -354,7 +357,7 @@ test('multisig - batch failed', async function (t) { await t.execution(core.append(b4a.from('hello'), { signature: multisig })) - const core2 = new Hypercore(ram, { manifest }) + const core2 = await create(t, { manifest }) const s1 = core.replicate(true) const s2 = core2.replicate(false) @@ -362,7 +365,7 @@ test('multisig - batch failed', async function (t) { const p = new Promise((resolve, reject) => { core2.on('verification-error', reject) - setImmediate(resolve) + setTimeout(resolve, 100) }) s1.pipe(s2).pipe(s1) @@ -370,18 +373,19 @@ test('multisig - batch failed', async function (t) { await t.exception(p) t.is(core2.length, 0) + + await batch.close() }) test('multisig - patches', async function (t) { const signers = [] - for (let i = 0; i < 3; i++) signers.push(new Hypercore(ram, { compat: false })) + for (let i = 0; i < 3; i++) signers.push(await create(t, { compat: false })) await Promise.all(signers.map(s => s.ready())) const manifest = createMultiManifest(signers) let multisig = null - const core = new Hypercore(ram, { manifest }) - await core.ready() + const core = await create(t, { manifest }) await signers[0].append(b4a.from('0')) await signers[0].append(b4a.from('1')) @@ -404,7 +408,7 @@ test('multisig - patches', async function (t) { t.is(core.length, 1) - const core2 = new Hypercore(ram, { manifest }) + const core2 = await create(t, { manifest }) const s1 = core.replicate(true) const s2 = core2.replicate(false) @@ -418,6 +422,7 @@ test('multisig - patches', async function (t) { s1.pipe(s2).pipe(s1) + await p await t.execution(p) t.is(core2.length, core.length) @@ -429,14 +434,13 @@ test('multisig - patches', async function (t) { test('multisig - batch append', async function (t) { const signers = [] - for (let i = 0; i < 3; i++) signers.push(new Hypercore(ram, { compat: false })) + for (let i = 0; i < 3; i++) signers.push(await create(t, { compat: false })) await Promise.all(signers.map(s => s.ready())) const manifest = createMultiManifest(signers) let multisig = null - const core = new Hypercore(ram, { manifest }) - await core.ready() + const core = await create(t, { manifest }) await signers[0].append(b4a.from('0')) await signers[0].append(b4a.from('1')) @@ -468,7 +472,7 @@ test('multisig - batch append', async function (t) { t.is(core.length, 4) - const core2 = new Hypercore(ram, { manifest }) + const core2 = await create(t, { manifest }) const s1 = core.replicate(true) const s2 = core2.replicate(false) @@ -496,14 +500,13 @@ test('multisig - batch append', async function (t) { test('multisig - batch append with patches', async function (t) { const signers = [] - for (let i = 0; i < 3; i++) signers.push(new Hypercore(ram, { compat: false })) + for (let i = 0; i < 3; i++) signers.push(await create(t, { compat: false })) await Promise.all(signers.map(s => s.ready())) const manifest = createMultiManifest(signers) let multisig = null - const core = new Hypercore(ram, { manifest }) - await core.ready() + const core = await create(t, { manifest }) t.alike(Hypercore.key(manifest), core.key) @@ -539,7 +542,7 @@ test('multisig - batch append with patches', async function (t) { t.is(core.length, 4) - const core2 = new Hypercore(ram, { manifest }) + const core2 = await create(t, { manifest }) const s1 = core.replicate(true) const s2 = core2.replicate(false) @@ -567,15 +570,14 @@ test('multisig - batch append with patches', async function (t) { test('multisig - cannot divide batch', async function (t) { const signers = [] - for (let i = 0; i < 3; i++) signers.push(new Hypercore(ram, { compat: false })) + for (let i = 0; i < 3; i++) signers.push(await create(t, { compat: false })) await Promise.all(signers.map(s => s.ready())) const manifest = createMultiManifest(signers) let multisig = null - const core = new Hypercore(ram, { manifest }) - await core.ready() + const core = await create(t, { manifest }) await signers[0].append(b4a.from('0')) await signers[0].append(b4a.from('1')) @@ -603,7 +605,7 @@ test('multisig - cannot divide batch', async function (t) { signature: multisig })) - const core2 = new Hypercore(ram, { manifest }) + const core2 = await create(t, { manifest }) const s1 = core.replicate(true) const s2 = core2.replicate(false) @@ -624,7 +626,7 @@ test('multisig - cannot divide batch', async function (t) { test('multisig - multiple appends', async function (t) { const signers = [] - for (let i = 0; i < 3; i++) signers.push(new Hypercore(ram, { compat: false })) + for (let i = 0; i < 3; i++) signers.push(await create(t, { compat: false })) await Promise.all(signers.map(s => s.ready())) const manifest = createMultiManifest(signers) @@ -632,8 +634,7 @@ test('multisig - multiple appends', async function (t) { let multisig1 = null let multisig2 = null - const core = new Hypercore(ram, { manifest }) - await core.ready() + const core = await create(t, { manifest }) await signers[0].append(b4a.from('0')) await signers[0].append(b4a.from('1')) @@ -666,7 +667,7 @@ test('multisig - multiple appends', async function (t) { await partialCoreSignature(core, signers[1], len) ]) - const core2 = new Hypercore(ram, { manifest }) + const core2 = await create(t, { manifest }) const s1 = core.replicate(true) const s2 = core2.replicate(false) @@ -709,10 +710,11 @@ test('multisig - multiple appends', async function (t) { }) test('multisig - persist to disk', async function (t) { - const storage = await tmpDir(t) + const dir = await tmpDir(t) + const storage = await createStorage(t, dir) const signers = [] - for (let i = 0; i < 3; i++) signers.push(new Hypercore(ram, { compat: false })) + for (let i = 0; i < 3; i++) signers.push(await create(t, { compat: false })) await Promise.all(signers.map(s => s.ready())) const manifest = createMultiManifest(signers) @@ -739,11 +741,12 @@ test('multisig - persist to disk', async function (t) { t.is(core.length, 1) await core.close() + await storage.close() - const clone = new Hypercore(storage, { manifest }) + const clone = new Hypercore(await createStorage(t, dir), { manifest }) await t.execution(clone.ready()) - const core2 = new Hypercore(ram, { manifest }) + const core2 = await create(t, { manifest }) const s1 = clone.replicate(true) const s2 = core2.replicate(false) @@ -770,7 +773,7 @@ test('multisig - persist to disk', async function (t) { test('multisig - overlapping appends', async function (t) { const signers = [] - for (let i = 0; i < 3; i++) signers.push(new Hypercore(ram, { compat: false })) + for (let i = 0; i < 3; i++) signers.push(await create(t, { compat: false })) await Promise.all(signers.map(s => s.ready())) @@ -779,10 +782,9 @@ test('multisig - overlapping appends', async function (t) { let multisig1 = null let multisig2 = null - const core = new Hypercore(ram, { manifest }) - await core.ready() + const core = await create(t, { manifest }) - const core2 = new Hypercore(ram, { manifest }) + const core2 = await create(t, { manifest }) await core.ready() await signers[0].append(b4a.from('0')) @@ -854,9 +856,9 @@ test('multisig - normal operating mode', async function (t) { for (let i = 0; i < 0xff; i++) inputs.push(b4a.from([i])) const signers = [] - signers.push(new Hypercore(ram, { compat: false })) - signers.push(new Hypercore(ram, { compat: false })) - signers.push(new Hypercore(ram, { compat: false })) + signers.push(await create(t, { compat: false })) + signers.push(await create(t, { compat: false })) + signers.push(await create(t, { compat: false })) const [a, b, d] = signers @@ -866,10 +868,9 @@ test('multisig - normal operating mode', async function (t) { const signer1 = signer(a, b) const signer2 = signer(b, d) - const core = new Hypercore(ram, { manifest, sign: signer1.sign }) - await core.ready() + const core = await create(t, { manifest, sign: signer1.sign }) - const core2 = new Hypercore(ram, { manifest, sign: signer2.sign }) + const core2 = await create(t, { manifest, sign: signer2.sign }) await core.ready() let ai = 0 @@ -948,15 +949,14 @@ test('multisig - normal operating mode', async function (t) { test('multisig - large patches', async function (t) { const signers = [] - for (let i = 0; i < 3; i++) signers.push(new Hypercore(ram, { compat: false })) + for (let i = 0; i < 3; i++) signers.push(await create(t, { compat: false })) await Promise.all(signers.map(s => s.ready())) const manifest = createMultiManifest(signers) let multisig = null - const core = new Hypercore(ram, { manifest }) - await core.ready() + const core = await create(t, { manifest }) for (let i = 0; i < 10000; i++) { await signers[0].append(b4a.from(i.toString(10))) @@ -976,7 +976,7 @@ test('multisig - large patches', async function (t) { t.is(core.length, 1) - const core2 = new Hypercore(ram, { manifest }) + const core2 = await create(t, { manifest }) const s1 = core.replicate(true) const s2 = core2.replicate(false) @@ -1034,7 +1034,7 @@ test('multisig - large patches', async function (t) { test('multisig - prologue', async function (t) { const signers = [] - for (let i = 0; i < 2; i++) signers.push(new Hypercore(ram, { compat: false })) + for (let i = 0; i < 2; i++) signers.push(await create(t, { compat: false })) await Promise.all(signers.map(s => s.ready())) await signers[0].append(b4a.from('0')) @@ -1047,10 +1047,9 @@ test('multisig - prologue', async function (t) { let multisig = null - const core = new Hypercore(ram, { manifest }) - await core.ready() + const core = await create(t, { manifest }) - const prologued = new Hypercore(ram, { manifest: manifestWithPrologue }) + const prologued = await create(t, { manifest: manifestWithPrologue }) await prologued.ready() await signers[1].append(b4a.from('0')) @@ -1088,7 +1087,7 @@ test('multisig - prologue', async function (t) { test('multisig - prologue replicate', async function (t) { const signers = [] - for (let i = 0; i < 2; i++) signers.push(new Hypercore(ram, { compat: false })) + for (let i = 0; i < 2; i++) signers.push(await create(t, { compat: false })) await Promise.all(signers.map(s => s.ready())) await signers[0].append(b4a.from('0')) @@ -1100,10 +1099,9 @@ test('multisig - prologue replicate', async function (t) { let multisig = null - const core = new Hypercore(ram, { manifest }) - await core.ready() + const core = await create(t, { manifest }) - const remote = new Hypercore(ram, { manifest }) + const remote = await create(t, { manifest }) await remote.ready() await signers[1].append(b4a.from('0')) @@ -1133,7 +1131,7 @@ test('multisig - prologue replicate', async function (t) { test('multisig - prologue verify hash', async function (t) { const signers = [] - for (let i = 0; i < 2; i++) signers.push(new Hypercore(ram, { compat: false })) + for (let i = 0; i < 2; i++) signers.push(await create(t, { compat: false })) await Promise.all(signers.map(s => s.ready())) await signers[0].append(b4a.from('0')) @@ -1143,19 +1141,22 @@ test('multisig - prologue verify hash', async function (t) { const manifest = createMultiManifest(signers, { hash, length: 2 }) - const core = new Hypercore(ram, { manifest }) - await core.ready() + const core = await create(t, { manifest }) t.is(core.length, 0) - const proof = await signers[0].core.tree.proof({ upgrade: { start: 0, length: 2 } }) + const batch = signers[0].core.storage.createReadBatch() + const p = await signers[0].core.tree.proof(batch, { upgrade: { start: 0, length: 2 } }) + batch.tryFlush() + + const proof = await p.settle() proof.upgrade.signature = null await t.execution(core.core.verify(proof)) t.is(core.length, 2) - const remote = new Hypercore(ram, { manifest }) + const remote = await create(t, { manifest }) await remote.ready() t.is(core.length, 2) @@ -1178,7 +1179,7 @@ test('multisig - prologue morphs request', async function (t) { let multisig = null - for (let i = 0; i < 2; i++) signers.push(new Hypercore(ram, { compat: false })) + for (let i = 0; i < 2; i++) signers.push(await create(t, { compat: false })) await Promise.all(signers.map(s => s.ready())) await signers[0].append(b4a.from('0')) @@ -1196,12 +1197,15 @@ test('multisig - prologue morphs request', async function (t) { const hash = b4a.from(signers[0].core.tree.hash()) const manifest = createMultiManifest(signers, { hash, length: 4 }) - const core = new Hypercore(ram, { manifest }) - await core.ready() + const core = await create(t, { manifest }) t.is(core.length, 0) - const proof = await signers[0].core.tree.proof({ upgrade: { start: 0, length: 4 } }) + const batch = signers[0].core.storage.createReadBatch() + const p = await signers[0].core.tree.proof(batch, { upgrade: { start: 0, length: 4 } }) + batch.tryFlush() + + const proof = await p.settle() proof.upgrade.signature = null await t.execution(core.core.verify(proof)) @@ -1220,7 +1224,7 @@ test('multisig - prologue morphs request', async function (t) { t.is(core.length, 5) - const remote = new Hypercore(ram, { manifest }) + const remote = await create(t, { manifest }) await remote.ready() t.is(core.length, 5) @@ -1239,12 +1243,16 @@ test('multisig - prologue morphs request', async function (t) { t.is(remote.length, 5) - await t.execution(remote.core.tree.proof({ upgrade: { start: 0, length: 4 } })) + const rb = remote.core.storage.createReadBatch() + const rp = await remote.core.tree.proof(rb, { upgrade: { start: 0, length: 4 } }) + rb.tryFlush() + + await t.execution(rp.settle()) }) test('multisig - append/truncate before prologue', async function (t) { const signers = [] - for (let i = 0; i < 2; i++) signers.push(new Hypercore(ram, { compat: false })) + for (let i = 0; i < 2; i++) signers.push(await create(t, { compat: false })) await Promise.all(signers.map(s => s.ready())) await signers[0].append(b4a.from('0')) @@ -1259,8 +1267,7 @@ test('multisig - append/truncate before prologue', async function (t) { let multisig = null let partialMultisig = null - const core = new Hypercore(ram, { manifest }) - await core.ready() + const core = await create(t, { manifest }) const proof = await partialSignature(signers[0].core.tree, 0, 2) const proof2 = await partialSignature(signers[1].core.tree, 1, 2) @@ -1433,8 +1440,8 @@ test('create verifier - open existing core with manifest', async function (t) { const key = Verifier.manifestHash(manifest) - const storage = ram.reusable() - const core = new Hypercore(storage, key, { compat: false }) + const create = await createStored(t) + const core = await create(key, { compat: false }) await core.ready() t.is(core.manifest, null) @@ -1445,12 +1452,12 @@ test('create verifier - open existing core with manifest', async function (t) { manifest.signers[0].publicKey = b4a.alloc(32, 0) - const wrongCore = new Hypercore(storage, null, { manifest, compat: false }) + const wrongCore = await create(null, { manifest, compat: false }) await t.exception(wrongCore.ready(), /STORAGE_CONFLICT/) manifest.signers[0].publicKey = keyPair.publicKey - const manifestCore = new Hypercore(storage, null, { manifest, compat: false }) + const manifestCore = await create(null, { manifest, compat: false }) await manifestCore.ready() t.not(manifestCore.manifest, null) @@ -1459,8 +1466,10 @@ test('create verifier - open existing core with manifest', async function (t) { await manifestCore.close() - const compatCore = new Hypercore(storage, null, { manifest, compat: true }) + const compatCore = await create(null, { manifest, compat: true }) await t.execution(compatCore.ready()) // compat flag is unset internally + + await compatCore.close() }) function createMultiManifest (signers, prologue = null) { diff --git a/test/merkle-tree.js b/test/merkle-tree.js index 998e705c..e529b611 100644 --- a/test/merkle-tree.js +++ b/test/merkle-tree.js @@ -1,13 +1,11 @@ -const path = require('path') const test = require('brittle') -const RAM = require('random-access-memory') const b4a = require('b4a') -const RandomAccessFile = require('random-access-file') const createTempDir = require('test-tmp') +const CoreStorage = require('hypercore-on-the-rocks') const Tree = require('../lib/merkle-tree') test('nodes', async function (t) { - const tree = await create() + const { storage, tree } = await create(t) const b = tree.batch() @@ -15,7 +13,9 @@ test('nodes', async function (t) { b.append(b4a.from([i])) } - b.commit() + const wb = storage.createWriteBatch() + await b.commit(wb) + await wb.flush() t.is(await tree.nodes(0), 0) @@ -23,12 +23,17 @@ test('nodes', async function (t) { }) test('proof only block', async function (t) { - const tree = await create(10) + const { storage, tree } = await create(t, 10) - const proof = await tree.proof({ + const b = storage.createReadBatch() + const p = await tree.proof(b, { block: { index: 4, nodes: 2 } }) + b.tryFlush() + + const proof = await p.settle() + t.is(proof.upgrade, null) t.is(proof.seek, null) t.is(proof.block.index, 4) @@ -37,13 +42,17 @@ test('proof only block', async function (t) { }) test('proof with upgrade', async function (t) { - const tree = await create(10) + const { storage, tree } = await create(t, 10) - const proof = await tree.proof({ + const b = storage.createReadBatch() + const p = await tree.proof(b, { block: { index: 4, nodes: 0 }, upgrade: { start: 0, length: 10 } }) + b.tryFlush() + const proof = await p.settle() + t.is(proof.seek, null) t.is(proof.block.index, 4) t.is(proof.block.nodes.length, 3) @@ -55,13 +64,17 @@ test('proof with upgrade', async function (t) { }) test('proof with upgrade + additional', async function (t) { - const tree = await create(10) + const { storage, tree } = await create(t, 10) - const proof = await tree.proof({ + const b = storage.createReadBatch() + const p = await tree.proof(b, { block: { index: 4, nodes: 0 }, upgrade: { start: 0, length: 8 } }) + b.tryFlush() + const proof = await p.settle() + t.is(proof.seek, null) t.is(proof.block.index, 4) t.is(proof.block.nodes.length, 3) @@ -73,13 +86,17 @@ test('proof with upgrade + additional', async function (t) { }) test('proof with upgrade from existing state', async function (t) { - const tree = await create(10) + const { storage, tree } = await create(t, 10) - const proof = await tree.proof({ + const b = storage.createReadBatch() + const p = await tree.proof(b, { block: { index: 1, nodes: 0 }, upgrade: { start: 1, length: 9 } }) + b.tryFlush() + const proof = await p.settle() + t.is(proof.seek, null) t.is(proof.block.index, 1) t.is(proof.block.nodes.length, 0) @@ -91,13 +108,17 @@ test('proof with upgrade from existing state', async function (t) { }) test('proof with upgrade from existing state + additional', async function (t) { - const tree = await create(10) + const { storage, tree } = await create(t, 10) - const proof = await tree.proof({ + const b = storage.createReadBatch() + const p = await tree.proof(b, { block: { index: 1, nodes: 0 }, upgrade: { start: 1, length: 5 } }) + b.tryFlush() + const proof = await p.settle() + t.is(proof.seek, null) t.is(proof.block.index, 1) t.is(proof.block.nodes.length, 0) @@ -109,13 +130,17 @@ test('proof with upgrade from existing state + additional', async function (t) { }) test('proof block and seek, no upgrade', async function (t) { - const tree = await create(10) + const { storage, tree } = await create(t, 10) - const proof = await tree.proof({ + const b = storage.createReadBatch() + const p = await tree.proof(b, { seek: { bytes: 8, padding: 0 }, block: { index: 4, nodes: 2 } }) + b.tryFlush() + const proof = await p.settle() + t.is(proof.upgrade, null) t.is(proof.seek, null) // seek included in the block t.is(proof.block.index, 4) @@ -124,13 +149,17 @@ test('proof block and seek, no upgrade', async function (t) { }) test('proof block and seek #2, no upgrade', async function (t) { - const tree = await create(10) + const { storage, tree } = await create(t, 10) - const proof = await tree.proof({ + const b = storage.createReadBatch() + const p = await tree.proof(b, { seek: { bytes: 10, padding: 0 }, block: { index: 4, nodes: 2 } }) + b.tryFlush() + const proof = await p.settle() + t.is(proof.upgrade, null) t.is(proof.seek, null) // seek included in the block t.is(proof.block.index, 4) @@ -139,13 +168,17 @@ test('proof block and seek #2, no upgrade', async function (t) { }) test('proof block and seek #3, no upgrade', async function (t) { - const tree = await create(10) + const { storage, tree } = await create(t, 10) - const proof = await tree.proof({ + const b = storage.createReadBatch() + const p = await tree.proof(b, { seek: { bytes: 13, padding: 0 }, block: { index: 4, nodes: 2 } }) + b.tryFlush() + const proof = await p.settle() + t.is(proof.upgrade, null) t.alike(proof.seek.nodes.map(n => n.index), [12, 14]) t.is(proof.block.index, 4) @@ -154,40 +187,52 @@ test('proof block and seek #3, no upgrade', async function (t) { }) test('proof seek with padding, no upgrade', async function (t) { - const tree = await create(16) + const { storage, tree } = await create(t, 16) - const proof = await tree.proof({ + const b = storage.createReadBatch() + const p = await tree.proof(b, { seek: { bytes: 7, padding: 1 }, block: { index: 0, nodes: 4 } }) + b.tryFlush() + const proof = await p.settle() + t.is(proof.upgrade, null) t.alike(proof.block.nodes.map(n => n.index), [2, 5, 23]) t.alike(proof.seek.nodes.map(n => n.index), [12, 14, 9]) }) test('proof block and seek that results in tree, no upgrade', async function (t) { - const tree = await create(16) + const { storage, tree } = await create(t, 16) - const proof = await tree.proof({ + const b = storage.createReadBatch() + const p = await tree.proof(b, { seek: { bytes: 26, padding: 0 }, block: { index: 0, nodes: 4 } }) + b.tryFlush() + const proof = await p.settle() + t.is(proof.upgrade, null) t.alike(proof.block.nodes.map(n => n.index), [2, 5, 11]) t.alike(proof.seek.nodes.map(n => n.index), [19, 27]) }) test('proof block and seek, with upgrade', async function (t) { - const tree = await create(10) + const { storage, tree } = await create(t, 10) - const proof = await tree.proof({ + const b = storage.createReadBatch() + const p = await tree.proof(b, { seek: { bytes: 13, padding: 0 }, block: { index: 4, nodes: 2 }, upgrade: { start: 8, length: 2 } }) + b.tryFlush() + const proof = await p.settle() + t.alike(proof.seek.nodes.map(n => n.index), [12, 14]) t.is(proof.block.index, 4) t.is(proof.block.nodes.length, 1) @@ -199,13 +244,17 @@ test('proof block and seek, with upgrade', async function (t) { }) test('proof seek with upgrade', async function (t) { - const tree = await create(10) + const { storage, tree } = await create(t, 10) - const proof = await tree.proof({ + const b = storage.createReadBatch() + const p = await tree.proof(b, { seek: { bytes: 13, padding: 0 }, upgrade: { start: 0, length: 10 } }) + b.tryFlush() + const proof = await p.settle() + t.alike(proof.seek.nodes.map(n => n.index), [12, 14, 9, 3]) t.is(proof.block, null) t.is(proof.upgrade.start, 0) @@ -215,204 +264,263 @@ test('proof seek with upgrade', async function (t) { }) test('verify proof #1', async function (t) { - const tree = await create(10) - const clone = await create() + const { storage, tree } = await create(t, 10) + const clone = await create(t) - const p = await tree.proof({ + const batch = storage.createReadBatch() + const proof = await tree.proof(batch, { hash: { index: 6, nodes: 0 }, upgrade: { start: 0, length: 10 } }) - const b = await clone.verify(p) - b.commit() + batch.tryFlush() + const p = await proof.settle() + + const b = await clone.tree.verify(p) - t.is(clone.length, tree.length) - t.is(clone.byteLength, tree.byteLength) - t.is(await clone.byteOffset(6), await tree.byteOffset(6)) - t.is(await clone.get(6), await tree.get(6)) + const wb = clone.storage.createWriteBatch() + b.commit(wb) + + await wb.flush() + b.finalise() + + t.is(clone.tree.length, tree.length) + t.is(clone.tree.byteLength, tree.byteLength) + t.is(await clone.tree.byteOffset(6), await tree.byteOffset(6)) + t.alike(await clone.tree.get(6), await tree.get(6)) }) test('verify proof #2', async function (t) { - const tree = await create(10) - const clone = await create() + const { storage, tree } = await create(t, 10) + const clone = await create(t) - const p = await tree.proof({ + const batch = storage.createReadBatch() + const proof = await tree.proof(batch, { seek: { bytes: 10, padding: 0 }, upgrade: { start: 0, length: 10 } }) - const b = await clone.verify(p) - b.commit() + batch.tryFlush() + const p = await proof.settle() - t.is(clone.length, tree.length) - t.is(clone.byteLength, tree.byteLength) - t.alike(await clone.byteRange(10), await tree.byteRange(10)) + const b = await clone.tree.verify(p) + const wb = clone.storage.createWriteBatch() + await b.commit(wb) + await wb.flush() + b.finalise() + + t.is(clone.tree.length, tree.length) + t.is(clone.tree.byteLength, tree.byteLength) + t.alike(await clone.tree.byteRange(10), await tree.byteRange(10)) }) test('upgrade edgecase when no roots need upgrade', async function (t) { - const tree = await create(4) - const clone = await create() + const { tree, storage } = await create(t, 4) + const clone = await create(t) { - const proof = await tree.proof({ + const batch = storage.createReadBatch() + const p = await tree.proof(batch, { upgrade: { start: 0, length: 4 } }) - const b = await clone.verify(proof) - b.commit() + batch.tryFlush() + const proof = await p.settle() + + const b = await clone.tree.verify(proof) + const wb = clone.storage.createWriteBatch() + await b.commit(wb) + await wb.flush() + b.finalise() } const b = tree.batch() b.append(b4a.from('#5')) - b.commit() + const wb = storage.createWriteBatch() + await b.commit(wb) + await wb.flush() + b.finalise() { - const proof = await tree.proof({ + const batch = storage.createReadBatch() + const p = await tree.proof(batch, { upgrade: { start: 4, length: 1 } }) - const b = await clone.verify(proof) - b.commit() + batch.tryFlush() + const proof = await p.settle() + + const b = await clone.tree.verify(proof) + const wb = clone.tree.storage.createWriteBatch() + await b.commit(wb) + await wb.flush() + b.finalise() } t.is(tree.length, 5) }) test('lowest common ancestor - small gap', async function (t) { - const tree = await create(10) - const clone = await create(8) - const ancestors = await reorg(clone, tree) + const { storage, tree } = await create(t, 10) + const clone = await create(t, 8) + const ancestors = await reorg(clone, { tree, storage }) t.is(ancestors, 8) - t.is(clone.length, tree.length) + t.is(clone.tree.length, tree.length) }) test('lowest common ancestor - bigger gap', async function (t) { - const tree = await create(20) - const clone = await create(1) - const ancestors = await reorg(clone, tree) + const { storage, tree } = await create(t, 20) + const clone = await create(t, 1) + const ancestors = await reorg(clone, { tree, storage }) t.is(ancestors, 1) - t.is(clone.length, tree.length) + t.is(clone.tree.length, tree.length) }) test('lowest common ancestor - remote is shorter than local', async function (t) { - const tree = await create(5) - const clone = await create(10) - const ancestors = await reorg(clone, tree) + const { storage, tree } = await create(t, 5) + const clone = await create(t, 10) + const ancestors = await reorg(clone, { tree, storage }) t.is(ancestors, 5) - t.is(clone.length, tree.length) + t.is(clone.tree.length, tree.length) }) test('lowest common ancestor - simple fork', async function (t) { - const tree = await create(5) - const clone = await create(5) + const { tree, storage } = await create(t, 5) + const clone = await create(t, 5) { const b = tree.batch() b.append(b4a.from('fork #1')) - b.commit() + const wb = storage.createWriteBatch() + await b.commit(wb) + await wb.flush() + b.finalise() } { - const b = clone.batch() + const b = clone.tree.batch() b.append(b4a.from('fork #2')) - b.commit() + const wb = clone.storage.createWriteBatch() + await b.commit(wb) + await wb.flush() + b.finalise() } - const ancestors = await reorg(clone, tree) + const ancestors = await reorg(clone, { tree, storage }) t.is(ancestors, 5) - t.is(clone.length, tree.length) + t.is(clone.tree.length, tree.length) }) test('lowest common ancestor - long fork', async function (t) { - const tree = await create(5) - const clone = await create(5) + const { tree, storage } = await create(t, 5) + const clone = await create(t, 5) { const b = tree.batch() b.append(b4a.from('fork #1')) - b.commit() + const wb = storage.createWriteBatch() + await b.commit(wb) + await wb.flush() + b.finalise() } { - const b = clone.batch() + const b = clone.tree.batch() b.append(b4a.from('fork #2')) - b.commit() + const wb = clone.storage.createWriteBatch() + await b.commit(wb) + await wb.flush() + b.finalise() } { const b = tree.batch() for (let i = 0; i < 100; i++) b.append(b4a.from('#' + i)) - b.commit() + const wb = storage.createWriteBatch() + await b.commit(wb) + await wb.flush() + b.finalise() } { - const b = clone.batch() + const b = clone.tree.batch() for (let i = 0; i < 100; i++) b.append(b4a.from('#' + i)) - b.commit() + const wb = clone.storage.createWriteBatch() + await b.commit(wb) + await wb.flush() + b.finalise() } - const ancestors = await reorg(clone, tree) + const ancestors = await reorg(clone, { tree, storage }) t.is(ancestors, 5) - t.is(clone.length, tree.length) + t.is(clone.tree.length, tree.length) t.ok(await audit(tree)) - await tree.flush() t.ok(await audit(tree)) }) test('tree hash', async function (t) { - const a = await create(5) - const b = await create(5) + const a = await create(t, 5) + const b = await create(t, 5) - t.alike(a.hash(), b.hash()) + t.alike(a.tree.hash(), b.tree.hash()) { - const b = a.batch() - t.alike(b.hash(), a.hash()) + const b = a.tree.batch() + t.alike(b.hash(), a.tree.hash()) b.append(b4a.from('hi')) const h = b.hash() - t.unlike(h, a.hash()) - b.commit() - t.alike(h, a.hash()) + t.unlike(h, a.tree.hash()) + const wb = a.storage.createWriteBatch() + await b.commit(wb) + await wb.flush() + b.finalise() + t.alike(h, a.tree.hash()) } { - const ba = b.batch() + const ba = b.tree.batch() ba.append(b4a.from('hi')) const h = ba.hash() - t.unlike(h, b.hash()) - t.alike(h, a.hash()) - ba.commit() - t.alike(h, b.hash()) + t.unlike(h, b.tree.hash()) + t.alike(h, a.tree.hash()) + const wba = b.storage.createWriteBatch() + await ba.commit(wba) + await wba.flush() + ba.finalise() + t.alike(h, b.tree.hash()) } }) test('basic tree seeks', async function (t) { - const a = await create(5) + const a = await create(t, 5) { - const b = a.batch() + const b = a.tree.batch() b.append(b4a.from('bigger')) b.append(b4a.from('block')) b.append(b4a.from('tiny')) b.append(b4a.from('s')) b.append(b4a.from('another')) - b.commit() + const wb = a.storage.createWriteBatch() + await b.commit(wb) + await wb.flush() + b.finalise() } - t.is(a.length, 10) - t.is(a.byteLength, 33) + t.is(a.tree.length, 10) + t.is(a.tree.byteLength, 33) for (let i = 0; i < a.byteLength; i++) { - const s = a.seek(i) + const s = a.tree.seek(i) const actual = await s.update() - const expected = await linearSeek(a, i) + const expected = await linearSeek(a.tree, i) if (actual[0] !== expected[0] || actual[1] !== expected[1]) { t.is(actual, expected, 'bad seek at ' + i) @@ -433,16 +541,18 @@ test('basic tree seeks', async function (t) { }) test('clear full tree', async function (t) { - const a = await create(5) + const a = await create(t, 5) - t.is(a.length, 5) + t.is(a.tree.length, 5) - await a.clear() + const w = a.storage.createWriteBatch() + a.tree.clear(w) + await w.flush() - t.is(a.length, 0) + t.is(a.tree.length, 0) try { - await a.get(2) + await a.tree.get(2) t.fail('node should not exist now') } catch { t.pass('node should fail') @@ -450,46 +560,52 @@ test('clear full tree', async function (t) { }) test('get older roots', async function (t) { - const a = await create(5) + const a = await create(t, 5) - const roots = await a.getRoots(5) - t.alike(roots, a.roots, 'same roots') + const roots = await a.tree.getRoots(5) + t.alike(roots, a.tree.roots, 'same roots') { - const b = a.batch() + const b = a.tree.batch() b.append(b4a.from('next')) b.append(b4a.from('next')) b.append(b4a.from('next')) - b.commit() + const wb = a.storage.createWriteBatch() + await b.commit(wb) + await wb.flush() + b.finalise() } - const oldRoots = await a.getRoots(5) + const oldRoots = await a.tree.getRoots(5) t.alike(oldRoots, roots, 'same old roots') const expected = [] - const len = a.length + const len = a.tree.length for (let i = 0; i < 40; i++) { - expected.push([...a.roots]) + expected.push([...a.tree.roots]) { - const b = a.batch() + const b = a.tree.batch() b.append(b4a.from('tick')) - b.commit() + const wb = a.storage.createWriteBatch() + await b.commit(wb) + await wb.flush() + b.finalise() } } const actual = [] for (let i = 0; i < 40; i++) { - actual.push(await a.getRoots(len + i)) + actual.push(await a.tree.getRoots(len + i)) } t.alike(actual, expected, 'check a bunch of different roots') }) test('check if a length is upgradeable', async function (t) { - const tree = await create(5) - const clone = await create() + const { storage, tree } = await create(t, 5) + const clone = await create(t) // Full clone, has it all @@ -500,12 +616,19 @@ test('check if a length is upgradeable', async function (t) { t.is(await tree.upgradeable(4), true) t.is(await tree.upgradeable(5), true) - const p = await tree.proof({ + const batch = storage.createReadBatch() + const proof = await tree.proof(batch, { upgrade: { start: 0, length: 5 } }) - const b = await clone.verify(p) - b.commit() + batch.tryFlush() + const p = await proof.settle() + + const b = await clone.tree.verify(p) + const wb = clone.storage.createWriteBatch() + await b.commit(wb) + await wb.flush() + b.finalise() /* Merkle tree looks like @@ -523,18 +646,18 @@ test('check if a length is upgradeable', async function (t) { So length = 0, length = 4 (node 3) and length = 5 (node 8 + 3) should be upgradeable */ - t.is(await clone.upgradeable(0), true) - t.is(await clone.upgradeable(1), false) - t.is(await clone.upgradeable(2), false) - t.is(await clone.upgradeable(3), false) - t.is(await clone.upgradeable(4), true) - t.is(await clone.upgradeable(5), true) + t.is(await clone.tree.upgradeable(0), true) + t.is(await clone.tree.upgradeable(1), false) + t.is(await clone.tree.upgradeable(2), false) + t.is(await clone.tree.upgradeable(3), false) + t.is(await clone.tree.upgradeable(4), true) + t.is(await clone.tree.upgradeable(5), true) }) test('clone a batch', async t => { - const a = await create(5) + const a = await create(t, 5) - const b = a.batch() + const b = a.tree.batch() const c = b.clone() t.is(b.fork, c.fork) @@ -556,7 +679,10 @@ test('clone a batch', async t => { b.append(b4a.from('s')) b.append(b4a.from('another')) - b.commit() + const wb = a.storage.createWriteBatch() + await b.commit(wb) + await wb.flush() + b.finalise() let same = b.roots.length === c.roots.length for (let i = 0; i < b.roots.length; i++) { @@ -569,8 +695,8 @@ test('clone a batch', async t => { }) test('prune nodes in a batch', async t => { - const a = await create(0) - const b = a.batch() + const a = await create(t, 0) + const b = a.tree.batch() for (let i = 0; i < 16; i++) { b.append(b4a.from('tick tock')) @@ -584,8 +710,8 @@ test('prune nodes in a batch', async t => { }) test('checkout nodes in a batch', async t => { - const a = await create(0) - const b = a.batch() + const a = await create(t, 0) + const b = a.tree.batch() for (let i = 0; i < 16; i++) { b.append(b4a.from('tick tock')) @@ -603,7 +729,7 @@ test('checkout nodes in a batch', async t => { }) test('roots get unslabbed', async function (t) { - const tree = await create() + const { tree, storage } = await create(t) const b = tree.batch() @@ -611,7 +737,10 @@ test('roots get unslabbed', async function (t) { b.append(b4a.from([i])) } - b.commit() + const wb = storage.createWriteBatch() + await b.commit(wb) + await wb.flush() + b.finalise() t.is(tree.roots.length > 1, true, 'sanity check') @@ -635,21 +764,17 @@ test('roots get unslabbed', async function (t) { ) }) -test('buffer of cached nodes is copied to small slab', async function (t) { +test.skip('buffer of cached nodes is copied to small slab', async function (t) { // RAM does not use slab-allocated memory, // so we need to us random-access-file to reproduce this issue - const dir = await createTempDir(t) - const storage = new RandomAccessFile(path.join(dir, 'tree')) - - const tree = await Tree.open(storage) + const { tree, storage } = await create(t) const b = tree.batch() b.append(b4a.from('tree-entry')) - b.commit() - - // Note: if the batch is committed but not yet flushed, - // it still uses the original slab-allocated memory - await tree.flush() + const wb = storage.createWriteBatch() + await b.commit(wb) + await wb.flush() + b.finalise() const node = await tree.get(0) t.is(node.hash.buffer.byteLength, 32, 'created a new memory slab of the correct (small) size') @@ -657,6 +782,36 @@ test('buffer of cached nodes is copied to small slab', async function (t) { await tree.close() }) +test('reopen a tree', async t => { + const dir = await createTempDir(t) + + const a = await create(t, 16, dir) + const b = a.tree.batch() + + for (let i = 0; i < 16; i++) { + b.append(b4a.from('#' + i)) + } + + const wb = a.storage.createWriteBatch() + await b.commit(wb) + await wb.flush() + b.finalise() + + t.alike(a.tree.length, 32) + + const byteLength = a.tree.byteLength + + t.alike(a.tree.roots.map(n => n.index), [31]) + + await a.storage.db.close() + + const a1 = await create(t, 0, dir) + + t.alike(a1.tree.length, 32) + t.alike(a1.tree.byteLength, byteLength) + t.alike(a1.tree.roots.map(n => n.index), [31]) +}) + async function audit (tree) { const flat = require('flat-tree') const expectedRoots = flat.fullRoots(tree.length * 2) @@ -684,26 +839,57 @@ async function audit (tree) { } async function reorg (local, remote) { - const upgrade = { start: 0, length: remote.length } - const r = await local.reorg(await remote.proof({ upgrade })) + const upgrade = { start: 0, length: remote.tree.length } + + const batch = remote.storage.createReadBatch() + const proof = await remote.tree.proof(batch, { upgrade }) + + batch.tryFlush() + const r = await local.tree.reorg(await proof.settle()) while (!r.finished) { const index = 2 * (r.want.end - 1) const nodes = r.want.nodes - await r.update(await remote.proof({ hash: { index, nodes } })) + const batch = remote.storage.createReadBatch() + const proof = await remote.tree.proof(batch, { hash: { index, nodes } }) + + batch.tryFlush() + + await r.update(await proof.settle()) } - r.commit() + const wb = local.storage.createWriteBatch() + r.commit(wb) + await wb.flush() + r.finalise() return r.ancestors } -async function create (length = 0) { - const tree = await Tree.open(new RAM()) +async function create (t, length = 0, dir) { + if (!dir) dir = await createTempDir(t) + + const db = new CoreStorage(dir) + + t.teardown(() => db.close()) + + const dkey = b4a.alloc(32) + + const storage = (await db.resume(dkey)) || (await db.create({ key: dkey, discoveryKey: dkey })) + + const tree = await Tree.open(storage) + + if (!length) return { storage, tree } + const b = tree.batch() for (let i = 0; i < length; i++) { b.append(b4a.from('#' + i)) } - b.commit() - return tree + + const wb = storage.createWriteBatch() + await b.commit(wb) + await wb.flush() + b.finalise() + + return { storage, tree } } diff --git a/test/preload.js b/test/preload.js index 9b91cf89..7d76433a 100644 --- a/test/preload.js +++ b/test/preload.js @@ -1,13 +1,15 @@ const crypto = require('hypercore-crypto') const test = require('brittle') -const RAM = require('random-access-memory') const b4a = require('b4a') const Hypercore = require('../') +const { create, createStorage } = require('./helpers') test('preload - storage', async function (t) { + const storage = await createStorage(t) + const core = new Hypercore(null, { preload: () => { - return { storage: RAM } + return { storage } } }) await core.ready() @@ -15,13 +17,14 @@ test('preload - storage', async function (t) { await core.append('hello world') t.is(core.length, 1) t.alike(await core.get(0), b4a.from('hello world')) + + await core.close() }) test('preload - from another core', async function (t) { t.plan(2) - const first = new Hypercore(RAM) - await first.ready() + const first = await create(t) const second = new Hypercore(null, { preload: () => { @@ -32,11 +35,15 @@ test('preload - from another core', async function (t) { t.alike(first.key, second.key) t.is(first.sessions, second.sessions) + + await second.close() }) test('preload - custom keypair', async function (t) { const keyPair = crypto.keyPair() - const core = new Hypercore(RAM, keyPair.publicKey, { + const storage = await createStorage(t) + + const core = new Hypercore(storage, keyPair.publicKey, { preload: () => { return { keyPair } } @@ -45,15 +52,18 @@ test('preload - custom keypair', async function (t) { t.ok(core.writable) t.alike(core.key, keyPair.publicKey) + + await core.close() }) test('preload - sign/storage', async function (t) { const keyPair = crypto.keyPair() + const storage = await createStorage(t) const core = new Hypercore(null, keyPair.publicKey, { valueEncoding: 'utf-8', preload: () => { return { - storage: RAM, + storage, keyPair } } @@ -64,4 +74,6 @@ test('preload - sign/storage', async function (t) { await core.append('hello world') t.is(core.length, 1) t.is(await core.get(0), 'hello world') + + await core.close() }) diff --git a/test/remote-bitfield.js b/test/remote-bitfield.js index aed70d7a..ebf93077 100644 --- a/test/remote-bitfield.js +++ b/test/remote-bitfield.js @@ -34,9 +34,9 @@ test('remote congituous length consistency (remote-bitfield findFirst edge case) // Indirectly tests the findFirst method for the case where // a position > 0 is passed in, while _maxSegments is still 0 // because nothing was set. - const a = await create() - const b = await create(a.key) - const c = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) + const c = await create(t, a.key) replicate(a, b, t) replicate(b, c, t) diff --git a/test/remote-length.js b/test/remote-length.js index e42271fc..c5def6e4 100644 --- a/test/remote-length.js +++ b/test/remote-length.js @@ -4,8 +4,8 @@ const RemoteBitfield = require('../lib/remote-bitfield') const { create, replicate } = require('./helpers') test('when the writer appends he broadcasts the new contiguous length', async function (t) { - const a = await create() - const b = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) replicate(a, b, t) await new Promise(resolve => setTimeout(resolve, 100)) @@ -22,9 +22,9 @@ test('when the writer appends he broadcasts the new contiguous length', async fu }) test('contiguous-length announce-on-update flow', async function (t) { - const a = await create() - const b = await create(a.key) - const c = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) + const c = await create(t, a.key) replicate(a, b, t) replicate(b, c, t) @@ -43,9 +43,9 @@ test('contiguous-length announce-on-update flow', async function (t) { test('announce-range-on-update flow with big core (multiple bitfield pages)', async function (t) { t.timeout(1000 * 60 * 5) // Expected to take around 15s. Additional headroom in case of slow CI machine - const a = await create() - const b = await create(a.key) - const c = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) + const c = await create(t, a.key) replicate(a, b, t) replicate(b, c, t) @@ -107,8 +107,8 @@ test('announce-range-on-update flow with big core (multiple bitfield pages)', as }) test('truncates by the writer result in the updated contiguous length being announced', async function (t) { - const a = await create() - const b = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) replicate(a, b, t) await new Promise(resolve => setTimeout(resolve, 100)) diff --git a/test/replicate.js b/test/replicate.js index 5c37094e..6d113935 100644 --- a/test/replicate.js +++ b/test/replicate.js @@ -1,17 +1,16 @@ const test = require('brittle') const b4a = require('b4a') -const RAM = require('random-access-memory') const NoiseSecretStream = require('@hyperswarm/secret-stream') -const { create, replicate, unreplicate, eventFlush } = require('./helpers') +const { create, createStored, replicate, unreplicate, eventFlush } = require('./helpers') const { makeStreamPair } = require('./helpers/networking.js') const Hypercore = require('../') test('basic replication', async function (t) { - const a = await create() + const a = await create(t) await a.append(['a', 'b', 'c', 'd', 'e']) - const b = await create(a.key) + const b = await create(t, a.key) let d = 0 b.on('download', () => d++) @@ -26,11 +25,11 @@ test('basic replication', async function (t) { }) test('basic replication stats', async function (t) { - const a = await create() + const a = await create(t) await a.append(['a', 'b', 'c', 'd', 'e']) - const b = await create(a.key) + const b = await create(t, a.key) const aStats = a.replicator.stats const bStats = b.replicator.stats @@ -91,27 +90,34 @@ test('basic replication stats', async function (t) { // bitfield messages await b.clear(1) - const c = await create(a.key) + const c = await create(t, a.key) replicate(c, b, t) c.get(1).catch(() => {}) await new Promise(resolve => setImmediate(resolve)) + await c.storage.idle() const cStats = c.replicator.stats t.ok(cStats.wireBitfield.rx > 0, 'bitfield incremented') t.is(bStats.wireBitfield.tx, cStats.wireBitfield.rx, 'bitfield received == transmitted') t.is(initStatsLength, [...Object.keys(aStats)].length, 'No stats were dynamically added') + + await a.close() + await b.close() + await c.close() }) -test('basic downloading is set immediately after ready', function (t) { +test('basic downloading is set immediately after ready', async function (t) { t.plan(2) - const a = new Hypercore(RAM) + const createA = await createStored(t) + const a = await createA() a.on('ready', function () { t.ok(a.replicator.downloading) }) - const b = new Hypercore(RAM, { active: false }) + const createB = await createStored(t) + const b = await createB({ active: false }) b.on('ready', function () { t.absent(b.replicator.downloading) @@ -124,7 +130,7 @@ test('basic downloading is set immediately after ready', function (t) { }) test('basic replication from fork', async function (t) { - const a = await create() + const a = await create(t) await a.append(['a', 'b', 'c', 'd', 'e']) await a.truncate(4) @@ -132,7 +138,7 @@ test('basic replication from fork', async function (t) { t.is(a.fork, 1) - const b = await create(a.key) + const b = await create(t, a.key) replicate(a, b, t) @@ -148,21 +154,20 @@ test('basic replication from fork', async function (t) { }) test('eager replication from bigger fork', async function (t) { - const a = await create() - const b = await create(a.key) - - replicate(a, b, t) + const a = await create(t) + const b = await create(t, a.key) await a.append(['a', 'b', 'c', 'd', 'e', 'g', 'h', 'i', 'j', 'k']) await a.truncate(4) await a.append(['FORKED', 'g', 'h', 'i', 'j', 'k']) + // replication has to start here so that fork is not set in upgrade + replicate(a, b, t) + t.is(a.fork, 1) let d = 0 - b.on('download', (index) => { - d++ - }) + b.on('download', () => d++) const r = b.download({ start: 0, end: a.length }) await r.done() @@ -172,8 +177,8 @@ test('eager replication from bigger fork', async function (t) { }) test('eager replication of updates per default', async function (t) { - const a = await create() - const b = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) replicate(a, b, t) @@ -189,8 +194,8 @@ test('eager replication of updates per default', async function (t) { }) test('bigger download range', async function (t) { - const a = await create() - const b = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) replicate(a, b, t) @@ -210,8 +215,8 @@ test('bigger download range', async function (t) { }) test('high latency reorg', async function (t) { - const a = await create() - const b = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) const s = replicate(a, b, t, { teardown: false }) @@ -252,8 +257,8 @@ test('high latency reorg', async function (t) { test('invalid signature fails', async function (t) { t.plan(1) - const a = await create(null) - const b = await create(a.key) + const a = await create(t, null) + const b = await create(t, a.key) a.core.verifier = { sign () { @@ -274,14 +279,14 @@ test('invalid signature fails', async function (t) { }) test('more invalid signatures fails', async function (t) { - const a = await create(null) + const a = await create(t, null) await a.append(['a', 'b'], { signature: b4a.alloc(64) }) await t.test('replication fails after bad append', async function (sub) { sub.plan(1) - const b = await create(a.key) + const b = await create(t, a.key) replicate(a, b, sub) b.on('verification-error', function (err) { @@ -297,7 +302,7 @@ test('more invalid signatures fails', async function (t) { await t.test('replication fails after bad truncate', async function (sub) { sub.plan(1) - const b = await create(a.key) + const b = await create(t, a.key) replicate(a, b, sub) b.on('verification-error', function (err) { @@ -311,7 +316,7 @@ test('more invalid signatures fails', async function (t) { await a.append('good') await t.test('replication works again', async function (sub) { - const b = await create(a.key) + const b = await create(t, a.key) replicate(a, b, sub) await new Promise(resolve => setImmediate(resolve)) @@ -325,8 +330,8 @@ test('more invalid signatures fails', async function (t) { test('invalid capability fails', async function (t) { t.plan(2) - const a = await create() - const b = await create() + const a = await create(t) + const b = await create(t) b.replicator.discoveryKey = a.discoveryKey @@ -356,8 +361,8 @@ test('invalid capability fails', async function (t) { }) test('update with zero length', async function (t) { - const a = await create() - const b = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) replicate(a, b, t) @@ -366,11 +371,11 @@ test('update with zero length', async function (t) { }) test('basic multiplexing', async function (t) { - const a1 = await create() - const a2 = await create() + const a1 = await create(t) + const a2 = await create(t) - const b1 = await create(a1.key) - const b2 = await create(a2.key) + const b1 = await create(t, a1.key) + const b2 = await create(t, a2.key) const a = a1.replicate(a2.replicate(true, { keepAlive: false })) const b = b1.replicate(b2.replicate(false, { keepAlive: false })) @@ -388,18 +393,18 @@ test('basic multiplexing', async function (t) { }) test('async multiplexing', async function (t) { - const a1 = await create() - const b1 = await create(a1.key) + const a1 = await create(t) + const b1 = await create(t, a1.key) const a = a1.replicate(true, { keepAlive: false }) const b = b1.replicate(false, { keepAlive: false }) a.pipe(b).pipe(a) - const a2 = await create() + const a2 = await create(t) await a2.append('ho') - const b2 = await create(a2.key) + const b2 = await create(t, a2.key) // b2 doesn't replicate immediately. a2.replicate(a) @@ -416,11 +421,11 @@ test('async multiplexing', async function (t) { }) test('multiplexing with external noise stream', async function (t) { - const a1 = await create() - const a2 = await create() + const a1 = await create(t) + const a2 = await create(t) - const b1 = await create(a1.key) - const b2 = await create(a2.key) + const b1 = await create(t, a1.key) + const b2 = await create(t, a2.key) const n1 = new NoiseSecretStream(true) const n2 = new NoiseSecretStream(false) @@ -446,11 +451,11 @@ test('multiplexing with external noise stream', async function (t) { test('multiplexing with createProtocolStream (ondiscoverykey is not called)', async function (t) { t.plan(2) - const a1 = await create() - const a2 = await create() + const a1 = await create(t) + const a2 = await create(t) - const b1 = await create(a1.key) - const b2 = await create(a2.key) + const b1 = await create(t, a1.key) + const b2 = await create(t, a2.key) const n1 = new NoiseSecretStream(true) const n2 = new NoiseSecretStream(false) @@ -487,11 +492,11 @@ test('multiplexing with createProtocolStream (ondiscoverykey is not called)', as test('multiplexing with createProtocolStream (ondiscoverykey is called)', async function (t) { t.plan(4) - const a1 = await create() - const a2 = await create() + const a1 = await create(t) + const a2 = await create(t) - const b1 = await create(a1.key) - const b2 = await create(a2.key) + const b1 = await create(t, a1.key) + const b2 = await create(t, a2.key) const n1 = new NoiseSecretStream(true) const n2 = new NoiseSecretStream(false) @@ -530,8 +535,8 @@ test('multiplexing with createProtocolStream (ondiscoverykey is called)', async }) test('seeking while replicating', async function (t) { - const a = await create() - const b = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) replicate(a, b, t) @@ -543,8 +548,8 @@ test('seeking while replicating', async function (t) { test('seek with no wait', async function (t) { t.plan(2) - const a = await create() - const b = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) replicate(a, b, t) @@ -558,7 +563,7 @@ test('seek with no wait', async function (t) { test('seek with timeout', async function (t) { t.plan(1) - const a = await create() + const a = await create(t) try { await a.seek(6, { timeout: 1 }) @@ -571,7 +576,7 @@ test('seek with timeout', async function (t) { test('seek with session options', async function (t) { t.plan(3) - const a = await create() + const a = await create(t) const s1 = a.session({ wait: false }) @@ -594,11 +599,11 @@ test('seek with session options', async function (t) { }) test('multiplexing multiple times over the same stream', async function (t) { - const a1 = await create() + const a1 = await create(t) await a1.append('hi') - const b1 = await create(a1.key) + const b1 = await create(t, a1.key) const n1 = new NoiseSecretStream(true) const n2 = new NoiseSecretStream(false) @@ -621,11 +626,11 @@ test('multiplexing multiple times over the same stream', async function (t) { }) test('destroying a stream and re-replicating works', async function (t) { - const core = await create() + const core = await create(t) while (core.length < 33) await core.append(b4a.from('#' + core.length)) - const clone = await create(core.key) + const clone = await create(t, core.key) let s1 = core.replicate(true, { keepAlive: false }) let s2 = clone.replicate(false, { keepAlive: false }) @@ -661,11 +666,11 @@ test('destroying a stream and re-replicating works', async function (t) { }) test('replicate discrete range', async function (t) { - const a = await create() + const a = await create(t) await a.append(['a', 'b', 'c', 'd', 'e']) - const b = await create(a.key) + const b = await create(t, a.key) let d = 0 b.on('download', () => d++) @@ -682,11 +687,11 @@ test('replicate discrete range', async function (t) { }) test('replicate discrete empty range', async function (t) { - const a = await create() + const a = await create(t) await a.append(['a', 'b', 'c', 'd', 'e']) - const b = await create(a.key) + const b = await create(t, a.key) let d = 0 b.on('download', () => d++) @@ -701,11 +706,11 @@ test('replicate discrete empty range', async function (t) { }) test('get with { wait: false } returns null if block is not available', async function (t) { - const a = await create() + const a = await create(t) await a.append('a') - const b = await create(a.key, { valueEncoding: 'utf-8' }) + const b = await create(t, a.key, { valueEncoding: 'utf-8' }) replicate(a, b, t) @@ -716,8 +721,8 @@ test('get with { wait: false } returns null if block is not available', async fu test('request cancellation regression', async function (t) { t.plan(2) - const a = await create() - const b = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) let errored = 0 @@ -727,6 +732,9 @@ test('request cancellation regression', async function (t) { b.get(1).catch(onerror) b.get(2).catch(onerror) + // have to wait for the storage lookup here, TODO: add a flush sort of api for testing this + await new Promise(resolve => setTimeout(resolve, 500)) + // No explict api to trigger this (maybe we add a cancel signal / abort controller?) but cancel get(1) b.activeRequests[1].context.detach(b.activeRequests[1]) @@ -743,8 +751,8 @@ test('request cancellation regression', async function (t) { test('findingPeers makes update wait for first peer', async function (t) { t.plan(2) - const a = await create() - const b = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) await a.append('hi') @@ -764,8 +772,8 @@ test('findingPeers makes update wait for first peer', async function (t) { test('findingPeers + done makes update return false if no peers', async function (t) { t.plan(2) - const a = await create() - const b = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) await a.append('hi') @@ -781,12 +789,12 @@ test('findingPeers + done makes update return false if no peers', async function }) test.skip('can disable downloading from a peer', async function (t) { - const a = await create() + const a = await create(t) await a.append(['a', 'b', 'c', 'd', 'e']) - const b = await create(a.key, { valueEncoding: 'utf-8' }) - const c = await create(a.key, { valueEncoding: 'utf-8' }) + const b = await create(t, a.key, { valueEncoding: 'utf-8' }) + const c = await create(t, a.key, { valueEncoding: 'utf-8' }) const [aStream] = replicate(b, a, t) replicate(b, c, t) @@ -823,12 +831,12 @@ test.skip('can disable downloading from a peer', async function (t) { }) test('contiguous length', async function (t) { - const a = await create() + const a = await create(t) await a.append(['a', 'b', 'c', 'd', 'e']) t.is(a.contiguousLength, 5, 'a has all blocks') - const b = await create(a.key) + const b = await create(t, a.key) t.is(b.contiguousLength, 0) replicate(a, b, t) @@ -844,8 +852,8 @@ test('contiguous length', async function (t) { }) test('contiguous length after fork', async function (t) { - const a = await create() - const b = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) const s = replicate(a, b, t, { teardown: false }) @@ -864,8 +872,8 @@ test('contiguous length after fork', async function (t) { }) test('one inflight request to a peer per block', async function (t) { - const a = await create() - const b = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) let uploads = 0 a.on('upload', function (index) { @@ -888,9 +896,9 @@ test('one inflight request to a peer per block', async function (t) { t.is(uploads, 1) }) -test('non-sparse replication', async function (t) { - const a = await create() - const b = await create(a.key, { sparse: false }) +test.skip('non-sparse replication', async function (t) { + const a = await create(t) + const b = await create(t, a.key, { sparse: false }) await a.append(['a', 'b', 'c', 'd', 'e']) @@ -929,8 +937,8 @@ test('non-sparse replication', async function (t) { }) test('download blocks if available', async function (t) { - const a = await create() - const b = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) replicate(a, b, t) @@ -947,8 +955,8 @@ test('download blocks if available', async function (t) { }) test('download range if available', async function (t) { - const a = await create() - const b = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) replicate(a, b, t) @@ -965,8 +973,8 @@ test('download range if available', async function (t) { }) test('download blocks if available, destroy midway', async function (t) { - const a = await create() - const b = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) const s = replicate(a, b, t, { teardown: false }) @@ -985,9 +993,9 @@ test('download blocks if available, destroy midway', async function (t) { }) test('download blocks available from when only a partial set is available', async function (t) { - const a = await create() - const b = await create(a.key) - const c = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) + const c = await create(t, a.key) replicate(a, b, t) replicate(b, c, t) @@ -1009,8 +1017,8 @@ test('download blocks available from when only a partial set is available', asyn }) test('download range resolves immediately if no peers', async function (t) { - const a = await create() - const b = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) // no replication @@ -1021,8 +1029,8 @@ test('download range resolves immediately if no peers', async function (t) { }) test('download available blocks on non-sparse update', async function (t) { - const a = await create() - const b = await create(a.key, { sparse: false }) + const a = await create(t) + const b = await create(t, a.key, { sparse: false }) replicate(a, b, t) @@ -1033,11 +1041,11 @@ test('download available blocks on non-sparse update', async function (t) { }) test('downloaded blocks are unslabbed if small', async function (t) { - const a = await create() + const a = await create(t) await a.append(Buffer.alloc(1)) - const b = await create(a.key) + const b = await create(t, a.key) replicate(a, b, t) @@ -1048,7 +1056,7 @@ test('downloaded blocks are unslabbed if small', async function (t) { }) test('downloaded blocks are not unslabbed if bigger than half of slab size', async function (t) { - const a = await create() + const a = await create(t) await a.append(Buffer.alloc(5000)) t.is( @@ -1057,7 +1065,7 @@ test('downloaded blocks are not unslabbed if bigger than half of slab size', asy 'Sanity check (adapt test if fails)' ) - const b = await create(a.key) + const b = await create(t, a.key) replicate(a, b, t) @@ -1074,8 +1082,8 @@ test('downloaded blocks are not unslabbed if bigger than half of slab size', asy test('sparse replication without gossiping', async function (t) { t.plan(4) - const a = await create() - const b = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) await a.append(['a', 'b', 'c']) @@ -1092,7 +1100,7 @@ test('sparse replication without gossiping', async function (t) { await unreplicate(s) await t.test('block', async function (t) { - const c = await create(a.key) + const c = await create(t, a.key) s = replicate(b, c, t, { teardown: false }) t.teardown(() => unreplicate(s)) @@ -1101,7 +1109,7 @@ test('sparse replication without gossiping', async function (t) { }) await t.test('range', async function (t) { - const c = await create(a.key) + const c = await create(t, a.key) replicate(b, c, t) @@ -1110,7 +1118,7 @@ test('sparse replication without gossiping', async function (t) { }) await t.test('discrete range', async function (t) { - const c = await create(a.key) + const c = await create(t, a.key) replicate(b, c, t) @@ -1120,7 +1128,7 @@ test('sparse replication without gossiping', async function (t) { }) await t.test('seek', async function (t) { - const c = await create(a.key) + const c = await create(t, a.key) replicate(b, c, t) @@ -1129,8 +1137,8 @@ test('sparse replication without gossiping', async function (t) { }) test('force update writable cores', async function (t) { - const a = await create() - const b = await create(a.key, { header: a.core.header.manifest }) + const a = await create(t) + const b = await create(t, a.key, { header: a.core.header.manifest }) await a.append(['a', 'b', 'c', 'd', 'e']) @@ -1151,8 +1159,8 @@ test('force update writable cores', async function (t) { }) test('replicate to writable cores after clearing', async function (t) { - const a = await create() - const b = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) await a.append(['a', 'b', 'c', 'd', 'e']) @@ -1172,11 +1180,11 @@ test('replicate to writable cores after clearing', async function (t) { test('large linear download', async function (t) { const n = 1000 - const a = await create() + const a = await create(t) for (let i = 0; i < n; i++) await a.append(i.toString()) - const b = await create(a.key) + const b = await create(t, a.key) let d = 0 b.on('download', () => d++) @@ -1191,8 +1199,8 @@ test('large linear download', async function (t) { }) test('replication session', async function (t) { - const a = await create() - const b = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) await a.append(['a', 'b', 'c', 'd', 'e']) @@ -1200,8 +1208,8 @@ test('replication session', async function (t) { t.is(a.sessions.length, 1) t.is(b.sessions.length, 1) - t.is(a.core.active, 2) - t.is(b.core.active, 2) + t.is(a.core.state.active, 2) + t.is(b.core.state.active, 2) s1.destroy() s2.destroy() @@ -1210,13 +1218,13 @@ test('replication session', async function (t) { t.is(a.sessions.length, 1) t.is(b.sessions.length, 1) - t.is(a.core.active, 1) - t.is(b.core.active, 1) + t.is(a.core.state.active, 1) + t.is(b.core.state.active, 1) }) test('replication session after stream opened', async function (t) { - const a = await create() - const b = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) await a.append(['a', 'b', 'c', 'd', 'e']) @@ -1227,8 +1235,8 @@ test('replication session after stream opened', async function (t) { t.is(a.sessions.length, 1) t.is(b.sessions.length, 1) - t.is(a.core.active, 2) - t.is(b.core.active, 2) + t.is(a.core.state.active, 2) + t.is(b.core.state.active, 2) s1.destroy() s2.destroy() @@ -1237,13 +1245,13 @@ test('replication session after stream opened', async function (t) { t.is(a.sessions.length, 1) t.is(b.sessions.length, 1) - t.is(a.core.active, 1) - t.is(b.core.active, 1) + t.is(a.core.state.active, 1) + t.is(b.core.state.active, 1) }) test('replication session keeps the core open', async function (t) { - const a = await create() - const b = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) await a.append(['a', 'b', 'c', 'd', 'e']) @@ -1257,11 +1265,30 @@ test('replication session keeps the core open', async function (t) { t.alike(blk, b4a.from('c'), 'still replicating due to session') }) +test('force close kills replication session', async function (t) { + const a = await create(t) + const b = await create(t, a.key) + + await a.append(['a', 'b', 'c', 'd', 'e']) + + replicate(a, b, t, { session: true }) + + await a.close({ force: true }) + await eventFlush() + + const blk = b.get(2, { timeout: 1000 }) + + t.ok(a.core.closed) + t.ok(a.replicator.destroyed) + + await t.exception(blk, /REQUEST_TIMEOUT/) +}) + test('replicate range that fills initial size of bitfield page', async function (t) { - const a = await create() + const a = await create(t) await a.append(new Array(2 ** 15).fill('a')) - const b = await create(a.key) + const b = await create(t, a.key) let d = 0 b.on('download', () => d++) @@ -1275,10 +1302,10 @@ test('replicate range that fills initial size of bitfield page', async function }) test('replicate range that overflows initial size of bitfield page', async function (t) { - const a = await create() + const a = await create(t) await a.append(new Array(2 ** 15 + 1).fill('a')) - const b = await create(a.key) + const b = await create(t, a.key) let d = 0 b.on('download', () => d++) @@ -1292,10 +1319,10 @@ test('replicate range that overflows initial size of bitfield page', async funct }) test('replicate ranges in reverse order', async function (t) { - const a = await create() + const a = await create(t) await a.append(['a', 'b']) - const b = await create(a.key) + const b = await create(t, a.key) let d = 0 b.on('download', () => d++) @@ -1315,8 +1342,8 @@ test('replicate ranges in reverse order', async function (t) { test('cancel block', async function (t) { t.plan(4) - const a = await create() - const b = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) await a.append(['a', 'b', 'c']) @@ -1338,6 +1365,7 @@ test('cancel block', async function (t) { await a.close() await b.close() + await session.close() t.ok(a.replicator.stats.wireCancel.rx > 0, 'wireCancel stats incremented') t.is(a.replicator.stats.wireCancel.rx, b.replicator.stats.wireCancel.tx, 'wireCancel stats consistent') @@ -1349,8 +1377,8 @@ test('cancel block', async function (t) { test('try cancel block from a different session', async function (t) { t.plan(3) - const a = await create() - const b = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) await a.append(['a', 'b', 'c']) @@ -1389,9 +1417,9 @@ test('try cancel block from a different session', async function (t) { test('retry failed block requests to another peer', async function (t) { t.plan(6) - const a = await create() - const b = await create(a.key) - const c = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) + const c = await create(t, a.key) await a.append(['1', '2', '3']) @@ -1445,8 +1473,8 @@ test('retry failed block requests to another peer', async function (t) { }) test('idle replication sessions auto gc', async function (t) { - const a = await create({ active: false, notDownloadingLinger: 50 }) - const b = await create(a.key, { autoClose: true, active: false, notDownloadingLinger: 50 }) + const a = await create(t, { active: false, notDownloadingLinger: 50 }) + const b = await create(t, a.key, { autoClose: true, active: false, notDownloadingLinger: 50 }) await a.append('test') const s = b.session() @@ -1470,8 +1498,8 @@ test('idle replication sessions auto gc', async function (t) { test('manifests eagerly sync', async function (t) { t.plan(1) - const a = await create({ compat: false }) - const b = await create(a.key) + const a = await create(t, { compat: false }) + const b = await create(t, a.key) replicate(a, b, t) @@ -1483,9 +1511,9 @@ test('manifests eagerly sync', async function (t) { test('manifests gossip eagerly sync', async function (t) { t.plan(2) - const a = await create({ compat: false }) - const b = await create(a.key) - const c = await create(a.key) + const a = await create(t, { compat: false }) + const b = await create(t, a.key) + const c = await create(t, a.key) replicate(a, b, t) replicate(b, c, t) @@ -1500,9 +1528,9 @@ test('manifests gossip eagerly sync', async function (t) { }) test('remote has larger tree', async function (t) { - const a = await create() - const b = await create(a.key) - const c = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) + const c = await create(t, a.key) await a.append(['a', 'b', 'c', 'd', 'e']) @@ -1532,8 +1560,8 @@ test('remote has larger tree', async function (t) { }) test('range download, single block missing', async function (t) { - const a = await create() - const b = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) const n = 100 @@ -1548,8 +1576,8 @@ test('range download, single block missing', async function (t) { }) test('range download, repeated', async function (t) { - const a = await create() - const b = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) const n = 100 @@ -1564,16 +1592,16 @@ test('range download, repeated', async function (t) { } }) -test('replication updates on core copy', async function (t) { - const a = await create() +test.skip('replication updates on core copy', async function (t) { + const a = await create(t) const n = 100 for (let i = 0; i < n; i++) await a.append(b4a.from([0])) const manifest = { prologue: { hash: await a.treeHash(), length: a.length } } - const b = await create({ manifest }) - const c = await create({ manifest }) + const b = await create(t, { manifest }) + const c = await create(t, { manifest }) replicate(b, c, t) @@ -1585,10 +1613,10 @@ test('replication updates on core copy', async function (t) { }) test('can define default max-inflight blocks for replicator peers', async function (t) { - const a = new Hypercore(RAM, { inflightRange: [123, 123] }) + const a = await create(t, { inflightRange: [123, 123] }) await a.append('some block') - const b = await create(a.key) + const b = await create(t, a.key) replicate(a, b, t) await b.get(0) @@ -1608,8 +1636,8 @@ test('session id reuse does not stall', async function (t) { t.plan(2) t.timeout(90_000) - const a = await create() - const b = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) const n = 500 @@ -1649,8 +1677,8 @@ test('session id reuse does not stall', async function (t) { test('restore after cancelled block request', async function (t) { t.plan(2) - const a = await create() - const b = await create(a.key) + const a = await create(t) + const b = await create(t, a.key) for (let i = 0; i < 4; i++) await a.append(b4a.from([i])) @@ -1678,11 +1706,11 @@ test('restore after cancelled block request', async function (t) { }) test('handshake is unslabbed', async function (t) { - const a = await create() + const a = await create(t) await a.append(['a']) - const b = await create(a.key) + const b = await create(t, a.key) replicate(a, b, t) const r = b.download({ start: 0, end: a.length }) @@ -1701,10 +1729,10 @@ test('handshake is unslabbed', async function (t) { }) test('merkle-tree signature gets unslabbed', async function (t) { - const a = await create() + const a = await create(t) await a.append(['a']) - const b = await create(a.key) + const b = await create(t, a.key) replicate(a, b, t) await b.get(0) @@ -1716,10 +1744,10 @@ test('merkle-tree signature gets unslabbed', async function (t) { }) test('seek against non sparse peer', async function (t) { - const a = await create() + const a = await create(t) await a.append(['a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i', 'j', 'k', 'l', 'm', 'n']) - const b = await create(a.key) + const b = await create(t, a.key) replicate(a, b, t) await b.get(a.length - 1) @@ -1733,9 +1761,9 @@ test('seek against non sparse peer', async function (t) { test('replication count should never go negative', async function (t) { t.plan(2 + 3) - const a = await create({ autoClose: true }) - const b = await create(a.key, { autoClose: true }) - const c = await create(a.key, { autoClose: true }) + const a = await create(t, { autoClose: true }) + const b = await create(t, a.key, { autoClose: true }) + const c = await create(t, a.key, { autoClose: true }) const refA = a.session() const refB = b.session() diff --git a/test/sessions.js b/test/sessions.js index 542a9156..09fe4659 100644 --- a/test/sessions.js +++ b/test/sessions.js @@ -1,17 +1,17 @@ const test = require('brittle') -const RAM = require('random-access-memory') const crypto = require('hypercore-crypto') const c = require('compact-encoding') const b4a = require('b4a') -const { create } = require('./helpers') +const { create, createStorage } = require('./helpers') const Hypercore = require('../') test('sessions - can create writable sessions from a read-only core', async function (t) { t.plan(5) + const storage = await createStorage(t) const keyPair = crypto.keyPair() - const core = new Hypercore(RAM, keyPair.publicKey, { + const core = new Hypercore(storage, keyPair.publicKey, { valueEncoding: 'utf-8' }) await core.ready() @@ -36,10 +36,14 @@ test('sessions - can create writable sessions from a read-only core', async func } t.is(core.length, 1) + + await session.close() + await core.close() }) test('sessions - auto close', async function (t) { - const core = new Hypercore(RAM, { autoClose: true }) + const storage = await createStorage(t) + const core = new Hypercore(storage, { autoClose: true }) let closed = false core.on('close', function () { @@ -57,7 +61,8 @@ test('sessions - auto close', async function (t) { }) test('sessions - auto close different order', async function (t) { - const core = new Hypercore(RAM, { autoClose: true }) + const storage = await createStorage(t) + const core = new Hypercore(storage, { autoClose: true }) const a = core.session() const b = core.session() @@ -75,7 +80,8 @@ test('sessions - auto close different order', async function (t) { }) test('sessions - auto close with all closing', async function (t) { - const core = new Hypercore(RAM, { autoClose: true }) + const storage = await createStorage(t) + const core = new Hypercore(storage, { autoClose: true }) const a = core.session() const b = core.session() @@ -90,7 +96,8 @@ test('sessions - auto close with all closing', async function (t) { }) test('sessions - auto close when using from option', async function (t) { - const core1 = new Hypercore(RAM, { + const storage = await createStorage(t) + const core1 = new Hypercore(storage, { autoClose: true }) const core2 = new Hypercore({ @@ -105,7 +112,8 @@ test('sessions - auto close when using from option', async function (t) { }) test('sessions - close with from option', async function (t) { - const core1 = new Hypercore(RAM) + const storage = await createStorage(t) + const core1 = new Hypercore(storage) await core1.append('hello world') const core2 = new Hypercore({ @@ -119,10 +127,12 @@ test('sessions - close with from option', async function (t) { t.absent(core1.closed) t.alike(await core1.get(0), b4a.from('hello world')) + await core1.close() }) test('sessions - custom valueEncoding on session', async function (t) { - const core1 = new Hypercore(RAM) + const storage = await createStorage(t) + const core1 = new Hypercore(storage) await core1.append(c.encode(c.raw.json, { a: 1 })) const core2 = core1.session({ valueEncoding: 'json' }) @@ -130,13 +140,17 @@ test('sessions - custom valueEncoding on session', async function (t) { t.alike(await core2.get(0), { a: 1 }) t.alike(await core2.get(1), { b: 2 }) + + await core2.close() + await core1.close() }) test('sessions - custom preload hook on first/later sessions', async function (t) { const preloadsTest = t.test('both preload hooks called') preloadsTest.plan(2) - const core1 = new Hypercore(RAM, { + const storage = await createStorage(t) + const core1 = new Hypercore(storage, { preload: () => { preloadsTest.pass('first hook called') return null @@ -151,17 +165,23 @@ test('sessions - custom preload hook on first/later sessions', async function (t await core2.ready() await preloadsTest + + await core2.close() + await core1.close() }) test('session inherits non-sparse setting', async function (t) { - const a = await create({ sparse: false }) + const a = await create(t, { sparse: false }) const s = a.session() t.is(s.sparse, false) + + await s.close() + await a.close() }) test('session on a from instance, pre-ready', async function (t) { - const a = await create() + const a = await create(t) const b = new Hypercore({ from: a }) const c = b.session() @@ -172,10 +192,13 @@ test('session on a from instance, pre-ready', async function (t) { t.is(a.sessions, b.sessions) t.is(a.sessions, c.sessions) + + await b.close() + await c.close() }) test('session on a from instance does not inject itself to other sessions', async function (t) { - const a = await create({ }) + const a = await create(t, { }) const b = new Hypercore({ from: a, encryptionKey: null }) await b.ready() @@ -191,4 +214,8 @@ test('session on a from instance does not inject itself to other sessions', asyn t.absent(b.encryption) t.ok(c.encryption) t.absent(d.encryption) + + await b.close() + await c.close() + await d.close() }) diff --git a/test/snapshots.js b/test/snapshots.js index f83cc724..e9eabb34 100644 --- a/test/snapshots.js +++ b/test/snapshots.js @@ -4,8 +4,8 @@ const { replicate, unreplicate, create, createStored } = require('./helpers') test('implicit snapshot - gets are snapshotted at call time', async function (t) { t.plan(8) - const core = await create() - const clone = await create(core.key, { valueEncoding: 'utf-8' }) + const core = await create(t) + const clone = await create(t, core.key, { valueEncoding: 'utf-8' }) clone.on('truncate', function (len) { t.is(len, 2, 'remote truncation') @@ -31,6 +31,8 @@ test('implicit snapshot - gets are snapshotted at call time', async function (t) const p2 = clone.get(1) const p3 = clone.get(2) + const exception = t.exception(p3, 'should fail cause snapshot not available') + await core.truncate(2) await core.append('block #2.1') @@ -39,7 +41,7 @@ test('implicit snapshot - gets are snapshotted at call time', async function (t) replicate(core, clone, t) t.is(await p2, 'block #1.0') - t.exception(p3, 'should fail cause snapshot not available') + await exception t.is(await clone.get(2), 'block #2.1') @@ -53,9 +55,9 @@ test('implicit snapshot - gets are snapshotted at call time', async function (t) test('snapshots wait for ready', async function (t) { t.plan(10) - const create = createStored() + const create = await createStored(t) - const core = create() + const core = await create() const s1 = core.snapshot() await core.append('block #0.0') @@ -83,7 +85,7 @@ test('snapshots wait for ready', async function (t) { await core.close() - const coreCopy = create() + const coreCopy = await create() // if a snapshot is made on an opening core, it should wait until opened const s3 = coreCopy.snapshot() @@ -101,13 +103,19 @@ test('snapshots wait for ready', async function (t) { t.is(s3.length, 4, 'no changes') t.is(s4.length, 4, 'no changes') + + await coreCopy.close() + await s1.close() + await s2.close() + await s3.close() + await s4.close() }) test('snapshots are consistent', async function (t) { t.plan(6) - const core = await create() - const clone = await create(core.key) + const core = await create(t) + const clone = await create(t, core.key) await core.append('block #0.0') await core.append('block #1.0') @@ -127,10 +135,15 @@ test('snapshots are consistent', async function (t) { await core.append('block #1.1') await core.append('block #2.1') + // wait for clone to update + await new Promise(resolve => clone.once('truncate', resolve)) + t.is(clone.fork, 1, 'clone updated') const b = snapshot.get(0) t.exception(snapshot.get(1)) t.exception(snapshot.get(2)) t.is(await b, 'block #0.0') + + await snapshot.close() }) diff --git a/test/storage.js b/test/storage.js index afb0b8c4..37bdb8ed 100644 --- a/test/storage.js +++ b/test/storage.js @@ -17,6 +17,8 @@ test('storage layout', async function (t) { } snapshot(t, core) + + await core.close() }) test('encrypted storage layout', async function (t) { @@ -27,6 +29,8 @@ test('encrypted storage layout', async function (t) { } snapshot(t, core) + + await core.close() }) function snapshot (t, core) { diff --git a/test/streams.js b/test/streams.js index 9dba1003..8638a22f 100644 --- a/test/streams.js +++ b/test/streams.js @@ -4,7 +4,7 @@ const b4a = require('b4a') const { create } = require('./helpers') test('basic read stream', async function (t) { - const core = await create() + const core = await create(t) const expected = [ 'hello', @@ -23,7 +23,7 @@ test('basic read stream', async function (t) { }) test('read stream with start / end', async function (t) { - const core = await create() + const core = await create(t) const datas = [ 'hello', @@ -56,7 +56,7 @@ test('read stream with start / end', async function (t) { }) test('basic write+read stream', async function (t) { - const core = await create() + const core = await create(t) const expected = [ 'hello', @@ -80,7 +80,7 @@ test('basic write+read stream', async function (t) { }) test('basic byte stream', async function (t) { - const core = await create() + const core = await create(t) const expected = [ 'hello', @@ -99,7 +99,7 @@ test('basic byte stream', async function (t) { }) test('basic byte stream with byteOffset / byteLength', async function (t) { - const core = await create() + const core = await create(t) await core.append([ 'hello', @@ -122,7 +122,7 @@ test('basic byte stream with byteOffset / byteLength', async function (t) { }) test('basic byte stream with byteOffset / byteLength of a core that has valueEncoding', async function (t) { - const core = await create({ valueEncoding: 'utf8' }) + const core = await create(t, { valueEncoding: 'utf8' }) await core.append([ 'hello', @@ -146,7 +146,7 @@ test('basic byte stream with byteOffset / byteLength of a core that has valueEnc }) test('byte stream with lower byteLength than byteOffset', async function (t) { - const core = await create() + const core = await create(t) await core.append([ 'hello', @@ -168,7 +168,7 @@ test('byte stream with lower byteLength than byteOffset', async function (t) { }) test('basic byte stream with custom byteOffset but default byteLength', async function (t) { - const core = await create() + const core = await create(t) await core.append([ 'hello', diff --git a/test/timeouts.js b/test/timeouts.js index 478e3c36..697bef4c 100644 --- a/test/timeouts.js +++ b/test/timeouts.js @@ -1,13 +1,13 @@ const test = require('brittle') -const { create } = require('./helpers') +const { create, createStorage } = require('./helpers') const Hypercore = require('../') -const RAM = require('random-access-memory') const b4a = require('b4a') test('core and session timeout property', async function (t) { t.plan(3) - const core = new Hypercore(RAM) + const storage = await createStorage(t) + const core = new Hypercore(storage) t.is(core.timeout, 0) const a = core.session() @@ -15,12 +15,19 @@ test('core and session timeout property', async function (t) { const b = core.session({ timeout: 50 }) t.is(b.timeout, 50) + + await new Promise(resolve => setTimeout(resolve, 100)) + + await core.close() + await a.close() + await b.close() }) test('core session inherits timeout property', async function (t) { t.plan(3) - const core = new Hypercore(RAM, { timeout: 50 }) + const storage = await createStorage(t) + const core = new Hypercore(storage, { timeout: 50 }) t.is(core.timeout, 50) const a = core.session() @@ -28,12 +35,18 @@ test('core session inherits timeout property', async function (t) { const b = core.session({ timeout: 0 }) t.is(b.timeout, 0) + + await new Promise(resolve => setTimeout(resolve, 100)) + + await core.close() + await a.close() + await b.close() }) test('get before timeout', async function (t) { t.plan(1) - const core = await create() + const core = await create(t) const get = core.get(0, { timeout: 30000 }) setTimeout(() => core.append('hi'), 100) @@ -43,7 +56,7 @@ test('get before timeout', async function (t) { test('get after timeout', async function (t) { t.plan(1) - const core = await create() + const core = await create(t) try { await core.get(0, { timeout: 1 }) @@ -56,7 +69,7 @@ test('get after timeout', async function (t) { test('get after timeout with constructor', async function (t) { t.plan(1) - const core = await create({ timeout: 1 }) + const core = await create(t, { timeout: 1 }) try { await core.get(0) @@ -69,7 +82,7 @@ test('get after timeout with constructor', async function (t) { test('session get after timeout', async function (t) { t.plan(1) - const core = await create() + const core = await create(t) const session = core.session({ timeout: 1 }) try { @@ -78,12 +91,14 @@ test('session get after timeout', async function (t) { } catch (err) { t.is(err.code, 'REQUEST_TIMEOUT') } + + await session.close() }) test('session get after inherited timeout', async function (t) { t.plan(1) - const core = await create({ timeout: 1 }) + const core = await create(t, { timeout: 1 }) const session = core.session() try { @@ -92,12 +107,14 @@ test('session get after inherited timeout', async function (t) { } catch (err) { t.is(err.code, 'REQUEST_TIMEOUT') } + + await session.close() }) test('core constructor timeout but disable on get', async function (t) { t.plan(1) - const core = await create({ timeout: 1 }) + const core = await create(t, { timeout: 1 }) const get = core.get(0, { timeout: 0 }) setTimeout(() => core.append('hi'), 100) @@ -107,7 +124,7 @@ test('core constructor timeout but disable on get', async function (t) { test('core constructor timeout but increase on get', async function (t) { t.plan(1) - const core = await create({ timeout: 1 }) + const core = await create(t, { timeout: 1 }) const get = core.get(0, { timeout: 30000 }) setTimeout(() => core.append('hi'), 100) @@ -117,7 +134,7 @@ test('core constructor timeout but increase on get', async function (t) { test('block request gets cancelled before timeout', async function (t) { t.plan(1) - const core = await create() + const core = await create(t) const a = core.session() const promise = a.get(0, { timeout: 1 }) diff --git a/test/user-data.js b/test/user-data.js index 03f260cd..eaf0a5cc 100644 --- a/test/user-data.js +++ b/test/user-data.js @@ -1,18 +1,16 @@ const test = require('brittle') -const tmp = require('test-tmp') const b4a = require('b4a') -const { create } = require('./helpers') -const Hypercore = require('../') +const { create, createStored } = require('./helpers') test('userdata - can set through setUserData', async function (t) { - const core = await create() + const core = await create(t) await core.setUserData('hello', b4a.from('world')) t.alike(await core.getUserData('hello'), b4a.from('world')) }) test('userdata - can set through constructor option', async function (t) { - const core = await create({ + const core = await create(t, { userData: { hello: b4a.from('world') } @@ -22,9 +20,9 @@ test('userdata - can set through constructor option', async function (t) { }) test('userdata - persists across restarts', async function (t) { - const dir = await tmp(t) + const create = await createStored(t) - let core = new Hypercore(dir, { + let core = await create({ userData: { hello: b4a.from('world') } @@ -32,7 +30,7 @@ test('userdata - persists across restarts', async function (t) { await core.ready() await core.close() - core = new Hypercore(dir, { + core = await create({ userData: { other: b4a.from('another') } @@ -45,7 +43,7 @@ test('userdata - persists across restarts', async function (t) { }) test('userdata - big userdata gets swapped to external header', async function (t) { - const core = await create() + const core = await create(t) await core.setUserData('hello', b4a.alloc(20000)) await core.setUserData('world', b4a.alloc(20000)) await core.setUserData('world2', b4a.alloc(20000))