replace async-append-only-log with our own log

This commit is contained in:
Andre Staltz 2023-11-07 09:53:56 +02:00
parent c1f527b5d5
commit db915d0287
No known key found for this signature in database
GPG Key ID: 9EDE23EA7E8A4890
16 changed files with 1693 additions and 61 deletions

View File

@ -1,14 +1,11 @@
const Path = require('path')
// @ts-ignore
const push = require('push-stream')
// @ts-ignore
const AAOL = require('async-append-only-log')
const promisify = require('promisify-4loc')
const b4a = require('b4a')
const base58 = require('bs58')
// @ts-ignore
const Obz = require('obz')
const Keypair = require('ppppp-keypair')
const Log = require('./log')
const MsgV3 = require('./msg-v3')
const {
SIGNATURE_TAG_ACCOUNT_ADD,
@ -29,7 +26,6 @@ const { decrypt } = require('./encryption')
* @typedef {import('./msg-v3').AccountPower} AccountPower
* @typedef {import('./encryption').EncryptionFormat} EncryptionFormat
* @typedef {import('./msg-v3/tangle')} Tangle
*
* @typedef {Buffer | Uint8Array} B4A
*/
@ -133,8 +129,7 @@ function initDB(peer, config) {
const onRecordAdded = Obz()
const log = AAOL(Path.join(config.path, 'db.bin'), {
cacheSize: 1,
const log = Log(Path.join(config.path, 'db.bin'), {
blockSize: 64 * 1024,
codec: {
/**
@ -145,6 +140,7 @@ function initDB(peer, config) {
},
/**
* @param {B4A} buf
* @returns {Msg}
*/
decode(buf) {
return JSON.parse(b4a.toString(buf, 'utf8'))
@ -174,14 +170,13 @@ function initDB(peer, config) {
const scannedLog = new ReadyGate()
// setTimeout to let peer.db.* secret-stack become available
// needed by decrypt()
setTimeout(() => {
let i = -1
log.stream({ offsets: true, values: true, sizes: true }).pipe(
push.drain(
// @ts-ignore
function drainEach({ offset, value, size }) {
log.scan(
function scanEach(offset, logRec, size) {
i += 1
if (!value) {
if (!logRec) {
// deleted record
/** @type {RecDeleted} */
const rec = { misc: { offset, size, seq: i } }
@ -191,20 +186,20 @@ function initDB(peer, config) {
// TODO: for performance, dont decrypt on startup, instead decrypt on
// demand, or decrypt in the background. Or then store the log with
// decrypted msgs and only encrypt when moving it to the network.
const rec = decrypt(value, peer, config)
// @ts-ignore // FIXME:
const rec = decrypt(logRec, peer, config)
rec.misc ??= /** @type {Rec['misc']} */ ({})
rec.misc.offset = offset
rec.misc.size = size
rec.misc.seq = i
recs.push(rec)
},
function drainEnd(/** @type {any} */ err) {
function scanEnd(err) {
// prettier-ignore
if (err) throw new Error('Failed to initially scan the log', { cause: err });
scannedLog.setReady()
}
)
)
})
/**
@ -956,7 +951,7 @@ function initDB(peer, config) {
if (!opts.msgID || typeof opts.msgID !== 'string') return cb(new Error('ghosts.add() requires msgID of the deleted msg in `opts.msgID`'))
// prettier-ignore
if (!opts.span || typeof opts.span !== 'number') return cb(new Error('ghosts.add() requires span in `opts.span`'))
const { tangleID, msgID, span} = opts
const { tangleID, msgID, span } = opts
const rec = getRecord(msgID)
if (!rec) return cb()
if (!rec.msg) return cb()

85
lib/log/errors.js Normal file
View File

@ -0,0 +1,85 @@
class ErrorWithCode extends Error {
/**
* @param {string} message
* @param {string} code
*/
constructor(message, code) {
super(message)
this.code = code
}
}
/**
* @param {number} offset
*/
function nanOffsetErr(offset) {
return new ErrorWithCode(
`Offset ${offset} is not a number`,
'ERR_AAOL_INVALID_OFFSET'
)
}
/**
* @param {number} offset
*/
function negativeOffsetErr(offset) {
return new ErrorWithCode(
`Offset ${offset} is negative`,
'ERR_AAOL_INVALID_OFFSET'
)
}
/**
* @param {number} offset
* @param {number} logSize
*/
function outOfBoundsOffsetErr(offset, logSize) {
return new ErrorWithCode(
`Offset ${offset} is beyond log size ${logSize}`,
'ERR_AAOL_OFFSET_OUT_OF_BOUNDS'
)
}
function deletedRecordErr() {
return new ErrorWithCode('Record has been deleted', 'ERR_AAOL_DELETED_RECORD')
}
function delDuringCompactErr() {
return new Error('Cannot delete while compaction is in progress')
}
function compactWithMaxLiveStreamErr() {
return new Error(
'Compaction cannot run if there are live streams ' +
'configured with opts.lt or opts.lte'
)
}
function appendLargerThanBlockErr() {
return new Error('Data to be appended is larger than block size')
}
function appendTransactionWantsArrayErr() {
return new Error('appendTransaction expects first argument to be an array')
}
function unexpectedTruncationErr() {
return new Error(
'truncate() is trying to *increase* the log size, ' +
'which is totally unexpected. ' +
'There may be a logic bug in async-append-only-log'
)
}
module.exports = {
ErrorWithCode,
nanOffsetErr,
negativeOffsetErr,
outOfBoundsOffsetErr,
deletedRecordErr,
delDuringCompactErr,
compactWithMaxLiveStreamErr,
appendLargerThanBlockErr,
appendTransactionWantsArrayErr,
unexpectedTruncationErr,
}

878
lib/log/index.js Normal file
View File

@ -0,0 +1,878 @@
const fs = require('fs')
const b4a = require('b4a')
const Cache = require('@alloc/quick-lru') // @ts-ignore
const RAF = require('polyraf') // @ts-ignore
const Obv = require('obz') // @ts-ignore
const AtomicFile = require('atomic-file-rw') // @ts-ignore
const debounce = require('lodash.debounce') // @ts-ignore
const isBufferZero = require('is-buffer-zero') // @ts-ignore
const debug = require('debug')('async-append-only-log') // @ts-ignore
const mutexify = require('mutexify')
const {
deletedRecordErr,
nanOffsetErr,
negativeOffsetErr,
outOfBoundsOffsetErr,
// delDuringCompactErr,
appendLargerThanBlockErr,
appendTransactionWantsArrayErr,
unexpectedTruncationErr,
// compactWithMaxLiveStreamErr,
} = require('./errors')
const Record = require('./record')
// const Compaction = require('./compaction') // FIXME:
/**
* @typedef {Buffer | Uint8Array} B4A
* @typedef {number} BlockIndex
* @typedef {{
* encode: (data: any) => B4A,
* decode: (data: B4A) => any
* }} Codec
* @typedef {{
* blockSize?: number,
* codec?: Codec,
* writeTimeout?: number,
* validateRecord?: (data: B4A) => boolean
* }} Options
*/
/**
* @template T
* @typedef {T extends void ?
* (...args: [Error] | []) => void :
* (...args: [Error] | [null, T]) => void
* } CB
*/
/**
* @param {unknown} check
* @param {string} message
* @returns {asserts check}
*/
function assert(check, message) {
if (!check) throw new Error(message)
}
/**
* The "End of Block" is a special field used to mark the end of a block, and
* in practice it's like a Record header "length" field, with the value 0.
* In most cases, the end region of a block will have a larger length than this,
* but we want to guarantee there is at *least* this many bytes at the end.
*/
const EOB = {
SIZE: Record.HEADER_SIZE,
asNumber: 0,
}
/** @type {Codec} */
const DEFAULT_CODEC = { encode: (x) => x, decode: (x) => x }
const DEFAULT_BLOCK_SIZE = 65536
const DEFAULT_WRITE_TIMEOUT = 250
const DEFAULT_VALIDATE = () => true
// const COMPACTION_PROGRESS_EMIT_INTERVAL = 1000 // FIXME:
/**
* @param {string} filename
* @param {Options} opts
*/
function AsyncAppendOnlyLog(filename, opts) {
const cache = new Cache({ maxSize: 1024 }) // This is potentially 64 MiB!
const raf = RAF(filename)
const statsFilename = filename + '.stats'
const blockSize = opts?.blockSize ?? DEFAULT_BLOCK_SIZE
const codec = opts?.codec ?? DEFAULT_CODEC
const writeTimeout = opts?.writeTimeout ?? DEFAULT_WRITE_TIMEOUT
const validateRecord = opts?.validateRecord ?? DEFAULT_VALIDATE
/**
* @type {Array<CallableFunction>}
*/
const waitingLoad = []
/** @type {Map<BlockIndex, Array<CallableFunction>>} */
const waitingDrain = new Map() // blockIndex -> []
/** @type {Array<CB<any>>} */
const waitingFlushDelete = []
/** @type {Map<BlockIndex, {blockBuf: B4A; offset: number}>} */
const blocksToBeWritten = new Map() // blockIndex -> { blockBuf, offset }
/** @type {Map<BlockIndex, B4A>} */
const blocksWithDeletables = new Map() // blockIndex -> blockBuf
let flushingDelete = false
let writingBlockIndex = -1
let latestBlockBuf = /** @type {B4A | null} */ (null)
let latestBlockIndex = /** @type {number | null} */ (null)
let nextOffsetInBlock = /** @type {number | null} */ (null)
let deletedBytes = 0
const since = Obv() // offset of last written record
// let compaction = null // FIXME:
// const compactionProgress = Obv()
// if (typeof window !== 'undefined') {
// // fs sync not working in browser
// compactionProgress.set({ percent: 1, done: true, sizeDiff: 0 })
// } else {
// compactionProgress.set(
// Compaction.stateFileExists(filename)
// ? { percent: 0, done: false }
// : { percent: 1, done: true, sizeDiff: 0 }
// )
// }
// const waitingCompaction = []
// onLoad(function maybeResumeCompaction() {
// // fs sync not working in browser
// if (typeof window !== 'undefined') return
// if (Compaction.stateFileExists(filename)) {
// compact(function onCompactDone(err) {
// if (err) throw err
// })
// }
// })()
AtomicFile.readFile(
statsFilename,
'utf8',
/** @type {CB<string>} */
function statsUp(err, json) {
if (err) {
debug('error loading stats: %s', err.message)
deletedBytes = 0
} else {
try {
const stats = JSON.parse(json)
deletedBytes = stats.deletedBytes
} catch (err) {
debug('error parsing stats: %s', /** @type {Error} */ (err).message)
deletedBytes = 0
}
}
raf.stat(
/** @type {CB<{size: number}>} */
function onRAFStatDone(err, stat) {
if (err) debug('failed to stat ' + filename, err)
const fileSize = stat ? stat.size : -1
if (fileSize <= 0) {
debug('empty file')
latestBlockBuf = b4a.alloc(blockSize)
latestBlockIndex = 0
nextOffsetInBlock = 0
cache.set(0, latestBlockBuf)
since.set(-1)
// @ts-ignore
while (waitingLoad.length) waitingLoad.shift()()
} else {
const blockStart = fileSize - blockSize
loadLatestBlock(blockStart, function onLoadedLatestBlock(err) {
if (err) throw err
debug('opened file, since: %d', since.value)
// @ts-ignore
while (waitingLoad.length) waitingLoad.shift()()
})
}
}
)
}
)
/**
* @param {number} blockStart
* @param {CB<void>} cb
*/
function loadLatestBlock(blockStart, cb) {
raf.read(
blockStart,
blockSize,
/** @type {CB<B4A>} */
(
function onRAFReadLastDone(err, blockBuf) {
if (err) return cb(err)
getLastGoodRecord(
blockBuf,
blockStart,
function gotLastGoodRecord(err, offsetInBlock) {
if (err) return cb(err)
latestBlockBuf = blockBuf
latestBlockIndex = blockStart / blockSize
const recSize = Record.readSize(blockBuf, offsetInBlock)
nextOffsetInBlock = offsetInBlock + recSize
since.set(blockStart + offsetInBlock)
cb()
}
)
}
)
)
}
/**
* @param {number} offset
*/
function getOffsetInBlock(offset) {
return offset % blockSize
}
/**
* @param {number} offset
*/
function getBlockStart(offset) {
return offset - getOffsetInBlock(offset)
}
/**
* @param {number} offset
*/
function getNextBlockStart(offset) {
return getBlockStart(offset) + blockSize
}
/**
* @param {number} offset
*/
function getBlockIndex(offset) {
return getBlockStart(offset) / blockSize
}
/** @type {(fn: (unlock: (cb: CB<any>, ...args: ([Error] | [null, any])) => void) => void) => void} */
const writeLock = mutexify()
/**
* @template T
* @param {number} blockStart
* @param {B4A | undefined} blockBuf
* @param {T} successValue
* @param {CB<T>} cb
*/
function writeWithFSync(blockStart, blockBuf, successValue, cb) {
writeLock(function onWriteLockReleased(unlock) {
raf.write(
blockStart,
blockBuf,
function onRAFWriteDone(/** @type {Error | null} */ err) {
if (err) return unlock(cb, err)
if (raf.fd) {
fs.fsync(raf.fd, function onFSyncDone(err) {
if (err) unlock(cb, err)
else unlock(cb, null, successValue)
})
} else unlock(cb, null, successValue)
}
)
})
}
/**
* @param {number} newSize
* @param {CB<void>} cb
*/
function truncateWithFSync(newSize, cb) {
writeLock(function onWriteLockReleasedForTruncate(unlock) {
raf.del(
newSize,
Infinity,
function onRAFDeleteDone(/** @type {Error | null} */ err) {
if (err) return unlock(cb, err)
if (raf.fd) {
fs.fsync(raf.fd, function onFSyncDoneForTruncate(err) {
if (err) unlock(cb, err)
else unlock(cb, null, undefined)
})
} else unlock(cb, null, undefined)
}
)
})
}
/**
* @param {B4A} blockBuf
* @param {number} badOffsetInBlock
* @param {number} blockStart
* @param {number} successValue
* @param {CB<number>} cb
*/
function fixBlock(blockBuf, badOffsetInBlock, blockStart, successValue, cb) {
debug('found invalid record at %d, fixing last block', badOffsetInBlock)
blockBuf.fill(0, badOffsetInBlock, blockSize)
writeWithFSync(blockStart, blockBuf, successValue, cb)
}
/**
* @param {B4A} blockBuf
* @param {number} blockStart
* @param {CB<number>} cb
*/
function getLastGoodRecord(blockBuf, blockStart, cb) {
let lastGoodOffset = 0
for (let offsetInRecord = 0; offsetInRecord < blockSize; ) {
const length = Record.readDataLength(blockBuf, offsetInRecord)
if (length === EOB.asNumber) break
const [dataBuf, recSize] = Record.read(blockBuf, offsetInRecord)
const isLengthCorrupt = offsetInRecord + recSize > blockSize
const isDataCorrupt = !isBufferZero(dataBuf) && !validateRecord(dataBuf)
if (isLengthCorrupt || isDataCorrupt) {
fixBlock(blockBuf, offsetInRecord, blockStart, lastGoodOffset, cb)
return
}
lastGoodOffset = offsetInRecord
offsetInRecord += recSize
}
cb(null, lastGoodOffset)
}
/**
* @param {number} offset
* @param {CB<B4A>} cb
*/
function getBlock(offset, cb) {
const blockIndex = getBlockIndex(offset)
if (cache.has(blockIndex)) {
debug('getting offset %d from cache', offset)
const cachedBlockBuf = cache.get(blockIndex)
cb(null, cachedBlockBuf)
} else {
debug('getting offset %d from disc', offset)
const blockStart = getBlockStart(offset)
raf.read(
blockStart,
blockSize,
/** @type {CB<B4A>} */
(
function onRAFReadDone(err, blockBuf) {
if (err) return cb(err)
cache.set(blockIndex, blockBuf)
cb(null, blockBuf)
}
)
)
}
}
/**
* @param {number} offset
* @param {CB<B4A>} cb
*/
function get(offset, cb) {
assert(typeof latestBlockIndex === 'number', 'latestBlockIndex not set')
assert(typeof nextOffsetInBlock === 'number', 'nextOffsetInBlock not set')
const logSize = latestBlockIndex * blockSize + nextOffsetInBlock
if (typeof offset !== 'number') return cb(nanOffsetErr(offset))
if (isNaN(offset)) return cb(nanOffsetErr(offset))
if (offset < 0) return cb(negativeOffsetErr(offset))
if (offset >= logSize) return cb(outOfBoundsOffsetErr(offset, logSize))
getBlock(offset, function gotBlock(err, blockBuf) {
if (err) return cb(err)
const [dataBuf] = Record.read(blockBuf, getOffsetInBlock(offset))
if (isBufferZero(dataBuf)) return cb(deletedRecordErr())
cb(null, codec.decode(dataBuf))
})
}
/**
* Returns [nextOffset, decodedRecord, recordSize] where nextOffset can take 3
* forms:
* * `-1`: end of log
* * `0`: need a new block
* * `>0`: next record within block
* @param {Buffer} blockBuf
* @param {number} offset
* @param {boolean} asRaw
*/
function getDataNextOffset(blockBuf, offset, asRaw = false) {
const offsetInBlock = getOffsetInBlock(offset)
const [dataBuf, recSize] = Record.read(blockBuf, offsetInBlock)
const nextLength = Record.readDataLength(blockBuf, offsetInBlock + recSize)
let nextOffset
if (nextLength === EOB.asNumber) {
if (getNextBlockStart(offset) > since.value) nextOffset = -1
else nextOffset = 0
} else {
nextOffset = offset + recSize
}
if (isBufferZero(dataBuf)) return [nextOffset, null, recSize]
else return [nextOffset, asRaw ? dataBuf : codec.decode(dataBuf), recSize]
}
/**
* @param {(offset: number, value: B4A, size: number) => void} onNext
* @param {(error?: Error) => void} onDone
*/
function scan(onNext, onDone) {
let cursor = 0
const gotNextBlock =
/** @type {CB<B4A>} */
(
(err, blockBuf) => {
if (err) return onDone(err)
if (isBufferZero(blockBuf)) return onDone()
while (true) {
const [offset, value, size] = getDataNextOffset(blockBuf, cursor)
onNext(cursor, value, size)
if (offset === 0) {
cursor = getNextBlockStart(cursor)
getNextBlock()
return
} else if (offset === -1) {
onDone()
return
} else {
cursor = offset
}
}
}
)
function getNextBlock() {
setTimeout(getBlock, 0, cursor, gotNextBlock)
}
getNextBlock()
}
/**
* @param {number} offset
* @param {CB<void>} cb
*/
function del(offset, cb) {
// if (compaction) { // FIXME:
// cb(delDuringCompactErr())
// return
// }
const blockIndex = getBlockIndex(offset)
if (blocksToBeWritten.has(blockIndex)) {
onDrain(function delAfterDrained() {
del(offset, cb)
})
return
}
const gotBlockForDelete = /** @type {CB<B4A>} */ (
(err, blockBuf) => {
if (err) return cb(err)
assert(blockBuf, 'blockBuf should be defined in gotBlockForDelete')
const actualBlockBuf = blocksWithDeletables.get(blockIndex) ?? blockBuf
Record.overwriteWithZeroes(actualBlockBuf, getOffsetInBlock(offset))
deletedBytes += Record.readSize(
actualBlockBuf,
getOffsetInBlock(offset)
)
blocksWithDeletables.set(blockIndex, actualBlockBuf)
scheduleFlushDelete()
cb()
}
)
if (blocksWithDeletables.has(blockIndex)) {
const blockBuf = /** @type {any} */ (blocksWithDeletables.get(blockIndex))
gotBlockForDelete(null, blockBuf)
} else {
getBlock(offset, gotBlockForDelete)
}
}
/**
* @param {Uint8Array} dataBuf
* @param {number} offsetInBlock
*/
function hasNoSpaceFor(dataBuf, offsetInBlock) {
return offsetInBlock + Record.size(dataBuf) + EOB.SIZE > blockSize
}
const scheduleFlushDelete = debounce(flushDelete, writeTimeout)
function flushDelete() {
if (blocksWithDeletables.size === 0) {
for (const cb of waitingFlushDelete) cb()
waitingFlushDelete.length = 0
return
}
const blockIndex = blocksWithDeletables.keys().next().value
const blockStart = blockIndex * blockSize
const blockBuf = blocksWithDeletables.get(blockIndex)
blocksWithDeletables.delete(blockIndex)
flushingDelete = true
writeWithFSync(blockStart, blockBuf, null, function flushedDelete(err, _) {
saveStats(function onSavedStats(err, _) {
if (err) debug('error saving stats: %s', err.message)
flushingDelete = false
if (err) {
for (const cb of waitingFlushDelete) cb(err)
waitingFlushDelete.length = 0
return
}
flushDelete() // next
})
})
}
/**
* @param {CB<void>} cb
*/
function onDeletesFlushed(cb) {
if (flushingDelete || blocksWithDeletables.size > 0) {
waitingFlushDelete.push(cb)
} else cb()
}
/**
* @param {any} data
* @returns {number}
*/
function appendSingle(data) {
let encodedData = codec.encode(data)
if (typeof encodedData === 'string') encodedData = b4a.from(encodedData)
if (Record.size(encodedData) + EOB.SIZE > blockSize)
throw appendLargerThanBlockErr()
assert(typeof latestBlockIndex === 'number', 'latestBlockIndex not set')
assert(typeof nextOffsetInBlock === 'number', 'nextOffsetInBlock not set')
if (hasNoSpaceFor(encodedData, nextOffsetInBlock)) {
const nextBlockBuf = b4a.alloc(blockSize)
latestBlockBuf = nextBlockBuf
latestBlockIndex += 1
nextOffsetInBlock = 0
debug("data doesn't fit current block, creating new")
}
assert(latestBlockBuf, 'latestBlockBuf not set')
Record.write(latestBlockBuf, nextOffsetInBlock, encodedData)
cache.set(latestBlockIndex, latestBlockBuf) // update cache
const offset = latestBlockIndex * blockSize + nextOffsetInBlock
blocksToBeWritten.set(latestBlockIndex, {
blockBuf: latestBlockBuf,
offset,
})
nextOffsetInBlock += Record.size(encodedData)
scheduleWrite()
debug('data inserted at offset %d', offset)
return offset
}
/**
* @param {any | Array<any>} data
* @param {CB<number>} cb
*/
function append(data, cb) {
// if (compaction) { // FIXME:
// waitingCompaction.push(() => append(data, cb))
// return
// }
if (Array.isArray(data)) {
let offset = 0
for (let i = 0, length = data.length; i < length; ++i)
offset = appendSingle(data[i])
cb(null, offset)
} else cb(null, appendSingle(data))
}
/**
* @param {Array<any>} dataArray
* @param {CB<Array<number>>} cb
*/
function appendTransaction(dataArray, cb) {
if (!Array.isArray(dataArray)) {
return cb(appendTransactionWantsArrayErr())
}
// if (compaction) { // FIXME:
// waitingCompaction.push(() => appendTransaction(dataArray, cb))
// return
// }
let size = 0
const encodedDataArray = dataArray.map((data) => {
let encodedData = codec.encode(data)
if (typeof encodedData === 'string') encodedData = b4a.from(encodedData)
size += Record.size(encodedData)
return encodedData
})
size += EOB.SIZE
if (size > blockSize) return cb(appendLargerThanBlockErr())
assert(typeof nextOffsetInBlock === 'number', 'nextOffsetInBlock not set')
assert(typeof latestBlockIndex === 'number', 'latestBlockIndex not set')
if (nextOffsetInBlock + size > blockSize) {
// doesn't fit
const nextBlockBuf = b4a.alloc(blockSize)
latestBlockBuf = nextBlockBuf
latestBlockIndex += 1
nextOffsetInBlock = 0
debug("data doesn't fit current block, creating new")
}
assert(latestBlockBuf, 'latestBlockBuf not set')
const offsets = /** @type {Array<number>} */ ([])
for (const encodedData of encodedDataArray) {
Record.write(latestBlockBuf, nextOffsetInBlock, encodedData)
cache.set(latestBlockIndex, latestBlockBuf) // update cache
const offset = latestBlockIndex * blockSize + nextOffsetInBlock
offsets.push(offset)
blocksToBeWritten.set(latestBlockIndex, {
blockBuf: latestBlockBuf,
offset,
})
nextOffsetInBlock += Record.size(encodedData)
debug('data inserted at offset %d', offset)
}
scheduleWrite()
return cb(null, offsets)
}
const scheduleWrite = debounce(write, writeTimeout)
function write() {
if (blocksToBeWritten.size === 0) return
const blockIndex = blocksToBeWritten.keys().next().value
const blockStart = blockIndex * blockSize
const { blockBuf, offset } =
/** @type {{ blockBuf: B4A, offset: number }} */ (
blocksToBeWritten.get(blockIndex)
)
blocksToBeWritten.delete(blockIndex)
// prettier-ignore
debug('writing block of size: %d, to offset: %d', blockBuf.length, blockIndex * blockSize)
writingBlockIndex = blockIndex
writeWithFSync(blockStart, blockBuf, null, function onBlockWritten(err, _) {
const drainsBefore = (waitingDrain.get(blockIndex) || []).slice(0)
writingBlockIndex = -1
if (err) {
debug('failed to write block %d', blockIndex)
throw err
} else {
since.set(offset)
// prettier-ignore
debug('draining the waiting queue for %d, items: %d', blockIndex, drainsBefore.length)
for (let i = 0; i < drainsBefore.length; ++i) drainsBefore[i]()
// the resumed streams might have added more to waiting
let drainsAfter = waitingDrain.get(blockIndex) || []
if (drainsBefore.length === drainsAfter.length)
waitingDrain.delete(blockIndex)
else if (drainsAfter.length === 0) waitingDrain.delete(blockIndex)
else
waitingDrain.set(
blockIndex,
// @ts-ignore // FIXME:
waitingDrain.get(blockIndex).slice(drainsBefore.length)
)
write() // next!
}
})
}
/**
* @param {number} blockIndex
* @param {B4A } blockBuf
* @param {CB<null>} cb
*/
function overwrite(blockIndex, blockBuf, cb) {
cache.set(blockIndex, blockBuf)
const blockStart = blockIndex * blockSize
writeWithFSync(blockStart, blockBuf, null, cb)
}
/**
* @param {number} newLatestBlockIndex
* @param {CB<number>} cb
*/
function truncate(newLatestBlockIndex, cb) {
assert(typeof latestBlockIndex === 'number', 'latestBlockIndex not set')
if (newLatestBlockIndex > latestBlockIndex) {
return cb(unexpectedTruncationErr())
}
if (newLatestBlockIndex === latestBlockIndex) {
const blockStart = latestBlockIndex * blockSize
loadLatestBlock(blockStart, function onTruncateLoadedLatestBlock1(err) {
if (err) cb(err)
else cb(null, 0)
})
return
}
const size = (latestBlockIndex + 1) * blockSize
const newSize = (newLatestBlockIndex + 1) * blockSize
for (let i = newLatestBlockIndex + 1; i < latestBlockIndex; ++i) {
cache.delete(i)
}
truncateWithFSync(newSize, function onTruncateWithFSyncDone(err) {
if (err) return cb(err)
const blockStart = newSize - blockSize
loadLatestBlock(blockStart, function onTruncateLoadedLatestBlock2(err) {
if (err) return cb(err)
const sizeDiff = size - newSize
cb(null, sizeDiff)
})
})
}
/**
* @param {CB<{ totalBytes: number; deletedBytes: number }>} cb
*/
function stats(cb) {
if (since.value == null) {
since((/** @type {number} */ totalBytes) => {
cb(null, { totalBytes: Math.max(0, totalBytes), deletedBytes })
return false
})
} else {
cb(null, { totalBytes: Math.max(0, since.value), deletedBytes })
}
}
/**
* @param {CB<unknown>} cb
*/
function saveStats(cb) {
const stats = JSON.stringify({ deletedBytes })
AtomicFile.writeFile(statsFilename, stats, 'utf8', cb)
}
/**
* @param {CB<unknown>} cb
*/
// function compact(cb) { // FIXME:
// if (compaction) {
// debug('compaction already in progress')
// waitingCompaction.push(cb)
// return
// }
// for (const stream of self.streams) {
// if (stream.live && (stream.max || stream.max_inclusive)) {
// return cb(compactWithMaxLiveStreamErr())
// }
// }
// onStreamsDone(function startCompactAfterStreamsDone() {
// onDrain(function startCompactAfterDrain() {
// onDeletesFlushed(function startCompactAfterDeletes() {
// if (compactionProgress.value.done) {
// compactionProgress.set({ percent: 0, done: false })
// }
// compaction = new Compaction(self, (/** @type {any} */ err, /** @type {any} */ stats) => {
// compaction = null
// if (err) return cb(err)
// deletedBytes = 0
// saveStats(function onSavedStatsAfterCompaction(err) {
// if (err)
// debug('error saving stats after compaction: %s', err.message)
// })
// for (const stream of self.streams) {
// if (stream.live) stream.postCompactionReset(since.value)
// }
// compactionProgress.set({ ...stats, percent: 1, done: true })
// for (const callback of waitingCompaction) callback()
// waitingCompaction.length = 0
// cb()
// })
// let prevUpdate = 0
// compaction.progress((/** @type {any} */ stats) => {
// const now = Date.now()
// if (now - prevUpdate > COMPACTION_PROGRESS_EMIT_INTERVAL) {
// prevUpdate = now
// compactionProgress.set({ ...stats, done: false })
// }
// })
// })
// })
// })
// }
/**
* @param {CB<unknown>} cb
*/
function close(cb) {
onDrain(function closeAfterHavingDrained() {
onDeletesFlushed(function closeAfterDeletesFlushed() {
raf.close(cb)
})
})
}
/**
* @template T
* @param {T} fn
* @returns {T}
*/
function onLoad(fn) {
const fun = /** @type {(this: null | void, ...args: Array<any> )=>void} */ (
fn
)
return /** @type {any} */ (
function waitForLogLoaded(/** @type {any[]} */ ...args) {
if (latestBlockBuf === null) waitingLoad.push(fun.bind(null, ...args))
else fun(...args)
}
)
}
/**
* @param {CallableFunction} fn
*/
function onDrain(fn) {
// if (compaction) { // FIXME:
// waitingCompaction.push(fn)
// return
// }
if (blocksToBeWritten.size === 0 && writingBlockIndex === -1) fn()
else {
const latestBlockIndex = /** @type {number} */ (
blocksToBeWritten.size > 0
? last(blocksToBeWritten.keys())
: writingBlockIndex
)
const drains = waitingDrain.get(latestBlockIndex) || []
drains.push(fn)
waitingDrain.set(latestBlockIndex, drains)
}
}
/**
* @param {IterableIterator<number>} iterable
*/
function last(iterable) {
let res = null
for (let x of iterable) res = x
return res
}
return {
// Public API:
get: onLoad(get),
scan: onLoad(scan),
del: onLoad(del), // TODO
append: onLoad(append), // TODO
appendTransaction: onLoad(appendTransaction),
close: onLoad(close), // TODO
onDrain: onLoad(onDrain), // TODO
onDeletesFlushed: onLoad(onDeletesFlushed),
// compact: onLoad(compact), // FIXME:
// compactionProgress,
since,
stats, // TODO
// Internals needed by ./compaction.js:
filename,
blockSize,
overwrite,
truncate,
hasNoSpaceFor,
}
}
module.exports = AsyncAppendOnlyLog

94
lib/log/record.js Normal file
View File

@ -0,0 +1,94 @@
const b4a = require('b4a')
/**
* @typedef {Buffer | Uint8Array} B4A
*/
/*
Binary format for a Record:
<record>
<dataLength: UInt16LE>
<dataBuf: Arbitrary Bytes>
</record>
The "Header" is the first two bytes for the dataLength.
*/
const HEADER_SIZE = 2 // uint16
/**
* @param {B4A} dataBuf
*/
function size(dataBuf) {
return HEADER_SIZE + dataBuf.length
}
/**
* @param {B4A} blockBuf
* @param {number} offsetInBlock
*/
function readDataLength(blockBuf, offsetInBlock) {
const view = new DataView(
blockBuf.buffer,
blockBuf.byteOffset,
blockBuf.byteLength
)
return view.getUint16(offsetInBlock, true)
}
/**
* @param {B4A} blockBuf
* @param {number} offsetInBlock
*/
function readSize(blockBuf, offsetInBlock) {
const dataLength = readDataLength(blockBuf, offsetInBlock)
return HEADER_SIZE + dataLength
}
/**
* @param {B4A} blockBuf
* @param {number} offsetInBlock
* @returns {[B4A, number]}
*/
function read(blockBuf, offsetInBlock) {
const dataLength = readDataLength(blockBuf, offsetInBlock)
const dataStart = offsetInBlock + HEADER_SIZE
const dataBuf = blockBuf.slice(dataStart, dataStart + dataLength)
const size = HEADER_SIZE + dataLength
return [dataBuf, size]
}
/**
* @param {B4A} blockBuf
* @param {number} offsetInBlock
* @param {B4A} dataBuf
*/
function write(blockBuf, offsetInBlock, dataBuf) {
// write dataLength
const view = new DataView(blockBuf.buffer, blockBuf.byteOffset, blockBuf.byteLength)
view.setUint16(offsetInBlock, dataBuf.length, true)
// write dataBuf
b4a.copy(dataBuf, blockBuf, offsetInBlock + HEADER_SIZE)
}
/**
* @param {B4A} blockBuf
* @param {number} offsetInBlock
*/
function overwriteWithZeroes(blockBuf, offsetInBlock) {
const dataLength = readDataLength(blockBuf, offsetInBlock)
const dataStart = offsetInBlock + HEADER_SIZE
const dataEnd = dataStart + dataLength
blockBuf.fill(0, dataStart, dataEnd)
}
module.exports = {
HEADER_SIZE,
size,
readDataLength,
readSize,
read,
write,
overwriteWithZeroes,
}

View File

@ -27,16 +27,20 @@
}
},
"dependencies": {
"async-append-only-log": "~4.3.10",
"@alloc/quick-lru": "^5.2.0",
"atomic-file-rw": "~0.3.0",
"blake3": "~2.1.7",
"b4a": "~1.6.4",
"bs58": "~5.0.0",
"debug": "^4.3.0",
"is-buffer-zero": "^1.0.0",
"json-canon": "~1.0.0",
"lodash.debounce": "~4.0.8",
"multicb": "~1.2.2",
"mutexify": "~1.4.0",
"obz": "~1.1.0",
"ppppp-keypair": "github:staltz/ppppp-keypair",
"polyraf": "^1.1.0",
"promisify-4loc": "~1.0.0",
"push-stream": "~11.2.0",
"set.prototype.union": "~1.0.2"
@ -44,6 +48,7 @@
"devDependencies": {
"@types/b4a": "^1.6.0",
"c8": "^7.11.0",
"flumecodec": "~0.0.1",
"husky": "^4.3.0",
"ppppp-caps": "github:staltz/ppppp-caps",
"prettier": "^2.6.2",
@ -58,6 +63,7 @@
"prepublishOnly": "npm run clean-check && tsc --build",
"postpublish": "npm run clean-check",
"test": "npm run clean-check && node --test",
"test-verbose": "VERBOSE=1 npm run test",
"format-code": "prettier --write \"(lib|test)/**/*.js\"",
"format-code-staged": "pretty-quick --staged --pattern \"(lib|test)/**/*.js\"",
"coverage": "c8 --reporter=lcov npm run test"

View File

@ -0,0 +1,67 @@
const test = require('node:test')
const assert = require('node:assert')
const fs = require('node:fs')
const p = require('node:util').promisify
const Log = require('../../lib/log')
test('Log get() handles bad offset NaN', async function (t) {
const file = '/tmp/ppppp-db-log-test-bad-offset.log'
try {
fs.unlinkSync(file)
} catch (_) {}
const log = Log(file, { blockSize: 2 * 1024 })
const msg = Buffer.from('testing')
const offset1 = await p(log.append)(msg)
assert.equal(offset1, 0)
await assert.rejects(p(log.get)(NaN), (err) => {
assert.match(err.message, /Offset NaN is not a number/, err.message)
assert.equal(err.code, 'ERR_AAOL_INVALID_OFFSET')
return true
})
await p(log.close)()
})
test('Log get() handles bad offset -1', async function (t) {
const file = '/tmp/ppppp-db-log-test-bad-offset.log'
try {
fs.unlinkSync(file)
} catch (_) {}
const log = Log(file, { blockSize: 2 * 1024 })
const msg = Buffer.from('testing')
const offset1 = await p(log.append)(msg)
assert.equal(offset1, 0)
await assert.rejects(p(log.get)(-1), (err) => {
assert.match(err.message, /Offset -1 is negative/, err.message)
assert.equal(err.code, 'ERR_AAOL_INVALID_OFFSET')
return true
})
await p(log.close)()
})
test('Log get() handles bad offset out of bounds', async function (t) {
const file = '/tmp/ppppp-db-log-test-bad-offset.log'
try {
fs.unlinkSync(file)
} catch (_) {}
const log = Log(file, { blockSize: 2 * 1024 })
const msg = Buffer.from('testing')
const offset1 = await p(log.append)(msg)
assert.equal(offset1, 0)
await assert.rejects(p(log.get)(10240), (err) => {
assert.match(err.message, /Offset 10240 is beyond log size/, err.message)
assert.equal(err.code, 'ERR_AAOL_OFFSET_OUT_OF_BOUNDS')
return true
})
await p(log.close)()
})

77
test/log/basic.test.js Normal file
View File

@ -0,0 +1,77 @@
const test = require('node:test')
const assert = require('node:assert')
const fs = require('node:fs')
const p = require('node:util').promisify
const Log = require('../../lib/log')
test('Log handles basic binary records', async function (t) {
const file = '/tmp/ppppp-db-log-test-basic-binary.log'
try {
fs.unlinkSync(file)
} catch (_) {}
const log = Log(file, { blockSize: 2 * 1024 })
const msg1 = Buffer.from('testing')
const msg2 = Buffer.from('testing2')
const offset1 = await p(log.append)(msg1)
assert.equal(offset1, 0)
const offset2 = await p(log.append)(msg2)
assert.equal(offset2, msg1.length + 2)
const b1 = await p(log.get)(offset1)
assert.equal(b1.toString(), msg1.toString())
const b2 = await p(log.get)(offset2)
assert.equal(b2.toString(), msg2.toString())
await p(log.close)()
})
const json1 = { text: 'testing' }
const json2 = { test: 'testing2' }
test('Log handles basic json records', async function (t) {
const file = '/tmp/ppppp-db-log-test-basic-json.log'
try {
fs.unlinkSync(file)
} catch (_) {}
const log = Log(file, {
blockSize: 2 * 1024,
codec: require('flumecodec/json'),
})
const offset1 = await p(log.append)(json1)
assert.equal(offset1, 0)
const offset2 = await p(log.append)(json2)
assert.equal(offset2, 20)
const rec1 = await p(log.get)(offset1)
assert.deepEqual(rec1, json1)
const rec2 = await p(log.get)(offset2)
assert.deepEqual(rec2, json2)
await p(log.close)()
})
test('Log handles basic json record re-reading', async function (t) {
const file = '/tmp/ppppp-db-log-test-basic-json.log'
const log = Log(file, {
blockSize: 2 * 1024,
codec: require('flumecodec/json'),
})
await p(log.onDrain)()
assert.equal(log.since.value, 20)
const rec1 = await p(log.get)(0)
assert.deepEqual(rec1, json1)
const rec2 = await p(log.get)(20)
assert.deepEqual(rec2, json2)
await p(log.close)()
})

View File

@ -0,0 +1,175 @@
const test = require('node:test')
const assert = require('node:assert')
const fs = require('node:fs')
const p = require('node:util').promisify
const RAF = require('polyraf')
const Log = require('../../lib/log')
function encode(json) {
if (Buffer.isBuffer(json)) return json
return Buffer.from(JSON.stringify(json), 'utf8')
}
function decode(buf) {
return JSON.parse(buf.toString('utf8'))
}
test('Log handles corrupted records', async (t) => {
const file = '/tmp/ppppp-db-log-corrupt-records.log'
await t.test('Simulate corruption', async (t) => {
try {
fs.unlinkSync(file)
} catch (_) {}
const log = Log(file, {
blockSize: 64 * 1024,
codec: { encode, decode },
})
const msg1 = encode({ text: 'testing' })
const msg2 = encode({ bool: true, test: 'x' })
msg2[0] = 0x00
await p(log.append)(msg1)
await p(log.append)(msg2)
await p(log.onDrain)()
})
await test('Re-read without validation', async (t) => {
const log = Log(file, { blockSize: 64 * 1024 })
await p(log.onDrain)()
const arr = []
await new Promise((resolve, reject) => {
log.scan(
(offset, rec, size) => {
arr.push(rec)
},
(err) => {
if (err) reject(err)
else resolve()
}
)
})
// Because these are just buffers we won't see the corruption
assert.equal(arr.length, 2)
await p(log.close)()
})
await test('Re-read with validation', async (t) => {
const log = Log(file, {
blockSize: 64 * 1024,
validateRecord(buf) {
try {
decode(buf)
return true
} catch {
return false
}
},
})
await p(log.onDrain)()
const arr = []
await new Promise((resolve, reject) => {
log.scan(
(offset, rec, size) => {
arr.push(rec)
},
(err) => {
if (err) reject(err)
else resolve()
}
)
})
assert.equal(arr.length, 1)
await p(log.close)()
})
})
test('Log handles corrupted length', async (t) => {
const file = '/tmp/ppppp-db-log-corrupt-length.log'
await t.test('Simulate length corruption', async (t) => {
try {
fs.unlinkSync(file)
} catch (_) {}
const raf = RAF(file)
let block = Buffer.alloc(64 * 1024)
const msg1 = encode({ text: 'testing' })
const msg2 = encode({ bool: true, test: 'testing2' })
block.writeUInt16LE(msg1.length, 0)
msg1.copy(block, 2)
block.writeUInt16LE(65534, 2 + msg1.length) // corrupt!
msg2.copy(block, 2 + msg1.length + 2)
await p(raf.write.bind(raf))(0, block)
await p(raf.close.bind(raf))()
})
await t.test('Re-read without validation', async (t) => {
const log = Log(file, { blockSize: 64 * 1024 })
await p(log.onDrain)()
const arr = []
await new Promise((resolve, reject) => {
log.scan(
(offset, rec, size) => {
arr.push(rec)
},
(err) => {
if (err) reject(err)
else resolve()
}
)
})
assert.equal(arr.length, 1)
const msg = encode({ bool: true, test: 'testing2' })
await p(log.append)(msg)
await p(log.close)()
})
await t.test('Re-read with validation', async (t) => {
const log = Log(file, {
blockSize: 64 * 1024,
validateRecord: (d) => {
try {
decode(d)
return true
} catch (ex) {
return false
}
},
})
await p(log.onDrain)()
const arr = []
await new Promise((resolve, reject) => {
log.scan(
(offset, rec, size) => {
arr.push(rec)
},
(err) => {
if (err) reject(err)
else resolve()
}
)
})
assert.equal(arr.length, 2)
await p(log.close)()
})
})

189
test/log/delete.test.js Normal file
View File

@ -0,0 +1,189 @@
const test = require('node:test')
const assert = require('node:assert')
const fs = require('node:fs')
const p = require('node:util').promisify
const Log = require('../../lib/log')
const msg1 = Buffer.from(
'hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world hello world'
)
const msg2 = Buffer.from(
'hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db hello offset db'
)
const msg3 = Buffer.from(
'hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db hello offsetty db'
)
test('Log performing simple delete', async (t) => {
const file = '/tmp/ppppp-db-log-test-del.log'
try {
fs.unlinkSync(file)
} catch (_) {}
const log = Log(file, { blockSize: 2 * 1024 })
const offset1 = await p(log.append)(msg1)
assert.equal(offset1, 0)
const offset2 = await p(log.append)(msg2)
assert.ok(offset2 > offset1)
const offset3 = await p(log.append)(msg3)
assert.ok(offset3 > offset2)
const buf1 = await p(log.get)(offset1)
assert.equal(buf1.toString(), msg1.toString())
const buf2 = await p(log.get)(offset2)
assert.equal(buf2.toString(), msg2.toString())
const buf3 = await p(log.get)(offset3)
assert.equal(buf3.toString(), msg3.toString())
await p(log.del)(offset2)
await p(log.onDeletesFlushed)()
await assert.rejects(p(log.get)(offset2), (err) => {
assert.ok(err)
assert.equal(err.message, 'Record has been deleted')
assert.equal(err.code, 'ERR_AAOL_DELETED_RECORD')
return true
})
await p(log.close)()
})
test('Log deleted records are not invalid upon re-opening', async (t) => {
const file = '/tmp/ppppp-db-log-test-del-invalid.log'
try {
fs.unlinkSync(file)
} catch (_) {}
const opts = {
blockSize: 2 * 1024,
codec: {
encode(msg) {
return Buffer.from(JSON.stringify(msg), 'utf8')
},
decode(buf) {
return JSON.parse(buf.toString('utf8'))
},
},
validateRecord(buf) {
try {
JSON.parse(buf.toString('utf8'))
return true
} catch {
return false
}
},
}
const log = Log(file, opts)
const offset1 = await p(log.append)({ text: 'm0' })
const offset2 = await p(log.append)({ text: 'm1' })
const offset3 = await p(log.append)({ text: 'm2' })
await p(log.del)(offset2)
await p(log.onDeletesFlushed)()
await p(log.close)()
const log2 = Log(file, opts)
let arr = []
await new Promise((resolve) => {
log2.scan(
(offset, value, size) => {
arr.push(value)
},
(err) => {
assert.ifError(err)
assert.deepEqual(arr, [{ text: 'm0' }, null, { text: 'm2' }])
resolve()
}
)
})
await assert.rejects(p(log2.get)(offset2), (err) => {
assert.ok(err)
assert.equal(err.message, 'Record has been deleted')
assert.equal(err.code, 'ERR_AAOL_DELETED_RECORD')
return true
})
await p(log2.close)()
})
test('Log deletes are handled by scan()', async (t) => {
const file = '/tmp/offset-test_' + Date.now() + '.log'
const log = Log(file, { blockSize: 64 * 1024 })
const buf1 = Buffer.from('hello one')
const buf2 = Buffer.from('hello two')
const offset1 = await p(log.append)(buf1)
const offset2 = await p(log.append)(buf2)
await p(log.del)(offset1)
await p(log.onDrain)()
await p(log.onDeletesFlushed)()
const arr = []
await new Promise((resolve) => {
log.scan(
(offset, rec, length) => {
arr.push(rec)
},
(err) => {
assert.ifError(err)
resolve()
}
)
})
assert.deepEqual(arr, [null, buf2])
await p(log.close)()
})
test('Log can handle many deleted records', { timeout: 60e3 }, async (t) => {
const file = '/tmp/aaol-test-delete-many' + Date.now() + '.log'
const log = Log(file, { blockSize: 64 * 1024 })
const TOTAL = 100000
const offsets = []
const logAppend = p(log.append)
if (process.env.VERBOSE) console.time('append ' + TOTAL)
for (let i = 0; i < TOTAL; i += 1) {
const offset = await logAppend(Buffer.from(`hello ${i}`))
offsets.push(offset)
}
assert('appended records')
if (process.env.VERBOSE) console.timeEnd('append ' + TOTAL)
await p(log.onDrain)()
const logDel = p(log.del)
if (process.env.VERBOSE) console.time('delete ' + TOTAL / 2)
for (let i = 0; i < TOTAL; i += 2) {
await logDel(offsets[i])
}
if (process.env.VERBOSE) console.timeEnd('delete ' + TOTAL / 2)
assert('deleted messages')
await p(log.onDeletesFlushed)()
await new Promise((resolve) => {
let i = 0
log.scan(
(offset, rec, length) => {
if (i % 2 === 0) {
if (rec !== null) assert.fail('record ' + i + ' should be deleted')
} else {
if (rec === null) assert.fail('record ' + i + ' should be present')
}
i += 1
},
(err) => {
assert.ifError(err)
resolve()
}
)
})
await p(log.close)()
})

View File

@ -0,0 +1,66 @@
const test = require('node:test')
const assert = require('node:assert')
const fs = require('node:fs')
const p = require('node:util').promisify
const Log = require('../../lib/log')
var file = '/tmp/ds-test_restart.log'
var msg1 = { text: 'hello world hello world' }
var msg2 = { text: 'hello world hello world 2' }
test('Log (fix buggy write) simple', async (t) => {
try {
fs.unlinkSync(file)
} catch (_) {}
const log = Log(file, {
block: 16 * 1024,
codec: require('flumecodec/json'),
})
const offset1 = await p(log.append)(msg1)
assert.equal(offset1, 0)
const offset2 = await p(log.append)(msg2)
assert.equal(offset2, 36)
await p(log.onDrain)()
let arr = []
await new Promise((resolve) => {
log.scan(
(offset, msg, size) => {
arr.push(msg)
},
(err) => {
assert.ifError(err)
resolve()
}
)
})
assert.deepEqual(arr, [msg1, msg2])
await p(log.close)()
})
test('Log (fix buggy write) reread', async (t) => {
const log = Log(file, {
block: 16 * 1024,
codec: require('flumecodec/json'),
})
await p(log.onDrain)()
let arr = []
await new Promise((resolve) => {
log.scan(
(offset, msg, size) => {
arr.push(msg)
},
(err) => {
assert.ifError(err)
resolve()
}
)
})
assert.deepEqual(arr, [msg1, msg2])
await p(log.close)()
})

View File

@ -8,7 +8,7 @@ test('MsgV3.createAccount()', (t) => {
const keypair = Keypair.generate('ed25519', 'alice')
const accountMsg0 = MsgV3.createAccount(keypair, 'person', 'MYNONCE')
console.log(JSON.stringify(accountMsg0, null, 2))
if (process.env.VERBOSE) console.log(JSON.stringify(accountMsg0, null, 2))
assert.deepEqual(
accountMsg0.data,
@ -45,7 +45,7 @@ test('MsgV3.createMoot()', (t) => {
const keypair = Keypair.generate('ed25519', 'alice')
moot = MsgV3.createMoot(account, 'post', keypair)
console.log(JSON.stringify(moot, null, 2))
if (process.env.VERBOSE) console.log(JSON.stringify(moot, null, 2))
assert.equal(moot.data, null, 'data')
assert.equal(moot.metadata.dataHash, null, 'hash')
@ -78,7 +78,7 @@ test('MsgV3.create()', (t) => {
[mootID]: tangle1,
},
})
console.log(JSON.stringify(msg1, null, 2))
if (process.env.VERBOSE) console.log(JSON.stringify(msg1, null, 2))
assert.deepEqual(msg1.data, data, 'data')
assert.deepEqual(
@ -142,7 +142,7 @@ test('MsgV3.create()', (t) => {
[mootID]: tangle2,
},
})
console.log(JSON.stringify(msg2, null, 2))
if (process.env.VERBOSE) console.log(JSON.stringify(msg2, null, 2))
assert.deepEqual(msg2.data, data2, 'data')
assert.deepEqual(
@ -189,7 +189,7 @@ test('MsgV3.create()', (t) => {
assert.deepEqual(MsgV3.getMsgID(msg2), 'LxWgRRr4wXd29sLDNGNTkr', 'getMsgID')
})
test('create() handles DAG tips correctly', (t) => {
test('MsgV3.create() handles DAG tips correctly', (t) => {
const keypair = Keypair.generate('ed25519', 'alice')
const tangle = new MsgV3.Tangle(mootID)
tangle.add(mootID, moot)

View File

@ -3,7 +3,7 @@ const assert = require('node:assert')
const Keypair = require('ppppp-keypair')
const MsgV3 = require('../../lib/msg-v3')
test('invalid domain not a string', (t) => {
test('MsgV3 invalid domain not a string', (t) => {
const keypair = Keypair.generate('ed25519', 'alice')
assert.throws(
@ -19,7 +19,7 @@ test('invalid domain not a string', (t) => {
)
})
test('invalid domain with "/" character', (t) => {
test('MsgV3 invalid domain with "/" character', (t) => {
const keypair = Keypair.generate('ed25519', 'alice')
assert.throws(
@ -35,7 +35,7 @@ test('invalid domain with "/" character', (t) => {
)
})
test('invalid domain with "*" character', (t) => {
test('MsgV3 invalid domain with "*" character', (t) => {
const keypair = Keypair.generate('ed25519', 'alice')
assert.throws(
@ -51,7 +51,7 @@ test('invalid domain with "*" character', (t) => {
)
})
test('invalid domain too short', (t) => {
test('MsgV3 invalid domain too short', (t) => {
const keypair = Keypair.generate('ed25519', 'alice')
assert.throws(
@ -67,7 +67,7 @@ test('invalid domain too short', (t) => {
)
})
test('invalid domain too long', (t) => {
test('MsgV3 invalid domain too long', (t) => {
const keypair = Keypair.generate('ed25519', 'alice')
assert.throws(

View File

@ -10,7 +10,7 @@ const account = MsgV3.getMsgID(
)
const pubkeys = new Set([keypair.public])
test('invalid msg with non-array prev', (t) => {
test('MsgV3 invalid msg with non-array prev', (t) => {
const keypair = Keypair.generate('ed25519', 'alice')
const moot = MsgV3.createMoot(account, 'post', keypair)
@ -41,7 +41,7 @@ test('invalid msg with non-array prev', (t) => {
)
})
test('invalid msg with bad prev', (t) => {
test('MsgV3 invalid msg with bad prev', (t) => {
const keypair = Keypair.generate('ed25519', 'alice')
const moot = MsgV3.createMoot(account, 'post', keypair)
@ -86,7 +86,7 @@ test('invalid msg with bad prev', (t) => {
)
})
test('invalid msg with URI in prev', (t) => {
test('MsgV3 invalid msg with URI in prev', (t) => {
const keypair = Keypair.generate('ed25519', 'alice')
const moot = MsgV3.createMoot(account, 'post', keypair)
@ -129,7 +129,7 @@ test('invalid msg with URI in prev', (t) => {
assert.match(err, /prev item ".*" is a URI/, 'invalid 2nd msg description')
})
test('invalid msg with unknown prev', (t) => {
test('MsgV3 invalid msg with unknown prev', (t) => {
const keypair = Keypair.generate('ed25519', 'alice')
const moot = MsgV3.createMoot(account, 'post', keypair)
@ -189,7 +189,7 @@ test('invalid msg with unknown prev', (t) => {
)
})
test('invalid feed msg with a different pubkey', (t) => {
test('MsgV3 invalid feed msg with a different pubkey', (t) => {
const keypairA = Keypair.generate('ed25519', 'alice')
const keypairB = Keypair.generate('ed25519', 'bob')
@ -223,7 +223,7 @@ test('invalid feed msg with a different pubkey', (t) => {
)
})
test('invalid feed msg with a different domain', (t) => {
test('MsgV3 invalid feed msg with a different domain', (t) => {
const keypairA = Keypair.generate('ed25519', 'alice')
const moot = MsgV3.createMoot(account, 'post', keypair)
@ -252,7 +252,7 @@ test('invalid feed msg with a different domain', (t) => {
)
})
test('invalid feed msg with non-alphabetical prev', (t) => {
test('MsgV3 invalid feed msg with non-alphabetical prev', (t) => {
const keypair = Keypair.generate('ed25519', 'alice')
const moot = MsgV3.createMoot(account, 'post', keypair)
@ -317,7 +317,7 @@ test('invalid feed msg with non-alphabetical prev', (t) => {
)
})
test('invalid feed msg with duplicate prev', (t) => {
test('MsgV3 invalid feed msg with duplicate prev', (t) => {
const keypair = Keypair.generate('ed25519', 'alice')
const moot = MsgV3.createMoot(account, 'post', keypair)

View File

@ -3,7 +3,7 @@ const assert = require('node:assert')
const Keypair = require('ppppp-keypair')
const MsgV3 = require('../../lib/msg-v3')
test('lipmaa prevs', (t) => {
test('MsgV3 lipmaa prevs', (t) => {
const keypair = Keypair.generate('ed25519', 'alice')
const account = MsgV3.getMsgID(
MsgV3.createAccount(keypair, 'person', 'MYNONCE')

View File

@ -3,7 +3,7 @@ const assert = require('node:assert')
const Keypair = require('ppppp-keypair')
const MsgV3 = require('../../lib/msg-v3')
test('simple multi-author tangle', (t) => {
test('MsgV3.Tangle simple multi-author tangle', (t) => {
const keypairA = Keypair.generate('ed25519', 'alice')
const keypairB = Keypair.generate('ed25519', 'bob')
const accountA = MsgV3.getMsgID(
@ -86,7 +86,7 @@ test('simple multi-author tangle', (t) => {
)
})
test('lipmaa in multi-author tangle', (t) => {
test('MsgV3.Tangle lipmaa in multi-author tangle', (t) => {
const keypairA = Keypair.generate('ed25519', 'alice')
const keypairB = Keypair.generate('ed25519', 'bob')
const accountA = MsgV3.getMsgID(

View File

@ -3,7 +3,7 @@ const assert = require('node:assert')
const Keypair = require('ppppp-keypair')
const MsgV3 = require('../../lib/msg-v3')
test('validate root msg', (t) => {
test('MsgV3 validate root msg', (t) => {
const keypair = Keypair.generate('ed25519', 'alice')
const account = MsgV3.getMsgID(
MsgV3.createAccount(keypair, 'person', 'alice')
@ -19,7 +19,7 @@ test('validate root msg', (t) => {
assert.ifError(err, 'valid root msg')
})
test('validate account tangle', (t) => {
test('MsgV3 validate account tangle', (t) => {
const pubkeys = new Set()
const keypair1 = Keypair.generate('ed25519', 'alice')
pubkeys.add(keypair1.public)
@ -66,7 +66,7 @@ test('validate account tangle', (t) => {
assert.ifError(err, 'valid account msg')
})
test('validate 2nd msg with existing root', (t) => {
test('MsgV3 validate 2nd msg with existing root', (t) => {
const keypair = Keypair.generate('ed25519', 'alice')
const account = MsgV3.getMsgID(
MsgV3.createAccount(keypair, 'person', 'alice')
@ -95,7 +95,7 @@ test('validate 2nd msg with existing root', (t) => {
assert.ifError(err, 'valid 2nd msg')
})
test('validate 2nd forked msg', (t) => {
test('MsgV3 validate 2nd forked msg', (t) => {
const keypair = Keypair.generate('ed25519', 'alice')
const account = MsgV3.getMsgID(
MsgV3.createAccount(keypair, 'person', 'alice')
@ -137,7 +137,7 @@ test('validate 2nd forked msg', (t) => {
assert.ifError(err, 'valid 2nd forked msg')
})
test('validate erased msg', (t) => {
test('MsgV3 validate erased msg', (t) => {
const keypair = Keypair.generate('ed25519', 'alice')
const account = MsgV3.getMsgID(
MsgV3.createAccount(keypair, 'person', 'alice')