Compare commits

...

1 commit

Author SHA1 Message Date
Alex Good
9332ed4ad9
wip 2022-03-20 14:48:30 +00:00
63 changed files with 7781 additions and 84 deletions

2
.gitignore vendored
View file

@ -2,3 +2,5 @@
/.direnv
perf.*
/Cargo.lock
automerge/proptest-regressions/
.vim

3
.vim/coc-settings.json Normal file
View file

@ -0,0 +1,3 @@
{
"rust-analyzer.cargo.features": ["optree-visualisation", "storage-v2"]
}

View file

@ -1,6 +1,7 @@
[workspace]
members = [
"automerge",
"automerge-cli",
"automerge-wasm",
"automerge-cli",
"edit-trace",

View file

@ -9,6 +9,7 @@ license = "MIT"
[features]
optree-visualisation = ["dot"]
wasm = ["js-sys", "wasm-bindgen"]
storage-v2 = []
[dependencies]
hex = "^0.4.3"

View file

@ -51,7 +51,7 @@ fn main() {
doc1.merge(&mut doc2).unwrap();
for change in doc1.get_changes(&[]) {
let length = doc1.length_at(&cards, &[change.hash]);
let length = doc1.length_at(&cards, &[change.hash()]);
println!("{} {}", change.message().unwrap(), length);
}
}

View file

@ -1,9 +1,11 @@
use crate::exid::ExId;
use crate::transaction::{CommitOptions, Transactable};
use crate::{
change::export_change, transaction::TransactionInner, ActorId, Automerge, AutomergeError,
transaction::TransactionInner, ActorId, Automerge, AutomergeError,
Change, ChangeHash, Prop, Value,
};
#[cfg(not(feature = "storage-v2"))]
use crate::change::export_change;
use crate::{sync, Keys, KeysAt, ObjType, ScalarValue};
/// An automerge document that automatically manages transactions.
@ -83,12 +85,12 @@ impl AutoCommit {
.get(&actor)
.and_then(|v| v.get(seq as usize - 1))
.and_then(|&i| self.doc.history.get(i))
.map(|c| c.hash)
.map(|c| c.hash())
.ok_or(AutomergeError::InvalidSeq(seq))
}
fn update_history(&mut self, change: Change) -> usize {
self.doc.max_op = std::cmp::max(self.doc.max_op, change.start_op + change.len() as u64 - 1);
self.doc.max_op = std::cmp::max(self.doc.max_op, change.start_op() + change.len() as u64 - 1);
self.update_deps(&change);
@ -100,17 +102,17 @@ impl AutoCommit {
.or_default()
.push(history_index);
self.doc.history_index.insert(change.hash, history_index);
self.doc.history_index.insert(change.hash(), history_index);
self.doc.history.push(change);
history_index
}
fn update_deps(&mut self, change: &Change) {
for d in &change.deps {
for d in change.deps() {
self.doc.deps.remove(d);
}
self.doc.deps.insert(change.hash);
self.doc.deps.insert(change.hash());
}
pub fn fork(&mut self) -> Self {
@ -123,8 +125,7 @@ impl AutoCommit {
fn ensure_transaction_closed(&mut self) {
if let Some(tx) = self.transaction.take() {
self.update_history(export_change(
tx,
self.update_history(tx.export(
&self.doc.ops.m.actors,
&self.doc.ops.m.props,
));

View file

@ -1,6 +1,9 @@
use std::collections::{HashMap, HashSet, VecDeque};
#[cfg(not(feature = "storage-v2"))]
use crate::change::encode_document;
#[cfg(feature = "storage-v2")]
use crate::columnar_2::{load, rowblock::Key as EncodedKey};
use crate::exid::ExId;
use crate::keys::Keys;
use crate::op_set::OpSet;
@ -396,6 +399,7 @@ impl Automerge {
}
/// Load a document.
#[cfg(not(feature = "storage-v2"))]
pub fn load(data: &[u8]) -> Result<Self, AutomergeError> {
let changes = Change::load_document(data)?;
let mut doc = Self::new();
@ -403,20 +407,51 @@ impl Automerge {
Ok(doc)
}
#[cfg(feature = "storage-v2")]
pub fn load(data: &[u8]) -> Result<Self, AutomergeError> {
let (remaining, loaded) = load::load_opset(data)?;
let mut am = match loaded {
load::LoadOpset::Document { op_set, history_index, history, actor_to_history } => Self {
queue: vec![],
history,
history_index,
states: actor_to_history,
ops: op_set,
deps: Default::default(),
saved: Default::default(),
actor: Actor::Unused(ActorId::random()),
max_op: 0,
},
load::LoadOpset::Change(change) => {
let mut am = Self::new();
am.apply_change(change);
am
}
};
for change in load::load(remaining)? {
am.apply_change(change);
}
Ok(am)
}
/// Load an incremental save of a document.
pub fn load_incremental(&mut self, data: &[u8]) -> Result<usize, AutomergeError> {
#[cfg(not(feature = "storage-v2"))]
let changes = Change::load_document(data)?;
#[cfg(feature = "storage-v2")]
let changes = load::load(data)?;
let start = self.ops.len();
self.apply_changes(changes)?;
let delta = self.ops.len() - start;
Ok(delta)
}
fn duplicate_seq(&self, change: &Change) -> bool {
let mut dup = false;
if let Some(actor_index) = self.ops.m.actors.lookup(change.actor_id()) {
if let Some(s) = self.states.get(&actor_index) {
dup = s.len() >= change.seq as usize;
dup = s.len() >= change.seq() as usize;
}
}
dup
@ -425,10 +460,10 @@ impl Automerge {
/// Apply changes to this document.
pub fn apply_changes(&mut self, changes: Vec<Change>) -> Result<(), AutomergeError> {
for c in changes {
if !self.history_index.contains_key(&c.hash) {
if !self.history_index.contains_key(&c.hash()) {
if self.duplicate_seq(&c) {
return Err(AutomergeError::DuplicateSeqNumber(
c.seq,
c.seq(),
c.actor_id().clone(),
));
}
@ -447,7 +482,7 @@ impl Automerge {
/// Apply a single change to this document.
fn apply_change(&mut self, change: Change) {
let ops = self.import_ops(&change);
let ops = self.import_ops(&change, 0);
self.update_history(change, ops.len());
for (obj, op) in ops {
self.insert_op(&obj, op);
@ -456,7 +491,7 @@ impl Automerge {
fn is_causally_ready(&self, change: &Change) -> bool {
change
.deps
.deps()
.iter()
.all(|d| self.history_index.contains_key(d))
}
@ -472,7 +507,8 @@ impl Automerge {
None
}
fn import_ops(&mut self, change: &Change) -> Vec<(ObjId, Op)> {
#[cfg(not(feature = "storage-v2"))]
fn import_ops(&mut self, change: &Change, change_id: usize) -> Vec<(ObjId, Op)> {
change
.iter_ops()
.enumerate()
@ -510,6 +546,33 @@ impl Automerge {
.collect()
}
#[cfg(feature = "storage-v2")]
fn import_ops(&mut self, change: &Change, change_id: usize) -> Vec<(ObjId, Op)> {
change
.iter_ops()
.enumerate()
.map(|(i, c)| {
let actor = self.ops.m.actors.cache(change.actor_id().clone());
let id = OpId(change.start_op() + i as u64, actor);
let key = match &c.key {
EncodedKey::Prop(n) => Key::Map(self.ops.m.props.cache(n.to_string())),
EncodedKey::Elem(e) => Key::Seq(*e),
};
(
c.obj,
Op {
id,
action: OpType::from_index_and_value(c.action, c.val.into()).unwrap(),
key,
succ: Default::default(),
pred: c.pred,
insert: c.insert,
}
)
})
.collect()
}
/// Takes all the changes in `other` which are not in `self` and applies them
pub fn merge(&mut self, other: &mut Self) -> Result<Vec<ChangeHash>, AutomergeError> {
// TODO: Make this fallible and figure out how to do this transactionally
@ -526,8 +589,10 @@ impl Automerge {
pub fn save(&mut self) -> Vec<u8> {
let heads = self.get_heads();
let c = self.history.iter();
let ops = self.ops.iter();
let bytes = encode_document(heads, c, ops, &self.ops.m.actors, &self.ops.m.props.cache);
#[cfg(not(feature = "storage-v2"))]
let bytes = encode_document(heads, c, self.ops.iter(), &self.ops.m.actors, &self.ops.m.props.cache);
#[cfg(feature = "storage-v2")]
let bytes = crate::columnar_2::save::save_document(c, self.ops.iter(), &self.ops.m.actors, &self.ops.m.props, &heads);
self.saved = self.get_heads();
bytes
}
@ -590,7 +655,7 @@ impl Automerge {
.history_index
.get(hash)
.and_then(|i| self.history.get(*i))
.map(|c| c.deps.as_slice())
.map(|c| c.deps())
.unwrap_or_default()
{
// if we just removed something from our hashes then it is likely there is more
@ -609,10 +674,10 @@ impl Automerge {
/// Get the hashes of the changes in this document that aren't transitive dependencies of the
/// given `heads`.
pub fn get_missing_deps(&self, heads: &[ChangeHash]) -> Vec<ChangeHash> {
let in_queue: HashSet<_> = self.queue.iter().map(|change| change.hash).collect();
let in_queue: HashSet<_> = self.queue.iter().map(|change| change.hash()).collect();
let mut missing = HashSet::new();
for head in self.queue.iter().flat_map(|change| &change.deps) {
for head in self.queue.iter().flat_map(|change| change.deps()) {
if !self.history_index.contains_key(head) {
missing.insert(head);
}
@ -645,16 +710,16 @@ impl Automerge {
+ 1;
let mut missing_changes = vec![];
let mut has_seen: HashSet<_> = have_deps.iter().collect();
let mut has_seen: HashSet<_> = have_deps.into_iter().copied().collect();
for change in &self.history[lowest_idx..] {
let deps_seen = change.deps.iter().filter(|h| has_seen.contains(h)).count();
let deps_seen = change.deps().into_iter().filter(|h| has_seen.contains(*h)).count();
if deps_seen > 0 {
if deps_seen != change.deps.len() {
if deps_seen != change.deps().len() {
// future change depends on something we haven't seen - fast path cant work
return None;
}
missing_changes.push(change);
has_seen.insert(&change.hash);
has_seen.insert(change.hash());
}
}
@ -678,13 +743,13 @@ impl Automerge {
.get(hash)
.and_then(|i| self.history.get(*i))
{
stack.extend(change.deps.iter());
stack.extend(change.deps().iter());
}
has_seen.insert(hash);
}
self.history
.iter()
.filter(|change| !has_seen.contains(&change.hash))
.filter(|change| !has_seen.contains(&change.hash()))
.collect()
}
@ -712,7 +777,7 @@ impl Automerge {
// FIXME - faster
while let Some(hash) = to_see.pop() {
if let Some(c) = self.get_change_by_hash(&hash) {
for h in &c.deps {
for h in c.deps() {
if !seen.contains(h) {
to_see.push(*h);
}
@ -744,7 +809,7 @@ impl Automerge {
seen_hashes.insert(hash);
added_change_hashes.push(hash);
if let Some(change) = other.get_change_by_hash(&hash) {
stack.extend(&change.deps);
stack.extend(change.deps());
}
}
}
@ -769,12 +834,12 @@ impl Automerge {
.get(&actor)
.and_then(|v| v.get(seq as usize - 1))
.and_then(|&i| self.history.get(i))
.map(|c| c.hash)
.map(|c| c.hash())
.ok_or(AutomergeError::InvalidSeq(seq))
}
pub(crate) fn update_history(&mut self, change: Change, num_ops: usize) -> usize {
self.max_op = std::cmp::max(self.max_op, change.start_op + num_ops as u64 - 1);
self.max_op = std::cmp::max(self.max_op, change.start_op() + change.len() as u64 - 1);
self.update_deps(&change);
@ -785,17 +850,17 @@ impl Automerge {
.or_default()
.push(history_index);
self.history_index.insert(change.hash, history_index);
self.history_index.insert(change.hash(), history_index);
self.history.push(change);
history_index
}
fn update_deps(&mut self, change: &Change) {
for d in &change.deps {
for d in change.deps() {
self.deps.remove(d);
}
self.deps.insert(change.hash);
self.deps.insert(change.hash());
}
pub fn import(&self, s: &str) -> Result<ExId, AutomergeError> {
@ -1329,9 +1394,9 @@ mod tests {
let mut tx = doc.transaction();
tx.set(ROOT, "a", 1).unwrap();
tx.commit();
let hash = doc.get_last_local_change().unwrap().hash;
let hash = doc.get_last_local_change().unwrap().hash();
let bytes = doc.save();
let doc = Automerge::load(&bytes).unwrap();
assert_eq!(doc.get_change_by_hash(&hash).unwrap().hash, hash);
assert_eq!(doc.get_change_by_hash(&hash).unwrap().hash(), hash);
}
}

View file

@ -0,0 +1,89 @@
use serde::ser::{SerializeSeq, SerializeMap};
use crate::{ObjId, Automerge, Value, ObjType};
pub struct AutoSerde<'a>(&'a Automerge);
impl<'a> From<&'a Automerge> for AutoSerde<'a> {
fn from(a: &'a Automerge) -> Self {
AutoSerde(a)
}
}
impl<'a> serde::Serialize for AutoSerde<'a> {
fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
where
S: serde::Serializer {
AutoSerdeMap{doc: self.0, obj: ObjId::Root}.serialize(serializer)
}
}
struct AutoSerdeMap<'a>{doc: &'a Automerge, obj: ObjId}
impl<'a> serde::Serialize for AutoSerdeMap<'a> {
fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
where
S: serde::Serializer {
let mut map_ser = serializer.serialize_map(Some(self.doc.length(&ObjId::Root)))?;
for key in self.doc.keys(&self.obj) {
// SAFETY: This only errors if the object ID is unknown, but we construct this type
// with a known real object ID
let (val, obj) = self.doc.value(&self.obj, &key).unwrap().unwrap();
let serdeval = AutoSerdeVal{
doc: &self.doc,
val,
obj,
};
map_ser.serialize_entry(&key, &serdeval)?;
}
map_ser.end()
}
}
struct AutoSerdeSeq<'a>{doc: &'a Automerge, obj: ObjId}
impl<'a> serde::Serialize for AutoSerdeSeq<'a> {
fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
where
S: serde::Serializer {
let mut seq_ser = serializer.serialize_seq(None)?;
for i in 0..self.doc.length(&self.obj) {
// SAFETY: This only errors if the object ID is unknown, but we construct this type
// with a known real object ID
let (val, obj) = self.doc.value(&self.obj, i).unwrap().unwrap();
let serdeval = AutoSerdeVal{
doc: &self.doc,
val,
obj,
};
seq_ser.serialize_element(&serdeval)?;
}
seq_ser.end()
}
}
struct AutoSerdeVal<'a>{
doc: &'a Automerge,
val: Value,
obj: ObjId,
}
impl<'a> serde::Serialize for AutoSerdeVal<'a> {
fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
where
S: serde::Serializer {
match &self.val {
Value::Object(ObjType::Map | ObjType::Table) => {
let map = AutoSerdeMap{doc: &self.doc, obj: self.obj.clone()};
map.serialize(serializer)
},
Value::Object(ObjType::List | ObjType::Text) => {
let seq = AutoSerdeSeq{doc: &self.doc, obj: self.obj.clone()};
seq.serialize(serializer)
},
Value::Scalar(v) => {
v.serialize(serializer)
}
}
}
}

View file

@ -360,6 +360,22 @@ impl Change {
self.start_op + (self.len() as u64) - 1
}
pub fn deps(&self) -> &[amp::ChangeHash] {
&self.deps
}
pub fn seq(&self) -> u64 {
self.seq
}
pub fn hash(&self) -> amp::ChangeHash {
self.hash
}
pub fn start_op(&self) -> u64 {
self.start_op
}
pub fn message(&self) -> Option<String> {
let m = &self.bytes.uncompressed()[self.message.clone()];
if m.is_empty() {

179
automerge/src/change_v2.rs Normal file
View file

@ -0,0 +1,179 @@
use crate::{
columnar_2::{
rowblock::{
change_op_columns::{ChangeOp, ChangeOpsColumns},
RowBlock,
},
storage::{Change as StoredChange, Chunk, ChunkType},
},
types::{ActorId, ChangeHash},
};
#[derive(Clone, Debug)]
pub struct Change {
stored: StoredChange<'static>,
hash: ChangeHash,
len: usize,
}
impl Change {
pub(crate) fn new(stored: StoredChange<'static>, hash: ChangeHash, len: usize) -> Self {
Self{
stored,
hash,
len,
}
}
pub fn actor_id(&self) -> &ActorId {
&self.stored.actor
}
pub fn is_empty(&self) -> bool {
self.len() == 0
}
pub fn len(&self) -> usize {
self.len
}
pub fn max_op(&self) -> u64 {
self.stored.start_op + (self.len as u64) - 1
}
pub fn start_op(&self) -> u64 {
self.stored.start_op
}
pub fn message(&self) -> Option<&String> {
self.stored.message.as_ref()
}
pub fn deps(&self) -> &[ChangeHash] {
&self.stored.dependencies
}
pub fn hash(&self) -> ChangeHash {
self.hash
}
pub fn seq(&self) -> u64 {
self.stored.seq
}
pub fn timestamp(&self) -> i64 {
self.stored.timestamp
}
pub fn compress(&mut self) {}
pub fn raw_bytes(&self) -> Vec<u8> {
let vec = self.stored.write();
let chunk = Chunk::new_change(&vec);
chunk.write()
}
pub(crate) fn iter_ops<'a>(&'a self) -> impl Iterator<Item= ChangeOp<'a>> {
let rb = RowBlock::new(self.stored.ops_meta.iter(), self.stored.ops_data.clone()).unwrap();
let crb: RowBlock<ChangeOpsColumns> = rb.into_change_ops().unwrap();
let unwrapped = crb.into_iter().map(|r| r.unwrap().into_owned()).collect::<Vec<_>>();
return OperationIterator{
inner: unwrapped.into_iter(),
}
}
pub fn extra_bytes(&self) -> &[u8] {
self.stored.extra_bytes.as_ref()
}
// TODO replace all uses of this with TryFrom<&[u8]>
pub fn from_bytes(bytes: Vec<u8>) -> Result<Self, LoadError> {
Self::try_from(&bytes[..])
}
}
struct OperationIterator<'a> {
inner: std::vec::IntoIter<ChangeOp<'a>>,
}
impl<'a> Iterator for OperationIterator<'a> {
type Item = ChangeOp<'a>;
fn next(&mut self) -> Option<Self::Item> {
self.inner.next()
}
}
impl AsRef<StoredChange<'static>> for Change {
fn as_ref(&self) -> &StoredChange<'static> {
&self.stored
}
}
#[derive(thiserror::Error, Debug)]
pub enum LoadError {
#[error("unable to parse change: {0}")]
Parse(Box<dyn std::error::Error>),
#[error("leftover data after parsing")]
LeftoverData,
#[error("wrong chunk type")]
WrongChunkType,
}
impl<'a> TryFrom<&'a [u8]> for Change {
type Error = LoadError;
fn try_from(value: &'a [u8]) -> Result<Self, Self::Error> {
use crate::columnar_2::rowblock::change_op_columns::ReadChangeOpError;
let (remaining, chunk) = Chunk::parse(value).map_err(|e| LoadError::Parse(Box::new(e)))?;
if remaining.len() > 0 {
return Err(LoadError::LeftoverData);
}
match chunk.typ() {
ChunkType::Change => {
let chunkbytes = chunk.data();
let (_, c) = StoredChange::parse(chunkbytes.as_ref())
.map_err(|e| LoadError::Parse(Box::new(e)))?;
let rb = RowBlock::new(c.ops_meta.iter(), c.ops_data.clone()).unwrap();
let crb: RowBlock<ChangeOpsColumns> = rb.into_change_ops().unwrap();
let mut iter = crb.into_iter();
let ops_len = iter
.try_fold::<_, _, Result<_, ReadChangeOpError>>(0, |acc, op| {
op?;
Ok(acc + 1)
})
.map_err(|e| LoadError::Parse(Box::new(e)))?;
Ok(Self {
stored: c.into_owned(),
hash: chunk.hash(),
len: ops_len,
})
}
_ => Err(LoadError::WrongChunkType),
}
}
}
impl<'a> TryFrom<StoredChange<'a>> for Change {
type Error = LoadError;
fn try_from(c: StoredChange) -> Result<Self, Self::Error> {
use crate::columnar_2::rowblock::change_op_columns::ReadChangeOpError;
let rb = RowBlock::new(c.ops_meta.iter(), c.ops_data.clone()).unwrap();
let crb: RowBlock<ChangeOpsColumns> = rb.into_change_ops().unwrap();
let mut iter = crb.into_iter();
let ops_len = iter
.try_fold::<_, _, Result<_, ReadChangeOpError>>(0, |acc, op| {
op?;
Ok(acc + 1)
})
.map_err(|e| LoadError::Parse(Box::new(e)))?;
let chunkbytes = c.write();
let chunk = Chunk::new_change(chunkbytes.as_ref());
Ok(Self {
stored: c.into_owned(),
hash: chunk.hash(),
len: ops_len,
})
}
}

View file

@ -5,6 +5,7 @@ use std::{
borrow::Cow,
cmp::Ordering,
collections::HashMap,
convert::TryFrom,
io,
io::{Read, Write},
ops::Range,
@ -34,45 +35,10 @@ impl Encodable for Action {
}
}
impl Encodable for [ActorId] {
fn encode<R: Write>(&self, buf: &mut R) -> io::Result<usize> {
let mut len = self.len().encode(buf)?;
for i in self {
len += i.to_bytes().encode(buf)?;
}
Ok(len)
}
}
fn actor_index(actor: &ActorId, actors: &[ActorId]) -> usize {
actors.iter().position(|a| a == actor).unwrap()
}
impl Encodable for ActorId {
fn encode_with_actors<R: Write>(&self, buf: &mut R, actors: &[ActorId]) -> io::Result<usize> {
actor_index(self, actors).encode(buf)
}
fn encode<R: Write>(&self, _buf: &mut R) -> io::Result<usize> {
// we instead encode actors as their position on a sequence
Ok(0)
}
}
impl Encodable for Vec<u8> {
fn encode<R: Write>(&self, buf: &mut R) -> io::Result<usize> {
self.as_slice().encode(buf)
}
}
impl Encodable for &[u8] {
fn encode<R: Write>(&self, buf: &mut R) -> io::Result<usize> {
let head = self.len().encode(buf)?;
buf.write_all(self)?;
Ok(head + self.len())
}
}
pub(crate) struct OperationIterator<'a> {
pub(crate) action: RleDecoder<'a, Action>,
pub(crate) objs: ObjIterator<'a>,
@ -1285,6 +1251,14 @@ const ACTIONS: [Action; 7] = [
Action::MakeTable,
];
impl TryFrom<u64> for Action {
type Error = u64;
fn try_from(value: u64) -> Result<Self, Self::Error> {
ACTIONS.get(value as usize).copied().ok_or(value)
}
}
impl Decodable for Action {
fn decode<R>(bytes: &mut R) -> Option<Self>
where

View file

@ -0,0 +1,271 @@
#[derive(Eq, PartialEq, Clone, Copy)]
pub(crate) struct ColumnSpec(u32);
impl ColumnSpec {
pub(crate) fn new(id: ColumnId, col_type: ColumnType, deflate: bool) -> Self {
let mut raw = id.0 << 4;
raw |= u8::from(col_type) as u32;
if deflate {
raw |= 0b00001000;
} else {
raw &= 0b11110111;
}
ColumnSpec(raw)
}
pub(crate) fn col_type(&self) -> ColumnType {
self.0.to_be_bytes()[3].into()
}
pub(crate) fn id(&self) -> ColumnId {
ColumnId(self.0 >> 4)
}
pub(crate) fn deflate(&self) -> bool {
self.0 & 0b00001000 > 0
}
pub(crate) fn normalize(&self) -> Normalized {
Normalized(self.0 & 0b11110111)
}
}
#[derive(PartialEq, PartialOrd)]
pub(crate) struct Normalized(u32);
impl std::fmt::Debug for ColumnSpec {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
write!(
f,
"ColumnSpec(id: {:?}, type: {}, deflate: {})",
self.id(),
self.col_type(),
self.deflate()
)
}
}
#[derive(Eq, PartialEq, Clone, Copy)]
pub(crate) struct ColumnId(u32);
impl ColumnId {
pub const fn new(raw: u32) -> Self {
ColumnId(raw)
}
}
impl From<u32> for ColumnId {
fn from(raw: u32) -> Self {
Self(raw)
}
}
impl std::fmt::Debug for ColumnId {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
self.0.fmt(f)
}
}
#[derive(Eq, PartialEq, Clone, Copy, Debug)]
pub(crate) enum ColumnType {
Group,
Actor,
Integer,
DeltaInteger,
Boolean,
String,
ValueMetadata,
Value,
}
impl std::fmt::Display for ColumnType {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
match self {
Self::Group => write!(f, "Group"),
Self::Actor => write!(f, "Actor"),
Self::Integer => write!(f, "Integer"),
Self::DeltaInteger => write!(f, "DeltaInteger"),
Self::Boolean => write!(f, "Boolean"),
Self::String => write!(f, "String"),
Self::ValueMetadata => write!(f, "ValueMetadata"),
Self::Value => write!(f, "Value"),
}
}
}
impl From<u8> for ColumnType {
fn from(v: u8) -> Self {
let type_bits = v & 0b00000111;
match type_bits {
0 => Self::Group,
1 => Self::Actor,
2 => Self::Integer,
3 => Self::DeltaInteger,
4 => Self::Boolean,
5 => Self::String,
6 => Self::ValueMetadata,
7 => Self::Value,
_ => unreachable!(),
}
}
}
impl From<ColumnType> for u8 {
fn from(ct: ColumnType) -> Self {
match ct {
ColumnType::Group => 0,
ColumnType::Actor => 1,
ColumnType::Integer => 2,
ColumnType::DeltaInteger => 3,
ColumnType::Boolean => 4,
ColumnType::String => 5,
ColumnType::ValueMetadata => 6,
ColumnType::Value => 7,
}
}
}
impl From<u32> for ColumnSpec {
fn from(raw: u32) -> Self {
ColumnSpec(raw)
}
}
impl From<ColumnSpec> for u32 {
fn from(spec: ColumnSpec) -> Self {
spec.0
}
}
impl From<[u8; 4]> for ColumnSpec {
fn from(raw: [u8; 4]) -> Self {
u32::from_be_bytes(raw).into()
}
}
#[cfg(test)]
mod tests {
use super::*;
#[test]
fn column_spec_encoding() {
struct Scenario {
id: ColumnId,
col_type: ColumnType,
int_val: u32,
}
let scenarios = vec![
Scenario {
id: ColumnId(7),
col_type: ColumnType::Group,
int_val: 112,
},
Scenario {
id: ColumnId(0),
col_type: ColumnType::Actor,
int_val: 1,
},
Scenario {
id: ColumnId(0),
col_type: ColumnType::Integer,
int_val: 2,
},
Scenario {
id: ColumnId(1),
col_type: ColumnType::DeltaInteger,
int_val: 19,
},
Scenario {
id: ColumnId(3),
col_type: ColumnType::Boolean,
int_val: 52,
},
Scenario {
id: ColumnId(1),
col_type: ColumnType::String,
int_val: 21,
},
Scenario {
id: ColumnId(5),
col_type: ColumnType::ValueMetadata,
int_val: 86,
},
Scenario {
id: ColumnId(5),
col_type: ColumnType::Value,
int_val: 87,
},
];
for (index, scenario) in scenarios.into_iter().enumerate() {
let spec = ColumnSpec::new(scenario.id, scenario.col_type, false);
let encoded_val = u32::from(spec);
if encoded_val != scenario.int_val {
panic!(
"Scenario {} failed encoding: expected {} but got {}",
index + 1,
scenario.int_val,
encoded_val
);
}
if spec.col_type() != scenario.col_type {
panic!(
"Scenario {} failed col type: expected {:?} but got {:?}",
index + 1,
scenario.col_type,
spec.col_type()
);
}
if spec.deflate() {
panic!(
"Scenario {} failed: spec returned true for deflate, should have been false",
index + 1
);
}
if spec.id() != scenario.id {
panic!(
"Scenario {} failed id: expected {:?} but got {:?}",
index + 1,
scenario.id,
spec.id()
);
}
let deflated = ColumnSpec::new(scenario.id, scenario.col_type, true);
if deflated.id() != spec.id() {
panic!("Scenario {} failed deflate id test", index + 1);
}
if deflated.col_type() != spec.col_type() {
panic!("Scenario {} failed col type test", index + 1);
}
if !deflated.deflate() {
panic!(
"Scenario {} failed: when deflate bit set deflate returned false",
index + 1
);
}
let expected = scenario.int_val | 0b00001000;
if expected != u32::from(deflated) {
panic!(
"Scenario {} failed deflate bit test, expected {} got {}",
index + 1,
expected,
u32::from(deflated)
);
}
if deflated.normalize() != spec.normalize() {
panic!("Scenario {} failed normalize test", index + 1);
}
}
}
}

View file

@ -0,0 +1,162 @@
use std::collections::HashMap;
use tracing::instrument;
use super::{rowblock, storage};
use crate::{op_set::OpSet, Change};
mod change_collector;
mod loading_document;
#[derive(Debug, thiserror::Error)]
pub enum Error {
#[error("unable to parse chunk: {0}")]
Parse(Box<dyn std::error::Error>),
#[error("invalid change columns: {0}")]
InvalidChangeColumns(Box<dyn std::error::Error>),
#[error("invalid ops columns: {0}")]
InvalidOpsColumns(Box<dyn std::error::Error>),
#[error("a chunk contained leftover data")]
LeftoverData,
#[error("error inflating document chunk ops: {0}")]
InflateDocument(Box<dyn std::error::Error>),
#[error("bad checksum")]
BadChecksum,
}
/// The result of `load_opset`. See the documentation for [`load_opset`] for details on why this is
/// necessary
pub(crate) enum LoadOpset {
/// The data was a "document" chunk so we loaded an op_set
Document {
/// The opset we loaded
op_set: OpSet,
/// The changes
history: Vec<Change>,
/// An index from history index to hash
history_index: HashMap<crate::types::ChangeHash, usize>,
/// An index from actor index to seq to change index
actor_to_history: HashMap<usize, Vec<usize>>,
},
/// The data was a change chunk so we just loaded the change
Change(Change),
}
/// The binary storage format defines several different "chunk types". When we're loading a
/// document for the first time we wish to distinguish between "document" chunk types, and all the
/// others. The reason for this is that the "document" chunk type contains operations encoded in a
/// particular order which we can take advantage of to quickly load an OpSet. For all other chunk
/// types we must proceed as usual by loading changes in order.
///
/// The tuple returned by this function contains as it's first component any data which was not
/// consumed (i.e. data which could be more chunks) and as it's second component the [`LoadOpset`]
/// which represents the two possible alternatives described above.
#[instrument(level = "trace", skip(data))]
pub(crate) fn load_opset<'a>(data: &'a [u8]) -> Result<(&'a [u8], LoadOpset), Error> {
let (remaining, chunk) = storage::Chunk::parse(data).map_err(|e| Error::Parse(Box::new(e)))?;
if !chunk.checksum_valid() {
return Err(Error::BadChecksum);
}
match chunk.typ() {
storage::ChunkType::Document => {
tracing::trace!("loading document chunk");
let data = chunk.data();
let (inner_remaining, doc) =
storage::Document::parse(&data).map_err(|e| Error::Parse(Box::new(e)))?;
if !inner_remaining.is_empty() {
tracing::error!(
remaining = inner_remaining.len(),
"leftover data when parsing document chunk"
);
return Err(Error::LeftoverData);
}
let change_rowblock =
rowblock::RowBlock::new(doc.change_metadata.iter(), doc.change_bytes)
.map_err(|e| Error::InvalidChangeColumns(Box::new(e)))?
.into_doc_change()
.map_err(|e| Error::InvalidChangeColumns(Box::new(e)))?;
let ops_rowblock = rowblock::RowBlock::new(doc.op_metadata.iter(), doc.op_bytes)
.map_err(|e| Error::InvalidOpsColumns(Box::new(e)))?
.into_doc_ops()
.map_err(|e| Error::InvalidOpsColumns(Box::new(e)))?;
let loading_document::Loaded {
op_set,
history,
history_index,
actor_to_history,
..
} = loading_document::load(
doc.actors,
doc.heads.into_iter().collect(),
change_rowblock.into_iter(),
ops_rowblock.into_iter(),
)
.map_err(|e| Error::InflateDocument(Box::new(e)))?;
// TODO: remove this unwrap because we already materialized all the ops
let history = history.into_iter().map(|h| h.try_into().unwrap()).collect();
Ok((
remaining,
LoadOpset::Document {
op_set,
history,
history_index,
actor_to_history,
},
))
}
storage::ChunkType::Change => {
tracing::trace!("loading change chunk");
let data = chunk.data();
let (inner_remaining, change_chunk) =
storage::Change::parse(&data).map_err(|e| Error::Parse(Box::new(e)))?;
if !inner_remaining.is_empty() {
tracing::error!(
remaining = inner_remaining.len(),
"leftover data when parsing document chunk"
);
return Err(Error::LeftoverData);
}
let change_rowblock =
rowblock::RowBlock::new(change_chunk.ops_meta.iter(), change_chunk.ops_data.clone())
.map_err(|e| Error::InvalidOpsColumns(Box::new(e)))?
.into_change_ops()
.map_err(|e| Error::InvalidOpsColumns(Box::new(e)))?;
let len = (&change_rowblock).into_iter().try_fold(0, |acc, c| {
c.map_err(|e| Error::InvalidChangeColumns(Box::new(e)))?;
Ok(acc + 1)
})?;
Ok((
remaining,
LoadOpset::Change(Change::new(change_chunk.into_owned(), chunk.hash(), len)),
))
}
storage::ChunkType::Compressed => panic!(),
}
}
/// Load all the chunks in `data` returning a vector of changes. Note that this will throw an error
/// if there is data left over.
pub(crate) fn load(data: &[u8]) -> Result<Vec<Change>, Error> {
let mut changes = Vec::new();
let mut data = data;
while data.len() > 0 {
let (remaining, load_result) = load_opset(data)?;
match load_result {
LoadOpset::Change(c) => changes.push(c),
LoadOpset::Document { history, .. } => {
for stored_change in history {
changes.push(
Change::try_from(stored_change)
.map_err(|e| Error::InvalidOpsColumns(Box::new(e)))?,
);
}
}
}
data = remaining;
}
Ok(changes)
}

View file

@ -0,0 +1,259 @@
use std::{borrow::Cow, collections::{BTreeSet, HashMap}};
use tracing::instrument;
use crate::{
indexed_cache::IndexedCache,
columnar_2::{
rowblock::{
change_op_columns::{ChangeOp, ChangeOpsColumns},
doc_change_columns::ChangeMetadata,
Key as StoredKey, PrimVal,
},
storage::Change as StoredChange,
},
types::{ActorId, ChangeHash, ElemId, Key, Op, ObjId},
OpType,
};
#[derive(Debug, thiserror::Error)]
pub(crate) enum Error {
#[error("a change referenced an actor index we couldn't find")]
MissingActor,
#[error("changes out of order")]
ChangesOutOfOrder,
#[error("missing change")]
MissingChange,
#[error("some ops were missing")]
MissingOps,
#[error("unable to read change metadata: {0}")]
ReadChange(Box<dyn std::error::Error>),
}
pub(crate) struct ChangeCollector<'a> {
changes_by_actor: HashMap<usize, Vec<PartialChange<'a>>>,
}
pub(crate) struct CollectedChanges<'a> {
pub(crate) history: Vec<StoredChange<'a>>,
pub(crate) history_index: HashMap<ChangeHash, usize>,
pub(crate) actor_to_history: HashMap<usize, Vec<usize>>,
pub(crate) heads: BTreeSet<ChangeHash>,
}
impl<'a> ChangeCollector<'a> {
pub(crate) fn new<E: std::error::Error + 'static, I>(
changes: I,
) -> Result<ChangeCollector<'a>, Error>
where
I: IntoIterator<Item = Result<ChangeMetadata<'a>, E>>,
{
let mut changes_by_actor: HashMap<usize, Vec<PartialChange<'_>>> = HashMap::new();
for (index, change) in changes.into_iter().enumerate() {
tracing::trace!(?change, "importing change metadata");
let change = change.map_err(|e| Error::ReadChange(Box::new(e)))?;
let actor_changes = changes_by_actor.entry(change.actor).or_default();
if let Some(prev) = actor_changes.last() {
if prev.max_op >= change.max_op {
return Err(Error::ChangesOutOfOrder);
}
}
actor_changes.push(PartialChange {
index,
deps: change.deps,
actor: change.actor,
seq: change.seq,
timestamp: change.timestamp,
max_op: change.max_op,
message: change.message,
extra_bytes: change.extra,
ops: Vec::new(),
})
}
let num_changes: usize = changes_by_actor.values().map(|v| v.len()).sum();
tracing::trace!(num_changes, ?changes_by_actor, "change collection context created");
Ok(ChangeCollector { changes_by_actor })
}
#[instrument(skip(self))]
pub(crate) fn collect(&mut self, obj: ObjId, op: Op) -> Result<(), Error> {
let actor_changes = self
.changes_by_actor
.get_mut(&op.id.actor())
.ok_or_else(||{
tracing::error!(missing_actor=op.id.actor(), "missing actor for op");
Error::MissingActor
})?;
let change_index = actor_changes.partition_point(|c| c.max_op < op.id.counter());
let change = actor_changes
.get_mut(change_index)
.ok_or_else(||{
tracing::error!(missing_change_index=change_index, "missing change for op");
Error::MissingChange
})?;
change.ops.push((obj, op));
Ok(())
}
#[instrument(skip(self, actors, props))]
pub(crate) fn finish(
self,
actors: &IndexedCache<ActorId>,
props: &IndexedCache<String>,
) -> Result<CollectedChanges<'static>, Error> {
let mut changes_in_order =
Vec::with_capacity(self.changes_by_actor.values().map(|c| c.len()).sum());
for (_, changes) in self.changes_by_actor {
let mut start_op = 0;
let mut seq = None;
for change in changes {
if change.max_op != start_op + (change.ops.len() as u64) {
tracing::error!(?change, start_op, "missing operations");
return Err(Error::MissingOps);
} else {
start_op = change.max_op;
}
if let Some(seq) = seq {
if seq != change.seq - 1 {
return Err(Error::ChangesOutOfOrder);
}
} else if change.seq != 1 {
return Err(Error::ChangesOutOfOrder);
}
seq = Some(change.seq);
changes_in_order.push(change);
}
}
changes_in_order.sort_by_key(|c| c.index);
let mut hashes_by_index = HashMap::new();
let mut history = Vec::new();
let mut actor_to_history: HashMap<usize, Vec<usize>> = HashMap::new();
let mut heads = BTreeSet::new();
for (index, change) in changes_in_order.into_iter().enumerate() {
actor_to_history
.entry(change.actor)
.or_default()
.push(index);
let finished = change.finish(&hashes_by_index, actors, props)?;
let hash = finished.hash();
hashes_by_index.insert(index, hash);
for dep in &finished.dependencies {
heads.remove(dep);
}
tracing::trace!(?hash, "processing change hash");
heads.insert(hash);
history.push(finished.into_owned());
}
let indices_by_hash = hashes_by_index.into_iter().map(|(k, v)| (v, k)).collect();
Ok(CollectedChanges {
history,
history_index: indices_by_hash,
actor_to_history,
heads,
})
}
}
#[derive(Debug)]
struct PartialChange<'a> {
index: usize,
deps: Vec<u64>,
actor: usize,
seq: u64,
max_op: u64,
timestamp: i64,
message: Option<smol_str::SmolStr>,
extra_bytes: Cow<'a, [u8]>,
ops: Vec<(ObjId, Op)>,
}
impl<'a> PartialChange<'a> {
/// # Panics
///
/// If any op references a property index which is not in `props`
#[instrument(skip(self, known_changes, actors, props))]
fn finish(
self,
known_changes: &HashMap<usize, ChangeHash>,
actors: &IndexedCache<ActorId>,
props: &IndexedCache<String>,
) -> Result<StoredChange<'a>, Error> {
let deps_len = self.deps.len();
let mut deps =
self.deps
.into_iter()
.try_fold(Vec::with_capacity(deps_len), |mut acc, dep| {
acc.push(
known_changes
.get(&(dep as usize))
.cloned()
.ok_or_else(|| {
tracing::error!(dependent_index=self.index, dep_index=dep, "could not find dependency");
Error::MissingChange
})?,
);
Ok(acc)
})?;
deps.sort();
let other_actors =
self.ops
.iter()
.try_fold(Vec::with_capacity(self.ops.len()), |mut acc, (_, op)| {
match op.key {
Key::Seq(ElemId(elem)) => {
if elem.actor() != self.actor {
acc.push(
actors
.safe_get(elem.actor())
.cloned()
.ok_or(Error::MissingActor)?,
);
}
}
Key::Map(_) => {}
};
Ok(acc)
})?;
let mut ops_data = Vec::new();
let num_ops = self.ops.len() as u64;
let columns = ChangeOpsColumns::empty().encode(
self.ops.into_iter().map(|(obj, op)| {
let action_index = op.action.action_index();
ChangeOp {
key: match op.key {
// SAFETY: The caller must ensure that all props in the ops are in the propmap
Key::Map(idx) => StoredKey::Prop(props.safe_get(idx).unwrap().into()),
Key::Seq(elem) => StoredKey::Elem(elem),
},
insert: op.insert,
val: match op.action {
OpType::Make(_) | OpType::Del => PrimVal::Null,
OpType::Inc(i) => PrimVal::Int(i),
OpType::Set(v) => v.into(),
},
action: action_index,
pred: op.pred,
obj,
}
}),
&mut ops_data,
);
Ok(StoredChange {
dependencies: deps,
actor: actors
.safe_get(self.actor)
.cloned()
.ok_or(Error::MissingActor)?,
other_actors,
seq: self.seq,
start_op: self.max_op - num_ops,
timestamp: self.timestamp,
message: self.message.map(|s| s.to_string()),
ops_meta: columns.metadata(),
ops_data: Cow::Owned(ops_data),
extra_bytes: self.extra_bytes,
})
}
}

View file

@ -0,0 +1,241 @@
use fxhash::FxBuildHasher;
use std::collections::{HashMap, BTreeSet};
use tracing::instrument;
use super::change_collector::ChangeCollector;
use crate::{
columnar_2::{
storage::Change as StoredChange,
rowblock::{
Key as DocOpKey,
doc_change_columns::ChangeMetadata,
doc_op_columns::DocOp,
PrimVal,
}
},
op_set::OpSet,
op_tree::{OpSetMetadata, OpTree},
types::{ActorId, ChangeHash, ElemId, Key, ObjId, ObjType, Op, OpId, OpType},
};
#[derive(Debug, thiserror::Error)]
pub(crate) enum Error {
#[error("the document contained ops which were out of order")]
OpsOutOfOrder,
#[error("error reading operation: {0:?}")]
ReadOp(Box<dyn std::error::Error>),
#[error("an operation contained an invalid action")]
InvalidAction,
#[error("an operation referenced a missing actor id")]
MissingActor,
#[error("invalid changes: {0}")]
InvalidChanges(#[from] super::change_collector::Error),
#[error("mismatching heads")]
MismatchingHeads,
}
struct LoadingObject {
id: ObjId,
ops: Vec<Op>,
obj_type: ObjType,
preds: HashMap<OpId, Vec<OpId>>,
}
impl LoadingObject {
fn root() -> Self {
LoadingObject {
id: ObjId::root(),
ops: Vec::new(),
obj_type: ObjType::Map,
preds: HashMap::new(),
}
}
fn new(id: ObjId, obj_type: ObjType) -> Self {
LoadingObject {
id: id.into(),
ops: Vec::new(),
obj_type,
preds: HashMap::new(),
}
}
fn append_op(&mut self, op: Op) -> Result<(), Error> {
if let Some(previous_op) = self.ops.last() {
if op.key < previous_op.key {
tracing::error!(
?op,
?previous_op,
"op key was smaller than key of previous op"
);
return Err(Error::OpsOutOfOrder);
}
}
for succ in &op.succ {
self.preds.entry(*succ).or_default().push(op.id);
}
self.ops.push(op);
Ok(())
}
fn finish(mut self) -> (ObjId, ObjType, OpTree) {
let mut op_tree = OpTree::new();
for (index, mut op) in self.ops.into_iter().enumerate() {
if let Some(preds) = self.preds.remove(&op.id) {
op.pred = preds;
}
op_tree.insert(index, op);
}
(self.id, self.obj_type, op_tree)
}
}
pub(crate) struct Loaded<'a> {
pub(crate) op_set: OpSet,
pub(crate) history: Vec<StoredChange<'a>>,
pub(crate) history_index: HashMap<ChangeHash, usize>,
pub(crate) actor_to_history: HashMap<usize, Vec<usize>>,
}
#[instrument(skip(actors, expected_heads, changes, ops))]
pub(crate) fn load<'a, I, C, OE, CE>(
actors: Vec<ActorId>,
expected_heads: BTreeSet<ChangeHash>,
changes: C,
ops: I,
) -> Result<Loaded<'static>, Error>
where
OE: std::error::Error + 'static,
CE: std::error::Error + 'static,
I: Iterator<Item = Result<DocOp<'a>, OE>>,
C: Iterator<Item = Result<ChangeMetadata<'a>, CE>>,
{
let mut metadata = OpSetMetadata::from_actors(actors);
let mut completed_objects = HashMap::<_, _, FxBuildHasher>::default();
let mut current_object = LoadingObject::root();
let mut collector = ChangeCollector::new(changes)?;
let mut obj_types = HashMap::new();
obj_types.insert(ObjId::root(), ObjType::Map);
for op_res in ops {
let doc_op = op_res.map_err(|e| Error::ReadOp(Box::new(e)))?;
let obj = doc_op.object;
let op = import_op(&mut metadata, doc_op)?;
tracing::trace!(?op, "processing op");
collector.collect(current_object.id, op.clone())?;
// We have to record the object types of make operations so that when the object ID the
// incoming operations refer to switches we can lookup the object type for the new object.
// Ultimately we need this because the OpSet needs to know the object ID _and type_ for
// each OpTree it tracks.
if obj == current_object.id {
match op.action {
OpType::Make(obj_type) => {
obj_types.insert(op.id.into(), obj_type.clone());
}
_ => {}
};
current_object.append_op(op)?;
} else {
let new_obj_type = match obj_types.get(&obj) {
Some(t) => Ok(t.clone()),
None => {
tracing::error!(
?op,
"operation referenced an object which we haven't seen a create op for yet"
);
Err(Error::OpsOutOfOrder)
}
}?;
if obj < current_object.id {
tracing::error!(?op, previous_obj=?current_object.id, "op referenced an object ID which was less than the previous object ID");
return Err(Error::OpsOutOfOrder);
} else {
let (id, obj_type, op_tree) = current_object.finish();
current_object = LoadingObject::new(obj, new_obj_type);
current_object.append_op(op)?;
completed_objects.insert(id, (obj_type, op_tree));
}
}
}
let super::change_collector::CollectedChanges{
history,
history_index,
actor_to_history,
heads,
} = collector.finish(
&metadata.actors,
&metadata.props,
)?;
if expected_heads != heads {
tracing::error!(?expected_heads, ?heads, "mismatching heads");
return Err(Error::MismatchingHeads);
}
let (id, obj_type, op_tree) = current_object.finish();
completed_objects.insert(id, (obj_type, op_tree));
let op_set = OpSet::from_parts(completed_objects, metadata);
Ok(Loaded {
op_set,
history,
history_index,
actor_to_history,
})
}
#[instrument(skip(m))]
fn import_op<'a>(m: &mut OpSetMetadata, op: DocOp<'a>) -> Result<Op, Error> {
let key = match op.key {
DocOpKey::Prop(s) => Key::Map(m.import_prop(s)),
DocOpKey::Elem(ElemId(op)) => Key::Seq(ElemId(check_opid(m, op)?)),
};
for opid in &op.succ {
if m.actors.safe_get(opid.actor()).is_none() {
tracing::error!(?opid, "missing actor");
return Err(Error::MissingActor);
}
}
Ok(Op {
id: check_opid(m, op.id)?,
action: parse_optype(op.action, op.value)?,
key,
succ: op.succ,
pred: Vec::new(),
insert: op.insert,
})
}
/// We construct the OpSetMetadata directly from the vector of actors which are encoded in the
/// start of the document. Therefore we need to check for each opid in the docuemnt that the actor
/// ID which it references actually exists in the metadata.
#[tracing::instrument(skip(m))]
fn check_opid(m: &OpSetMetadata, opid: OpId) -> Result<OpId, Error> {
match m.actors.safe_get(opid.actor()) {
Some(_) => Ok(opid),
None => {
tracing::error!("missing actor");
Err(Error::MissingActor)
}
}
}
fn parse_optype<'a>(action_index: usize, value: PrimVal<'a>) -> Result<OpType, Error> {
match action_index {
0 => Ok(OpType::Make(ObjType::Map)),
1 => Ok(OpType::Set(value.into())),
2 => Ok(OpType::Make(ObjType::List)),
3 => Ok(OpType::Del),
4 => Ok(OpType::Make(ObjType::Text)),
5 => match value {
PrimVal::Int(i) => Ok(OpType::Inc(i)),
_ => {
tracing::error!(?value, "invalid value for counter op");
Err(Error::InvalidAction)
}
},
6 => Ok(OpType::Make(ObjType::Table)),
other => {
tracing::error!(action = other, "unknown action type");
Err(Error::InvalidAction)
}
}
}

View file

@ -0,0 +1,8 @@
mod column_specification;
#[cfg(feature = "storage-v2")]
pub(crate) mod load;
#[cfg(feature = "storage-v2")]
pub(crate) mod save;
pub(crate) mod rowblock;
pub(crate) mod storage;
pub(crate) use column_specification::{ColumnId, ColumnSpec};

View file

@ -0,0 +1,454 @@
use std::{borrow::Borrow, convert::TryFrom, ops::Range};
use crate::{
columnar_2::{
column_specification::ColumnType,
rowblock::{
column_layout::{column::{GroupColRange, ColumnRanges}, ColumnLayout, MismatchingColumn, assert_col_type},
column_range::{
ActorRange, BooleanRange, DeltaIntRange, RawRange, RleIntRange, RleStringRange,
},
encoding::{
BooleanDecoder, DecodeColumnError, Key, KeyDecoder, ObjDecoder,
OpIdListDecoder, RleDecoder, ValueDecoder,
},
PrimVal,
}, ColumnSpec, ColumnId, storage::ColumnMetadata
},
types::{ElemId, ObjId, OpId},
};
#[derive(Clone, Debug, PartialEq)]
pub(crate) struct ChangeOp<'a> {
pub(crate) key: Key,
pub(crate) insert: bool,
pub(crate) val: PrimVal<'a>,
pub(crate) pred: Vec<OpId>,
pub(crate) action: u64,
pub(crate) obj: ObjId,
}
impl<'a> ChangeOp<'a> {
pub(crate) fn into_owned(self) -> ChangeOp<'static> {
ChangeOp {
key: self.key,
insert: self.insert,
val: self.val.into_owned(),
pred: self.pred,
action: self.action,
obj: self.obj,
}
}
}
#[derive(Clone, Debug)]
pub(crate) struct ChangeOpsColumns {
obj_actor: ActorRange,
obj_counter: RleIntRange,
key_actor: ActorRange,
key_counter: DeltaIntRange,
key_string: RleStringRange,
insert: BooleanRange,
action: RleIntRange,
val_meta: RleIntRange,
val_raw: RawRange,
pred_group: RleIntRange,
pred_actor: RleIntRange,
pred_ctr: DeltaIntRange,
}
impl ChangeOpsColumns {
pub(crate) fn empty() -> Self {
ChangeOpsColumns {
obj_actor: (0..0).into(),
obj_counter: (0..0).into(),
key_actor: (0..0).into(),
key_counter: (0..0).into(),
key_string: (0..0).into(),
insert: (0..0).into(),
action: (0..0).into(),
val_meta: (0..0).into(),
val_raw: (0..0).into(),
pred_group: (0..0).into(),
pred_actor: (0..0).into(),
pred_ctr: (0..0).into(),
}
}
pub(crate) fn iter<'a>(&self, data: &'a [u8]) -> ChangeOpsIter<'a> {
ChangeOpsIter {
failed: false,
obj: ObjDecoder::new(self.obj_actor.decoder(data), self.obj_counter.decoder(data)),
key: KeyDecoder::new(
self.key_actor.decoder(data),
self.key_counter.decoder(data),
self.key_string.decoder(data),
),
insert: self.insert.decoder(data),
action: self.action.decoder(data),
val: ValueDecoder::new(self.val_meta.decoder(data), self.val_raw.decoder(data)),
pred: OpIdListDecoder::new(
self.pred_group.decoder(data),
self.pred_actor.decoder(data),
self.pred_ctr.decoder(data),
),
}
}
pub(crate) fn encode<'a, I, C: Borrow<ChangeOp<'a>>>(&self, ops: I, out: &mut Vec<u8>) -> ChangeOpsColumns
where
I: Iterator<Item = C> + Clone,
{
let obj_actor = self.obj_actor.decoder(&[]).splice(
0..0,
ops.clone().map(|o| Some(OpId::from(o.borrow().obj).actor() as u64)),
out,
);
let obj_counter = self.obj_counter.decoder(&[]).splice(
0..0,
ops.clone().map(|o| Some(OpId::from(o.borrow().obj).counter())),
out,
);
let key_actor = self.key_actor.decoder(&[]).splice(
0..0,
ops.clone().map(|o| match o.borrow().key {
Key::Prop(_) => None,
Key::Elem(ElemId(o)) => Some(o.actor() as u64),
}),
out,
);
let key_counter = self.key_counter.decoder(&[]).splice(
0..0,
ops.clone().map(|o| match o.borrow().key {
Key::Prop(_) => None,
Key::Elem(ElemId(o)) => Some(o.counter() as i64),
}),
out,
);
let key_string = self.key_string.decoder(&[]).splice(
0..0,
ops.clone().map(|o| match &o.borrow().key {
Key::Prop(k) => Some(k.clone()),
Key::Elem(_) => None,
}),
out,
);
let insert = self
.insert
.decoder(&[])
.splice(0..0, ops.clone().map(|o| o.borrow().insert), out);
let action =
self.action
.decoder(&[])
.splice(0..0, ops.clone().map(|o| Some(o.borrow().action)), out);
let mut val_dec = ValueDecoder::new(self.val_meta.decoder(&[]), self.val_raw.decoder(&[]));
let (val_meta, val_raw) = val_dec.splice(0..0, ops.clone().map(|o| o.borrow().val.clone()), out);
let mut pred_dec = OpIdListDecoder::new(
self.pred_group.decoder(&[]),
self.pred_actor.decoder(&[]),
self.pred_ctr.decoder(&[]),
);
let (pred_group, pred_actor, pred_ctr) =
pred_dec.splice(0..0, ops.map(|o| o.borrow().pred.clone()), out);
Self {
obj_actor: obj_actor.into(),
obj_counter: obj_counter.into(),
key_actor: key_actor.into(),
key_counter: key_counter.into(),
key_string: key_string.into(),
insert: insert.into(),
action: action.into(),
val_meta: val_meta.into(),
val_raw: val_raw.into(),
pred_group: pred_group.into(),
pred_actor: pred_actor.into(),
pred_ctr: pred_ctr.into(),
}
}
pub(crate) fn metadata(&self) -> ColumnMetadata {
const OBJ_COL_ID: ColumnId = ColumnId::new(0);
const KEY_COL_ID: ColumnId = ColumnId::new(1);
const INSERT_COL_ID: ColumnId = ColumnId::new(3);
const ACTION_COL_ID: ColumnId = ColumnId::new(4);
const VAL_COL_ID: ColumnId = ColumnId::new(5);
const PRED_COL_ID: ColumnId = ColumnId::new(7);
let mut cols = vec![
(ColumnSpec::new(OBJ_COL_ID, ColumnType::Actor, false), self.obj_actor.clone().into()),
(ColumnSpec::new(OBJ_COL_ID, ColumnType::Integer, false), self.obj_counter.clone().into()),
(ColumnSpec::new(KEY_COL_ID, ColumnType::Actor, false), self.key_actor.clone().into()),
(ColumnSpec::new(KEY_COL_ID, ColumnType::DeltaInteger, false), self.key_counter.clone().into()),
(ColumnSpec::new(KEY_COL_ID, ColumnType::String, false), self.key_string.clone().into()),
(ColumnSpec::new(INSERT_COL_ID, ColumnType::Boolean, false), self.insert.clone().into()),
(ColumnSpec::new(ACTION_COL_ID, ColumnType::Integer, false), self.action.clone().into()),
(ColumnSpec::new(VAL_COL_ID, ColumnType::ValueMetadata, false), self.val_meta.clone().into()),
];
if self.val_raw.len() > 0 {
cols.push((
ColumnSpec::new(VAL_COL_ID, ColumnType::Value, false), self.val_raw.clone().into()
));
}
cols.push(
(ColumnSpec::new(PRED_COL_ID, ColumnType::Group, false), self.pred_group.clone().into()),
);
if self.pred_actor.len() > 0 {
cols.extend([
(ColumnSpec::new(PRED_COL_ID, ColumnType::Actor, false), self.pred_actor.clone().into()),
(ColumnSpec::new(PRED_COL_ID, ColumnType::DeltaInteger, false), self.pred_ctr.clone().into()),
]);
}
cols.into_iter().collect()
}
}
#[derive(thiserror::Error, Debug)]
pub(crate) enum ReadChangeOpError {
#[error("unexpected null in column {0}")]
UnexpectedNull(String),
#[error("invalid value in column {column}: {description}")]
InvalidValue { column: String, description: String },
}
pub(crate) struct ChangeOpsIter<'a> {
failed: bool,
obj: ObjDecoder<'a>,
key: KeyDecoder<'a>,
insert: BooleanDecoder<'a>,
action: RleDecoder<'a, u64>,
val: ValueDecoder<'a>,
pred: OpIdListDecoder<'a>,
}
impl<'a> ChangeOpsIter<'a> {
fn done(&self) -> bool {
[
self.obj.done(),
self.key.done(),
self.insert.done(),
self.action.done(),
self.val.done(),
self.pred.done(),
]
.iter()
.all(|e| *e)
}
fn try_next(&mut self) -> Result<Option<ChangeOp<'a>>, ReadChangeOpError> {
if self.failed {
Ok(None)
} else if self.done() {
Ok(None)
} else {
let obj = self
.obj
.next()
.transpose()
.map_err(|e| self.handle_error("object", e))?
.ok_or(ReadChangeOpError::UnexpectedNull("object".to_string()))?;
let key = self
.key
.next()
.transpose()
.map_err(|e| self.handle_error("key", e))?
.ok_or(ReadChangeOpError::UnexpectedNull("key".to_string()))?;
let insert = self
.insert
.next()
.ok_or(ReadChangeOpError::UnexpectedNull("insert".to_string()))?;
let action = self
.action
.next()
.flatten()
.ok_or(ReadChangeOpError::UnexpectedNull("action".to_string()))?;
let val = self
.val
.next()
.transpose()
.map_err(|e| self.handle_error("value", e))?
.ok_or(ReadChangeOpError::UnexpectedNull("value".to_string()))?;
let pred = self
.pred
.next()
.transpose()
.map_err(|e| self.handle_error("pred", e))?
.ok_or(ReadChangeOpError::UnexpectedNull("pred".to_string()))?;
Ok(Some(ChangeOp {
obj: obj.into(),
key,
insert,
action,
val,
pred,
}))
}
}
fn handle_error(
&mut self,
outer_col: &'static str,
err: DecodeColumnError,
) -> ReadChangeOpError {
match err {
DecodeColumnError::InvalidValue {
column,
description,
} => ReadChangeOpError::InvalidValue {
column: format!("{}:{}", outer_col, column),
description,
},
DecodeColumnError::UnexpectedNull(col) => {
ReadChangeOpError::UnexpectedNull(format!("{}:{}", outer_col, col))
}
}
}
}
impl<'a> Iterator for ChangeOpsIter<'a> {
type Item = Result<ChangeOp<'a>, ReadChangeOpError>;
fn next(&mut self) -> Option<Self::Item> {
match self.try_next() {
Ok(v) => v.map(Ok),
Err(e) => {
self.failed = true;
Some(Err(e))
}
}
}
}
#[derive(thiserror::Error, Debug)]
pub(crate) enum ParseChangeColumnsError {
#[error("mismatching column at {index}.")]
MismatchingColumn { index: usize },
#[error("not enough columns")]
NotEnoughColumns,
}
impl From<MismatchingColumn> for ParseChangeColumnsError {
fn from(m: MismatchingColumn) -> Self {
Self::MismatchingColumn{index: m.index}
}
}
impl TryFrom<ColumnLayout> for ChangeOpsColumns {
type Error = ParseChangeColumnsError;
fn try_from(columns: ColumnLayout) -> Result<Self, Self::Error> {
let mut obj_actor: Option<Range<usize>> = None;
let mut obj_ctr: Option<Range<usize>> = None;
let mut key_actor: Option<Range<usize>> = None;
let mut key_ctr: Option<Range<usize>> = None;
let mut key_str: Option<Range<usize>> = None;
let mut insert: Option<Range<usize>> = None;
let mut action: Option<Range<usize>> = None;
let mut val_meta: Option<Range<usize>> = None;
let mut val_raw: Option<Range<usize>> = None;
let mut pred_group: Option<Range<usize>> = None;
let mut pred_actor: Option<Range<usize>> = None;
let mut pred_ctr: Option<Range<usize>> = None;
let mut other = ColumnLayout::empty();
for (index, col) in columns.into_iter().enumerate() {
match index {
0 => assert_col_type(index, col, ColumnType::Actor, &mut obj_actor)?,
1 => assert_col_type(index, col, ColumnType::Integer, &mut obj_ctr)?,
2 => assert_col_type(index, col, ColumnType::Actor, &mut key_actor)?,
3 => assert_col_type(index, col, ColumnType::DeltaInteger, &mut key_ctr)?,
4 => assert_col_type(index, col, ColumnType::String, &mut key_str)?,
5 => assert_col_type(index, col, ColumnType::Boolean, &mut insert)?,
6 => assert_col_type(index, col, ColumnType::Integer, &mut action)?,
7 => match col.ranges() {
ColumnRanges::Value{meta, val} => {
val_meta = Some(meta);
val_raw = Some(val);
},
_ => return Err(ParseChangeColumnsError::MismatchingColumn{ index }),
},
8 => match col.ranges() {
ColumnRanges::Group{num, mut cols} => {
pred_group = Some(num.into());
// If there was no data in the group at all then the columns won't be
// present
if cols.len() == 0 {
pred_actor = Some((0..0).into());
pred_ctr = Some((0..0).into());
} else {
let first = cols.next();
let second = cols.next();
match (first, second) {
(Some(GroupColRange::Single(actor_range)), Some(GroupColRange::Single(ctr_range))) =>
{
pred_actor = Some(actor_range.into());
pred_ctr = Some(ctr_range.into());
},
_ => return Err(ParseChangeColumnsError::MismatchingColumn{ index }),
}
}
if let Some(_) = cols.next() {
return Err(ParseChangeColumnsError::MismatchingColumn{ index });
}
},
_ => return Err(ParseChangeColumnsError::MismatchingColumn{ index }),
},
_ => {
other.append(col);
}
}
}
Ok(ChangeOpsColumns {
obj_actor: obj_actor.ok_or(ParseChangeColumnsError::NotEnoughColumns)?.into(),
obj_counter: obj_ctr.ok_or(ParseChangeColumnsError::NotEnoughColumns)?.into(),
key_actor: key_actor.ok_or(ParseChangeColumnsError::NotEnoughColumns)?.into(),
key_counter: key_ctr.ok_or(ParseChangeColumnsError::NotEnoughColumns)?.into(),
key_string: key_str.ok_or(ParseChangeColumnsError::NotEnoughColumns)?.into(),
insert: insert.ok_or(ParseChangeColumnsError::NotEnoughColumns)?.into(),
action: action.ok_or(ParseChangeColumnsError::NotEnoughColumns)?.into(),
val_meta: val_meta.ok_or(ParseChangeColumnsError::NotEnoughColumns)?.into(),
val_raw: val_raw.ok_or(ParseChangeColumnsError::NotEnoughColumns)?.into(),
pred_group: pred_group.ok_or(ParseChangeColumnsError::NotEnoughColumns)?.into(),
pred_actor: pred_actor.ok_or(ParseChangeColumnsError::NotEnoughColumns)?.into(),
pred_ctr: pred_ctr.ok_or(ParseChangeColumnsError::NotEnoughColumns)?.into(),
})
}
}
#[cfg(test)]
mod tests {
use super::*;
use crate::columnar_2::rowblock::encoding::properties::{key, opid, value};
use proptest::prelude::*;
prop_compose! {
fn change_op()
(key in key(),
value in value(),
pred in proptest::collection::vec(opid(), 0..20),
action in 0_u64..6,
obj in opid(),
insert in any::<bool>()) -> ChangeOp<'static> {
ChangeOp {
obj: obj.into(),
key,
val: value,
pred,
action,
insert,
}
}
}
proptest! {
#[test]
fn test_encode_decode_change_ops(ops in proptest::collection::vec(change_op(), 0..100)) {
let cols = ChangeOpsColumns::empty();
let mut out = Vec::new();
let cols2 = cols.encode(ops.iter(), &mut out);
let decoded = cols2.iter(&out[..]).collect::<Result<Vec<_>, _>>().unwrap();
assert_eq!(ops, decoded);
}
}
}

View file

@ -0,0 +1,603 @@
use std::{borrow::Cow, ops::Range};
use super::{
super::{
encoding::{
generic::{GenericColDecoder, GroupDecoder, SingleLogicalColDecoder},
RawDecoder, RleDecoder, RleEncoder, SimpleColDecoder, ValueDecoder, BooleanDecoder, DeltaDecoder,
},
CellValue, ColumnId, ColumnSpec,
},
ColumnSpliceError,
};
use crate::columnar_2::column_specification::ColumnType;
#[derive(Debug)]
pub(crate) struct Column(ColumnInner);
impl Column {
pub(crate) fn range(&self) -> Range<usize> {
self.0.range()
}
pub(crate) fn ranges<'a>(&'a self) -> ColumnRanges<'a> {
self.0.ranges()
}
pub(crate) fn decoder<'a>(&self, data: &'a [u8]) -> GenericColDecoder<'a> {
self.0.decoder(data)
}
pub(crate) fn splice<'a, I>(
&self,
source: &[u8],
output: &mut Vec<u8>,
replace: Range<usize>,
replace_with: I,
) -> Result<Self, ColumnSpliceError>
where
I: Iterator<Item=CellValue<'a>> + Clone
{
Ok(Self(self.0.splice(source, output, replace, replace_with)?))
}
pub(crate) fn col_type(&self) -> ColumnType {
self.0.col_type()
}
pub fn id(&self) -> ColumnId {
match self.0 {
ColumnInner::Single(SingleColumn { spec: s, .. }) => s.id(),
ColumnInner::Composite(CompositeColumn::Value(ValueColumn { spec, .. })) => spec.id(),
ColumnInner::Composite(CompositeColumn::Group(GroupColumn { spec, .. })) => spec.id(),
}
}
pub(crate) fn spec(&self) -> ColumnSpec {
match &self.0 {
ColumnInner::Single(s) => s.spec,
ColumnInner::Composite(CompositeColumn::Value(v)) => v.spec,
ColumnInner::Composite(CompositeColumn::Group(g)) => g.spec,
}
}
}
#[derive(Debug, Clone)]
enum ColumnInner {
Single(SingleColumn),
Composite(CompositeColumn),
}
pub(crate) enum ColumnRanges<'a> {
Single(Range<usize>),
Group{
num: Range<usize>,
cols: ColRangeIter<'a>,
},
Value {
meta: Range<usize>,
val: Range<usize>,
}
}
pub(crate) enum GroupColRange {
Single(Range<usize>),
Value{
meta: Range<usize>,
val: Range<usize>,
}
}
pub(crate) struct ColRangeIter<'a> {
offset: usize,
cols: &'a [GroupedColumn]
}
impl<'a> Iterator for ColRangeIter<'a> {
type Item = GroupColRange;
fn next(&mut self) -> Option<Self::Item> {
match self.cols.get(self.offset) {
None => None,
Some(GroupedColumn::Single(SingleColumn{range, ..})) => {
self.offset += 1;
Some(GroupColRange::Single(range.clone()))
},
Some(GroupedColumn::Value(ValueColumn{meta, value, ..})) => {
self.offset += 1;
Some(GroupColRange::Value{meta: meta.clone(), val: value.clone()})
}
}
}
}
impl<'a> ExactSizeIterator for ColRangeIter<'a> {
fn len(&self) -> usize {
self.cols.len()
}
}
impl<'a> From<&'a [GroupedColumn]> for ColRangeIter<'a> {
fn from(cols: &'a [GroupedColumn]) -> Self {
ColRangeIter{
cols,
offset: 0,
}
}
}
impl ColumnInner {
pub(crate) fn range(&self) -> Range<usize> {
match self {
Self::Single(SingleColumn { range: r, .. }) => r.clone(),
Self::Composite(CompositeColumn::Value(ValueColumn { meta, value, .. })) => {
meta.start..value.end
}
Self::Composite(CompositeColumn::Group(GroupColumn { num, values, .. })) => {
num.start..values.last().map(|v| v.range().end).unwrap_or(num.end)
}
}
}
pub(crate) fn ranges<'a>(&'a self) -> ColumnRanges<'a> {
match self {
Self::Single(SingleColumn{range, ..}) => ColumnRanges::Single(range.clone()),
Self::Composite(CompositeColumn::Value(ValueColumn{ meta, value, ..})) => ColumnRanges::Value {
meta: meta.clone(),
val: value.clone(),
},
Self::Composite(CompositeColumn::Group(GroupColumn{num, values, ..})) => ColumnRanges::Group {
num: num.clone(),
cols: (&values[..]).into(),
}
}
}
pub(crate) fn decoder<'a>(&self, data: &'a [u8]) -> GenericColDecoder<'a> {
match self {
Self::Single(SingleColumn {
range, col_type, ..
}) => {
let simple = col_type.decoder(&data[range.clone()]);
GenericColDecoder::new_simple(simple)
},
Self::Composite(CompositeColumn::Value(ValueColumn{meta, value,..})) => GenericColDecoder::new_value(
ValueDecoder::new(
RleDecoder::from(Cow::Borrowed(&data[meta.clone()])),
RawDecoder::from(Cow::Borrowed(&data[value.clone()])),
)
),
Self::Composite(CompositeColumn::Group(GroupColumn{num, values, ..})) => {
let num_coder = RleDecoder::from(Cow::from(&data[num.clone()]));
let values = values
.iter()
.map(|gc| match gc {
GroupedColumn::Single(SingleColumn{col_type, range, ..}) => SingleLogicalColDecoder::Simple(
col_type.decoder(&data[range.clone()])
),
GroupedColumn::Value(ValueColumn{ meta, value, .. }) => {
SingleLogicalColDecoder::Value(ValueDecoder::new(
RleDecoder::from(Cow::Borrowed(&data[meta.clone()])),
RawDecoder::from(Cow::Borrowed(&data[value.clone()])),
))
}
})
.collect();
GenericColDecoder::new_group(GroupDecoder::new(num_coder, values))
}
}
}
pub(crate) fn splice<'a, I>(
&self,
source: &[u8],
output: &mut Vec<u8>,
replace: Range<usize>,
replace_with: I,
) -> Result<Self, ColumnSpliceError>
where
I: Iterator<Item=CellValue<'a>> + Clone
{
match self {
Self::Single(s) => Ok(Self::Single(s.splice(
source,
output,
replace,
replace_with,
)?)),
Self::Composite(s) => Ok(Self::Composite(s.splice(
source,
output,
replace,
replace_with,
)?)),
}
}
pub(crate) fn col_type(&self) -> ColumnType {
match self {
Self::Single(SingleColumn{spec, ..}) => spec.col_type(),
Self::Composite(CompositeColumn::Value(..)) => ColumnType::Value,
Self::Composite(CompositeColumn::Group(..)) => ColumnType::Group,
}
}
}
#[derive(Clone, Debug)]
struct SingleColumn {
pub(crate) spec: ColumnSpec,
pub(crate) col_type: SimpleColType,
pub(crate) range: Range<usize>,
}
impl SingleColumn {
fn splice<'a, I>(
&self,
source: &[u8],
output: &mut Vec<u8>,
replace: Range<usize>,
replace_with: I,
) -> Result<Self, ColumnSpliceError>
where
I: Iterator<Item=CellValue<'a>> + Clone
{
let output_start = output.len();
let mut decoder = self.col_type.decoder(&source[self.range.clone()]);
let end = decoder.splice(output, replace, replace_with)? + output_start;
Ok(Self {
spec: self.spec,
col_type: self.col_type,
range: (output_start..end).into(),
})
}
}
#[derive(Debug, Clone)]
enum CompositeColumn {
Value(ValueColumn),
Group(GroupColumn),
}
impl CompositeColumn {
fn splice<'a, I>(
&self,
source: &[u8],
output: &mut Vec<u8>,
replace: Range<usize>,
replace_with: I,
) -> Result<Self, ColumnSpliceError>
where
I: Iterator<Item=CellValue<'a>> + Clone
{
match self {
Self::Value(value) => Ok(Self::Value(value.splice(
source,
replace,
replace_with,
output,
)?)),
Self::Group(group) => Ok(Self::Group(group.splice(
source,
output,
replace,
replace_with,
)?)),
}
}
}
#[derive(Clone, Debug)]
struct ValueColumn {
spec: ColumnSpec,
meta: Range<usize>,
value: Range<usize>,
}
impl ValueColumn {
fn splice<'a, I>(
&self,
source: &[u8],
replace: Range<usize>,
replace_with: I,
output: &mut Vec<u8>,
) -> Result<Self, ColumnSpliceError>
where
I: Iterator<Item=CellValue<'a>> + Clone
{
let mut decoder = ValueDecoder::new(
RleDecoder::from(&source[self.meta.clone()]),
RawDecoder::from(&source[self.value.clone()]),
);
let replacements = replace_with.enumerate().map(|(i, r)| match r {
CellValue::Value(p) => Ok(p),
_ => Err(ColumnSpliceError::InvalidValueForRow(i)),
});
let (new_meta, new_data) = decoder.try_splice(replace, replacements, output)?;
Ok(ValueColumn {
spec: self.spec,
meta: new_meta.into(),
value: new_data.into(),
})
}
}
#[derive(Debug, Clone)]
struct GroupColumn {
spec: ColumnSpec,
num: Range<usize>,
values: Vec<GroupedColumn>,
}
#[derive(Eq, PartialEq, Clone, Copy, Debug)]
enum SimpleColType {
Actor,
Integer,
DeltaInteger,
Boolean,
String,
}
impl SimpleColType {
fn decoder<'a>(self, data: &'a [u8]) -> SimpleColDecoder<'a> {
match self {
SimpleColType::Actor => SimpleColDecoder::new_uint(RleDecoder::from(Cow::from(data))),
SimpleColType::Integer => SimpleColDecoder::new_uint(RleDecoder::from(Cow::from(data))),
SimpleColType::String => SimpleColDecoder::new_string(RleDecoder::from(Cow::from(data))),
SimpleColType::Boolean => SimpleColDecoder::new_bool(BooleanDecoder::from(Cow::from(data))),
SimpleColType::DeltaInteger => SimpleColDecoder::new_delta(DeltaDecoder::from(Cow::from(data))),
}
}
}
#[derive(Clone, Debug)]
enum GroupedColumn {
Single(SingleColumn),
Value(ValueColumn),
}
impl GroupedColumn {
fn range(&self) -> Range<usize> {
match self {
Self::Single(SingleColumn{range, ..}) => range.clone(),
Self::Value(ValueColumn { meta, value, .. }) => (meta.start..value.end),
}
}
}
impl GroupColumn {
fn splice<'a, I>(
&self,
source: &[u8],
output: &mut Vec<u8>,
replace: Range<usize>,
replace_with: I,
) -> Result<Self, ColumnSpliceError>
where
I: Iterator<Item=CellValue<'a>> + Clone
{
// This is a little like ValueDecoder::splice. First we want to read off the values from `num`
// and insert them into the output - inserting replacements lengths as we go. Then we re-read
// num and use it to also iterate over the grouped values, inserting those into the subsidiary
// columns as we go.
// First encode the lengths
let output_start = output.len();
let mut num_decoder =
RleDecoder::<'_, u64>::from(Cow::from(&source[self.num.clone()]));
let mut num_encoder = RleEncoder::from(output);
let mut idx = 0;
while idx < replace.start {
match num_decoder.next() {
Some(next_num) => {
num_encoder.append(next_num.as_ref());
}
None => {
panic!("out of bounds");
}
}
idx += 1;
}
let mut num_replace_with = replace_with.clone();
while let Some(replacement) = num_replace_with.next() {
let rows = match &replacement {
CellValue::List(rows) => rows,
_ => return Err(ColumnSpliceError::InvalidValueForRow(idx)),
};
for row in rows {
if row.len() != self.values.len() {
return Err(ColumnSpliceError::WrongNumberOfValues {
row: idx - replace.start,
expected: self.values.len(),
actual: row.len(),
});
}
num_encoder.append(Some(&(rows.len() as u64)));
}
idx += 1;
}
while let Some(num) = num_decoder.next() {
num_encoder.append(num.as_ref());
idx += 1;
}
let _num_range = output_start..num_encoder.finish();
// Now encode the values
let _num_decoder =
RleDecoder::<'_, u64>::from(Cow::from(&source[self.num.clone()]));
panic!()
}
}
pub(crate) struct ColumnBuilder {
}
impl ColumnBuilder {
pub(crate) fn build_actor(spec: ColumnSpec, range: Range<usize>) -> Column {
Column(ColumnInner::Single(SingleColumn{spec, col_type: SimpleColType::Actor, range: range.into()}))
}
pub(crate) fn build_string(spec: ColumnSpec, range: Range<usize>) -> Column {
Column(ColumnInner::Single(SingleColumn{spec, col_type: SimpleColType::String, range: range.into()}))
}
pub(crate) fn build_integer(spec: ColumnSpec, range: Range<usize>) -> Column {
Column(ColumnInner::Single(SingleColumn{spec, col_type: SimpleColType::Integer, range: range.into()}))
}
pub(crate) fn build_delta_integer(spec: ColumnSpec, range: Range<usize>) -> Column {
Column(ColumnInner::Single(SingleColumn{spec, col_type: SimpleColType::Integer, range: range.into()}))
}
pub(crate) fn build_boolean(spec: ColumnSpec, range: Range<usize>) -> Column {
Column(ColumnInner::Single(SingleColumn{spec, col_type: SimpleColType::Boolean, range: range.into()}))
}
pub(crate) fn start_value(spec: ColumnSpec, meta: Range<usize>) -> AwaitingRawColumnValueBuilder {
AwaitingRawColumnValueBuilder { spec, meta }
}
pub(crate) fn start_group(spec: ColumnSpec, num: Range<usize>) -> GroupBuilder {
GroupBuilder{spec, num_range: num, columns: Vec::new()}
}
}
pub(crate) struct AwaitingRawColumnValueBuilder {
spec: ColumnSpec,
meta: Range<usize>,
}
impl AwaitingRawColumnValueBuilder {
pub(crate) fn id(&self) -> ColumnId {
self.spec.id()
}
pub(crate) fn meta_range(&self) -> &Range<usize> {
&self.meta
}
pub(crate) fn build(&mut self, raw: Range<usize>) -> Column {
Column(ColumnInner::Composite(CompositeColumn::Value(ValueColumn{
spec: self.spec,
meta: self.meta.clone().into(),
value: raw.into(),
})))
}
}
#[derive(Debug)]
pub(crate) struct GroupBuilder{
spec: ColumnSpec,
num_range: Range<usize>,
columns: Vec<GroupedColumn>,
}
impl GroupBuilder {
pub(crate) fn range(&self) -> Range<usize> {
let start = self.num_range.start;
let end = self.columns.last().map(|c| c.range().end).unwrap_or(self.num_range.end);
start..end
}
pub(crate) fn add_actor(&mut self, spec: ColumnSpec, range: Range<usize>) {
self.columns.push(GroupedColumn::Single(SingleColumn{
col_type: SimpleColType::Actor,
range: range.into(),
spec,
}));
}
pub(crate) fn add_string(&mut self, spec: ColumnSpec, range: Range<usize>) {
self.columns.push(GroupedColumn::Single(SingleColumn{
col_type: SimpleColType::String,
range: range.into(),
spec,
}));
}
pub(crate) fn add_integer(&mut self, spec: ColumnSpec, range: Range<usize>) {
self.columns.push(GroupedColumn::Single(SingleColumn{
col_type: SimpleColType::Integer,
range: range.into(),
spec,
}));
}
pub(crate) fn add_delta_integer(&mut self, spec: ColumnSpec, range: Range<usize>) {
self.columns.push(GroupedColumn::Single(SingleColumn{
col_type: SimpleColType::DeltaInteger,
range: range.into(),
spec,
}));
}
pub(crate) fn add_boolean(&mut self, spec: ColumnSpec, range: Range<usize>) {
self.columns.push(GroupedColumn::Single(SingleColumn{
col_type: SimpleColType::Boolean,
range: range.into(),
spec,
}));
}
pub(crate) fn start_value(&mut self, spec: ColumnSpec, meta: Range<usize>) -> GroupAwaitingValue {
GroupAwaitingValue {
spec,
num_range: self.num_range.clone(),
columns: std::mem::take(&mut self.columns),
val_spec: spec,
val_meta: meta,
}
}
pub(crate) fn finish(&mut self) -> Column {
Column(ColumnInner::Composite(CompositeColumn::Group(GroupColumn{
spec: self.spec,
num: self.num_range.clone(),
values: std::mem::take(&mut self.columns),
})))
}
}
#[derive(Debug)]
pub(crate) struct GroupAwaitingValue {
spec: ColumnSpec,
num_range: Range<usize>,
columns: Vec<GroupedColumn>,
val_spec: ColumnSpec,
val_meta: Range<usize>,
}
impl GroupAwaitingValue {
pub(crate) fn finish_empty(&mut self) -> GroupBuilder {
self.columns.push(GroupedColumn::Value(ValueColumn{
meta: self.val_meta.clone(),
value: 0..0,
spec: self.val_spec,
}));
GroupBuilder {
spec: self.spec,
num_range: self.num_range.clone(),
columns: std::mem::take(&mut self.columns),
}
}
pub(crate) fn finish_value(&mut self, raw: Range<usize>) -> GroupBuilder {
self.columns.push(GroupedColumn::Value(ValueColumn{
spec: self.val_spec,
value: raw.into(),
meta: self.val_meta.clone(),
}));
GroupBuilder {
spec: self.spec,
num_range: self.num_range.clone(),
columns: std::mem::take(&mut self.columns),
}
}
pub(crate) fn range(&self) -> Range<usize> {
self.num_range.start..self.val_meta.end
}
}

View file

@ -0,0 +1,480 @@
use std::{borrow::{Borrow, Cow}, convert::TryFrom, ops::Range};
use tracing::instrument;
use crate::columnar_2::{
column_specification::ColumnType,
rowblock::{
column_range::{ActorRange, DeltaIntRange, RawRange, RleIntRange, RleStringRange},
encoding::{DecodeColumnError, DeltaDecoder, RawDecoder, RleDecoder, ValueDecoder},
PrimVal,
},
ColumnId, ColumnSpec,
storage::ColumnMetadata,
};
use super::{
assert_col_type,
column::{ColumnRanges, GroupColRange},
ColumnLayout, MismatchingColumn,
};
#[derive(Debug)]
pub(crate) struct ChangeMetadata<'a> {
pub(crate) actor: usize,
pub(crate) seq: u64,
pub(crate) max_op: u64,
pub(crate) timestamp: i64,
pub(crate) message: Option<smol_str::SmolStr>,
pub(crate) deps: Vec<u64>,
pub(crate) extra: Cow<'a, [u8]>,
}
pub(crate) struct DocChangeColumns {
actor: ActorRange,
seq: DeltaIntRange,
max_op: DeltaIntRange,
time: DeltaIntRange,
message: RleStringRange,
deps_group: RleIntRange,
deps_index: DeltaIntRange,
extra_meta: RleIntRange,
extra_val: RawRange,
other: ColumnLayout,
}
impl DocChangeColumns {
pub(crate) fn iter<'a>(&self, data: &'a [u8]) -> DocChangeColumnIter<'a> {
DocChangeColumnIter {
actors: self.actor.decoder(data),
seq: self.seq.decoder(data),
max_op: self.max_op.decoder(data),
time: self.time.decoder(data),
message: self.message.decoder(data),
deps: DepsDecoder {
group: self.deps_group.decoder(data),
deps: self.deps_index.decoder(data),
},
extra: ExtraDecoder {
val: ValueDecoder::new(self.extra_meta.decoder(data), self.extra_val.decoder(data)),
},
}
}
pub(crate) fn encode<'a, I, C: Borrow<ChangeMetadata<'a>>>(changes: I, out: &mut Vec<u8>) -> DocChangeColumns
where
I: Iterator<Item = C> + Clone,
{
let actor = ActorRange::from(0..0).decoder(&[]).splice(
0..0,
// TODO: make this fallible once iterators have a try_splice
changes.clone().map(|c| Some(c.borrow().actor as u64)),
out,
);
let seq = DeltaDecoder::from(&[] as &[u8]).splice(
0..0,
changes.clone().map(|c| Some(c.borrow().seq as i64)),
out,
);
let max_op = DeltaDecoder::from(&[] as &[u8]).splice(
0..0,
changes.clone().map(|c| Some(c.borrow().max_op as i64)),
out,
);
let time = DeltaDecoder::from(&[] as &[u8]).splice(
0..0,
changes.clone().map(|c| Some(c.borrow().timestamp)),
out,
);
let message = RleDecoder::<'a, smol_str::SmolStr>::from(&[] as &[u8]).splice(
0..0,
changes.clone().map(|c| c.borrow().message.clone()),
out,
);
let (deps_group, deps_index) = DepsDecoder {
group: RleDecoder::from(&[] as &[u8]),
deps: DeltaDecoder::from(&[] as &[u8]),
}
.splice(0..0, changes.clone().map(|c| c.borrow().deps.clone()), out);
let (extra_meta, extra_val) = ValueDecoder::new(
RleDecoder::from(&[] as &[u8]),
RawDecoder::from(&[] as &[u8]),
)
.splice(0..0, changes.clone().map(|c| PrimVal::Bytes(c.borrow().extra.clone())), out);
DocChangeColumns {
actor: actor.into(),
seq: seq.into(),
max_op: max_op.into(),
time: time.into(),
message: message.into(),
deps_group: deps_group.into(),
deps_index: deps_index.into(),
extra_meta: extra_meta.into(),
extra_val: extra_val.into(),
other: ColumnLayout::empty(),
}
}
pub(crate) fn metadata(&self) -> ColumnMetadata {
const ACTOR_COL_ID: ColumnId = ColumnId::new(0);
const SEQ_COL_ID: ColumnId = ColumnId::new(0);
const MAX_OP_COL_ID: ColumnId = ColumnId::new(1);
const TIME_COL_ID: ColumnId = ColumnId::new(2);
const MESSAGE_COL_ID: ColumnId = ColumnId::new(3);
const DEPS_COL_ID: ColumnId = ColumnId::new(4);
const EXTRA_COL_ID: ColumnId = ColumnId::new(5);
let mut cols = vec![
(ColumnSpec::new(ACTOR_COL_ID, ColumnType::Actor, false), self.actor.clone().into()),
(ColumnSpec::new(SEQ_COL_ID, ColumnType::DeltaInteger, false), self.seq.clone().into()),
(ColumnSpec::new(MAX_OP_COL_ID, ColumnType::DeltaInteger, false), self.max_op.clone().into()),
(ColumnSpec::new(TIME_COL_ID, ColumnType::DeltaInteger, false), self.time.clone().into()),
(ColumnSpec::new(MESSAGE_COL_ID, ColumnType::String, false), self.message.clone().into()),
(ColumnSpec::new(DEPS_COL_ID, ColumnType::Group, false), self.deps_group.clone().into()),
];
if self.deps_index.len() > 0 {
cols.push((
ColumnSpec::new(DEPS_COL_ID, ColumnType::DeltaInteger, false), self.deps_index.clone().into()
))
}
cols.push(
(ColumnSpec::new(EXTRA_COL_ID, ColumnType::ValueMetadata, false), self.extra_meta.clone().into()),
);
if self.extra_val.len() > 0 {
cols.push((
ColumnSpec::new(EXTRA_COL_ID, ColumnType::Value, false), self.extra_val.clone().into()
))
}
cols.into_iter().collect()
}
}
#[derive(Debug, thiserror::Error)]
pub(crate) enum DecodeChangeError {
#[error("the depenencies column was invalid")]
InvalidDeps,
#[error("unexpected null value for {0}")]
UnexpectedNull(String),
#[error("mismatching column types for column {index}")]
MismatchingColumn { index: usize },
#[error("not enough columns")]
NotEnoughColumns,
#[error("incorrect value in extra bytes column")]
InvalidExtraBytes,
#[error("error reading column {column}: {description}")]
ReadColumn { column: String, description: String },
}
impl DecodeChangeError {
fn from_decode_col(col: &'static str, err: DecodeColumnError) -> Self {
match err {
DecodeColumnError::InvalidValue { description, .. } => Self::ReadColumn {
column: col.to_string(),
description,
},
DecodeColumnError::UnexpectedNull(inner_col) => {
Self::UnexpectedNull(format!("{}:{}", col, inner_col))
}
}
}
}
impl From<MismatchingColumn> for DecodeChangeError {
fn from(m: MismatchingColumn) -> Self {
Self::MismatchingColumn { index: m.index }
}
}
pub(crate) struct DocChangeColumnIter<'a> {
actors: RleDecoder<'a, u64>,
seq: DeltaDecoder<'a>,
max_op: DeltaDecoder<'a>,
time: DeltaDecoder<'a>,
message: RleDecoder<'a, smol_str::SmolStr>,
deps: DepsDecoder<'a>,
extra: ExtraDecoder<'a>,
}
macro_rules! next_or_invalid({$iter: expr, $col: literal} => {
match $iter.next() {
Some(Some(s)) => s,
Some(None) => return Some(Err(DecodeChangeError::UnexpectedNull($col.to_string()))),
None => return Some(Err(DecodeChangeError::UnexpectedNull($col.to_string()))),
}
});
impl<'a> Iterator for DocChangeColumnIter<'a> {
type Item = Result<ChangeMetadata<'a>, DecodeChangeError>;
fn next(&mut self) -> Option<Self::Item> {
let actor = match self.actors.next() {
Some(Some(actor)) => actor as usize,
Some(None) => return Some(Err(DecodeChangeError::UnexpectedNull("actor".to_string()))),
None => {
// The actor column should always have a value so if the actor iterator returns None that
// means we should be done, we check by asserting that all the other iterators
// return none (which is what Self::is_done does).
if self.is_done() {
return None;
} else {
return Some(Err(DecodeChangeError::UnexpectedNull("actor".to_string())));
}
}
};
let seq = match next_or_invalid!(self.seq, "seq").try_into() {
Ok(s) => s,
Err(_) => {
return Some(Err(DecodeChangeError::ReadColumn {
column: "seq".to_string(),
description: "negative value".to_string(),
}))
}
};
let max_op = match next_or_invalid!(self.max_op, "max_op").try_into() {
Ok(o) => o,
Err(_) => {
return Some(Err(DecodeChangeError::ReadColumn {
column: "max_op".to_string(),
description: "negative value".to_string(),
}))
}
};
let time = next_or_invalid!(self.time, "time");
let message = match self.message.next() {
Some(Some(s)) => Some(s),
Some(None) => None,
None => return Some(Err(DecodeChangeError::UnexpectedNull("msg".to_string()))),
};
let deps = match self.deps.next() {
Some(Ok(d)) => d,
Some(Err(e)) => return Some(Err(e)),
None => return Some(Err(DecodeChangeError::UnexpectedNull("deps".to_string()))),
};
let extra = match self.extra.next() {
Some(Ok(e)) => e,
Some(Err(e)) => return Some(Err(e)),
None => return Some(Err(DecodeChangeError::UnexpectedNull("extra".to_string()))),
};
Some(Ok(ChangeMetadata {
actor,
seq,
max_op,
timestamp: time,
message,
deps,
extra,
}))
}
}
impl<'a> DocChangeColumnIter<'a> {
/// Given that we have read a `None` value in the actor column, check that every other column
/// also returns `None`.
fn is_done(&mut self) -> bool {
let other_cols = [
self.seq.next().is_none(),
self.max_op.next().is_none(),
self.time.next().is_none(),
self.message.next().is_none(),
self.deps.next().is_none(),
];
other_cols.iter().all(|f| *f)
}
}
struct DepsDecoder<'a> {
group: RleDecoder<'a, u64>,
deps: DeltaDecoder<'a>,
}
impl<'a> DepsDecoder<'a> {
fn encode<'b, I>(deps: I, out: &'a mut Vec<u8>) -> DepsDecoder<'a>
where
I: Iterator<Item=&'b [u64]> + Clone
{
let group = RleDecoder::encode(deps.clone().map(|d| d.len() as u64), out);
let deps = DeltaDecoder::encode(deps.flat_map(|d| d.iter().map(|d| *d as i64)), out);
DepsDecoder{
group: RleDecoder::from(&out[group]),
deps: DeltaDecoder::from(&out[deps]),
}
}
fn splice<'b, I>(
&self,
replace_range: Range<usize>,
items: I,
out: &mut Vec<u8>,
) -> (Range<usize>, Range<usize>)
where
I: Iterator<Item = Vec<u64>> + Clone,
{
let mut replace_start = 0_usize;
let mut replace_len = 0_usize;
for (index, elems) in self.group.clone().enumerate() {
if let Some(elems) = elems {
if index < replace_range.start {
replace_start += elems as usize;
} else if index < replace_range.end {
replace_len += elems as usize;
}
}
}
let val_replace_range = replace_start..(replace_start + replace_len);
let group = self.group.clone().splice(
replace_range,
items.clone().map(|i| Some(i.len() as u64)),
out
);
let items = self.deps.clone().splice(
val_replace_range,
items.flat_map(|elems| elems.into_iter().map(|v| Some(v as i64))),
out,
);
(group, items)
}
}
impl<'a> Iterator for DepsDecoder<'a> {
type Item = Result<Vec<u64>, DecodeChangeError>;
fn next(&mut self) -> Option<Self::Item> {
let num = match self.group.next() {
Some(Some(n)) => n as usize,
Some(None) => return Some(Err(DecodeChangeError::InvalidDeps)),
None => return None,
};
let mut result = Vec::with_capacity(num);
while result.len() < num {
match self.deps.next() {
Some(Some(elem)) => {
let elem = match u64::try_from(elem) {
Ok(e) => e,
Err(e) => {
tracing::error!(err=?e, dep=elem, "error converting dep index to u64");
return Some(Err(DecodeChangeError::InvalidDeps));
}
};
result.push(elem);
}
_ => return Some(Err(DecodeChangeError::InvalidDeps)),
}
}
Some(Ok(result))
}
}
struct ExtraDecoder<'a> {
val: ValueDecoder<'a>,
}
impl<'a> Iterator for ExtraDecoder<'a> {
type Item = Result<Cow<'a, [u8]>, DecodeChangeError>;
fn next(&mut self) -> Option<Self::Item> {
match self.val.next() {
Some(Ok(PrimVal::Bytes(b))) => Some(Ok(b)),
Some(Ok(_)) => Some(Err(DecodeChangeError::InvalidExtraBytes)),
Some(Err(e)) => Some(Err(DecodeChangeError::from_decode_col("value", e))),
None => None,
}
}
}
impl TryFrom<ColumnLayout> for DocChangeColumns {
type Error = DecodeChangeError;
#[instrument]
fn try_from(columns: ColumnLayout) -> Result<Self, Self::Error> {
let mut actor: Option<Range<usize>> = None;
let mut seq: Option<Range<usize>> = None;
let mut max_op: Option<Range<usize>> = None;
let mut time: Option<Range<usize>> = None;
let mut message: Option<Range<usize>> = None;
let mut deps_group: Option<Range<usize>> = None;
let mut deps_index: Option<Range<usize>> = None;
let mut extra_meta: Option<Range<usize>> = None;
let mut extra_val: Option<Range<usize>> = None;
let mut other = ColumnLayout::empty();
for (index, col) in columns.into_iter().enumerate() {
match index {
0 => assert_col_type(index, col, ColumnType::Actor, &mut actor)?,
1 => assert_col_type(index, col, ColumnType::DeltaInteger, &mut seq)?,
2 => assert_col_type(index, col, ColumnType::DeltaInteger, &mut max_op)?,
3 => assert_col_type(index, col, ColumnType::DeltaInteger, &mut time)?,
4 => assert_col_type(index, col, ColumnType::String, &mut message)?,
5 => match col.ranges() {
ColumnRanges::Group { num, mut cols } => {
deps_group = Some(num.into());
let first = cols.next();
match first {
Some(GroupColRange::Single(index_range)) => {
deps_index = Some(index_range.into());
}
Some(_) => {
tracing::error!("deps column contained more than one grouped column");
return Err(DecodeChangeError::MismatchingColumn{index: 5});
}
None => {
deps_index = (0..0).into()
}
};
if let Some(_) = cols.next() {
return Err(DecodeChangeError::MismatchingColumn { index });
}
}
_ => return Err(DecodeChangeError::MismatchingColumn { index }),
},
6 => match col.ranges() {
ColumnRanges::Value { meta, val } => {
extra_meta = Some(meta);
extra_val = Some(val);
}
_ => return Err(DecodeChangeError::MismatchingColumn { index }),
},
_ => {
other.append(col);
}
}
}
Ok(DocChangeColumns {
actor: actor.ok_or(DecodeChangeError::NotEnoughColumns)?.into(),
seq: seq.ok_or(DecodeChangeError::NotEnoughColumns)?.into(),
max_op: max_op.ok_or(DecodeChangeError::NotEnoughColumns)?.into(),
time: time.ok_or(DecodeChangeError::NotEnoughColumns)?.into(),
message: message.ok_or(DecodeChangeError::NotEnoughColumns)?.into(),
deps_group: deps_group
.ok_or(DecodeChangeError::NotEnoughColumns)?
.into(),
deps_index: deps_index
.ok_or(DecodeChangeError::NotEnoughColumns)?
.into(),
extra_meta: extra_meta
.ok_or(DecodeChangeError::NotEnoughColumns)?
.into(),
extra_val: extra_val.ok_or(DecodeChangeError::NotEnoughColumns)?.into(),
other,
})
}
}
#[cfg(test)]
mod tests {
use super::*;
use proptest::prelude::*;
use proptest::collection::vec as propvec;
fn encodable_u64() -> impl Strategy<Value = u64> + Clone {
0_u64..((i64::MAX / 2) as u64)
}
proptest!{
#[test]
fn encode_decode_deps(deps in propvec(propvec(encodable_u64(), 0..100), 0..100)) {
let mut out = Vec::new();
let decoder = DepsDecoder::encode(deps.iter().map(|d| &d[..]), &mut out);
let decoded = decoder.collect::<Result<Vec<_>, _>>().unwrap();
assert_eq!(deps, decoded);
}
}
}

View file

@ -0,0 +1,420 @@
use std::{convert::TryFrom, ops::Range};
use tracing::instrument;
use crate::columnar_2::storage::ColumnMetadata;
use super::{
super::{
super::column_specification::ColumnType,
column_range::{
ActorRange, BooleanRange, DeltaIntRange, RawRange, RleIntRange, RleStringRange,
},
encoding::{
BooleanDecoder, DecodeColumnError, Key, KeyDecoder, ObjDecoder, OpIdDecoder,
OpIdListDecoder, RleDecoder, ValueDecoder,
},
ColumnSpec, ColumnId,
},
assert_col_type,
column::{ColumnRanges, GroupColRange},
ColumnLayout, MismatchingColumn,
};
use crate::{
columnar_2::rowblock::{PrimVal, encoding::{RawDecoder, DeltaDecoder}},
types::{ObjId, OpId, ElemId},
};
/// The form operations take in the compressed document format.
#[derive(Debug)]
pub(crate) struct DocOp<'a> {
pub(crate) id: OpId,
pub(crate) object: ObjId,
pub(crate) key: Key,
pub(crate) insert: bool,
pub(crate) action: usize,
pub(crate) value: PrimVal<'a>,
pub(crate) succ: Vec<OpId>,
}
pub(crate) struct DocOpColumns {
actor: ActorRange,
ctr: RleIntRange,
key_actor: ActorRange,
key_ctr: DeltaIntRange,
key_str: RleStringRange,
id_actor: RleIntRange,
id_ctr: DeltaIntRange,
insert: BooleanRange,
action: RleIntRange,
val_meta: RleIntRange,
val_raw: RawRange,
succ_group: RleIntRange,
succ_actor: RleIntRange,
succ_ctr: DeltaIntRange,
other: ColumnLayout,
}
impl DocOpColumns {
pub(crate) fn empty() -> DocOpColumns {
Self {
actor: (0..0).into(),
ctr: (0..0).into(),
key_actor: (0..0).into(),
key_ctr: (0..0).into(),
key_str: (0..0).into(),
id_actor: (0..0).into(),
id_ctr: (0..0).into(),
insert: (0..0).into(),
action: (0..0).into(),
val_meta: (0..0).into(),
val_raw: (0..0).into(),
succ_group: (0..0).into(),
succ_actor: (0..0).into(),
succ_ctr: (0..0).into(),
other: ColumnLayout::empty(),
}
}
pub(crate) fn encode<'a, I>(ops: I, out: &mut Vec<u8>) -> DocOpColumns
where
I: Iterator<Item = DocOp<'a>> + Clone,
{
let obj_actor = ActorRange::from(0..0).decoder(&[]).splice(
0..0,
ops.clone().map(|o| Some(o.object.opid().actor() as u64)),
out,
);
let obj_ctr = RleIntRange::from(0..0).decoder(&[]).splice(
0..0,
ops.clone().map(|o| Some(o.object.opid().counter() as u64)),
out,
);
let key_actor = ActorRange::from(0..0).decoder(&[]).splice(
0..0,
ops.clone().map(|o| match o.key {
Key::Prop(_) => None,
Key::Elem(ElemId(opid)) if opid.actor() == 0 => None,
Key::Elem(ElemId(opid)) => Some(opid.actor() as u64),
}),
out,
);
let key_ctr = DeltaIntRange::from(0..0).decoder(&[]).splice(
0..0,
ops.clone().map(|o| match o.key {
Key::Prop(_) => None,
Key::Elem(ElemId(opid)) => Some(opid.counter() as i64),
}),
out,
);
let key_str = RleStringRange::from(0..0).decoder(&[]).splice(
0..0,
ops.clone().map(|o| match o.key {
Key::Prop(s) => Some(s),
Key::Elem(_) => None,
}),
out,
);
let id_actor = RleIntRange::from(0..0).decoder(&[]).splice(
0..0,
ops.clone().map(|o| Some(o.id.actor() as u64)),
out,
);
let id_counter = DeltaIntRange::from(0..0).decoder(&[]).splice(
0..0,
ops.clone().map(|o| Some(o.id.counter() as i64)),
out,
);
let insert = BooleanRange::from(0..0).decoder(&[]).splice(
0..0,
ops.clone().map(|o| o.insert),
out,
);
let action = RleIntRange::from(0..0).decoder(&[]).splice(
0..0,
ops.clone().map(|o| Some(o.action as u64)),
out,
);
let (val_meta, val_raw) = ValueDecoder::new(RleDecoder::from(&[] as &[u8]), RawDecoder::from(&[] as &[u8])).splice(
0..0,
ops.clone().map(|o| o.value),
out,
);
let mut succ_dec = OpIdListDecoder::new(
RleDecoder::from(&[] as &[u8]),
RleDecoder::from(&[] as &[u8]),
DeltaDecoder::from(&[] as &[u8]),
);
let (succ_group, succ_actor, succ_ctr) =
succ_dec.splice(0..0, ops.map(|o| o.succ.clone()), out);
Self {
actor: obj_actor.into(),
ctr: obj_ctr.into(),
key_actor: key_actor.into(),
key_ctr: key_ctr.into(),
key_str: key_str.into(),
id_actor: id_actor.into(),
id_ctr: id_counter.into(),
insert: insert.into(),
action: action.into(),
val_meta: val_meta.into(),
val_raw: val_raw.into(),
succ_group: succ_group.into(),
succ_actor: succ_actor.into(),
succ_ctr: succ_ctr.into(),
other: ColumnLayout::empty(),
}
}
pub(crate) fn iter<'a>(&self, data: &'a [u8]) -> DocOpColumnIter<'a> {
DocOpColumnIter {
id: OpIdDecoder::new(self.id_actor.decoder(data), self.id_ctr.decoder(data)),
action: self.action.decoder(data),
objs: ObjDecoder::new(self.actor.decoder(data), self.ctr.decoder(data)),
keys: KeyDecoder::new(
self.key_actor.decoder(data),
self.key_ctr.decoder(data),
self.key_str.decoder(data),
),
insert: self.insert.decoder(data),
value: ValueDecoder::new(self.val_meta.decoder(data), self.val_raw.decoder(data)),
succ: OpIdListDecoder::new(
self.succ_group.decoder(data),
self.succ_actor.decoder(data),
self.succ_ctr.decoder(data),
),
}
}
pub(crate) fn metadata(&self) -> ColumnMetadata {
const OBJ_COL_ID: ColumnId = ColumnId::new(0);
const KEY_COL_ID: ColumnId = ColumnId::new(1);
const ID_COL_ID: ColumnId = ColumnId::new(2);
const INSERT_COL_ID: ColumnId = ColumnId::new(3);
const ACTION_COL_ID: ColumnId = ColumnId::new(4);
const VAL_COL_ID: ColumnId = ColumnId::new(5);
const SUCC_COL_ID: ColumnId = ColumnId::new(8);
let mut cols = vec![
(ColumnSpec::new(OBJ_COL_ID, ColumnType::Actor, false), self.actor.clone().into()),
(ColumnSpec::new(OBJ_COL_ID, ColumnType::Integer, false), self.ctr.clone().into()),
(ColumnSpec::new(KEY_COL_ID, ColumnType::Actor, false), self.key_actor.clone().into()),
(ColumnSpec::new(KEY_COL_ID, ColumnType::DeltaInteger, false), self.key_ctr.clone().into()),
(ColumnSpec::new(KEY_COL_ID, ColumnType::String, false), self.key_str.clone().into()),
(ColumnSpec::new(ID_COL_ID, ColumnType::Actor, false), self.id_actor.clone().into()),
(ColumnSpec::new(ID_COL_ID, ColumnType::DeltaInteger, false), self.id_ctr.clone().into()),
(ColumnSpec::new(INSERT_COL_ID, ColumnType::Boolean, false), self.insert.clone().into()),
(ColumnSpec::new(ACTION_COL_ID, ColumnType::Integer, false), self.action.clone().into()),
(ColumnSpec::new(VAL_COL_ID, ColumnType::ValueMetadata, false), self.val_meta.clone().into()),
];
if self.val_raw.len() > 0 {
cols.push((
ColumnSpec::new(VAL_COL_ID, ColumnType::Value, false), self.val_raw.clone().into()
));
}
cols.push(
(ColumnSpec::new(SUCC_COL_ID, ColumnType::Group, false), self.succ_group.clone().into()),
);
if self.succ_actor.len() > 0 {
cols.extend([
(ColumnSpec::new(SUCC_COL_ID, ColumnType::Actor, false), self.succ_actor.clone().into()),
(ColumnSpec::new(SUCC_COL_ID, ColumnType::DeltaInteger, false), self.succ_ctr.clone().into()),
]);
}
cols.into_iter().collect()
}
}
pub(crate) struct DocOpColumnIter<'a> {
id: OpIdDecoder<'a>,
action: RleDecoder<'a, u64>,
objs: ObjDecoder<'a>,
keys: KeyDecoder<'a>,
insert: BooleanDecoder<'a>,
value: ValueDecoder<'a>,
succ: OpIdListDecoder<'a>,
}
impl<'a> DocOpColumnIter<'a> {
fn done(&self) -> bool {
[
self.id.done(),
self.action.done(),
self.objs.done(),
self.keys.done(),
self.insert.done(),
self.value.done(),
self.succ.done(),
]
.iter()
.all(|c| *c)
}
}
#[derive(Debug, thiserror::Error)]
pub(crate) enum DecodeOpError {
#[error("unexpected null in column {0}")]
UnexpectedNull(String),
#[error("invalid value in column {column}: {description}")]
InvalidValue { column: String, description: String },
}
macro_rules! next_or_invalid({$iter: expr, $col: literal} => {
match $iter.next() {
Some(Ok(id)) => id,
Some(Err(e)) => match e {
DecodeColumnError::UnexpectedNull(inner_col) => {
return Some(Err(DecodeOpError::UnexpectedNull(format!(
"{}:{}", $col, inner_col
))));
},
DecodeColumnError::InvalidValue{column, description} => {
let col = format!("{}:{}", $col, column);
return Some(Err(DecodeOpError::InvalidValue{column: col, description}))
}
}
None => return Some(Err(DecodeOpError::UnexpectedNull($col.to_string()))),
}
});
impl<'a> Iterator for DocOpColumnIter<'a> {
type Item = Result<DocOp<'a>, DecodeOpError>;
fn next(&mut self) -> Option<Self::Item> {
if self.done() {
None
} else {
let id = next_or_invalid!(self.id, "opid");
let action = match self.action.next() {
Some(Some(a)) => a,
Some(None) | None => {
return Some(Err(DecodeOpError::UnexpectedNull("action".to_string())))
}
};
let obj = next_or_invalid!(self.objs, "obj").into();
let key = next_or_invalid!(self.keys, "key");
let value = next_or_invalid!(self.value, "value");
let succ = next_or_invalid!(self.succ, "succ");
let insert = self.insert.next().unwrap_or(false);
Some(Ok(DocOp {
id,
value,
action: action as usize,
object: obj,
key,
succ,
insert,
}))
}
}
}
#[derive(Debug, thiserror::Error)]
pub(crate) enum Error {
#[error("mismatching column at {index}.")]
MismatchingColumn { index: usize },
#[error("not enough columns")]
NotEnoughColumns,
}
impl From<MismatchingColumn> for Error {
fn from(m: MismatchingColumn) -> Self {
Error::MismatchingColumn { index: m.index }
}
}
impl TryFrom<ColumnLayout> for DocOpColumns {
type Error = Error;
#[instrument]
fn try_from(columns: ColumnLayout) -> Result<Self, Self::Error> {
let mut obj_actor: Option<Range<usize>> = None;
let mut obj_ctr: Option<Range<usize>> = None;
let mut key_actor: Option<Range<usize>> = None;
let mut key_ctr: Option<Range<usize>> = None;
let mut key_str: Option<Range<usize>> = None;
let mut id_actor: Option<Range<usize>> = None;
let mut id_ctr: Option<Range<usize>> = None;
let mut insert: Option<Range<usize>> = None;
let mut action: Option<Range<usize>> = None;
let mut val_meta: Option<Range<usize>> = None;
let mut val_raw: Option<Range<usize>> = None;
let mut succ_group: Option<Range<usize>> = None;
let mut succ_actor: Option<Range<usize>> = None;
let mut succ_ctr: Option<Range<usize>> = None;
let mut other = ColumnLayout::empty();
for (index, col) in columns.into_iter().enumerate() {
match index {
0 => assert_col_type(index, col, ColumnType::Actor, &mut obj_actor)?,
1 => assert_col_type(index, col, ColumnType::Integer, &mut obj_ctr)?,
2 => assert_col_type(index, col, ColumnType::Actor, &mut key_actor)?,
3 => assert_col_type(index, col, ColumnType::DeltaInteger, &mut key_ctr)?,
4 => assert_col_type(index, col, ColumnType::String, &mut key_str)?,
5 => assert_col_type(index, col, ColumnType::Actor, &mut id_actor)?,
6 => assert_col_type(index, col, ColumnType::DeltaInteger, &mut id_ctr)?,
7 => assert_col_type(index, col, ColumnType::Boolean, &mut insert)?,
8 => assert_col_type(index, col, ColumnType::Integer, &mut action)?,
9 => match col.ranges() {
ColumnRanges::Value { meta, val } => {
val_meta = Some(meta);
val_raw = Some(val);
}
_ => {
tracing::error!("col 9 should be a value column");
return Err(Error::MismatchingColumn { index });
},
},
10 => match col.ranges() {
ColumnRanges::Group { num, mut cols } => {
let first = cols.next();
let second = cols.next();
succ_group = Some(num.into());
match (first, second) {
(
Some(GroupColRange::Single(actor_range)),
Some(GroupColRange::Single(ctr_range)),
) => {
succ_actor = Some(actor_range.into());
succ_ctr = Some(ctr_range.into());
},
(None, None) => {
succ_actor = Some((0..0).into());
succ_ctr = Some((0..0).into());
}
_ => {
tracing::error!("expected a two column group of (actor, rle int) for index 10");
return Err(Error::MismatchingColumn { index });
}
};
if let Some(_) = cols.next() {
return Err(Error::MismatchingColumn { index });
}
}
_ => return Err(Error::MismatchingColumn { index }),
},
_ => {
other.append(col);
}
}
}
Ok(DocOpColumns {
actor: obj_actor.ok_or(Error::NotEnoughColumns)?.into(),
ctr: obj_ctr.ok_or(Error::NotEnoughColumns)?.into(),
key_actor: key_actor.ok_or(Error::NotEnoughColumns)?.into(),
key_ctr: key_ctr.ok_or(Error::NotEnoughColumns)?.into(),
key_str: key_str.ok_or(Error::NotEnoughColumns)?.into(),
id_actor: id_actor.ok_or(Error::NotEnoughColumns)?.into(),
id_ctr: id_ctr.ok_or(Error::NotEnoughColumns)?.into(),
insert: insert.ok_or(Error::NotEnoughColumns)?.into(),
action: action.ok_or(Error::NotEnoughColumns)?.into(),
val_meta: val_meta.ok_or(Error::NotEnoughColumns)?.into(),
val_raw: val_raw.ok_or(Error::NotEnoughColumns)?.into(),
succ_group: succ_group.ok_or(Error::NotEnoughColumns)?.into(),
succ_actor: succ_actor.ok_or(Error::NotEnoughColumns)?.into(),
succ_ctr: succ_ctr.ok_or(Error::NotEnoughColumns)?.into(),
other,
})
}
}

View file

@ -0,0 +1,312 @@
use std::ops::Range;
use crate::columnar_2::{
column_specification::{ColumnId, ColumnSpec, ColumnType},
rowblock::column_layout::column::{
AwaitingRawColumnValueBuilder, Column, ColumnBuilder, GroupAwaitingValue, GroupBuilder,
},
};
#[derive(Debug)]
pub(crate) struct ColumnLayout(Vec<Column>);
impl ColumnLayout {
pub(crate) fn iter(&self) -> impl Iterator<Item = &Column> {
self.0.iter()
}
pub(crate) fn num_cols(&self) -> usize {
self.0.len()
}
pub(crate) fn parse<I: Iterator<Item = (ColumnSpec, Range<usize>)>>(
data_size: usize,
cols: I,
) -> Result<ColumnLayout, BadColumnLayout> {
let mut parser = ColumnLayoutParser::new(data_size, None);
for (col, range) in cols {
parser.add_column(col, range)?;
}
parser.build()
}
pub(crate) fn as_specs(&self) -> impl Iterator<Item=(ColumnSpec, Range<usize>)> {
panic!();
std::iter::empty()
}
pub(crate) fn empty() -> Self {
Self(Vec::new())
}
pub(crate) fn append(&mut self, col: Column) {
self.0.push(col)
}
pub(crate) fn unsafe_from_vec(v: Vec<Column>) -> Self {
Self(v)
}
}
impl FromIterator<Column> for Result<ColumnLayout, BadColumnLayout> {
fn from_iter<T: IntoIterator<Item = Column>>(iter: T) -> Self {
let iter = iter.into_iter();
let mut result = Vec::with_capacity(iter.size_hint().1.unwrap_or(0));
let mut last_column: Option<ColumnSpec> = None;
for col in iter {
if let Some(last_col) = last_column {
if col.spec().normalize() < last_col.normalize() {
return Err(BadColumnLayout::OutOfOrder);
}
}
last_column = Some(col.spec());
result.push(col);
}
Ok(ColumnLayout(result))
}
}
impl IntoIterator for ColumnLayout {
type Item = Column;
type IntoIter = std::vec::IntoIter<Column>;
fn into_iter(self) -> Self::IntoIter {
self.0.into_iter()
}
}
#[derive(Debug, thiserror::Error)]
pub(crate) enum BadColumnLayout {
#[error("duplicate column specifications: {0}")]
DuplicateColumnSpecs(u32),
#[error("out of order columns")]
OutOfOrder,
#[error("nested group")]
NestedGroup,
#[error("raw value column without metadata column")]
LoneRawValueColumn,
#[error("value metadata followed by value column with different column ID")]
MismatchingValueMetadataId,
#[error("non contiguous columns")]
NonContiguousColumns,
#[error("data out of range")]
DataOutOfRange,
}
struct ColumnLayoutParser {
columns: Vec<Column>,
last_spec: Option<ColumnSpec>,
state: LayoutParserState,
total_data_size: usize,
}
enum LayoutParserState {
Ready,
InValue(AwaitingRawColumnValueBuilder),
InGroup(ColumnId, GroupParseState),
}
#[derive(Debug)]
enum GroupParseState {
Ready(GroupBuilder),
InValue(GroupAwaitingValue),
}
impl ColumnLayoutParser {
fn new(data_size: usize, size_hint: Option<usize>) -> Self {
ColumnLayoutParser {
columns: Vec::with_capacity(size_hint.unwrap_or(0)),
last_spec: None,
state: LayoutParserState::Ready,
total_data_size: data_size,
}
}
fn build(mut self) -> Result<ColumnLayout, BadColumnLayout> {
match self.state {
LayoutParserState::Ready => Ok(ColumnLayout(self.columns)),
LayoutParserState::InValue(mut builder) => {
self.columns.push(builder.build(0..0));
Ok(ColumnLayout(self.columns))
}
LayoutParserState::InGroup(_, groupstate) => {
match groupstate {
GroupParseState::InValue(mut builder) => {
self.columns.push(builder.finish_empty().finish());
}
GroupParseState::Ready(mut builder) => {
self.columns.push(builder.finish());
}
};
Ok(ColumnLayout(self.columns))
}
}
}
fn add_column(
&mut self,
column: ColumnSpec,
range: Range<usize>,
) -> Result<(), BadColumnLayout> {
self.check_contiguous(&range)?;
self.check_bounds(&range)?;
if let Some(last_spec) = self.last_spec {
if last_spec.normalize() > column.normalize() {
return Err(BadColumnLayout::OutOfOrder);
} else if last_spec == column {
return Err(BadColumnLayout::DuplicateColumnSpecs(column.into()));
}
}
match &mut self.state {
LayoutParserState::Ready => match column.col_type() {
ColumnType::Group => {
self.state = LayoutParserState::InGroup(
column.id(),
GroupParseState::Ready(ColumnBuilder::start_group(column, range)),
);
Ok(())
}
ColumnType::ValueMetadata => {
self.state =
LayoutParserState::InValue(ColumnBuilder::start_value(column, range));
Ok(())
}
ColumnType::Value => Err(BadColumnLayout::LoneRawValueColumn),
ColumnType::Actor => {
self.columns.push(ColumnBuilder::build_actor(column, range));
Ok(())
}
ColumnType::String => {
self.columns
.push(ColumnBuilder::build_string(column, range));
Ok(())
}
ColumnType::Integer => {
self.columns
.push(ColumnBuilder::build_integer(column, range));
Ok(())
}
ColumnType::DeltaInteger => {
self.columns
.push(ColumnBuilder::build_delta_integer(column, range));
Ok(())
}
ColumnType::Boolean => {
self.columns
.push(ColumnBuilder::build_boolean(column, range));
Ok(())
}
},
LayoutParserState::InValue(builder) => match column.col_type() {
ColumnType::Value => {
if builder.id() != column.id() {
return Err(BadColumnLayout::MismatchingValueMetadataId);
}
self.columns.push(builder.build(range));
self.state = LayoutParserState::Ready;
Ok(())
}
_ => {
self.columns.push(builder.build(0..0));
self.state = LayoutParserState::Ready;
self.add_column(column, range)
}
},
LayoutParserState::InGroup(id, group_state) => {
if *id != column.id() {
match group_state {
GroupParseState::Ready(b) => self.columns.push(b.finish()),
GroupParseState::InValue(b) => self.columns.push(b.finish_empty().finish()),
};
std::mem::swap(&mut self.state, &mut LayoutParserState::Ready);
self.add_column(column, range)
} else {
match group_state {
GroupParseState::Ready(builder) => match column.col_type() {
ColumnType::Group => Err(BadColumnLayout::NestedGroup),
ColumnType::Value => Err(BadColumnLayout::LoneRawValueColumn),
ColumnType::ValueMetadata => {
*group_state =
GroupParseState::InValue(builder.start_value(column, range));
Ok(())
}
ColumnType::Actor => {
builder.add_actor(column, range);
Ok(())
}
ColumnType::Boolean => {
builder.add_boolean(column, range);
Ok(())
}
ColumnType::DeltaInteger => {
builder.add_delta_integer(column, range);
Ok(())
}
ColumnType::Integer => {
builder.add_integer(column, range);
Ok(())
}
ColumnType::String => {
builder.add_string(column, range);
Ok(())
}
},
GroupParseState::InValue(builder) => match column.col_type() {
ColumnType::Value => {
*group_state = GroupParseState::Ready(builder.finish_value(range));
Ok(())
}
_ => {
*group_state = GroupParseState::Ready(builder.finish_empty());
self.add_column(column, range)
}
},
}
}
}
}
}
fn check_contiguous(&self, next_range: &Range<usize>) -> Result<(), BadColumnLayout> {
match &self.state {
LayoutParserState::Ready => {
if let Some(prev) = self.columns.last() {
if prev.range().end != next_range.start {
Err(BadColumnLayout::NonContiguousColumns)
} else {
Ok(())
}
} else {
Ok(())
}
}
LayoutParserState::InValue(builder) => {
if builder.meta_range().end != next_range.start {
Err(BadColumnLayout::NonContiguousColumns)
} else {
Ok(())
}
}
LayoutParserState::InGroup(_, group_state) => {
let end = match group_state {
GroupParseState::InValue(b) => b.range().end,
GroupParseState::Ready(b) => b.range().end,
};
if end != next_range.start {
println!("Group state: {:?}", group_state);
Err(BadColumnLayout::NonContiguousColumns)
} else {
Ok(())
}
}
}
}
fn check_bounds(&self, next_range: &Range<usize>) -> Result<(), BadColumnLayout> {
if next_range.end > self.total_data_size {
Err(BadColumnLayout::DataOutOfRange)
} else {
Ok(())
}
}
}

View file

@ -0,0 +1,53 @@
use std::ops::Range;
pub(crate) mod column;
pub(crate) mod generic;
pub(crate) mod doc_op_columns;
pub(crate) mod doc_change_columns;
pub(crate) mod change_op_columns;
pub(crate) use generic::{BadColumnLayout, ColumnLayout};
pub(crate) use doc_op_columns::{DocOpColumns, Error as ParseDocColumnError};
#[derive(Debug, thiserror::Error)]
pub(crate) enum ColumnSpliceError {
#[error("invalid value for row {0}")]
InvalidValueForRow(usize),
#[error("wrong number of values for row {0}, expected {expected} but got {actual}")]
WrongNumberOfValues {
row: usize,
expected: usize,
actual: usize,
}
}
#[derive(Debug, thiserror::Error)]
#[error("mismatching column at {index}.")]
struct MismatchingColumn {
index: usize,
}
/// Given a `column::Column` assert that it is of the given `typ` and if so update `target` to be
/// `Some(range)`. Otherwise return a `MismatchingColumn{index}`
fn assert_col_type(
index: usize,
col: column::Column,
typ: crate::columnar_2::column_specification::ColumnType,
target: &mut Option<Range<usize>>,
) -> Result<(), MismatchingColumn> {
if col.col_type() == typ {
match col.ranges() {
column::ColumnRanges::Single(range) => {
*target = Some(range);
Ok(())
},
_ => {
tracing::error!("expected a single column range");
return Err(MismatchingColumn{ index });
}
}
} else {
tracing::error!(index, expected=?typ, actual=?col.col_type(), "unexpected columnt type");
Err(MismatchingColumn { index })
}
}

View file

@ -0,0 +1,60 @@
use std::{borrow::Cow, ops::Range};
use smol_str::SmolStr;
use super::encoding::{
BooleanDecoder, BooleanEncoder, DeltaDecoder, DeltaEncoder, RawDecoder, RawEncoder,
RleDecoder, RleEncoder,
};
macro_rules! make_col_range({$name: ident, $decoder_name: ident$(<$($dparam: tt),+>)?, $encoder_name: ident$(<$($eparam: tt),+>)?} => {
#[derive(Clone, Debug)]
pub(crate) struct $name(Range<usize>);
impl $name {
pub(crate) fn decoder<'a>(&self, data: &'a[u8]) -> $decoder_name $(<$($dparam,)+>)* {
$decoder_name::from(Cow::Borrowed(&data[self.0.clone()]))
}
pub(crate) fn encoder<'a>(&self, output: &'a mut Vec<u8>) -> $encoder_name $(<$($eparam,)+>)* {
$encoder_name::from(output)
}
pub(crate) fn len(&self) -> usize {
self.0.len()
}
pub(crate) fn is_empty(&self) -> bool {
self.0.is_empty()
}
}
impl AsRef<Range<usize>> for $name {
fn as_ref(&self) -> &Range<usize> {
&self.0
}
}
impl From<Range<usize>> for $name {
fn from(r: Range<usize>) -> $name {
$name(r)
}
}
impl From<$name> for Range<usize> {
fn from(r: $name) -> Range<usize> {
r.0
}
}
});
make_col_range!(ActorRange, RleDecoder<'a, u64>, RleEncoder<'a, u64>);
make_col_range!(RleIntRange, RleDecoder<'a, u64>, RleEncoder<'a, u64>);
make_col_range!(DeltaIntRange, DeltaDecoder<'a>, DeltaEncoder<'a>);
make_col_range!(
RleStringRange,
RleDecoder<'a, SmolStr>,
RleEncoder<'a, SmolStr>
);
make_col_range!(BooleanRange, BooleanDecoder<'a>, BooleanEncoder<'a>);
make_col_range!(RawRange, RawDecoder<'a>, RawEncoder<'a>);

View file

@ -0,0 +1,163 @@
use std::{borrow::Cow, ops::Range};
use super::{Encodable, RawDecoder};
/// Encodes booleans by storing the count of the same value.
///
/// The sequence of numbers describes the count of false values on even indices (0-indexed) and the
/// count of true values on odd indices (0-indexed).
///
/// Counts are encoded as usize.
pub(crate) struct BooleanEncoder<'a> {
written: usize,
buf: &'a mut Vec<u8>,
last: bool,
count: usize,
}
impl<'a> BooleanEncoder<'a> {
pub fn new(output: &'a mut Vec<u8>) -> BooleanEncoder<'a> {
BooleanEncoder {
written: 0,
buf: output,
last: false,
count: 0,
}
}
pub fn append(&mut self, value: bool) {
if value == self.last {
self.count += 1;
} else {
self.written += self.count.encode(&mut self.buf);
self.last = value;
self.count = 1;
}
}
pub fn finish(mut self) -> usize {
if self.count > 0 {
self.written += self.count.encode(&mut self.buf);
}
self.written
}
}
impl<'a> From<&'a mut Vec<u8>> for BooleanEncoder<'a> {
fn from(output: &'a mut Vec<u8>) -> Self {
BooleanEncoder::new(output)
}
}
/// See the discussion of [`BooleanEncoder`] for details on this encoding
pub(crate) struct BooleanDecoder<'a> {
decoder: RawDecoder<'a>,
last_value: bool,
count: usize,
}
impl<'a> BooleanDecoder<'a> {
pub(crate) fn done(&self) -> bool {
self.decoder.done()
}
pub(crate) fn splice<I: Iterator<Item=bool>>(&mut self, replace: Range<usize>, mut replace_with: I, out: &mut Vec<u8>) -> Range<usize> {
let start = out.len();
let mut encoder = BooleanEncoder::new(out);
let mut idx = 0;
while idx < replace.start {
match self.next() {
Some(elem) => encoder.append(elem),
None => panic!("out of bounds"),
}
idx += 1;
}
for _ in 0..replace.len() {
self.next();
if let Some(next) = replace_with.next() {
encoder.append(next);
}
}
while let Some(next) = replace_with.next() {
encoder.append(next);
}
while let Some(next) = self.next() {
encoder.append(next);
}
start..(start + encoder.finish())
}
}
impl<'a> From<Cow<'a, [u8]>> for BooleanDecoder<'a> {
fn from(bytes: Cow<'a, [u8]>) -> Self {
BooleanDecoder {
decoder: RawDecoder::from(bytes),
last_value: true,
count: 0,
}
}
}
impl<'a> From<&'a [u8]> for BooleanDecoder<'a> {
fn from(d: &'a [u8]) -> Self {
Cow::Borrowed(d).into()
}
}
// this is an endless iterator that returns false after input is exhausted
impl<'a> Iterator for BooleanDecoder<'a> {
type Item = bool;
fn next(&mut self) -> Option<bool> {
while self.count == 0 {
if self.decoder.done() && self.count == 0 {
return None;
}
self.count = self.decoder.read().unwrap_or_default();
self.last_value = !self.last_value;
}
self.count -= 1;
Some(self.last_value)
}
}
#[cfg(test)]
mod tests {
use super::*;
use crate::columnar_2::rowblock::encoding::properties::splice_scenario;
use proptest::prelude::*;
fn encode(vals: &[bool]) -> Vec<u8> {
let mut buf = Vec::new();
let mut encoder = BooleanEncoder::new(&mut buf);
for val in vals {
encoder.append(*val);
}
encoder.finish();
buf
}
fn decode(buf: &[u8]) -> Vec<bool> {
BooleanDecoder::from(buf).collect()
}
proptest!{
#[test]
fn encode_decode_bools(vals in proptest::collection::vec(any::<bool>(), 0..100)) {
assert_eq!(vals, decode(&encode(&vals)))
}
#[test]
fn splice_bools(scenario in splice_scenario(any::<bool>())) {
let encoded = encode(&scenario.initial_values);
let mut decoder = BooleanDecoder::from(&encoded[..]);
let mut out = Vec::new();
let r = decoder.splice(scenario.replace_range.clone(), scenario.replacements.iter().copied(), &mut out);
let result = decode(&out);
scenario.check(result);
assert_eq!(r.len(), out.len());
}
}
}

View file

@ -0,0 +1,158 @@
use std::{borrow::Cow, convert::TryFrom, str, io::Read};
use smol_str::SmolStr;
use super::Decodable;
use crate::ActorId;
impl Decodable for u8 {
fn decode<R>(bytes: &mut R) -> Option<Self>
where
R: Read,
{
let mut buffer = [0; 1];
bytes.read_exact(&mut buffer).ok()?;
Some(buffer[0])
}
}
impl Decodable for u32 {
fn decode<R>(bytes: &mut R) -> Option<Self>
where
R: Read,
{
u64::decode::<R>(bytes).and_then(|val| Self::try_from(val).ok())
}
}
impl Decodable for usize {
fn decode<R>(bytes: &mut R) -> Option<Self>
where
R: Read,
{
u64::decode::<R>(bytes).and_then(|val| Self::try_from(val).ok())
}
}
impl Decodable for isize {
fn decode<R>(bytes: &mut R) -> Option<Self>
where
R: Read,
{
i64::decode::<R>(bytes).and_then(|val| Self::try_from(val).ok())
}
}
impl Decodable for i32 {
fn decode<R>(bytes: &mut R) -> Option<Self>
where
R: Read,
{
i64::decode::<R>(bytes).and_then(|val| Self::try_from(val).ok())
}
}
impl Decodable for i64 {
fn decode<R>(bytes: &mut R) -> Option<Self>
where
R: Read,
{
leb128::read::signed(bytes).ok()
}
}
impl Decodable for f64 {
fn decode<R>(bytes: &mut R) -> Option<Self>
where
R: Read,
{
let mut buffer = [0; 8];
bytes.read_exact(&mut buffer).ok()?;
Some(Self::from_le_bytes(buffer))
}
}
impl Decodable for f32 {
fn decode<R>(bytes: &mut R) -> Option<Self>
where
R: Read,
{
let mut buffer = [0; 4];
bytes.read_exact(&mut buffer).ok()?;
Some(Self::from_le_bytes(buffer))
}
}
impl Decodable for u64 {
fn decode<R>(bytes: &mut R) -> Option<Self>
where
R: Read,
{
leb128::read::unsigned(bytes).ok()
}
}
impl Decodable for Vec<u8> {
fn decode<R>(bytes: &mut R) -> Option<Self>
where
R: Read,
{
let len = usize::decode::<R>(bytes)?;
if len == 0 {
return Some(vec![]);
}
let mut buffer = vec![0; len];
bytes.read_exact(buffer.as_mut_slice()).ok()?;
Some(buffer)
}
}
impl Decodable for SmolStr {
fn decode<R>(bytes: &mut R) -> Option<SmolStr>
where
R: Read,
{
let buffer = Vec::decode(bytes)?;
str::from_utf8(&buffer).map(|t| t.into()).ok()
}
}
impl Decodable for Cow<'static, SmolStr> {
fn decode<R>(bytes: &mut R) -> Option<Self>
where
R: std::io::Read {
SmolStr::decode(bytes).map(|s| Cow::Owned(s))
}
}
impl Decodable for String {
fn decode<R>(bytes: &mut R) -> Option<String>
where
R: Read,
{
let buffer = Vec::decode(bytes)?;
str::from_utf8(&buffer).map(|t| t.into()).ok()
}
}
impl Decodable for Option<String> {
fn decode<R>(bytes: &mut R) -> Option<Self>
where
R: Read,
{
let buffer = Vec::decode(bytes)?;
if buffer.is_empty() {
return Some(None);
}
Some(str::from_utf8(&buffer).map(|t| t.into()).ok())
}
}
impl Decodable for ActorId {
fn decode<R>(bytes: &mut R) -> Option<Self>
where
R: Read,
{
let buffer = Vec::decode(bytes)?;
Some(buffer.into())
}
}

View file

@ -0,0 +1,169 @@
use std::{borrow::Cow, ops::Range};
use super::{RleEncoder, RleDecoder};
/// Encodes integers as the change since the previous value.
///
/// The initial value is 0 encoded as u64. Deltas are encoded as i64.
///
/// Run length encoding is then applied to the resulting sequence.
pub(crate) struct DeltaEncoder<'a> {
rle: RleEncoder<'a, i64>,
absolute_value: i64,
}
impl<'a> DeltaEncoder<'a> {
pub fn new(output: &'a mut Vec<u8>) -> DeltaEncoder<'a> {
DeltaEncoder {
rle: RleEncoder::new(output),
absolute_value: 0,
}
}
pub fn append_value(&mut self, value: i64) {
self.rle
.append_value(&(value.saturating_sub(self.absolute_value)));
self.absolute_value = value;
}
pub fn append_null(&mut self) {
self.rle.append_null();
}
pub fn append(&mut self, val: Option<i64>) {
match val {
Some(v) => self.append_value(v),
None => self.append_null(),
}
}
pub fn finish(self) -> usize {
self.rle.finish()
}
}
impl<'a> From<&'a mut Vec<u8>> for DeltaEncoder<'a> {
fn from(output: &'a mut Vec<u8>) -> Self {
DeltaEncoder::new(output)
}
}
/// See discussion on [`DeltaEncoder`] for the format data is stored in.
#[derive(Clone)]
pub(crate) struct DeltaDecoder<'a> {
rle: RleDecoder<'a, i64>,
absolute_val: i64,
}
impl<'a> DeltaDecoder<'a> {
pub(crate) fn done(&self) -> bool {
self.rle.done()
}
pub(crate) fn encode<I>(items: I, out: &mut Vec<u8>) -> Range<usize>
where
I: Iterator<Item=i64>
{
let mut decoder = DeltaDecoder::from(&[] as &[u8]);
decoder.splice(0..0, items.map(Some), out)
}
pub(crate) fn splice<I: Iterator<Item=Option<i64>>>(&mut self, replace: Range<usize>, mut replace_with: I, out: &mut Vec<u8>) -> Range<usize> {
let start = out.len();
let mut encoder = DeltaEncoder::new(out);
let mut idx = 0;
while idx < replace.start {
match self.next() {
Some(elem) => encoder.append(elem),
None => panic!("out of bounds"),
}
idx += 1;
}
for _ in 0..replace.len() {
self.next();
if let Some(next) = replace_with.next() {
encoder.append(next);
}
}
while let Some(next) = replace_with.next() {
encoder.append(next);
}
while let Some(next) = self.next() {
encoder.append(next);
}
start..(start + encoder.finish())
}
}
impl<'a> From<Cow<'a, [u8]>> for DeltaDecoder<'a> {
fn from(bytes: Cow<'a, [u8]>) -> Self {
DeltaDecoder {
rle: RleDecoder::from(bytes),
absolute_val: 0,
}
}
}
impl<'a> From<&'a [u8]> for DeltaDecoder<'a> {
fn from(d: &'a [u8]) -> Self {
Cow::Borrowed(d).into()
}
}
impl<'a> Iterator for DeltaDecoder<'a> {
type Item = Option<i64>;
fn next(&mut self) -> Option<Option<i64>> {
match self.rle.next() {
Some(Some(delta)) => {
self.absolute_val = self.absolute_val.saturating_add(delta);
Some(Some(self.absolute_val))
},
Some(None) => Some(None),
None => None,
}
}
}
#[cfg(test)]
mod tests {
use super::*;
use proptest::prelude::*;
use crate::columnar_2::rowblock::encoding::properties::splice_scenario;
fn encode(vals: &[Option<i64>]) -> Vec<u8> {
let mut buf = Vec::<u8>::new();
let mut encoder = DeltaEncoder::from(&mut buf);
for val in vals {
encoder.append(val.clone());
}
encoder.finish();
buf
}
fn decode(buf: &[u8]) -> Vec<Option<i64>> {
DeltaDecoder::from(buf).collect()
}
fn encodable_int() -> impl Strategy<Value = i64> + Clone {
0..(i64::MAX / 2)
}
proptest!{
#[test]
fn encode_decode_delta(vals in proptest::collection::vec(proptest::option::of(encodable_int()), 0..100)) {
assert_eq!(vals, decode(&encode(&vals)));
}
#[test]
fn splice_delta(scenario in splice_scenario(proptest::option::of(encodable_int()))) {
let encoded = encode(&scenario.initial_values);
let mut decoder = DeltaDecoder::from(&encoded[..]);
let mut out = Vec::new();
let r = decoder.splice(scenario.replace_range.clone(), scenario.replacements.iter().cloned(), &mut out);
let decoded = decode(&out[..]);
scenario.check(decoded);
assert_eq!(r.len(), out.len());
}
}
}

View file

@ -0,0 +1,142 @@
use super::Encodable;
use std::borrow::Cow;
use smol_str::SmolStr;
use std::io::Write;
/// Encodes bytes without a length prefix
pub(crate) struct RawBytes<'a>(Cow<'a, [u8]>);
impl<'a> From<&'a [u8]> for RawBytes<'a> {
fn from(r: &'a [u8]) -> Self {
RawBytes(r.into())
}
}
impl<'a> From<Cow<'a, [u8]>> for RawBytes<'a> {
fn from(c: Cow<'a, [u8]>) -> Self {
RawBytes(c)
}
}
impl<'a> Encodable for RawBytes<'a> {
fn encode(&self, out: &mut Vec<u8>) -> usize {
out.write_all(&self.0).unwrap();
self.0.len()
}
}
impl Encodable for SmolStr {
fn encode(&self, buf: &mut Vec<u8>) -> usize {
let bytes = self.as_bytes();
let len_encoded = bytes.len().encode(buf);
let data_len = bytes.encode(buf);
len_encoded + data_len
}
}
impl<'a> Encodable for Cow<'a, SmolStr> {
fn encode(&self, buf: &mut Vec<u8>) -> usize {
self.as_ref().encode(buf)
}
}
impl Encodable for String {
fn encode(&self, buf: &mut Vec<u8>) ->usize {
let bytes = self.as_bytes();
let len_encoded = bytes.len().encode(buf);
let data_len = bytes.encode(buf);
len_encoded + data_len
}
}
impl Encodable for Option<String> {
fn encode(&self, buf: &mut Vec<u8>) -> usize {
if let Some(s) = self {
s.encode(buf)
} else {
0.encode(buf)
}
}
}
impl<'a> Encodable for Option<Cow<'a, SmolStr>> {
fn encode(&self, out: &mut Vec<u8>) -> usize {
if let Some(s) = self {
SmolStr::encode(s, out)
} else {
0.encode(out)
}
}
}
impl Encodable for u64 {
fn encode(&self, buf: &mut Vec<u8>) -> usize{
leb128::write::unsigned(buf, *self).unwrap()
}
}
impl Encodable for f64 {
fn encode(&self, buf: &mut Vec<u8>) -> usize {
let bytes = self.to_le_bytes();
buf.write_all(&bytes).unwrap();
bytes.len()
}
}
impl Encodable for f32 {
fn encode(&self, buf: &mut Vec<u8>) -> usize {
let bytes = self.to_le_bytes();
buf.write_all(&bytes).unwrap();
bytes.len()
}
}
impl Encodable for i64 {
fn encode(&self, buf: &mut Vec<u8>) -> usize {
leb128::write::signed(buf, *self).unwrap()
}
}
impl Encodable for usize {
fn encode(&self, buf: &mut Vec<u8>) -> usize{
(*self as u64).encode(buf)
}
}
impl Encodable for u32 {
fn encode(&self, buf: &mut Vec<u8>) -> usize {
u64::from(*self).encode(buf)
}
}
impl Encodable for i32 {
fn encode(&self, buf: &mut Vec<u8>) -> usize {
i64::from(*self).encode(buf)
}
}
impl Encodable for [u8] {
fn encode(&self, out: &mut Vec<u8>) -> usize {
out.write(self).unwrap()
}
}
impl Encodable for &[u8] {
fn encode(&self, out: &mut Vec<u8>) -> usize {
out.write(self).unwrap()
}
}
impl<'a> Encodable for Cow<'a, [u8]> {
fn encode(&self, out: &mut Vec<u8>) -> usize {
out.write(self).unwrap()
}
}
impl Encodable for Vec<u8> {
fn encode(&self, out: &mut Vec<u8>) -> usize {
Encodable::encode(&self[..], out)
}
}

View file

@ -0,0 +1,181 @@
use std::{
borrow::Cow,
ops::Range,
};
use crate::columnar_2::rowblock::{column_layout::ColumnSpliceError, value::CellValue};
use super::{
BooleanDecoder, DecodeColumnError, DeltaDecoder, RleDecoder,
ValueDecoder,
};
pub(crate) enum SimpleColDecoder<'a> {
RleUint(RleDecoder<'a, u64>),
RleString(RleDecoder<'a, smol_str::SmolStr>),
Delta(DeltaDecoder<'a>),
Bool(BooleanDecoder<'a>),
}
impl<'a> SimpleColDecoder<'a> {
pub(crate) fn new_uint(d: RleDecoder<'a, u64>) -> Self {
Self::RleUint(d)
}
pub(crate) fn new_string(d: RleDecoder<'a, smol_str::SmolStr>) -> Self {
Self::RleString(d)
}
pub(crate) fn new_delta(d: DeltaDecoder<'a>) -> Self {
Self::Delta(d)
}
pub(crate) fn new_bool(d: BooleanDecoder<'a>) -> Self {
Self::Bool(d)
}
pub(crate) fn done(&self) -> bool {
match self {
Self::RleUint(d) => d.done(),
Self::RleString(d) => d.done(),
Self::Delta(d) => d.done(),
Self::Bool(d) => d.done(),
}
}
pub(crate) fn next(&mut self) -> Option<CellValue<'a>> {
match self {
Self::RleUint(d) => d.next().and_then(|i| i.map(CellValue::Uint)),
Self::RleString(d) => d
.next()
.and_then(|s| s.map(|s| CellValue::String(Cow::Owned(s.into())))),
Self::Delta(d) => d.next().and_then(|i| i.map(CellValue::Int)),
Self::Bool(d) => d.next().map(CellValue::Bool),
}
}
pub(crate) fn splice<'b, I>(
&mut self,
out: &mut Vec<u8>,
replace: Range<usize>,
replace_with: I,
) -> Result<usize, ColumnSpliceError>
where
I: Iterator<Item=CellValue<'b>> + Clone
{
// Requires `try_splice` methods on all the basic decoders so that we can report an error
// if the cellvalue types don't match up
unimplemented!()
}
}
pub(crate) enum SingleLogicalColDecoder<'a> {
Simple(SimpleColDecoder<'a>),
Value(ValueDecoder<'a>),
}
impl<'a> Iterator for SingleLogicalColDecoder<'a> {
type Item = Result<CellValue<'a>, DecodeColumnError>;
fn next(&mut self) -> Option<Self::Item> {
match self {
Self::Simple(s) => s.next().map(Ok),
Self::Value(v) => v.next().map(|v| v.map(|v| CellValue::Value(v))),
}
}
}
pub(crate) enum GenericColDecoder<'a> {
Simple(SimpleColDecoder<'a>),
Value(ValueDecoder<'a>),
Group(GroupDecoder<'a>),
}
impl<'a> GenericColDecoder<'a> {
pub(crate) fn new_simple(s: SimpleColDecoder<'a>) -> Self {
Self::Simple(s)
}
pub(crate) fn new_value(v: ValueDecoder<'a>) -> Self {
Self::Value(v)
}
pub(crate) fn new_group(g: GroupDecoder<'a>) -> Self {
Self::Group(g)
}
pub(crate) fn done(&self) -> bool {
match self {
Self::Simple(s) => s.done(),
Self::Group(g) => g.done(),
Self::Value(v) => v.done(),
}
}
pub(crate) fn next(&mut self) -> Option<Result<CellValue<'a>, DecodeColumnError>> {
match self {
Self::Simple(s) => s.next().map(Ok),
Self::Value(v) => v.next().map(|v| v.map(|v| CellValue::Value(v))),
Self::Group(g) => g.next().map(|v| v.map(|v| CellValue::List(v))),
}
}
}
impl<'a> Iterator for GenericColDecoder<'a> {
type Item = Result<CellValue<'a>, DecodeColumnError>;
fn next(&mut self) -> Option<Self::Item> {
GenericColDecoder::next(self)
}
}
pub(crate) struct GroupDecoder<'a> {
num: RleDecoder<'a, u64>,
values: Vec<SingleLogicalColDecoder<'a>>,
}
impl<'a> GroupDecoder<'a> {
pub(crate) fn new(
num: RleDecoder<'a, u64>,
values: Vec<SingleLogicalColDecoder<'a>>,
) -> GroupDecoder<'a> {
GroupDecoder { num, values }
}
fn next(&mut self) -> Option<Result<Vec<Vec<CellValue<'a>>>, DecodeColumnError>> {
match self.num.next() {
Some(Some(num_rows)) => {
let mut result = Vec::with_capacity(num_rows as usize);
for _ in 0..num_rows {
let mut row = Vec::with_capacity(self.values.len());
for (index, column) in self.values.iter_mut().enumerate() {
match column.next() {
Some(Ok(v)) => row.push(v),
Some(Err(e)) => {
return Some(Err(DecodeColumnError::InvalidValue {
column: format!("group column {0}", index + 1),
description: e.to_string(),
}))
}
None => {
return Some(Err(DecodeColumnError::UnexpectedNull(format!(
"grouped column {0}",
index + 1
))))
}
}
}
result.push(row)
}
Some(Ok(result))
}
Some(None) => Some(Err(DecodeColumnError::UnexpectedNull("num".to_string()))),
_ => None,
}
}
fn done(&self) -> bool {
self.num.done()
}
}

View file

@ -0,0 +1,41 @@
use crate::types::{ElemId, Key, OpId};
use super::{DeltaDecoder, RleDecoder};
pub(crate) struct InternedKeyDecoder<'a> {
actor: RleDecoder<'a, u64>,
ctr: DeltaDecoder<'a>,
str_idx: RleDecoder<'a, u64>,
}
impl<'a> InternedKeyDecoder<'a> {
pub(crate) fn new(
actor: RleDecoder<'a, u64>,
ctr: DeltaDecoder<'a>,
str_idx: RleDecoder<'a, u64>,
) -> Self {
Self {
actor,
ctr,
str_idx,
}
}
pub(crate) fn done(&self) -> bool {
self.actor.done() && self.ctr.done() && self.str_idx.done()
}
}
impl<'a> Iterator for InternedKeyDecoder<'a> {
type Item = Key;
fn next(&mut self) -> Option<Key> {
match (self.actor.next(), self.ctr.next(), self.str_idx.next()) {
(None, None, Some(Some(key_idx))) => Some(Key::Map(key_idx as usize)),
(None, Some(Some(0)), None) => Some(Key::Seq(ElemId(OpId(0, 0)))),
(Some(Some(actor)), Some(Some(ctr)), None) => Some(Key::Seq(OpId(actor, ctr as usize).into())),
// TODO: This should be fallible and throw here
_ => None,
}
}
}

View file

@ -0,0 +1,119 @@
use std::{borrow::Cow, ops::Range};
use smol_str::SmolStr;
use super::{DecodeColumnError, DeltaDecoder, RleDecoder};
use crate::types::{ElemId, OpId};
#[derive(Clone, Debug, PartialEq)]
pub(crate) enum Key {
Prop(smol_str::SmolStr),
Elem(ElemId),
}
pub(crate) struct KeyDecoder<'a> {
actor: RleDecoder<'a, u64>,
ctr: DeltaDecoder<'a>,
str: RleDecoder<'a, SmolStr>,
}
impl<'a> KeyDecoder<'a> {
pub(crate) fn new(
actor: RleDecoder<'a, u64>,
ctr: DeltaDecoder<'a>,
str: RleDecoder<'a, SmolStr>,
) -> Self {
Self { actor, ctr, str }
}
pub(crate) fn empty() -> KeyDecoder<'static> {
KeyDecoder {
actor: RleDecoder::from(Cow::Owned(Vec::new())),
ctr: DeltaDecoder::from(Cow::Owned(Vec::new())),
str: RleDecoder::from(Cow::Owned(Vec::new())),
}
}
pub(crate) fn done(&self) -> bool {
self.actor.done() && self.ctr.done() && self.str.done()
}
/// Splice new keys into this set of keys, encoding the resulting actor, counter, and str
/// columns in `out`. The result is (actor, ctr, str) where actor is the range of the output which
/// contains the new actor column, ctr the counter column, and str the str column.
pub(crate) fn splice<'b, I: Iterator<Item = &'b Key> + Clone>(
&mut self,
replace: Range<usize>,
replace_with: I,
out: &mut Vec<u8>,
) -> (Range<usize>, Range<usize>, Range<usize>) {
panic!()
}
}
impl<'a> Iterator for KeyDecoder<'a> {
type Item = Result<Key, DecodeColumnError>;
fn next(&mut self) -> Option<Self::Item> {
match (self.actor.next(), self.ctr.next(), self.str.next()) {
(Some(Some(_)), Some(Some(_)), Some(Some(_))) => {
Some(Err(DecodeColumnError::InvalidValue {
column: "key".to_string(),
description: "too many values".to_string(),
}))
}
(Some(None), Some(None), Some(Some(string))) => Some(Ok(Key::Prop(string))),
(Some(None), Some(Some(0)), Some(None)) => Some(Ok(Key::Elem(ElemId(OpId(0, 0))))),
(Some(Some(actor)), Some(Some(ctr)), Some(None)) => match ctr.try_into() {
Ok(ctr) => Some(Ok(Key::Elem(ElemId(OpId(ctr, actor as usize))))),
Err(e) => Some(Err(DecodeColumnError::InvalidValue{
column: "counter".to_string(),
description: "negative value for counter".to_string(),
})),
}
(None, None, None) => None,
(None | Some(None), _, _) => {
Some(Err(DecodeColumnError::UnexpectedNull("actor".to_string())))
}
(_, None | Some(None), _) => {
Some(Err(DecodeColumnError::UnexpectedNull("ctr".to_string())))
}
(_, _, None) => Some(Err(DecodeColumnError::UnexpectedNull("str".to_string()))),
}
}
}
#[cfg(test)]
mod tests {
use super::*;
use proptest::prelude::*;
fn encode(vals: &[Key]) -> (Vec<u8>, Range<usize>, Range<usize>, Range<usize>) {
let mut out = Vec::new();
let mut decoder = KeyDecoder::empty();
let (actor, ctr, string) = decoder.splice(0..0, vals.iter(), &mut out);
(out, actor, ctr, string)
}
//proptest! {
//#[test]
//fn splice_key(scenario in splice_scenario(row_op_key())) {
//let (buf, actor, ctr, string) = encode(&scenario.initial_values[..]);
//let mut decoder = KeyDecoder::new(
//RleDecoder::from(&buf[actor]),
//DeltaDecoder::from(&buf[ctr]),
//RleDecoder::from(&buf[string]),
//);
//let mut out = Vec::new();
//let (actor, ctr, string) = decoder.splice(scenario.replace_range.clone(), scenario.replacements.iter(), &mut out);
//let decoder = KeyDecoder::new(
//RleDecoder::from(&buf[actor]),
//DeltaDecoder::from(&buf[ctr]),
//RleDecoder::from(&buf[string.clone()]),
//);
//let result = decoder.map(|c| c.unwrap()).collect();
//scenario.check(result);
//assert_eq!(string.end, out.len());
//}
//}
}

View file

@ -0,0 +1,52 @@
mod raw;
use std::borrow::Borrow;
pub(crate) use raw::{RawEncoder, RawDecoder};
mod rle;
pub(crate) use rle::{RleEncoder, RleDecoder};
mod boolean;
pub(crate) use boolean::{BooleanDecoder, BooleanEncoder};
mod delta;
pub(crate) use delta::{DeltaDecoder, DeltaEncoder};
mod value;
pub(crate) use value::ValueDecoder;
pub(crate) mod generic;
pub(crate) use generic::{GenericColDecoder, SimpleColDecoder};
mod opid;
pub(crate) use opid::OpIdDecoder;
mod opid_list;
pub(crate) use opid_list::OpIdListDecoder;
mod obj_id;
pub(crate) use obj_id::ObjDecoder;
mod key;
pub(crate) use key::{Key, KeyDecoder};
#[cfg(test)]
pub(crate) mod properties;
pub(crate) trait Encodable {
fn encode(&self, out: &mut Vec<u8>) -> usize;
}
mod encodable_impls;
pub(crate) use encodable_impls::RawBytes;
pub(crate) trait Decodable: Sized {
fn decode<R>(bytes: &mut R) -> Option<Self>
where
R: std::io::Read;
}
mod decodable_impls;
#[derive(Clone, thiserror::Error, Debug)]
pub(crate) enum DecodeColumnError {
#[error("unexpected null decoding column {0}")]
UnexpectedNull(String),
#[error("invalid value in column {column}: {description}")]
InvalidValue{
column: String,
description: String,
},
}

View file

@ -0,0 +1,35 @@
use crate::types::{OpId, ObjId};
use super::{DecodeColumnError, RleDecoder};
pub(crate) struct ObjDecoder<'a> {
actor: RleDecoder<'a, u64>,
ctr: RleDecoder<'a, u64>,
}
impl<'a> ObjDecoder<'a> {
pub(crate) fn new(actor: RleDecoder<'a, u64>, ctr: RleDecoder<'a, u64>) -> Self {
Self{
actor,
ctr,
}
}
pub(crate) fn done(&self) -> bool {
self.actor.done() || self.ctr.done()
}
}
impl<'a> Iterator for ObjDecoder<'a> {
type Item = Result<ObjId, DecodeColumnError>;
fn next(&mut self) -> Option<Self::Item> {
match (self.actor.next(), self.ctr.next()) {
(None, None) => None,
(Some(None), Some(None)) => Some(Ok(ObjId::root())),
(Some(Some(a)), Some(Some(c))) => Some(Ok(ObjId(OpId(c, a as usize)))),
(Some(None), _) | (None, _) => Some(Err(DecodeColumnError::UnexpectedNull("actor".to_string()))),
(_, Some(None)) | (_, None) => Some(Err(DecodeColumnError::UnexpectedNull("counter".to_string()))),
}
}
}

View file

@ -0,0 +1,110 @@
use std::{borrow::Cow, ops::Range};
use crate::types::OpId;
use super::{DecodeColumnError, DeltaDecoder, RleDecoder};
pub(crate) struct OpIdDecoder<'a> {
actor: RleDecoder<'a, u64>,
ctr: DeltaDecoder<'a>,
}
impl Default for OpIdDecoder<'static> {
fn default() -> Self {
Self::new(
RleDecoder::from(Cow::Owned(Vec::new())),
DeltaDecoder::from(Cow::Owned(Vec::new())),
)
}
}
impl<'a> OpIdDecoder<'a> {
pub(crate) fn new(actor: RleDecoder<'a, u64>, ctr: DeltaDecoder<'a>) -> Self {
Self { actor, ctr }
}
pub(crate) fn done(&self) -> bool {
self.actor.done() && self.ctr.done()
}
/// Splice new operations into this set of operations, encoding the resulting actor and counter
/// columns in `out`. The result is (actor, ctr) where actor is the range of the output which
/// contains the new actor column and ctr the counter column.
pub(crate) fn splice<'b, I: Iterator<Item = &'b OpId> + Clone>(
&mut self,
replace: Range<usize>,
replace_with: I,
out: &mut Vec<u8>,
) -> (Range<usize>, Range<usize>) {
// first splice actors, then counters
let actor = self.actor.splice(
replace.clone(),
replace_with.clone().map(|i| Some(i.actor() as u64)),
out,
);
let counter = self
.ctr
.splice(replace, replace_with.map(|i| Some(i.counter() as i64)), out);
(actor, counter)
}
}
impl<'a> Iterator for OpIdDecoder<'a> {
type Item = Result<OpId, DecodeColumnError>;
fn next(&mut self) -> Option<Self::Item> {
match (self.actor.next(), self.ctr.next()) {
(Some(Some(a)), Some(Some(c))) => match c.try_into() {
Ok(c) => Some(Ok(OpId(c, a as usize))),
Err(e) => Some(Err(DecodeColumnError::InvalidValue{
column: "counter".to_string(),
description: "negative value encountered".to_string(),
}))
},
(Some(None), _) => Some(Err(DecodeColumnError::UnexpectedNull("actor".to_string()))),
(_, Some(None)) => Some(Err(DecodeColumnError::UnexpectedNull("actor".to_string()))),
(Some(_), None) => Some(Err(DecodeColumnError::UnexpectedNull("ctr".to_string()))),
(None, Some(_)) => Some(Err(DecodeColumnError::UnexpectedNull("actor".to_string()))),
(None, None) => None,
}
}
}
#[cfg(test)]
mod tests {
use super::*;
use crate::columnar_2::rowblock::encoding::properties::{opid, splice_scenario};
use proptest::prelude::*;
fn encode(vals: &[OpId]) -> (Vec<u8>, Range<usize>, Range<usize>) {
let mut out = Vec::new();
let mut decoder = OpIdDecoder::default();
let (actor, ctr) = decoder.splice(0..0, vals.into_iter(), &mut out);
(out, actor, ctr)
}
fn decode(buf: &[u8], actor: Range<usize>, ctr: Range<usize>) -> Vec<OpId> {
OpIdDecoder::new(RleDecoder::from(&buf[actor]), DeltaDecoder::from(&buf[ctr]))
.map(|c| c.unwrap())
.collect()
}
proptest! {
#[test]
fn encode_decode_opid(opids in proptest::collection::vec(opid(), 0..100)) {
let (encoded, actor, ctr) = encode(&opids);
assert_eq!(opids, decode(&encoded[..], actor, ctr));
}
#[test]
fn splice_opids(scenario in splice_scenario(opid())) {
let (encoded, actor, ctr) = encode(&scenario.initial_values);
let mut decoder = OpIdDecoder::new(RleDecoder::from(&encoded[actor]), DeltaDecoder::from(&encoded[ctr]));
let mut out = Vec::new();
let (actor, ctr) = decoder.splice(scenario.replace_range.clone(), scenario.replacements.iter(), &mut out);
let result = decode(&out[..], actor, ctr.clone());
scenario.check(result);
assert_eq!(ctr.end, out.len());
}
}
}

View file

@ -0,0 +1,182 @@
use std::{borrow::Cow, ops::Range};
use crate::types::OpId;
use super::{DecodeColumnError, DeltaDecoder, RleDecoder};
pub(crate) struct OpIdListDecoder<'a> {
num: RleDecoder<'a, u64>,
actor: RleDecoder<'a, u64>,
ctr: DeltaDecoder<'a>,
}
impl<'a> OpIdListDecoder<'a> {
pub(crate) fn new(
num: RleDecoder<'a, u64>,
actor: RleDecoder<'a, u64>,
ctr: DeltaDecoder<'a>,
) -> Self {
Self { num, actor, ctr }
}
/// A decoder which references empty arrays, therefore has no elements
pub(crate) fn empty() -> OpIdListDecoder<'static> {
OpIdListDecoder {
num: RleDecoder::from(Cow::Owned(Vec::new())),
actor: RleDecoder::from(Cow::Owned(Vec::new())),
ctr: DeltaDecoder::from(Cow::Owned(Vec::new())),
}
}
pub(crate) fn done(&self) -> bool {
self.num.done()
}
/// Splice new lists of opids into this set of lists of opids, encoding the resulting num, actor and counter
/// columns in `out`. The result is (num, actor, ctr) where num is the range of the output which
/// contains the new num column, actor the actor column, and ctr the counter column
pub(crate) fn splice<'b, I, II, IE>(
&mut self,
replace: Range<usize>,
replace_with: I,
out: &mut Vec<u8>,
) -> (Range<usize>, Range<usize>, Range<usize>)
where
II: IntoIterator<Item = OpId, IntoIter=IE>,
IE: Iterator<Item=OpId> + ExactSizeIterator,
I: Iterator<Item = II> + Clone,
{
let group_replace = group_replace_range(replace.clone(), self.num.clone());
// first nums
let num = self.num.splice(
replace.clone(),
replace_with.clone().map(|elems| Some(elems.into_iter().len() as u64)),
out,
);
let actor = self.actor.splice(
group_replace.clone(),
replace_with
.clone()
.flat_map(|elem| elem.into_iter().map(|oid| Some(oid.actor() as u64))),
out,
);
let ctr = self.ctr.splice(
group_replace,
replace_with.flat_map(|elem| elem.into_iter().map(|oid| Some(oid.counter() as i64))),
out,
);
(num, actor, ctr)
}
}
/// Find the replace range for the grouped columns.
fn group_replace_range(replace: Range<usize>, mut num: RleDecoder<u64>) -> Range<usize> {
let mut idx = 0;
let mut grouped_replace_start: usize = 0;
let mut grouped_replace_len: usize = 0;
while idx < replace.start {
if let Some(Some(count)) = num.next() {
grouped_replace_start += count as usize;
}
idx += 1;
}
for _ in 0..replace.len() {
if let Some(Some(count)) = num.next() {
grouped_replace_len += count as usize;
}
}
grouped_replace_start..(grouped_replace_start + grouped_replace_len)
}
impl<'a> Iterator for OpIdListDecoder<'a> {
type Item = Result<Vec<OpId>, DecodeColumnError>;
fn next(&mut self) -> Option<Self::Item> {
let num = match self.num.next() {
Some(Some(n)) => n,
Some(None) => return Some(Err(DecodeColumnError::UnexpectedNull("num".to_string()))),
None => return None,
};
let mut p = Vec::with_capacity(num as usize);
for _ in 0..num {
match (self.actor.next(), self.ctr.next()) {
(Some(Some(a)), Some(Some(ctr))) => match ctr.try_into() {
Ok(ctr) => p.push(OpId(ctr, a as usize)),
Err(e) => return Some(Err(DecodeColumnError::InvalidValue{
column: "counter".to_string(),
description: "negative value for counter".to_string(),
}))
},
(Some(None) | None, _) => {
return Some(Err(DecodeColumnError::UnexpectedNull("actor".to_string())))
}
(_, Some(None) | None) => {
return Some(Err(DecodeColumnError::UnexpectedNull("ctr".to_string())))
}
}
}
Some(Ok(p))
}
}
#[cfg(test)]
mod tests {
use super::*;
use proptest::collection::vec as propvec;
use proptest::prelude::*;
use crate::columnar_2::rowblock::encoding::properties::{opid, splice_scenario};
fn encode(opids: Vec<Vec<OpId>>) -> (Vec<u8>, Range<usize>, Range<usize>, Range<usize>) {
let mut out = Vec::new();
let mut decoder = OpIdListDecoder::empty();
let (num, actor, ctr) = decoder.splice(
0..0,
opids.into_iter(),
&mut out,
);
(out, num, actor, ctr)
}
fn decode(
buf: &[u8],
num: Range<usize>,
actor: Range<usize>,
ctr: Range<usize>,
) -> Vec<Vec<OpId>> {
let decoder = OpIdListDecoder::new(
RleDecoder::from(&buf[num]),
RleDecoder::from(&buf[actor]),
DeltaDecoder::from(&buf[ctr]),
);
decoder.map(|c| c.unwrap()).collect()
}
proptest! {
#[test]
fn encode_decode_opid_list(opids in propvec(propvec(opid(), 0..100), 0..100)){
let (encoded, num, actor, ctr) = encode(opids.clone());
let result = decode(&encoded, num, actor, ctr);
assert_eq!(opids, result)
}
#[test]
fn splice_opid_list(scenario in splice_scenario(propvec(opid(), 0..100))) {
let (encoded, num, actor, ctr) = encode(scenario.initial_values.clone());
let mut decoder = OpIdListDecoder::new(
RleDecoder::from(&encoded[num]),
RleDecoder::from(&encoded[actor]),
DeltaDecoder::from(&encoded[ctr]),
);
let mut out = Vec::new();
let (num, actor, ctr) = decoder.splice(
scenario.replace_range.clone(),
scenario.replacements.clone().into_iter(),
&mut out
);
let result = decode(&out[..], num, actor, ctr.clone());
scenario.check(result);
assert_eq!(ctr.end, out.len())
}
}
}

View file

@ -0,0 +1,109 @@
//! Helpers for property tests.
use std::{borrow::Cow, fmt::Debug, ops::Range};
use proptest::prelude::*;
use smol_str::SmolStr;
use crate::{
columnar_2::rowblock::{PrimVal, Key},
types::{OpId, Key as InternedKey, ElemId}
};
#[derive(Clone, Debug)]
pub(crate) struct SpliceScenario<T> {
pub(crate) initial_values: Vec<T>,
pub(crate) replace_range: Range<usize>,
pub(crate) replacements: Vec<T>,
}
impl<T: Debug + PartialEq + Clone> SpliceScenario<T> {
pub(crate) fn check(&self, results: Vec<T>) {
let mut expected = self
.initial_values
.clone();
expected.splice(self.replace_range.clone(), self.replacements.clone());
assert_eq!(expected, results)
}
}
pub(crate) fn splice_scenario<S: Strategy<Value = T> + Clone, T: Debug + Clone + 'static>(
item_strat: S,
) -> impl Strategy<Value = SpliceScenario<T>> {
(
proptest::collection::vec(item_strat.clone(), 0..100),
proptest::collection::vec(item_strat, 0..10),
)
.prop_flat_map(move |(values, to_splice)| {
if values.len() == 0 {
Just(SpliceScenario {
initial_values: values.clone(),
replace_range: 0..0,
replacements: to_splice.clone(),
})
.boxed()
} else {
// This is somewhat awkward to write because we have to carry the `values` and
// `to_splice` through as `Just(..)` to please the borrow checker.
(0..values.len(), Just(values), Just(to_splice))
.prop_flat_map(move |(replace_range_start, values, to_splice)| {
(
0..(values.len() - replace_range_start),
Just(values),
Just(to_splice),
)
.prop_map(
move |(replace_range_len, values, to_splice)| SpliceScenario {
initial_values: values.clone(),
replace_range: replace_range_start
..(replace_range_start + replace_range_len),
replacements: to_splice.clone(),
},
)
})
.boxed()
}
})
}
pub(crate) fn opid() -> impl Strategy<Value = OpId> + Clone {
(0..(i64::MAX as usize), 0..(i64::MAX as u64)).prop_map(|(actor, ctr)| OpId(ctr, actor))
}
pub(crate) fn elemid() -> impl Strategy<Value = ElemId> + Clone {
opid().prop_map(ElemId)
}
pub(crate) fn interned_key() -> impl Strategy<Value = InternedKey> + Clone {
prop_oneof!{
elemid().prop_map(InternedKey::Seq),
(0..(i64::MAX as usize)).prop_map(InternedKey::Map),
}
}
pub(crate) fn key() -> impl Strategy<Value = Key> + Clone {
prop_oneof!{
elemid().prop_map(Key::Elem),
any::<String>().prop_map(|s| Key::Prop(s.into())),
}
}
pub(crate) fn value() -> impl Strategy<Value = PrimVal<'static>> + Clone {
prop_oneof! {
Just(PrimVal::Null),
any::<bool>().prop_map(|b| PrimVal::Bool(b)),
any::<u64>().prop_map(|i| PrimVal::Uint(i)),
any::<i64>().prop_map(|i| PrimVal::Int(i)),
any::<f64>().prop_map(|f| PrimVal::Float(f)),
any::<String>().prop_map(|s| PrimVal::String(Cow::Owned(s.into()))),
any::<Vec<u8>>().prop_map(|b| PrimVal::Bytes(Cow::Owned(b))),
any::<u64>().prop_map(|i| PrimVal::Counter(i)),
any::<u64>().prop_map(|i| PrimVal::Timestamp(i)),
(10..15_u8, any::<Vec<u8>>()).prop_map(|(c, b)| PrimVal::Unknown { type_code: c, data: b }),
}
}
fn smol_str() -> impl Strategy<Value = SmolStr> + Clone {
any::<String>().prop_map(SmolStr::from)
}

View file

@ -0,0 +1,96 @@
use std::{borrow::Cow, fmt::Debug};
use super::{Decodable, Encodable};
#[derive(Clone, Debug)]
pub(crate) struct RawDecoder<'a> {
pub offset: usize,
pub last_read: usize,
data: Cow<'a, [u8]>,
}
#[derive(thiserror::Error, Debug)]
pub(crate) enum Error {
#[error("no decoded value")]
NoDecodedValue,
#[error("buffer size did not change")]
BufferSizeDidNotChange,
#[error("trying to read past end")]
TryingToReadPastEnd,
}
impl<'a> RawDecoder<'a> {
pub fn new(data: Cow<'a, [u8]>) -> Self {
RawDecoder {
offset: 0,
last_read: 0,
data,
}
}
pub fn read<T: Decodable + Debug>(&mut self) -> Result<T, Error> {
let mut buf = &self.data[self.offset..];
let init_len = buf.len();
let val = T::decode::<&[u8]>(&mut buf).ok_or(Error::NoDecodedValue)?;
let delta = init_len - buf.len();
if delta == 0 {
Err(Error::BufferSizeDidNotChange)
} else {
self.last_read = delta;
self.offset += delta;
Ok(val)
}
}
pub fn read_bytes(&mut self, index: usize) -> Result<&[u8], Error> {
if self.offset + index > self.data.len() {
Err(Error::TryingToReadPastEnd)
} else {
let head = &self.data[self.offset..self.offset + index];
self.last_read = index;
self.offset += index;
Ok(head)
}
}
pub fn done(&self) -> bool {
self.offset >= self.data.len()
}
}
impl<'a> From<&'a [u8]> for RawDecoder<'a> {
fn from(d: &'a [u8]) -> Self {
Cow::Borrowed(d).into()
}
}
impl<'a> From<Cow<'a, [u8]>> for RawDecoder<'a> {
fn from(d: Cow<'a, [u8]>) -> Self {
RawDecoder::new(d)
}
}
pub(crate) struct RawEncoder<'a> {
written: usize,
output: &'a mut Vec<u8>,
}
impl<'a> RawEncoder<'a> {
pub(crate) fn append<I: Encodable>(&mut self, value: &I) -> usize {
let written = value.encode(&mut self.output);
self.written += written;
written
}
fn finish(self) -> usize {
self.written
}
}
impl<'a> From<&'a mut Vec<u8>> for RawEncoder<'a> {
fn from(output: &'a mut Vec<u8>) -> Self {
RawEncoder{ written: 0, output }
}
}

View file

@ -0,0 +1,364 @@
use std::{
borrow::{Borrow, Cow},
fmt::Debug,
ops::Range,
};
use super::{Encodable, Decodable, RawDecoder};
pub(crate) struct RleEncoder<'a, T>
where
T: Encodable + PartialEq + Clone,
{
buf: &'a mut Vec<u8>,
written: usize,
state: RleState<T>,
}
impl<'a, T> RleEncoder<'a, T>
where
T: Encodable + PartialEq + Clone,
{
pub fn new(output_buf: &'a mut Vec<u8>) -> RleEncoder<'a, T> {
RleEncoder {
buf: output_buf,
written: 0,
state: RleState::Empty,
}
}
pub fn finish(mut self) -> usize {
match self.take_state() {
// this covers `only_nulls`
RleState::NullRun(size) => {
self.flush_null_run(size);
}
RleState::LoneVal(value) => self.flush_lit_run(vec![value]),
RleState::Run(value, len) => self.flush_run(&value, len),
RleState::LiteralRun(last, mut run) => {
run.push(last);
self.flush_lit_run(run);
}
RleState::Empty => {}
}
self.written
}
fn flush_run(&mut self, val: &T, len: usize) {
self.encode(&(len as i64));
self.encode(val);
}
fn flush_null_run(&mut self, len: usize) {
self.encode::<i64>(&0);
self.encode(&len);
}
fn flush_lit_run(&mut self, run: Vec<T>) {
self.encode(&-(run.len() as i64));
for val in run {
self.encode(&val);
}
}
fn take_state(&mut self) -> RleState<T> {
let mut state = RleState::Empty;
std::mem::swap(&mut self.state, &mut state);
state
}
pub fn append_null(&mut self) {
self.state = match self.take_state() {
RleState::Empty => RleState::NullRun(1),
RleState::NullRun(size) => RleState::NullRun(size + 1),
RleState::LoneVal(other) => {
self.flush_lit_run(vec![other]);
RleState::NullRun(1)
}
RleState::Run(other, len) => {
self.flush_run(&other, len);
RleState::NullRun(1)
}
RleState::LiteralRun(last, mut run) => {
run.push(last);
self.flush_lit_run(run);
RleState::NullRun(1)
}
}
}
pub fn append_value(&mut self, value: &T) {
self.state = match self.take_state() {
RleState::Empty => RleState::LoneVal(value.clone()),
RleState::LoneVal(other) => {
if &other == value {
RleState::Run(value.clone(), 2)
} else {
let mut v = Vec::with_capacity(2);
v.push(other);
RleState::LiteralRun(value.clone(), v)
}
}
RleState::Run(other, len) => {
if &other == value {
RleState::Run(other, len + 1)
} else {
self.flush_run(&other, len);
RleState::LoneVal(value.clone())
}
}
RleState::LiteralRun(last, mut run) => {
if &last == value {
self.flush_lit_run(run);
RleState::Run(value.clone(), 2)
} else {
run.push(last);
RleState::LiteralRun(value.clone(), run)
}
}
RleState::NullRun(size) => {
self.flush_null_run(size);
RleState::LoneVal(value.clone())
}
}
}
pub fn append(&mut self, value: Option<&T>) {
match value {
Some(t) => self.append_value(t),
None => self.append_null(),
}
}
fn encode<V>(&mut self, val: &V)
where
V: Encodable,
{
self.written += val.encode(&mut self.buf);
}
}
enum RleState<T> {
Empty,
NullRun(usize),
LiteralRun(T, Vec<T>),
LoneVal(T),
Run(T, usize),
}
impl<'a, T: Clone + PartialEq + Encodable> From<&'a mut Vec<u8>> for RleEncoder<'a, T> {
fn from(output: &'a mut Vec<u8>) -> Self {
Self::new(output)
}
}
/// See discussion on [`RleEncoder`] for the format data is stored in.
#[derive(Clone, Debug)]
pub(crate) struct RleDecoder<'a, T> {
pub decoder: RawDecoder<'a>,
last_value: Option<T>,
count: isize,
literal: bool,
}
impl<'a, T> RleDecoder<'a, T> {
fn empty() -> Self {
RleDecoder{
decoder: RawDecoder::from(&[] as &[u8]),
last_value: None,
count: 0,
literal: false,
}
}
pub(crate) fn done(&self) -> bool {
self.decoder.done() && self.count == 0
}
}
impl<'a, T: Clone + Debug + Encodable + Decodable + Eq> RleDecoder<'a, T> {
pub(crate) fn encode<I>(items: I, out: &'a mut Vec<u8>) -> Range<usize>
where
I: Iterator<Item=T>
{
let mut empty = RleDecoder::empty();
let range = empty.splice(0..0, items.map(Some), out);
range
}
pub(crate) fn splice<I: Iterator<Item=Option<TB>>, TB: Borrow<T>>(&mut self, replace: Range<usize>, mut replace_with: I, out: &mut Vec<u8>) -> Range<usize> {
let start = out.len();
let mut encoder = RleEncoder::new(out);
let mut idx = 0;
while idx < replace.start {
match self.next() {
Some(elem) => encoder.append(elem.as_ref()),
None => panic!("out of bounds"),
}
idx += 1;
}
for _ in 0..replace.len() {
self.next();
if let Some(next) = replace_with.next() {
encoder.append(next.as_ref().map(|n| n.borrow()));
}
}
while let Some(next) = replace_with.next() {
encoder.append(next.as_ref().map(|n| n.borrow()));
}
while let Some(next) = self.next() {
encoder.append(next.as_ref());
}
start..(start + encoder.finish())
}
}
impl<'a, T> From<Cow<'a, [u8]>> for RleDecoder<'a, T> {
fn from(bytes: Cow<'a, [u8]>) -> Self {
RleDecoder {
decoder: RawDecoder::from(bytes),
last_value: None,
count: 0,
literal: false,
}
}
}
impl<'a, T> From<&'a [u8]> for RleDecoder<'a, T> {
fn from(d: &'a [u8]) -> Self {
Cow::Borrowed(d).into()
}
}
// this decoder needs to be able to send type T or 'null'
// it is an endless iterator that will return all 'null's
// once input is exhausted
impl<'a, T> Iterator for RleDecoder<'a, T>
where
T: Clone + Debug + Decodable,
{
type Item = Option<T>;
fn next(&mut self) -> Option<Option<T>> {
while self.count == 0 {
if self.decoder.done() {
return None;
}
match self.decoder.read::<i64>() {
Ok(count) if count > 0 => {
// normal run
self.count = count as isize;
self.last_value = self.decoder.read().ok();
self.literal = false;
}
Ok(count) if count < 0 => {
// literal run
self.count = count.abs() as isize;
self.literal = true;
}
Ok(_) => {
// null run
// FIXME(jeffa5): handle usize > i64 here somehow
self.count = self.decoder.read::<usize>().unwrap() as isize;
self.last_value = None;
self.literal = false;
}
Err(e) => {
tracing::warn!(error=?e, "error during rle decoding");
return None;
}
}
}
self.count -= 1;
if self.literal {
Some(self.decoder.read().ok())
} else {
Some(self.last_value.clone())
}
}
}
#[cfg(test)]
mod tests {
use std::borrow::Cow;
use super::*;
use proptest::prelude::*;
use super::super::properties::splice_scenario;
#[test]
fn rle_int_round_trip() {
let vals = [1,1,2,2,3,2,3,1,3];
let mut buf = Vec::with_capacity(vals.len() * 3);
let mut encoder: RleEncoder<'_, u64> = RleEncoder::new(&mut buf);
for val in vals {
encoder.append_value(&val)
}
let total_slice_len = encoder.finish();
let mut decoder: RleDecoder<'_, u64> = RleDecoder::from(Cow::Borrowed(&buf[0..total_slice_len]));
let mut result = Vec::new();
while let Some(Some(val)) = decoder.next() {
result.push(val);
}
assert_eq!(result, vals);
}
#[test]
fn rle_int_insert() {
let vals = [1,1,2,2,3,2,3,1,3];
let mut buf = Vec::with_capacity(vals.len() * 3);
let mut encoder: RleEncoder<'_, u64> = RleEncoder::new(&mut buf);
for i in 0..4 {
encoder.append_value(&vals[i])
}
encoder.append_value(&5);
for i in 4..vals.len() {
encoder.append_value(&vals[i]);
}
let total_slice_len = encoder.finish();
let mut decoder: RleDecoder<'_, u64> = RleDecoder::from(Cow::Borrowed(&buf[0..total_slice_len]));
let mut result = Vec::new();
while let Some(Some(val)) = decoder.next() {
result.push(val);
}
let expected = [1,1,2,2,5,3,2,3,1,3];
assert_eq!(result, expected);
}
fn encode<T: Clone + Encodable + PartialEq>(vals: &[Option<T>]) -> Vec<u8> {
let mut buf = Vec::with_capacity(vals.len() * 3);
let mut encoder: RleEncoder<'_, T> = RleEncoder::new(&mut buf);
for val in vals {
encoder.append(val.as_ref())
}
encoder.finish();
buf
}
fn decode<T: Clone + Decodable + Debug>(buf: Vec<u8>) -> Vec<Option<T>> {
let decoder = RleDecoder::<'_, T>::from(&buf[..]);
decoder.collect()
}
proptest!{
#[test]
fn splice_ints(scenario in splice_scenario(any::<Option<i32>>())) {
let buf = encode(&scenario.initial_values);
let mut decoder = RleDecoder::<'_, i32>::from(&buf[..]);
let mut out = Vec::new();
decoder.splice(scenario.replace_range.clone(), scenario.replacements.iter().cloned(), &mut out);
let result = decode::<i32>(out);
scenario.check(result)
}
#[test]
fn splice_strings(scenario in splice_scenario(any::<Option<String>>())) {
let buf = encode(&scenario.initial_values);
let mut decoder = RleDecoder::<'_, String>::from(&buf[..]);
let mut out = Vec::new();
decoder.splice(scenario.replace_range.clone(), scenario.replacements.iter().cloned(), &mut out);
let result = decode::<String>(out);
scenario.check(result)
}
}
}

View file

@ -0,0 +1,475 @@
use crate::columnar_2::rowblock::column_layout::ColumnSpliceError;
use std::{borrow::Cow, convert::TryInto, ops::Range};
use super::{DecodeColumnError, RawDecoder, RawEncoder, RleDecoder, RleEncoder, RawBytes};
use crate::columnar_2::rowblock::value::PrimVal;
#[derive(Clone)]
pub(crate) struct ValueDecoder<'a> {
meta: RleDecoder<'a, u64>,
raw: RawDecoder<'a>,
}
impl<'a> ValueDecoder<'a> {
pub(crate) fn new(meta: RleDecoder<'a, u64>, raw: RawDecoder<'a>) -> ValueDecoder<'a> {
ValueDecoder { meta, raw }
}
pub(crate) fn done(&self) -> bool {
self.meta.done()
}
pub(crate) fn next(&mut self) -> Option<Result<PrimVal<'a>, DecodeColumnError>> {
match self.meta.next() {
Some(Some(next)) => {
let val_meta = ValueMeta::from(next);
#[allow(clippy::redundant_slicing)]
match val_meta.type_code() {
ValueType::Null => Some(Ok(PrimVal::Null)),
ValueType::True => Some(Ok(PrimVal::Bool(true))),
ValueType::False => Some(Ok(PrimVal::Bool(false))),
ValueType::Uleb => self.parse_raw(val_meta, |mut bytes| {
let val = leb128::read::unsigned(&mut bytes).map_err(|e| {
DecodeColumnError::InvalidValue {
column: "value".to_string(),
description: e.to_string(),
}
})?;
Ok(PrimVal::Uint(val))
}),
ValueType::Leb => self.parse_raw(val_meta, |mut bytes| {
let val = leb128::read::signed(&mut bytes).map_err(|e| {
DecodeColumnError::InvalidValue {
column: "value".to_string(),
description: e.to_string(),
}
})?;
Ok(PrimVal::Int(val))
}),
ValueType::String => self.parse_raw(val_meta, |bytes| {
let val = Cow::Owned(
std::str::from_utf8(bytes)
.map_err(|e| DecodeColumnError::InvalidValue {
column: "value".to_string(),
description: e.to_string(),
})?
.into(),
);
Ok(PrimVal::String(val))
}),
ValueType::Float => self.parse_raw(val_meta, |bytes| {
if val_meta.length() != 8 {
return Err(DecodeColumnError::InvalidValue {
column: "value".to_string(),
description: format!(
"float should have length 8, had {0}",
val_meta.length()
),
});
}
let raw: [u8; 8] = bytes
.try_into()
// SAFETY: parse_raw() calls read_bytes(val_meta.length()) and we have
// checked that val_meta.length() == 8
.unwrap();
let val = f64::from_le_bytes(raw);
Ok(PrimVal::Float(val))
}),
ValueType::Counter => self.parse_raw(val_meta, |mut bytes| {
let val = leb128::read::unsigned(&mut bytes).map_err(|e| {
DecodeColumnError::InvalidValue {
column: "value".to_string(),
description: e.to_string(),
}
})?;
Ok(PrimVal::Counter(val))
}),
ValueType::Timestamp => self.parse_raw(val_meta, |mut bytes| {
let val = leb128::read::unsigned(&mut bytes).map_err(|e| {
DecodeColumnError::InvalidValue {
column: "value".to_string(),
description: e.to_string(),
}
})?;
Ok(PrimVal::Timestamp(val))
}),
ValueType::Unknown(code) => self.parse_raw(val_meta, |bytes| {
Ok(PrimVal::Unknown {
type_code: code,
data: bytes.to_vec(),
})
}),
ValueType::Bytes => match self.raw.read_bytes(val_meta.length()) {
Err(e) => Some(Err(DecodeColumnError::InvalidValue {
column: "value".to_string(),
description: e.to_string(),
})),
Ok(bytes) => Some(Ok(PrimVal::Bytes(Cow::Owned(bytes.to_vec())))),
},
}
}
Some(None) => Some(Err(DecodeColumnError::UnexpectedNull("meta".to_string()))),
None => None,
}
}
pub(crate) fn splice<'b, I>(
&'a mut self,
replace: Range<usize>,
replace_with: I,
out: &mut Vec<u8>,
) -> (Range<usize>, Range<usize>)
where
I: Iterator<Item = PrimVal<'a>> + Clone,
{
// SAFETY: try_splice only fails if the iterator fails, and this iterator is infallible
self.try_splice(replace, replace_with.map(|i| Ok(i)), out).unwrap()
}
pub(crate) fn try_splice<'b, I>(
&'a mut self,
replace: Range<usize>,
mut replace_with: I,
out: &mut Vec<u8>,
) -> Result<(Range<usize>, Range<usize>), ColumnSpliceError>
where
I: Iterator<Item = Result<PrimVal<'a>, ColumnSpliceError>> + Clone,
{
// Our semantics here are similar to those of Vec::splice. We can describe this
// imperatively like this:
//
// * First copy everything up to the start of `replace` into the output
// * For every index in `replace` skip that index from ourselves and if `replace_with`
// returns `Some` then copy that value to the output
// * Once we have iterated past `replace.end` we continue to call `replace_with` until it
// returns None, copying the results to the output
// * Finally we copy the remainder of our data into the output
//
// However, things are complicated by the fact that our data is stored in two columns. This
// means that we do this in two passes. First we execute the above logic for the metadata
// column. Then we do it all over again for the value column.
// First pass - metadata
//
// Copy the metadata decoder so we can iterate over it again when we read the values in the
// second pass
let start = out.len();
let mut meta_copy = self.meta.clone();
let mut meta_out = RleEncoder::from(&mut *out);
let mut idx = 0;
// Copy everything up to replace.start to the output
while idx < replace.start {
let val = meta_copy.next().unwrap_or(None);
meta_out.append(val.as_ref());
idx += 1;
}
// Now step through replace, skipping our data and inserting the replacement data (if there
// is any)
let mut meta_replace_with = replace_with.clone();
for _ in 0..replace.len() {
meta_copy.next();
if let Some(val) = meta_replace_with.next() {
let val = val?;
// Note that we are just constructing metadata values here.
let meta_val = &u64::from(ValueMeta::from(&val));
meta_out.append(Some(meta_val));
}
idx += 1;
}
// Copy any remaining input from the replacments to the output
while let Some(val) = meta_replace_with.next() {
let val = val?;
let meta_val = &u64::from(ValueMeta::from(&val));
meta_out.append(Some(meta_val));
idx += 1;
}
// Now copy any remaining data we have to the output
while !meta_copy.done() {
let val = meta_copy.next().unwrap_or(None);
meta_out.append(val.as_ref());
}
let meta_len = meta_out.finish();
let meta_range = start..(start + meta_len);
// Second pass, copying the values. For this pass we iterate over ourselves.
//
//
let mut value_range_len = 0;
let mut raw_encoder = RawEncoder::from(out);
idx = 0;
// Copy everything up to replace.start to the output
while idx < replace.start {
let val = self.next().unwrap().unwrap_or(PrimVal::Null);
value_range_len += encode_primval(&mut raw_encoder, &val);
idx += 1;
}
// Now step through replace, skipping our data and inserting the replacement data (if there
// is any)
for _ in 0..replace.len() {
self.next();
if let Some(val) = replace_with.next() {
let val = val?;
value_range_len += encode_primval(&mut raw_encoder, &val);
}
idx += 1;
}
// Copy any remaining input from the replacments to the output
while let Some(val) = replace_with.next() {
let val = val?;
value_range_len += encode_primval(&mut raw_encoder, &val);
idx += 1;
}
// Now copy any remaining data we have to the output
while !self.done() {
let val = self.next().unwrap().unwrap_or(PrimVal::Null);
value_range_len += encode_primval(&mut raw_encoder, &val);
}
let value_range = meta_range.end..(meta_range.end + value_range_len);
Ok((meta_range, value_range))
}
fn parse_raw<R, F: Fn(&[u8]) -> Result<R, DecodeColumnError>>(
&mut self,
meta: ValueMeta,
f: F,
) -> Option<Result<R, DecodeColumnError>> {
let raw = match self.raw.read_bytes(meta.length()) {
Err(e) => {
return Some(Err(DecodeColumnError::InvalidValue {
column: "value".to_string(),
description: e.to_string(),
}))
}
Ok(bytes) => bytes,
};
let val = match f(&mut &raw[..]) {
Ok(v) => v,
Err(e) => return Some(Err(e)),
};
Some(Ok(val))
}
}
fn encode_primval(out: &mut RawEncoder, val: &PrimVal) -> usize {
match val {
PrimVal::Uint(i) => out.append(i),
PrimVal::Int(i) => out.append(i),
PrimVal::Null => 0,
PrimVal::Bool(_) => 0,
PrimVal::Timestamp(i) => out.append(i),
PrimVal::Float(f) => out.append(f),
PrimVal::Counter(i) => out.append(i),
PrimVal::String(s) => out.append(&RawBytes::from(s.as_bytes())),
PrimVal::Bytes(b) => out.append(&RawBytes::from(&b[..])),
PrimVal::Unknown { data, .. } => out.append(&RawBytes::from(&data[..])),
}
}
impl<'a> Iterator for ValueDecoder<'a> {
type Item = Result<PrimVal<'a>, DecodeColumnError>;
fn next(&mut self) -> Option<Self::Item> {
ValueDecoder::next(self)
}
}
enum ValueType {
Null,
False,
True,
Uleb,
Leb,
Float,
String,
Bytes,
Counter,
Timestamp,
Unknown(u8),
}
#[derive(Copy, Clone)]
struct ValueMeta(u64);
impl ValueMeta {
fn type_code(&self) -> ValueType {
let low_byte = (self.0 & 0b00001111) as u8;
match low_byte {
0 => ValueType::Null,
1 => ValueType::False,
2 => ValueType::True,
3 => ValueType::Uleb,
4 => ValueType::Leb,
5 => ValueType::Float,
6 => ValueType::String,
7 => ValueType::Bytes,
8 => ValueType::Counter,
9 => ValueType::Timestamp,
other => ValueType::Unknown(other),
}
}
fn length(&self) -> usize {
(self.0 >> 4) as usize
}
}
impl<'a> From<&PrimVal<'a>> for ValueMeta {
fn from(p: &PrimVal<'a>) -> Self {
match p {
PrimVal::Uint(i) => Self((ulebsize(*i) << 4) | 3),
PrimVal::Int(i) => Self((lebsize(*i) << 4) | 4),
PrimVal::Null => Self(0),
PrimVal::Bool(b) => Self(match b {
false => 1,
true => 2,
}),
PrimVal::Timestamp(i) => Self((ulebsize(*i) << 4) | 9),
PrimVal::Float(_) => Self((8 << 4) | 5),
PrimVal::Counter(i) => Self((ulebsize(*i) << 4) | 8),
PrimVal::String(s) => Self(((s.as_bytes().len() as u64) << 4) | 6),
PrimVal::Bytes(b) => Self(((b.len() as u64) << 4) | 7),
PrimVal::Unknown { type_code, data } => {
Self(((data.len() as u64) << 4) | (*type_code as u64))
}
}
}
}
impl From<u64> for ValueMeta {
fn from(raw: u64) -> Self {
ValueMeta(raw)
}
}
impl From<ValueMeta> for u64 {
fn from(v: ValueMeta) -> Self {
v.0
}
}
impl<'a> From<&PrimVal<'a>> for ValueType {
fn from(p: &PrimVal) -> Self {
match p {
PrimVal::Uint(_) => ValueType::Uleb,
PrimVal::Int(_) => ValueType::Leb,
PrimVal::Null => ValueType::Null,
PrimVal::Bool(b) => match b {
true => ValueType::True,
false => ValueType::False,
},
PrimVal::Timestamp(_) => ValueType::Timestamp,
PrimVal::Float(_) => ValueType::Float,
PrimVal::Counter(_) => ValueType::Counter,
PrimVal::String(_) => ValueType::String,
PrimVal::Bytes(_) => ValueType::Bytes,
PrimVal::Unknown { type_code, .. } => ValueType::Unknown(*type_code),
}
}
}
impl From<ValueType> for u64 {
fn from(v: ValueType) -> Self {
match v {
ValueType::Null => 0,
ValueType::False => 1,
ValueType::True => 2,
ValueType::Uleb => 3,
ValueType::Leb => 4,
ValueType::Float => 5,
ValueType::String => 6,
ValueType::Bytes => 7,
ValueType::Counter => 8,
ValueType::Timestamp => 9,
ValueType::Unknown(other) => other as u64,
}
}
}
fn lebsize(val: i64) -> u64 {
if val == 0 {
return 1;
}
let numbits = (val as f64).abs().log2().ceil() as u64;
let mut numblocks = (numbits as f64 / 7.0).ceil() as u64;
// Make room for the sign bit
if numbits % 7 == 0 {
numblocks += 1;
}
return numblocks;
}
fn ulebsize(val: u64) -> u64 {
if val == 0 {
return 1;
}
let numbits = (val as f64).log2().ceil() as u64;
let numblocks = (numbits as f64 / 7.0).ceil() as u64;
return numblocks;
}
#[cfg(test)]
mod tests {
use super::*;
use crate::columnar_2::rowblock::encoding::{
properties::{splice_scenario, value}, RawDecoder, RleDecoder,
};
use proptest::prelude::*;
fn encode_values(vals: &[PrimVal]) -> (Range<usize>, Range<usize>, Vec<u8>) {
let mut decoder = ValueDecoder {
meta: RleDecoder::from(&[] as &[u8]),
raw: RawDecoder::from(&[] as &[u8]),
};
let mut out = Vec::new();
let (meta_range, val_range) = decoder
.try_splice(0..0, vals.iter().map(|v| Ok(v.clone())), &mut out)
.unwrap();
(meta_range, val_range, out)
}
proptest! {
#[test]
fn test_initialize_splice(values in proptest::collection::vec(value(), 0..100)) {
let (meta_range, val_range, out) = encode_values(&values);
let mut decoder = ValueDecoder{
meta: RleDecoder::from(&out[meta_range]),
raw: RawDecoder::from(&out[val_range]),
};
let mut testvals = Vec::new();
while !decoder.done() {
testvals.push(decoder.next().unwrap().unwrap());
}
assert_eq!(values, testvals);
}
#[test]
fn test_splice_values(scenario in splice_scenario(value())){
let (meta_range, val_range, out) = encode_values(&scenario.initial_values);
let mut decoder = ValueDecoder{
meta: RleDecoder::from(&out[meta_range]),
raw: RawDecoder::from(&out[val_range]),
};
let mut spliced = Vec::new();
let (spliced_meta, spliced_val) = decoder
.try_splice(
scenario.replace_range.clone(),
scenario.replacements.clone().into_iter().map(|i| Ok(i)),
&mut spliced,
).unwrap();
let mut spliced_decoder = ValueDecoder{
meta: RleDecoder::from(&spliced[spliced_meta]),
raw: RawDecoder::from(&spliced[spliced_val]),
};
let mut result_values = Vec::new();
while !spliced_decoder.done() {
result_values.push(spliced_decoder.next().unwrap().unwrap());
}
let mut expected: Vec<_> = scenario.initial_values.clone();
expected.splice(scenario.replace_range, scenario.replacements);
assert_eq!(result_values, expected);
}
}
}

View file

@ -0,0 +1,143 @@
use std::{borrow::Cow, convert::TryInto};
use self::column_layout::DocOpColumns;
use super::{ColumnId, ColumnSpec};
mod column_layout;
pub(crate) use column_layout::doc_change_columns;
pub(crate) use column_layout::doc_op_columns;
pub(crate) use column_layout::change_op_columns;
pub(crate) use column_layout::{BadColumnLayout, ColumnLayout};
mod column_range;
mod encoding;
pub(crate) use encoding::Key;
use encoding::{DecodeColumnError, GenericColDecoder};
mod value;
pub(crate) use value::{CellValue, PrimVal};
pub(crate) struct RowBlock<'a, C> {
columns: C,
data: Cow<'a, [u8]>,
}
impl<'a> RowBlock<'a, ColumnLayout> {
pub(crate) fn new<I: Iterator<Item = (ColumnSpec, std::ops::Range<usize>)>>(
cols: I,
data: Cow<'a, [u8]>,
) -> Result<RowBlock<'a, ColumnLayout>, BadColumnLayout> {
let layout = ColumnLayout::parse(data.len(), cols)?;
Ok(RowBlock {
columns: layout,
data,
})
}
pub(crate) fn into_doc_ops(
self,
) -> Result<RowBlock<'a, column_layout::DocOpColumns>, column_layout::ParseDocColumnError> {
let doc_cols: column_layout::DocOpColumns = self.columns.try_into()?;
Ok(RowBlock {
columns: doc_cols,
data: self.data,
})
}
pub(crate) fn into_doc_change(
self,
) -> Result<
RowBlock<'a, column_layout::doc_change_columns::DocChangeColumns>,
column_layout::doc_change_columns::DecodeChangeError,
> {
let doc_cols: column_layout::doc_change_columns::DocChangeColumns =
self.columns.try_into()?;
Ok(RowBlock {
columns: doc_cols,
data: self.data,
})
}
pub(crate) fn into_change_ops(
self
) -> Result<RowBlock<'a, change_op_columns::ChangeOpsColumns>, change_op_columns::ParseChangeColumnsError> {
let change_cols: change_op_columns::ChangeOpsColumns = self.columns.try_into()?;
Ok(RowBlock {
columns: change_cols,
data: self.data,
})
}
}
impl<'a, 'b> IntoIterator for &'a RowBlock<'b, ColumnLayout> {
type Item = Result<Vec<(usize, CellValue<'a>)>, DecodeColumnError>;
type IntoIter = RowBlockIter<'a>;
fn into_iter(self) -> Self::IntoIter {
RowBlockIter {
failed: false,
decoders: self
.columns
.iter()
.map(|c| (c.id(), c.decoder(&self.data)))
.collect(),
}
}
}
pub(crate) struct RowBlockIter<'a> {
failed: bool,
decoders: Vec<(ColumnId, GenericColDecoder<'a>)>,
}
impl<'a> Iterator for RowBlockIter<'a> {
type Item = Result<Vec<(usize, CellValue<'a>)>, DecodeColumnError>;
fn next(&mut self) -> Option<Self::Item> {
if self.failed {
return None;
}
if self.decoders.iter().all(|(_, d)| d.done()) {
None
} else {
let mut result = Vec::with_capacity(self.decoders.len());
for (col_index, (_, decoder)) in self.decoders.iter_mut().enumerate() {
match decoder.next() {
Some(Ok(c)) => result.push((col_index, c)),
Some(Err(e)) => {
self.failed = true;
return Some(Err(e));
},
None => {},
}
}
Some(Ok(result))
}
}
}
impl<'a> IntoIterator for &'a RowBlock<'a, DocOpColumns> {
type Item = Result<doc_op_columns::DocOp<'a>, column_layout::doc_op_columns::DecodeOpError>;
type IntoIter = column_layout::doc_op_columns::DocOpColumnIter<'a>;
fn into_iter(self) -> Self::IntoIter {
self.columns.iter(&self.data)
}
}
impl<'a> IntoIterator for &'a RowBlock<'a, doc_change_columns::DocChangeColumns> {
type Item = Result<doc_change_columns::ChangeMetadata<'a>, doc_change_columns::DecodeChangeError>;
type IntoIter = doc_change_columns::DocChangeColumnIter<'a>;
fn into_iter(self) -> Self::IntoIter {
self.columns.iter(&self.data)
}
}
impl<'a> IntoIterator for &'a RowBlock<'a, change_op_columns::ChangeOpsColumns> {
type Item = Result<change_op_columns::ChangeOp<'a>, change_op_columns::ReadChangeOpError>;
type IntoIter = change_op_columns::ChangeOpsIter<'a>;
fn into_iter(self) -> Self::IntoIter {
self.columns.iter(&self.data)
}
}

View file

@ -0,0 +1,105 @@
use crate::ScalarValue;
use std::borrow::Cow;
use smol_str::SmolStr;
#[derive(Debug)]
pub(crate) enum CellValue<'a> {
Uint(u64),
Int(i64),
Bool(bool),
String(Cow<'a, SmolStr>),
Value(PrimVal<'a>),
List(Vec<Vec<CellValue<'a>>>),
}
#[derive(Clone, Debug, PartialEq)]
pub(crate) enum PrimVal<'a> {
Null,
Bool(bool),
Uint(u64),
Int(i64),
Float(f64),
String(Cow<'a, SmolStr>),
Bytes(Cow<'a, [u8]>),
Counter(u64),
Timestamp(u64),
Unknown { type_code: u8, data: Vec<u8> },
}
impl<'a> PrimVal<'a> {
pub(crate) fn into_owned(self) -> PrimVal<'static> {
match self {
PrimVal::String(s) => PrimVal::String(Cow::Owned(s.into_owned().into())),
PrimVal::Bytes(b) => PrimVal::Bytes(Cow::Owned(b.to_vec())),
PrimVal::Null => PrimVal::Null,
PrimVal::Bool(b) => PrimVal::Bool(b),
PrimVal::Uint(u) => PrimVal::Uint(u),
PrimVal::Int(i) => PrimVal::Int(i),
PrimVal::Float(f) => PrimVal::Float(f),
PrimVal::Counter(u) => PrimVal::Counter(u),
PrimVal::Timestamp(u) => PrimVal::Timestamp(u),
PrimVal::Unknown { type_code, data } => PrimVal::Unknown{ type_code, data},
}
}
}
impl<'a> From<PrimVal<'a>> for ScalarValue {
fn from(p: PrimVal) -> Self {
match p {
PrimVal::Null => Self::Null,
PrimVal::Bool(b) => Self::Boolean(b),
PrimVal::Uint(u) => Self::Uint(u),
PrimVal::Int(i) => Self::Int(i),
PrimVal::Float(f) => Self::F64(f),
PrimVal::String(s) => Self::Str(s.into_owned()),
PrimVal::Bytes(b) => Self::Bytes(b.to_vec()),
PrimVal::Counter(c) => Self::Counter((c as i64).into()),
PrimVal::Timestamp(t) => Self::Timestamp(t as i64),
PrimVal::Unknown { data, .. } => Self::Bytes(data),
}
}
}
impl<'a> From<ScalarValue> for PrimVal<'static> {
fn from(s: ScalarValue) -> Self {
match s {
ScalarValue::Null => PrimVal::Null,
ScalarValue::Boolean(b) => PrimVal::Bool(b),
ScalarValue::Uint(u) => PrimVal::Uint(u),
ScalarValue::Int(i) => PrimVal::Int(i),
ScalarValue::F64(f) => PrimVal::Float(f),
ScalarValue::Str(s) => PrimVal::String(Cow::Owned(s)),
// This is bad, if there was an unknown type code in the primval we have lost it on the
// round trip
ScalarValue::Bytes(b) => PrimVal::Bytes(Cow::Owned(b)),
ScalarValue::Counter(c) => PrimVal::Counter(c.current as u64),
ScalarValue::Timestamp(t) => PrimVal::Timestamp(t as u64),
}
}
}
impl<'a> From<&ScalarValue> for PrimVal<'static> {
fn from(s: &ScalarValue) -> Self {
match s {
ScalarValue::Null => PrimVal::Null,
ScalarValue::Boolean(b) => PrimVal::Bool(*b),
ScalarValue::Uint(u) => PrimVal::Uint(*u),
ScalarValue::Int(i) => PrimVal::Int(*i),
ScalarValue::F64(f) => PrimVal::Float(*f),
ScalarValue::Str(s) => PrimVal::String(Cow::Owned(s.clone())),
// This is bad, if there was an unknown type code in the primval we have lost it on the
// round trip
ScalarValue::Bytes(b) => PrimVal::Bytes(Cow::Owned(b.clone())),
ScalarValue::Counter(c) => PrimVal::Counter(c.current as u64),
ScalarValue::Timestamp(t) => PrimVal::Timestamp((*t) as u64),
}
}
}
impl<'a> From<&'a [u8]> for PrimVal<'a> {
fn from(d: &'a [u8]) -> Self {
PrimVal::Bytes(Cow::Borrowed(d))
}
}

View file

@ -0,0 +1,256 @@
use std::{
borrow::Cow,
collections::{BTreeMap, BTreeSet},
iter::Iterator,
};
use itertools::Itertools;
use crate::{
columnar_2::{
rowblock::{
change_op_columns::{ChangeOp, ChangeOpsColumns},
doc_change_columns::{ChangeMetadata, DocChangeColumns},
doc_op_columns::{DocOp, DocOpColumns},
Key as EncodedKey, PrimVal,
},
storage::{Chunk, Document},
},
indexed_cache::IndexedCache,
types::{ActorId, ElemId, Key, ObjId, Op, OpId, OpType},
Change, ChangeHash,
};
/// # Panics
///
/// * If any of the `heads` are not in `changes`
/// * If any of ops in `ops` reference an actor which is not in `actors`
/// * If any of ops in `ops` reference a property which is not in `props`
/// * If any of the changes reference a dependency index which is not in `changes`
pub(crate) fn save_document<'a, I, O>(
changes: I,
ops: O,
actors: &'a IndexedCache<ActorId>,
props: &IndexedCache<String>,
heads: &[ChangeHash],
) -> Vec<u8>
where
I: Iterator<Item = &'a Change> + Clone + 'a,
O: Iterator<Item = (&'a ObjId, &'a Op)> + Clone,
{
let actor_lookup = actors.encode_index();
let doc_ops = ops.map(|(obj, op)| DocOp {
id: translate_opid(&op.id, &actor_lookup),
insert: op.insert,
object: translate_objid(obj, &actor_lookup),
key: translate_key(&op.key, props),
action: op.action.action_index() as usize,
value: match &op.action {
OpType::Set(v) => v.into(),
OpType::Inc(i) => PrimVal::Int(*i),
_ => PrimVal::Null,
},
succ: op
.succ
.iter()
.map(|o| translate_opid(o, &actor_lookup))
.collect(),
});
let mut ops_out = Vec::new();
let ops_meta = DocOpColumns::encode(doc_ops, &mut ops_out);
let mut change_out = Vec::new();
let hash_graph = HashGraph::new(changes.clone(), heads);
let cols = DocChangeColumns::encode(
changes.map(|c| hash_graph.construct_change(c, &actor_lookup, actors)),
&mut change_out,
);
let doc = Document {
actors: actors.sorted().cache,
heads: heads.to_vec(),
op_metadata: ops_meta.metadata(),
op_bytes: Cow::Owned(ops_out),
change_metadata: cols.metadata(),
change_bytes: Cow::Owned(change_out),
head_indices: hash_graph.head_indices,
};
let written = doc.write();
let chunk = Chunk::new_document(&written);
chunk.write()
}
pub(crate) fn encode_change_ops<'a, O>(
ops: O,
change_actor: ActorId,
actors: &IndexedCache<ActorId>,
props: &IndexedCache<String>,
) -> (ChangeOpsColumns, Vec<u8>, Vec<ActorId>)
where
O: Iterator<Item = (&'a ObjId, &'a Op)> + Clone,
{
let encoded_actors = actor_ids_in_change(ops.clone(), change_actor.clone(), actors);
let actor_lookup = actors
.cache
.iter()
.map(|a| encoded_actors.iter().position(|r| r == a).unwrap())
.collect::<Vec<_>>();
let change_ops = ops.map(|(obj, op)| ChangeOp {
insert: op.insert,
obj: translate_objid(obj, &actor_lookup),
key: translate_key(&op.key, props),
action: op.action.action_index(),
val: match &op.action {
OpType::Set(v) => v.into(),
OpType::Inc(i) => PrimVal::Int(*i),
_ => PrimVal::Null,
},
pred: op
.pred
.iter()
.map(|o| translate_opid(o, &actor_lookup))
.collect(),
});
let mut out = Vec::new();
let cols = ChangeOpsColumns::empty().encode(change_ops, &mut out);
let other_actors = encoded_actors.into_iter().skip(1).collect();
(cols, out, other_actors)
}
/// When encoding a change chunk we take all the actor IDs referenced by a change and place them in
/// an array. The array has the actor who authored the change as the first element and all
/// remaining actors (i.e. those referenced in object IDs in the target of an operation or in the
/// `pred` of an operation) lexicographically ordered following the change author.
fn actor_ids_in_change<'a, I>(
ops: I,
change_actor: ActorId,
actors: &IndexedCache<ActorId>,
) -> Vec<ActorId>
where
I: Iterator<Item = (&'a ObjId, &'a Op)> + Clone,
{
let mut other_ids: Vec<ActorId> = ops
.flat_map(|(obj, o)| opids_in_operation(&obj, &o, actors))
.filter(|a| *a != &change_actor)
.unique()
.cloned()
.collect();
other_ids.sort();
// Now prepend the change actor
std::iter::once(change_actor)
.chain(other_ids.into_iter())
.collect()
}
fn opids_in_operation<'a>(
obj: &'a ObjId,
op: &'a Op,
actors: &'a IndexedCache<ActorId>,
) -> impl Iterator<Item = &'a ActorId> {
let obj_actor_id = if obj.is_root() {
None
} else {
Some(actors.get(obj.opid().actor()))
};
let pred_ids = op.pred.iter().filter_map(|a| {
if a.counter() != 0 {
Some(actors.get(a.actor()))
} else {
None
}
});
let key_actor = match &op.key {
Key::Seq(ElemId(op)) if !op.counter() == 0 => Some(actors.get(op.actor())),
_ => None,
};
obj_actor_id
.into_iter()
.chain(key_actor.into_iter())
.chain(pred_ids)
}
fn translate_key(k: &Key, props: &IndexedCache<String>) -> EncodedKey {
match k {
Key::Seq(e) => EncodedKey::Elem(*e),
Key::Map(idx) => EncodedKey::Prop(props.get(*idx).into()),
}
}
fn translate_objid(obj: &ObjId, actors: &[usize]) -> ObjId {
if obj.is_root() {
*obj
} else {
ObjId(translate_opid(&obj.opid(), actors))
}
}
fn translate_opid(id: &OpId, actors: &[usize]) -> OpId {
OpId::new(actors[id.actor()], id.counter())
}
fn find_head_indices<'a, I>(changes: I, heads: &[ChangeHash]) -> Vec<u64>
where
I: Iterator<Item = &'a Change>,
{
let heads_set: BTreeSet<ChangeHash> = heads.iter().copied().collect();
let mut head_indices = BTreeMap::new();
for (index, change) in changes.enumerate() {
if heads_set.contains(&change.hash()) {
head_indices.insert(change.hash(), index as u64);
}
}
heads.iter().map(|h| head_indices[h]).collect()
}
struct HashGraph {
head_indices: Vec<u64>,
index_by_hash: BTreeMap<ChangeHash, usize>,
}
impl HashGraph {
fn new<'a, I>(changes: I, heads: &[ChangeHash]) -> Self
where
I: Iterator<Item = &'a Change>,
{
let heads_set: BTreeSet<ChangeHash> = heads.iter().copied().collect();
let mut head_indices = BTreeMap::new();
let mut index_by_hash = BTreeMap::new();
for (index, change) in changes.enumerate() {
if heads_set.contains(&change.hash()) {
head_indices.insert(change.hash(), index as u64);
}
index_by_hash.insert(change.hash(), index);
}
let head_indices = heads.iter().map(|h| head_indices[h]).collect();
Self {
head_indices,
index_by_hash,
}
}
fn change_index(&self, hash: &ChangeHash) -> usize {
self.index_by_hash[hash]
}
fn construct_change(
&self,
c: &Change,
actor_lookup: &[usize],
actors: &IndexedCache<ActorId>,
) -> ChangeMetadata<'static> {
ChangeMetadata {
actor: actor_lookup[actors.lookup(c.actor_id()).unwrap()],
seq: c.seq(),
max_op: c.max_op(),
timestamp: c.timestamp(),
message: c.message().map(|s| s.into()),
deps: c
.deps()
.iter()
.map(|d| self.change_index(d) as u64)
.collect(),
extra: Cow::Owned(c.extra_bytes().to_vec()),
}
}
}

View file

@ -0,0 +1,114 @@
use std::{borrow::Cow, io::Write};
use crate::{ActorId, ChangeHash};
use super::{parse, ColumnMetadata, Chunk};
#[derive(Clone, Debug)]
pub(crate) struct Change<'a> {
pub(crate) dependencies: Vec<ChangeHash>,
pub(crate) actor: ActorId,
pub(crate) other_actors: Vec<ActorId>,
pub(crate) seq: u64,
pub(crate) start_op: u64,
pub(crate) timestamp: i64,
pub(crate) message: Option<String>,
pub(crate) ops_meta: ColumnMetadata,
pub(crate) ops_data: Cow<'a, [u8]>,
pub(crate) extra_bytes: Cow<'a, [u8]>,
}
impl<'a> Change<'a> {
pub(crate) fn parse(input: &'a [u8]) -> parse::ParseResult<Change<'a>> {
let (i, deps) = parse::length_prefixed(parse::leb128_u64, parse::change_hash)(input)?;
let (i, actor) = parse::actor_id(i)?;
let (i, seq) = parse::leb128_u64(i)?;
let (i, start_op) = parse::leb128_u64(i)?;
let (i, timestamp) = parse::leb128_i64(i)?;
let (i, message_len) = parse::leb128_u64(i)?;
let (i, message) = parse::utf_8(message_len as usize, i)?;
let (i, other_actors) = parse::length_prefixed(parse::leb128_u64, parse::actor_id)(i)?;
let (i, ops_meta) = ColumnMetadata::parse(i)?;
let (i, ops_data) = parse::take_n(ops_meta.total_column_len(), i)?;
Ok((
&[],
Change {
dependencies: deps,
actor,
other_actors,
seq,
start_op,
timestamp,
message: if message.is_empty() {
None
} else {
Some(message)
},
ops_meta,
ops_data: Cow::Borrowed(ops_data),
extra_bytes: Cow::Borrowed(i),
},
))
}
fn byte_len(&self) -> usize {
(self.dependencies.len() * 32)
+ 8
+ self.actor.to_bytes().len()
+ 24 // seq, start op, timestamp
+ 8
+ self.message.as_ref().map(|m| m.as_bytes().len()).unwrap_or(0_usize)
+ self.other_actors.iter().map(|a| a.to_bytes().len() + 8_usize).sum::<usize>()
+ self.ops_meta.byte_len()
+ self.ops_data.len()
+ self.extra_bytes.len()
}
pub(crate) fn write(&self) -> Vec<u8> {
let mut out = Vec::with_capacity(self.byte_len());
leb128::write::unsigned(&mut out, self.dependencies.len() as u64).unwrap();
for dep in &self.dependencies {
out.write_all(dep.as_bytes()).unwrap();
}
length_prefixed_bytes(&self.actor, &mut out);
leb128::write::unsigned(&mut out, self.seq).unwrap();
leb128::write::unsigned(&mut out, self.start_op).unwrap();
leb128::write::signed(&mut out, self.timestamp).unwrap();
length_prefixed_bytes(self.message.as_ref().map(|m| m.as_bytes()).unwrap_or(&[]), &mut out);
leb128::write::unsigned(&mut out, self.other_actors.len() as u64).unwrap();
for actor in self.other_actors.iter() {
length_prefixed_bytes(&actor, &mut out);
}
self.ops_meta.write(&mut out);
out.write_all(self.ops_data.as_ref()).unwrap();
out.write_all(self.extra_bytes.as_ref()).unwrap();
out
}
pub(crate) fn hash(&self) -> ChangeHash {
let this = self.write();
let chunk = Chunk::new_change(&this);
chunk.hash()
}
pub(crate) fn into_owned(self) -> Change<'static> {
Change{
dependencies: self.dependencies,
actor: self.actor,
other_actors: self.other_actors,
seq: self.seq,
start_op: self.start_op,
timestamp: self.timestamp,
message: self.message,
ops_meta: self.ops_meta,
ops_data: Cow::Owned(self.ops_data.into_owned()),
extra_bytes: Cow::Owned(self.extra_bytes.into_owned()),
}
}
}
fn length_prefixed_bytes<B: AsRef<[u8]>>(b: B, out: &mut Vec<u8>) -> usize {
let prefix_len = leb128::write::unsigned(out, b.as_ref().len() as u64).unwrap();
out.write_all(b.as_ref()).unwrap();
prefix_len + b.as_ref().len()
}

View file

@ -0,0 +1,163 @@
use std::{borrow::Cow, convert::{TryFrom, TryInto}};
use sha2::{Digest, Sha256};
use crate::ChangeHash;
use super::parse;
const MAGIC_BYTES: [u8; 4] = [0x85, 0x6f, 0x4a, 0x83];
#[derive(Clone, Copy, Debug)]
pub(crate) enum ChunkType {
Document,
Change,
Compressed,
}
impl TryFrom<u8> for ChunkType {
type Error = u8;
fn try_from(value: u8) -> Result<Self, Self::Error> {
match value {
0 => Ok(Self::Document),
1 => Ok(Self::Change),
2 => Ok(Self::Compressed),
other => Err(other),
}
}
}
impl From<ChunkType> for u8 {
fn from(ct: ChunkType) -> Self {
match ct {
ChunkType::Document => 0,
ChunkType::Change => 1,
ChunkType::Compressed => 2,
}
}
}
#[derive(Clone, Copy, Debug, PartialEq)]
pub(crate) struct CheckSum([u8; 4]);
impl CheckSum {
fn bytes(&self) -> [u8; 4] {
self.0
}
}
impl From<[u8; 4]> for CheckSum {
fn from(raw: [u8; 4]) -> Self {
CheckSum(raw)
}
}
impl From<ChangeHash> for CheckSum {
fn from(h: ChangeHash) -> Self {
let bytes = h.as_bytes();
[bytes[0], bytes[1], bytes[2], bytes[3]].into()
}
}
#[derive(Debug)]
pub(crate) struct Chunk<'a> {
typ: ChunkType,
checksum: CheckSum,
data: Cow<'a, [u8]>,
}
impl<'a> Chunk<'a> {
pub(crate) fn new_change(data: &'a [u8]) -> Chunk<'a> {
let hash_result = hash(ChunkType::Change, data);
Chunk{
typ: ChunkType::Change,
checksum: hash_result.into(),
data: Cow::Borrowed(data),
}
}
pub(crate) fn new_document(data: &'a [u8]) -> Chunk<'a> {
let hash_result = hash(ChunkType::Document, data);
Chunk{
typ: ChunkType::Document,
checksum: hash_result.into(),
data: Cow::Borrowed(data),
}
}
pub(crate) fn parse(input: &'a [u8]) -> parse::ParseResult<Chunk<'a>> {
let (i, magic) = parse::take4(input)?;
if magic != MAGIC_BYTES {
return Err(parse::ParseError::Error(
parse::ErrorKind::InvalidMagicBytes,
));
}
let (i, checksum_bytes) = parse::take4(i)?;
let (i, raw_chunk_type) = parse::take1(i)?;
let chunk_type: ChunkType = raw_chunk_type
.try_into()
.map_err(|e| parse::ParseError::Error(parse::ErrorKind::UnknownChunkType(e)))?;
let (i, chunk_len) = parse::leb128_u64(i)?;
let (i, data) = parse::take_n(chunk_len as usize, i)?;
Ok((
i,
Chunk {
typ: chunk_type,
checksum: checksum_bytes.into(),
data: Cow::Borrowed(data),
},
))
}
fn byte_len(&self) -> usize {
MAGIC_BYTES.len()
+ 1 // chunk type
+ 4 // checksum
+ 5 //length
+ self.data.len()
}
pub(crate) fn write(&self) -> Vec<u8> {
let mut out = Vec::with_capacity(self.byte_len());
out.extend(MAGIC_BYTES);
out.extend(self.checksum.bytes());
out.push(u8::from(self.typ));
leb128::write::unsigned(&mut out, self.data.len() as u64).unwrap();
out.extend(self.data.as_ref());
out
}
pub(crate) fn checksum_valid(&self) -> bool {
let hash = self.hash();
let checksum = CheckSum(hash.checksum());
checksum == self.checksum
}
pub(crate) fn hash(&self) -> ChangeHash {
hash(self.typ, self.data.as_ref())
}
pub(crate) fn typ(&self) -> ChunkType {
self.typ
}
pub(crate) fn checksum(&self) -> CheckSum {
self.checksum
}
pub(crate) fn data(&self) -> Cow<'a, [u8]> {
self.data.clone()
}
}
fn hash(typ: ChunkType, data: &[u8]) -> ChangeHash {
let mut out = Vec::new();
out.push(u8::from(typ));
leb128::write::unsigned(&mut out, data.len() as u64).unwrap();
out.extend(data.as_ref());
let hash_result = Sha256::digest(out);
let array: [u8; 32] = hash_result.into();
ChangeHash(array)
}

View file

@ -0,0 +1,85 @@
use std::ops::Range;
use super::{super::ColumnSpec, parse};
#[derive(Clone, Debug)]
pub(crate) struct Column {
spec: ColumnSpec,
data: Range<usize>,
}
#[derive(Clone, Debug)]
pub(crate) struct ColumnMetadata(Vec<Column>);
impl FromIterator<Column> for ColumnMetadata {
fn from_iter<T: IntoIterator<Item = Column>>(iter: T) -> Self {
Self(iter.into_iter().collect())
}
}
impl FromIterator<(ColumnSpec, Range<usize>)> for ColumnMetadata {
fn from_iter<T: IntoIterator<Item = (ColumnSpec, Range<usize>)>>(iter: T) -> Self {
Self(iter.into_iter().map(|(spec, data)| Column{spec, data}).collect())
}
}
impl ColumnMetadata {
pub(crate) fn parse(input: &[u8]) -> parse::ParseResult<ColumnMetadata> {
let i = input;
let (i, num_columns) = parse::leb128_u64(i)?;
let (i, specs_and_lens) = parse::apply_n(
num_columns as usize,
parse::tuple2(
parse::map(parse::leb128_u32, ColumnSpec::from),
parse::leb128_u64,
),
)(i)?;
let columns = specs_and_lens
.into_iter()
.scan(0_usize, |offset, (spec, len)| {
let end = *offset + len as usize;
let data = *offset..end;
*offset = end;
Some(Column { spec, data })
})
.collect::<Vec<_>>();
if !are_normal_sorted(&columns) {
return Err(parse::ParseError::Error(
parse::ErrorKind::InvalidColumnMetadataSort,
));
}
Ok((i, ColumnMetadata(columns)))
}
pub(crate) fn write(&self, out: &mut Vec<u8>) -> usize {
let mut written = leb128::write::unsigned(out, self.0.len() as u64).unwrap();
for col in &self.0 {
written += leb128::write::unsigned(out, u32::from(col.spec) as u64).unwrap();
written += leb128::write::unsigned(out, col.data.len() as u64).unwrap();
}
written
}
pub(crate) fn total_column_len(&self) -> usize {
self.0.iter().map(|c| c.data.len()).sum()
}
pub(crate) fn iter(&self) -> impl Iterator<Item = (ColumnSpec, Range<usize>)> + '_ {
self.0.iter().map(|c| (c.spec, c.data.clone()))
}
pub(crate) fn byte_len(&self) -> usize {
self.0.len() * 16
}
}
fn are_normal_sorted(cols: &[Column]) -> bool {
if cols.len() > 1 {
for (i, col) in cols[1..].iter().enumerate() {
if col.spec.normalize() < cols[i].spec.normalize() {
return false;
}
}
}
true
}

View file

@ -0,0 +1,71 @@
use std::{borrow::Cow, io::Write};
use super::{column_metadata::ColumnMetadata, parse};
use crate::{ActorId, ChangeHash};
#[derive(Debug)]
pub(crate) struct Document<'a> {
pub(crate) actors: Vec<ActorId>,
pub(crate) heads: Vec<ChangeHash>,
pub(crate) op_metadata: ColumnMetadata,
pub(crate) op_bytes: Cow<'a, [u8]>,
pub(crate) change_metadata: ColumnMetadata,
pub(crate) change_bytes: Cow<'a, [u8]>,
pub(crate) head_indices: Vec<u64>,
}
impl<'a> Document<'a> {
pub(crate) fn parse(input: &'a [u8]) -> parse::ParseResult<Document<'a>> {
let (i, actors) = parse::length_prefixed(parse::leb128_u64, parse::actor_id)(input)?;
let (i, heads) = parse::length_prefixed(parse::leb128_u64, parse::change_hash)(i)?;
let (i, change_meta) = ColumnMetadata::parse(i)?;
let (i, ops_meta) = ColumnMetadata::parse(i)?;
let (i, change_data) = parse::take_n(change_meta.total_column_len(), i)?;
let (i, ops_data) = parse::take_n(ops_meta.total_column_len(), i)?;
let (i, head_indices) = parse::apply_n(heads.len(), parse::leb128_u64)(i)?;
Ok((
i,
Document {
actors,
heads,
op_metadata: ops_meta,
op_bytes: Cow::Borrowed(ops_data),
change_metadata: change_meta,
change_bytes: Cow::Borrowed(change_data),
head_indices,
},
))
}
fn byte_len(&self) -> usize {
self.actors.iter().map(|a| a.to_bytes().len()).sum::<usize>()
+ (32 * self.heads.len())
+ self.op_metadata.byte_len()
+ self.op_bytes.len()
+ self.change_metadata.byte_len()
+ self.change_bytes.len()
+ (64 * self.head_indices.len())
}
pub(crate) fn write(&self) -> Vec<u8> {
let mut out = Vec::with_capacity(self.byte_len());
leb128::write::unsigned(&mut out, self.actors.len() as u64).unwrap();
for actor in &self.actors {
leb128::write::unsigned(&mut out, actor.to_bytes().len() as u64).unwrap();
out.write_all(actor.to_bytes()).unwrap()
}
leb128::write::unsigned(&mut out, self.heads.len() as u64).unwrap();
for head in &self.heads {
out.write_all(head.as_bytes()).unwrap();
}
self.change_metadata.write(&mut out);
self.op_metadata.write(&mut out);
out.write_all(self.change_bytes.as_ref()).unwrap();
out.write_all(self.op_bytes.as_ref()).unwrap();
for index in &self.head_indices {
leb128::write::unsigned(&mut out, *index).unwrap();
}
out
}
}

View file

@ -0,0 +1,13 @@
mod change;
mod chunk;
mod column_metadata;
mod document;
mod parse;
pub(crate) use {
parse::ParseError,
change::Change,
chunk::{Chunk, ChunkType},
column_metadata::{Column, ColumnMetadata},
document::Document,
};

View file

@ -0,0 +1,82 @@
use core::{mem::size_of, num::NonZeroUsize};
use super::{ErrorKind, Needed, ParseError, ParseResult};
macro_rules! impl_leb {
($parser_name: ident, $ty: ty) => {
pub(in crate::columnar_2) fn $parser_name<'a>(input: &'a [u8]) -> ParseResult<'a, $ty> {
let mut res = 0;
let mut shift = 0;
for (pos, byte) in input.iter().enumerate() {
if (byte & 0x80) == 0 {
res |= (*byte as $ty) << shift;
return Ok((&input[pos + 1..], res));
} else if pos == leb128_size::<$ty>() - 1 {
return Err(ParseError::Error(ErrorKind::Leb128TooLarge));
} else {
res |= ((byte & 0x7F) as $ty) << shift;
}
shift += 7;
}
Err(ParseError::Incomplete(NEED_ONE))
}
};
}
impl_leb!(leb128_u64, u64);
impl_leb!(leb128_u32, u32);
impl_leb!(leb128_i64, i64);
impl_leb!(leb128_i32, i32);
/// Maximum LEB128-encoded size of an integer type
const fn leb128_size<T>() -> usize {
let bits = size_of::<T>() * 8;
(bits + 6) / 7 // equivalent to ceil(bits/7) w/o floats
}
const NEED_ONE: Needed = Needed::Size(unsafe { NonZeroUsize::new_unchecked(1) });
#[cfg(test)]
mod tests {
use super::*;
use std::convert::TryFrom;
#[test]
fn leb_128_unsigned() {
let scenarios: Vec<(&'static [u8], ParseResult<u64>)> = vec![
(&[0b00000001_u8], Ok((&[], 1))),
(&[0b10000001_u8], Err(ParseError::Incomplete(NEED_ONE))),
(&[0b10000001, 0b00000001], Ok((&[], 129))),
(&[0b00000001, 0b00000011], Ok((&[0b00000011], 1))),
(
&[129, 129, 129, 129, 129, 129, 129, 129, 129, 129, 129, 129],
Err(ParseError::Error(ErrorKind::Leb128TooLarge)),
),
];
for (index, (input, expected)) in scenarios.clone().into_iter().enumerate() {
let result = leb128_u64(input);
if result != expected {
panic!(
"Scenario {} failed for u64: expected {:?} got {:?}",
index + 1,
expected,
result
);
}
}
for (index, (input, expected)) in scenarios.into_iter().enumerate() {
let u32_expected = expected.map(|(i, e)| (i, u32::try_from(e).unwrap()));
let result = leb128_u32(input);
if result != u32_expected {
panic!(
"Scenario {} failed for u32: expected {:?} got {:?}",
index + 1,
u32_expected,
result
);
}
}
}
}

View file

@ -0,0 +1,184 @@
use core::num::NonZeroUsize;
use std::convert::TryInto;
mod leb128;
use crate::{ActorId, ChangeHash};
pub(in crate::columnar_2) use self::leb128::{leb128_u32, leb128_u64, leb128_i32, leb128_i64};
pub(crate) type ParseResult<'a, O> = Result<(&'a [u8], O), ParseError<ErrorKind>>;
pub(super) trait Parser<'a, O> {
fn parse(&mut self, input: &'a [u8]) -> ParseResult<'a, O>;
}
impl<'a, O, F> Parser<'a, O> for F
where
F: FnMut(&'a [u8]) -> ParseResult<'a, O>,
{
fn parse(&mut self, input: &'a [u8]) -> ParseResult<'a, O> {
(self)(input)
}
}
#[derive(Clone, Debug, PartialEq)]
pub(crate) enum ParseError<E> {
Error(E),
Incomplete(Needed),
}
impl<E: std::fmt::Display> std::fmt::Display for ParseError<E> {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
match self {
Self::Error(e) => write!(f, "{}", e),
Self::Incomplete(_) => write!(f, "not enough data"),
}
}
}
impl<E: std::fmt::Display + std::fmt::Debug> std::error::Error for ParseError<E> {}
#[derive(Clone, Debug, PartialEq)]
pub(crate) enum Needed {
Unknown,
Size(NonZeroUsize),
}
#[derive(Clone, Debug, PartialEq)]
pub(crate) enum ErrorKind {
Leb128TooLarge,
InvalidMagicBytes,
UnknownChunkType(u8),
InvalidColumnMetadataSort,
InvalidUtf8,
}
impl std::fmt::Display for ErrorKind {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
match self {
Self::Leb128TooLarge => write!(f, "invalid leb 128"),
Self::InvalidMagicBytes => write!(f, "invalid magic bytes"),
Self::UnknownChunkType(t) => write!(f, "unknown chunk type: {}", t),
Self::InvalidColumnMetadataSort => write!(f, "invalid sort of column metadata"),
Self::InvalidUtf8 => write!(f, "invalid utf8"),
}
}
}
pub(super) fn map<'a, O1, O2, F, G>(
mut parser: F,
mut f: G,
) -> impl FnMut(&'a [u8]) -> ParseResult<'a, O2>
where
F: Parser<'a, O1>,
G: FnMut(O1) -> O2,
{
move |input: &[u8]| {
let (input, o1) = parser.parse(input)?;
Ok((input, f(o1)))
}
}
pub(super) fn take1(input: &[u8]) -> ParseResult<u8> {
if let Some(need) = NonZeroUsize::new(1_usize.saturating_sub(input.len())) {
Err(ParseError::Incomplete(Needed::Size(need)))
} else {
let (result, remaining) = input.split_at(1);
Ok((remaining, result[0]))
}
}
pub(super) fn take4(input: &[u8]) -> ParseResult<[u8; 4]> {
if let Some(need) = NonZeroUsize::new(4_usize.saturating_sub(input.len())) {
Err(ParseError::Incomplete(Needed::Size(need)))
} else {
let (result, remaining) = input.split_at(4);
Ok((remaining, result.try_into().expect("we checked the length")))
}
}
pub(super) fn take_n<'a>(n: usize, input: &'a [u8]) -> ParseResult<&'a [u8]> {
if let Some(need) = NonZeroUsize::new(n.saturating_sub(input.len())) {
Err(ParseError::Incomplete(Needed::Size(need)))
} else {
let (result, remaining) = input.split_at(n);
Ok((remaining, result))
}
}
pub(super) fn length_prefixed<'a, F, G, E>(
mut f: F,
mut g: G,
) -> impl FnMut(&'a [u8]) -> ParseResult<'a, Vec<E>>
where
F: Parser<'a, u64>,
G: Parser<'a, E>,
{
move |input: &'a [u8]| {
let (i, count) = f.parse(input)?;
let mut res = Vec::new();
let mut input = i;
for _ in 0..count {
match g.parse(input) {
Ok((i, e)) => {
input = i;
res.push(e);
}
Err(e) => {
return Err(e);
}
}
}
Ok((input, res))
}
}
pub(super) fn tuple2<'a, F, E, G, H>(
mut f: F,
mut g: G,
) -> impl FnMut(&'a [u8]) -> ParseResult<(E, H)>
where
F: Parser<'a, E>,
G: Parser<'a, H>,
{
move |input: &'a [u8]| {
let (i, one) = f.parse(input)?;
let (i, two) = g.parse(i)?;
Ok((i, (one, two)))
}
}
pub(super) fn apply_n<'a, F, E>(n: usize, mut f: F) -> impl FnMut(&'a [u8]) -> ParseResult<Vec<E>>
where
F: Parser<'a, E>,
{
move |input: &'a [u8]| {
let mut i = input;
let mut result = Vec::new();
for _ in 0..n {
let (new_i, e) = f.parse(i)?;
result.push(e);
i = new_i;
}
Ok((i, result))
}
}
/// Parse a length prefixed actor ID
pub(super) fn actor_id(input: &[u8]) -> ParseResult<ActorId> {
let (i, length) = leb128_u64(input)?;
let (i, bytes) = take_n(length as usize, i)?;
Ok((i, bytes.into()))
}
pub(super) fn change_hash(input: &[u8]) -> ParseResult<ChangeHash> {
let (i, bytes) = take_n(32, input)?;
let byte_arr: ChangeHash = bytes.try_into().expect("we checked the length above");
Ok((i, byte_arr))
}
pub(super) fn utf_8(len: usize, input: &[u8]) -> ParseResult<String> {
let (i, bytes) = take_n(len, input)?;
let result = String::from_utf8(bytes.to_vec()).map_err(|_| ParseError::Error(ErrorKind::InvalidUtf8))?;
Ok((i, result))
}

View file

@ -49,6 +49,9 @@ pub enum Error {
Leb128(#[from] leb128::read::Error),
#[error(transparent)]
Io(#[from] io::Error),
#[cfg(feature = "storage-v2")]
#[error(transparent)]
Changev2LoadError(#[from] crate::change_v2::LoadError),
}
#[derive(thiserror::Error, Debug)]
@ -122,6 +125,12 @@ pub(crate) struct BooleanDecoder<'a> {
count: usize,
}
impl<'a> BooleanDecoder<'a> {
pub(crate) fn done(&self) -> bool {
self.decoder.done()
}
}
impl<'a> From<Cow<'a, [u8]>> for Decoder<'a> {
fn from(bytes: Cow<'a, [u8]>) -> Decoder<'a> {
Decoder::new(bytes)
@ -164,6 +173,12 @@ pub(crate) struct RleDecoder<'a, T> {
literal: bool,
}
impl<'a, T> RleDecoder<'a, T> {
pub(crate) fn done(&self) -> bool {
self.decoder.done()
}
}
impl<'a, T> From<Cow<'a, [u8]>> for RleDecoder<'a, T> {
fn from(bytes: Cow<'a, [u8]>) -> Self {
RleDecoder {
@ -229,6 +244,12 @@ pub(crate) struct DeltaDecoder<'a> {
absolute_val: u64,
}
impl<'a> DeltaDecoder<'a> {
pub(crate) fn done(&self) -> bool {
self.rle.done()
}
}
impl<'a> From<Cow<'a, [u8]>> for DeltaDecoder<'a> {
fn from(bytes: Cow<'a, [u8]>) -> Self {
DeltaDecoder {

View file

@ -8,10 +8,13 @@ use std::{
use flate2::{bufread::DeflateEncoder, Compression};
use smol_str::SmolStr;
#[cfg(not(feature = "storage-v2"))]
use crate::columnar::COLUMN_TYPE_DEFLATE;
use crate::ActorId;
pub(crate) const DEFLATE_MIN_SIZE: usize = 256;
#[cfg(feature = "storage-v2")]
const COLUMN_TYPE_DEFLATE: u32 = 8;
/// The error type for encoding operations.
#[derive(Debug, thiserror::Error)]
@ -331,6 +334,47 @@ impl Encodable for i32 {
}
}
impl Encodable for [ActorId] {
fn encode<R: Write>(&self, buf: &mut R) -> io::Result<usize> {
let mut len = self.len().encode(buf)?;
for i in self {
len += i.to_bytes().encode(buf)?;
}
Ok(len)
}
}
fn actor_index(actor: &ActorId, actors: &[ActorId]) -> usize {
actors.iter().position(|a| a == actor).unwrap()
}
impl Encodable for ActorId {
fn encode_with_actors<R: Write>(&self, buf: &mut R, actors: &[ActorId]) -> io::Result<usize> {
actor_index(self, actors).encode(buf)
}
fn encode<R: Write>(&self, _buf: &mut R) -> io::Result<usize> {
// we instead encode actors as their position on a sequence
Ok(0)
}
}
impl Encodable for Vec<u8> {
fn encode<R: Write>(&self, buf: &mut R) -> io::Result<usize> {
self.as_slice().encode(buf)
}
}
impl Encodable for &[u8] {
fn encode<R: Write>(&self, buf: &mut R) -> io::Result<usize> {
let head = self.len().encode(buf)?;
buf.write_all(self)?;
Ok(head + self.len())
}
}
#[derive(Debug)]
pub(crate) struct ColData {
pub col: u32,

View file

@ -1,6 +1,8 @@
use crate::decoding;
use crate::types::{ActorId, ScalarValue};
use crate::value::DataType;
#[cfg(feature = "storage-v2")]
use crate::columnar_2::load::Error as LoadError;
use thiserror::Error;
#[derive(Error, Debug)]
@ -23,6 +25,9 @@ pub enum AutomergeError {
DuplicateSeqNumber(u64, ActorId),
#[error("generic automerge error")]
Fail,
#[cfg(feature = "storage-v2")]
#[error(transparent)]
Load(#[from] LoadError),
}
impl From<std::io::Error> for AutomergeError {
@ -72,3 +77,11 @@ pub struct InvalidElementId(pub String);
#[derive(Error, Debug)]
#[error("Invalid OpID: {0}")]
pub struct InvalidOpId(pub String);
#[derive(Error, Debug)]
pub enum InvalidOpType {
#[error("unrecognized action index {0}")]
UnknownAction(u64),
#[error("non numeric argument for inc op")]
NonNumericInc,
}

View file

@ -39,6 +39,10 @@ where
self.cache.len()
}
pub fn safe_get(&self, index: usize) -> Option<&T> {
self.cache.get(index)
}
pub fn get(&self, index: usize) -> &T {
&self.cache[index]
}
@ -66,6 +70,25 @@ where
sorted
}
/// Create a vector from positions in this index to positions in an equivalent sorted index
///
/// This is useful primarily when encoding an `IndexedCache<ActorId>` in the document format.
/// In this case we encode the actors in sorted order in the document and all ops reference the
/// offset into this sorted actor array. But the `IndexedCache<ActorId>` we have in the
/// application does not contain actors in sorted order because we add them as we encounter
/// them, so we must map from the actor IDs in the application to the actor IDs in the document
/// format
///
/// # Examples
///
/// ```rust,ignore
/// let idx: IndexedCache<String> = IndexedCache::new();
/// let first_idx = idx.cache("b"); // first_idx is `0`
/// let second_idx = idx.cache("a"); // second_idx i `1`
/// let encoded = idx.encode_index();
/// // first_idx (0) maps to `1` whilst second_idx (1) maps to `0` because "a" < "b"
/// assert_eq!(encoded, vec![1,0])
/// ```
pub fn encode_index(&self) -> Vec<usize> {
let sorted: Vec<_> = self.cache.iter().sorted().cloned().collect();
self.cache
@ -90,3 +113,18 @@ impl<T> Index<usize> for IndexedCache<T> {
&self.cache[i]
}
}
impl<A: Hash + Eq + Clone> FromIterator<A> for IndexedCache<A> {
fn from_iter<T: IntoIterator<Item = A>>(iter: T) -> Self {
let mut cache = Vec::new();
let mut lookup = HashMap::new();
for (index, elem) in iter.into_iter().enumerate() {
cache.push(elem.clone());
lookup.insert(elem, index);
}
Self {
cache,
lookup,
}
}
}

View file

@ -26,9 +26,14 @@ macro_rules! __log {
mod autocommit;
mod automerge;
#[cfg(not(feature = "storage-v2"))]
mod change;
#[cfg(feature = "storage-v2")]
mod change_v2;
mod clock;
#[cfg(not(feature = "storage-v2"))]
mod columnar;
mod columnar_2;
mod decoding;
mod encoding;
mod error;
@ -42,6 +47,7 @@ mod op_tree;
mod query;
pub mod sync;
pub mod transaction;
mod autoserde;
mod types;
mod value;
#[cfg(feature = "optree-visualisation")]
@ -49,7 +55,10 @@ mod visualisation;
pub use crate::automerge::Automerge;
pub use autocommit::AutoCommit;
#[cfg(not(feature = "storage-v2"))]
pub use change::Change;
#[cfg(feature = "storage-v2")]
pub use change_v2::Change;
pub use error::AutomergeError;
pub use exid::ExId as ObjId;
pub use keys::Keys;
@ -57,5 +66,6 @@ pub use keys_at::KeysAt;
pub use legacy::Change as ExpandedChange;
pub use types::{ActorId, ChangeHash, ObjType, OpType, Prop};
pub use value::{ScalarValue, Value};
pub use autoserde::AutoSerde;
pub const ROOT: ObjId = ObjId::Root;

View file

@ -5,6 +5,7 @@ use crate::query::{self, TreeQuery};
use crate::types::{ActorId, Key, ObjId, Op, OpId, OpType};
use crate::ObjType;
use fxhash::FxBuildHasher;
use std::borrow::Borrow;
use std::cmp::Ordering;
use std::collections::HashMap;
@ -19,6 +20,19 @@ pub(crate) struct OpSetInternal<const B: usize> {
}
impl<const B: usize> OpSetInternal<B> {
pub(crate) fn from_parts(
trees: HashMap<ObjId, (ObjType, OpTreeInternal<B>), FxBuildHasher>,
meta: OpSetMetadata,
) -> Self {
let len = trees.values().map(|t| t.1.len()).sum();
Self {
trees,
length: len,
m: meta,
}
}
pub fn new() -> Self {
let mut trees: HashMap<_, _, _> = Default::default();
trees.insert(ObjId::root(), (ObjType::Map, Default::default()));
@ -143,6 +157,7 @@ impl<'a, const B: usize> IntoIterator for &'a OpSetInternal<B> {
}
}
#[derive(Clone)]
pub(crate) struct Iter<'a, const B: usize> {
inner: &'a OpSetInternal<B>,
index: usize,
@ -176,7 +191,24 @@ pub(crate) struct OpSetMetadata {
pub props: IndexedCache<String>,
}
impl Default for OpSetMetadata {
fn default() -> Self {
Self {
actors: IndexedCache::new(),
props: IndexedCache::new(),
}
}
}
impl OpSetMetadata {
pub fn from_actors(actors: Vec<ActorId>) -> Self {
Self {
props: IndexedCache::new(),
actors: actors.into_iter().collect(),
}
}
pub fn key_cmp(&self, left: &Key, right: &Key) -> Ordering {
match (left, right) {
(Key::Map(a), Key::Map(b)) => self.props[*a].cmp(&self.props[*b]),
@ -193,4 +225,8 @@ impl OpSetMetadata {
(OpId(a, _), OpId(b, _)) => a.cmp(&b),
}
}
pub fn import_prop<S: Borrow<str>>(&mut self, key: S) -> usize {
self.props.cache(key.borrow().to_string())
}
}

View file

@ -76,12 +76,12 @@ impl Automerge {
}
// deduplicate the changes to send with those we have already sent
changes_to_send.retain(|change| !sync_state.sent_hashes.contains(&change.hash));
changes_to_send.retain(|change| !sync_state.sent_hashes.contains(&change.hash()));
sync_state.last_sent_heads = our_heads.clone();
sync_state
.sent_hashes
.extend(changes_to_send.iter().map(|c| c.hash));
.extend(changes_to_send.iter().map(|c| c.hash()));
let sync_message = Message {
heads: our_heads,
@ -157,7 +157,7 @@ impl Automerge {
let new_changes = self.get_changes(&last_sync);
let hashes = new_changes
.into_iter()
.map(|change| change.hash)
.map(|change| change.hash())
.collect::<Vec<_>>();
Have {
last_sync,
@ -190,17 +190,17 @@ impl Automerge {
let mut hashes_to_send = HashSet::new();
for change in &changes {
change_hashes.insert(change.hash);
change_hashes.insert(change.hash());
for dep in &change.deps {
dependents.entry(*dep).or_default().push(change.hash);
for dep in change.deps() {
dependents.entry(*dep).or_default().push(change.hash());
}
if bloom_filters
.iter()
.all(|bloom| !bloom.contains_hash(&change.hash))
.all(|bloom| !bloom.contains_hash(&change.hash()))
{
hashes_to_send.insert(change.hash);
hashes_to_send.insert(change.hash());
}
}
@ -227,7 +227,7 @@ impl Automerge {
}
for change in changes {
if hashes_to_send.contains(&change.hash) {
if hashes_to_send.contains(&change.hash()) {
changes_to_send.push(change);
}
}

0
automerge/src/to_json.rs Normal file
View file

View file

@ -1,9 +1,21 @@
use std::borrow::Cow;
use crate::automerge::Actor;
use crate::Change;
use crate::exid::ExId;
use crate::indexed_cache::IndexedCache;
use crate::query::{self, OpIdSearch};
use crate::types::{Key, ObjId, OpId};
use crate::{change::export_change, types::Op, Automerge, ChangeHash, Prop};
use crate::{types::{ActorId, Op}, Automerge, ChangeHash, Prop};
use crate::{AutomergeError, ObjType, OpType, ScalarValue};
#[cfg(not(feature = "storage-v2"))]
use crate::change::export_change;
#[cfg(feature = "storage-v2")]
use crate::columnar_2::{
storage::{Chunk, Change as StoredChange},
save::encode_change_ops
};
#[derive(Debug, Clone)]
pub struct TransactionInner {
@ -40,13 +52,40 @@ impl TransactionInner {
}
let num_ops = self.operations.len();
let change = export_change(self, &doc.ops.m.actors, &doc.ops.m.props);
let hash = change.hash;
let change = self.export(&doc.ops.m.actors, &doc.ops.m.props);
let hash = change.hash();
doc.update_history(change, num_ops);
debug_assert_eq!(doc.get_heads(), vec![hash]);
hash
}
#[cfg(feature = "storage-v2")]
pub(crate) fn export(self, actors: &IndexedCache<ActorId>, props: &IndexedCache<String>) -> Change {
let actor = actors.get(self.actor).clone();
let ops = self.operations.iter().map(|o| (&o.0, &o.1));
let (ops, op_data, other_actors) = encode_change_ops(ops, actor.clone(), actors, props);
let stored = StoredChange {
actor,
message: self.message.clone(),
extra_bytes: Cow::Owned(vec![]),
start_op: self.start_op,
seq: self.seq,
dependencies: self.deps.clone(),
other_actors,
timestamp: self.time,
ops_meta: ops.metadata(),
ops_data: Cow::Owned(op_data),
};
let written = stored.write();
let chunk = Chunk::new_change(written.as_slice());
Change::new(stored, chunk.hash(), self.operations.len())
}
#[cfg(not(feature = "storage-v2"))]
pub(crate) fn export(self, actors: &IndexedCache<ActorId>, props: &IndexedCache<String>) -> Change {
export_change(self, actors, props)
}
/// Undo the operations added in this transaction, returning the number of cancelled
/// operations.
pub fn rollback(self, doc: &mut Automerge) -> usize {

View file

@ -60,6 +60,12 @@ impl TryFrom<&str> for ActorId {
}
}
impl AsRef<[u8]> for ActorId {
fn as_ref(&self) -> &[u8] {
&self.0
}
}
impl From<uuid::Uuid> for ActorId {
fn from(u: uuid::Uuid) -> Self {
ActorId(TinyVec::from(*u.as_bytes()))
@ -156,6 +162,40 @@ pub enum OpType {
Set(ScalarValue),
}
impl OpType {
/// The index into the action array as specified in [1]
///
/// [1]: https://alexjg.github.io/automerge-storage-docs/#action-array
pub(crate) fn action_index(&self) -> u64 {
match self {
Self::Make(ObjType::Map) => 0,
Self::Set(_) => 1,
Self::Make(ObjType::List) => 2,
Self::Del => 3,
Self::Make(ObjType::Text) => 4,
Self::Inc(_) => 5,
Self::Make(ObjType::Table) => 6,
}
}
pub(crate) fn from_index_and_value(index: u64, value: ScalarValue) -> Result<OpType, error::InvalidOpType> {
match index {
0 => Ok(Self::Make(ObjType::Map)),
1 => Ok(Self::Set(value)),
2 => Ok(Self::Make(ObjType::List)),
3 => Ok(Self::Del),
4 => Ok(Self::Make(ObjType::Text)),
5 => match value {
ScalarValue::Int(i) => Ok(Self::Inc(i)),
ScalarValue::Uint(i) => Ok(Self::Inc(i as i64)),
_ => Err(error::InvalidOpType::NonNumericInc),
},
6 => Ok(Self::Make(ObjType::Table)),
other => Err(error::InvalidOpType::UnknownAction(other)),
}
}
}
impl From<ObjType> for OpType {
fn from(v: ObjType) -> Self {
OpType::Make(v)
@ -235,6 +275,12 @@ impl Exportable for Key {
}
}
impl From<ObjId> for OpId {
fn from(o: ObjId) -> Self {
o.0
}
}
impl From<OpId> for ObjId {
fn from(o: OpId) -> Self {
ObjId(o)
@ -325,6 +371,12 @@ impl Key {
#[derive(Debug, Clone, PartialOrd, Ord, Eq, PartialEq, Copy, Hash, Default)]
pub(crate) struct OpId(pub u64, pub usize);
impl OpId {
pub(crate) fn new(actor: usize, counter: u64) -> Self {
Self(counter, actor)
}
}
#[derive(Debug, Clone, Copy, PartialOrd, Eq, PartialEq, Ord, Hash, Default)]
pub(crate) struct ObjId(pub OpId);
@ -332,6 +384,14 @@ impl ObjId {
pub fn root() -> Self {
ObjId(OpId(0, 0))
}
pub fn is_root(&self) -> bool {
self.0.counter() == 0
}
pub(crate) fn opid(&self) -> &OpId {
&self.0
}
}
#[derive(Debug, Clone, Copy, PartialOrd, Eq, PartialEq, Ord, Hash, Default)]
@ -451,6 +511,22 @@ pub struct Peer {}
#[derive(Eq, PartialEq, Hash, Clone, PartialOrd, Ord, Copy)]
pub struct ChangeHash(pub [u8; 32]);
impl ChangeHash {
pub(crate) fn as_bytes(&self) -> &[u8] {
&self.0
}
pub(crate) fn checksum(&self) -> [u8; 4] {
[self.0[0], self.0[1], self.0[2], self.0[3]]
}
}
impl AsRef<[u8]> for ChangeHash {
fn as_ref(&self) -> &[u8] {
&self.0
}
}
impl fmt::Debug for ChangeHash {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
f.debug_tuple("ChangeHash")

View file

@ -100,7 +100,9 @@ impl<'a, const B: usize> GraphVisualisation<'a, B> {
}
}
impl<'a, const B: usize> dot::GraphWalk<'a, &'a Node<'a, B>, Edge> for GraphVisualisation<'a, B> {
impl<'a, const B: usize> dot::GraphWalk<'a, &'a Node<'a, B>, Edge>
for GraphVisualisation<'a, B>
{
fn nodes(&'a self) -> dot::Nodes<'a, &'a Node<'a, B>> {
Cow::Owned(self.nodes.values().collect::<Vec<_>>())
}
@ -127,7 +129,9 @@ impl<'a, const B: usize> dot::GraphWalk<'a, &'a Node<'a, B>, Edge> for GraphVisu
}
}
impl<'a, const B: usize> dot::Labeller<'a, &'a Node<'a, B>, Edge> for GraphVisualisation<'a, B> {
impl<'a, const B: usize> dot::Labeller<'a, &'a Node<'a, B>, Edge>
for GraphVisualisation<'a, B>
{
fn graph_id(&'a self) -> dot::Id<'a> {
dot::Id::new("OpSet").unwrap()
}
@ -192,6 +196,7 @@ impl OpTable {
<td>prop</td>\
<td>action</td>\
<td>succ</td>\
<td>pred</td>\
</tr>\
<hr/>\
{}\
@ -207,6 +212,7 @@ struct OpTableRow {
prop: String,
op_description: String,
succ: String,
pred: String,
}
impl OpTableRow {
@ -217,6 +223,7 @@ impl OpTableRow {
&self.prop,
&self.op_description,
&self.succ,
&self.pred,
];
let row = rows
.iter()
@ -248,12 +255,18 @@ impl OpTableRow {
.iter()
.map(|s| format!(",{}", print_opid(s, actor_shorthands)))
.collect();
let pred = op
.pred
.iter()
.map(|s| format!(",{}", print_opid(s, actor_shorthands)))
.collect();
OpTableRow {
op_description,
obj_id: print_opid(&obj.0, actor_shorthands),
op_id: print_opid(&op.id, actor_shorthands),
prop,
succ,
pred,
}
}
}

116
howtosplit.adoc Normal file
View file

@ -0,0 +1,116 @@
= How to split a block
We have a block of column oriented storage. We want to split that block into two
separate blocks without using any intermediary storage.
A first stab at this algorithm:
[source]
----
for column in source_block:
if column is simple:
for each value up to index:
write value to first block
for each value after index
write value to second block
else if column is value:
read each metadata value up to index
write to first block
read each metadata value after index
write to second block
read each metadata value up to index
then read each value up to index
write value to first block
read each metadata value after index
then read each value after index
write value to second block
else if column is group
read each num up to index
write to first block
read each num after index
write to second block
read each num up to index
for each column in group columns
read num values
write to column in first block
read each num after index
for each column in group columns
read num values
write to column in second block
----
This logic does need to be implemented so we can handle future columns. However,
the logic might become much easier when we know the types of the values:
[source]
----
for column in source_block:
if column is simple:
for each value up to index:
write value to first block
for each value after index
write value to second block
if column is value: <1>
for each value in value:
...
----
<1> Here we know the kind of values. Does this mean we can avoid re-reading the
metadata column?
Turns out we can't avoid double reading the metadata column. And I think the
same is true for the group column. This means we always need to use the generic
encoding.
Therefore we can probably do this by definin a `split` operation on generic
rowblocks.
== Inserting?
What about `insert`? Can we think about this in terms of some abstract column
layout type?
[source,rust]
----
struct RowBlock<'a, C> {
layout: C,
data: Vec<u8>,
}
trait HasColumnLayout: Into<ColumnLayout> + TryFrom<ColumnLayout> {
type Item;
fn value_for_column(item: &Item, column_index: usize) -> Option<CellValue>;
}
enum BlockError {
InvalidValueForIndex,
}
impl<'a, C: Into<ColumnLayout> + TryFrom<ColumnLayout>> for RowBlock<'a, C> {
fn splice<R, I>(&self, range: R, items: I) -> Result<Self, <C as TryFrom<ColumnLayout>>::Error>
where
R: RangeBounds<usize>,
I: Iterator<Item = C::Item>
{
let generic = self.layout.into();
let new_data = Vec::with_capacity(self.data.len() + generic.max_row_size());
let new_generic_block = generic.splice(10..12, |row, col| {
if let Some(item) = i.next() {
Some(C::value_for_column(item, col))
} else {
None
}
})?;
Ok(new_generic_block.try_into()?)
}
fn split(&self, index: usize) -> Result<(Self, Self), <C as TryFrom<ColumnLayout>::Error>
{
let generic = self.layout.into();
let first_bock = ..
let second_block = ..
generic.split_into(index, first_block, second_block);
Ok((first_block.try_into()?, second_block.try_into()?)
}
}
----

BIN
testchange Normal file

Binary file not shown.

BIN
testdoc Normal file

Binary file not shown.

25
todo.adoc Normal file
View file

@ -0,0 +1,25 @@
== Doing
* Implement `columnar_2::save::save_document`
== Cleanups
* Add a `*Range` for each column type (including composite)
* For each of the `*Range` types add an `encode` method. Replace the ad hoc
`*Decoder::encode` methods with these methods
* Get rid of `PrimVal`, do everything with `ScalarValue`. i.e. move the unknown
type codes into `ScalarValue`.
* Make `Change` and `Document` directly hold a `*Columns` instead of
`ColumnMetadata`, construct these at parse time
* Make `Chunk` an actual enum and parse the contents into that enum
* Change `ChangeOpColumns::encode` to `ChangeOpColumns::splice`, include a
`replace_with` range, and create a new `encode` implementation which just
calls `ChangeOpColumns::empty().splice(..)`
* `try_splice` on all encoders
* Remove rowblock, just have structs for `ChangeOps`, `DocChangeOps`,
`DocChanges` which hold their own bytes. Maybe later we add a `GenericRows`
thing but we seem to not really need it
* Replace every use of `usize` as a lookup with a newtype wrapper