Start installing SQL schema. (#171) r=rnewman

* Start installing the SQLite store and bootstrapping the datom store.

* Review comment: Decomplect V2_IDENTS.

* Review comment: Decomplect V2_PARTS.

* Review comment: Pre: Expose Clojure's merge on Value instances.

* Review comment: Decomplect V2_SYMBOLIC_SCHEMA.

* Review comment: Decomplect V1_STATEMENTS.

* Review comment: Prefer ? to try!.

* Review comment: Fix typos; format; add TODOs.

* Review comment: Assert that Mentat `Schema` is valid upon creation.

* Review comment: Improve conversion to and from SQL values.

This patch factors the fundamental SQL conversion maps
between (rusqlite::Value, value_type_tag) and (edn::Value, ValueType)
through a new Mentat TypedValue.  (A future patch might rename this
fundamental type mentat::Value.)

To make certain conversion functions infallible, I removed
placeholders for :db.type/{instant,uuid,uri}.  (We could panic
instead, but there's no need to do that right now.)

* Review comment: Always uses bundled SQLite in rusqlite.

This avoids (runtime) failures in Travis CI due to old SQLite
versions.  See 432966ac77.

* Review comment: Move semantics in `from_sql_value_pair`.

* Review comment: DB_EXCISE_BEFORE_T instead of ...BEFORET (no underscore).

* Review comment: Move overview notes to the Wiki.
This commit is contained in:
Nick Alexander 2017-01-25 16:13:56 -08:00 committed by GitHub
parent 2592506288
commit 81af295948
18 changed files with 1560 additions and 2 deletions

2
.gitignore vendored
View file

@ -5,6 +5,8 @@
*~
.s*
.*.sw*
*.rs.bak
*.bak
.hg/
.hgignore
.lein-deps-sum

View file

@ -3,6 +3,7 @@ script:
- cargo build --verbose
- cargo test --verbose
- cargo test --verbose -p edn
- cargo test --verbose -p mentat_db
- cargo test --verbose -p mentat_query
- cargo test --verbose -p mentat_query_parser
- cargo test --verbose -p mentat_tx_parser

View file

@ -6,11 +6,18 @@ authors = ["Richard Newman <rnewman@twinql.com>", "Nicholas Alexander <nalexande
[dependencies]
clap = "2.19.3"
nickel = "0.9.0"
rusqlite = "0.8.0"
[dependencies.rusqlite]
version = "0.9.3"
# System sqlite might be very old.
features = ["bundled"]
[dependencies.edn]
path = "edn"
[dependencies.mentat_db]
path = "db"
[dependencies.mentat_query]
path = "query"

24
db/Cargo.toml Normal file
View file

@ -0,0 +1,24 @@
[package]
name = "mentat_db"
version = "0.0.1"
[dependencies]
error-chain = "0.8.0"
lazy_static = "0.2.2"
# TODO: don't depend on num and ordered-float; expose helpers in edn abstracting necessary constructors.
num = "0.1.35"
ordered-float = "0.3.0"
[dependencies.rusqlite]
version = "0.9.3"
# System sqlite might be very old.
features = ["bundled"]
[dependencies.edn]
path = "../edn"
[dependencies.mentat_tx]
path = "../tx"
[dependencies.mentat_tx_parser]
path = "../tx-parser"

3
db/README.md Normal file
View file

@ -0,0 +1,3 @@
This sub-crate implements the SQLite database layer: installing,
managing, and migrating forward the SQL schema underlying the datom
store.

263
db/src/bootstrap.rs Normal file
View file

@ -0,0 +1,263 @@
// Copyright 2016 Mozilla
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use
// this file except in compliance with the License. You may obtain a copy of the
// License at http://www.apache.org/licenses/LICENSE-2.0
// Unless required by applicable law or agreed to in writing, software distributed
// under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
// CONDITIONS OF ANY KIND, either express or implied. See the License for the
// specific language governing permissions and limitations under the License.
#![allow(dead_code)]
use {to_namespaced_keyword};
use edn;
use edn::types::Value;
use entids;
use errors::*;
use mentat_tx::entities::Entity;
use mentat_tx_parser;
use types::{IdentMap, Partition, PartitionMap, Schema, TypedValue};
use values;
lazy_static! {
static ref V1_IDENTS: Vec<(&'static str, i64)> = {
vec![(":db/ident", entids::DB_IDENT),
(":db.part/db", entids::DB_PART_DB),
(":db/txInstant", entids::DB_TX_INSTANT),
(":db.install/partition", entids::DB_INSTALL_PARTITION),
(":db.install/valueType", entids::DB_INSTALL_VALUETYPE),
(":db.install/attribute", entids::DB_INSTALL_ATTRIBUTE),
(":db/valueType", entids::DB_VALUE_TYPE),
(":db/cardinality", entids::DB_CARDINALITY),
(":db/unique", entids::DB_UNIQUE),
(":db/isComponent", entids::DB_IS_COMPONENT),
(":db/index", entids::DB_INDEX),
(":db/fulltext", entids::DB_FULLTEXT),
(":db/noHistory", entids::DB_NO_HISTORY),
(":db/add", entids::DB_ADD),
(":db/retract", entids::DB_RETRACT),
(":db.part/user", entids::DB_PART_USER),
(":db.part/tx", entids::DB_PART_TX),
(":db/excise", entids::DB_EXCISE),
(":db.excise/attrs", entids::DB_EXCISE_ATTRS),
(":db.excise/beforeT", entids::DB_EXCISE_BEFORE_T),
(":db.excise/before", entids::DB_EXCISE_BEFORE),
(":db.alter/attribute", entids::DB_ALTER_ATTRIBUTE),
(":db.type/ref", entids::DB_TYPE_REF),
(":db.type/keyword", entids::DB_TYPE_KEYWORD),
(":db.type/long", entids::DB_TYPE_LONG),
(":db.type/double", entids::DB_TYPE_DOUBLE),
(":db.type/string", entids::DB_TYPE_STRING),
(":db.type/boolean", entids::DB_TYPE_BOOLEAN),
(":db.type/instant", entids::DB_TYPE_INSTANT),
(":db.type/bytes", entids::DB_TYPE_BYTES),
(":db.cardinality/one", entids::DB_CARDINALITY_ONE),
(":db.cardinality/many", entids::DB_CARDINALITY_MANY),
(":db.unique/value", entids::DB_UNIQUE_VALUE),
(":db.unique/identity", entids::DB_UNIQUE_IDENTITY),
(":db/doc", entids::DB_DOC),
]
};
static ref V2_IDENTS: Vec<(&'static str, i64)> = {
[(*V1_IDENTS).clone(),
vec![(":db.schema/version", entids::DB_SCHEMA_VERSION),
(":db.schema/attribute", entids::DB_SCHEMA_ATTRIBUTE),
]].concat()
};
static ref V1_PARTS: Vec<(&'static str, i64, i64)> = {
vec![(":db.part/db", 0, (1 + V1_IDENTS.len()) as i64),
(":db.part/user", 0x10000, 0x10000),
(":db.part/tx", 0x10000000, 0x10000000),
]
};
static ref V2_PARTS: Vec<(&'static str, i64, i64)> = {
vec![(":db.part/db", 0, (1 + V2_IDENTS.len()) as i64),
(":db.part/user", 0x10000, 0x10000),
(":db.part/tx", 0x10000000, 0x10000000),
]
};
static ref V1_SYMBOLIC_SCHEMA: Value = {
let s = r#"
{:db/ident {:db/valueType :db.type/keyword
:db/cardinality :db.cardinality/one
:db/unique :db.unique/identity}
:db.install/partition {:db/valueType :db.type/ref
:db/cardinality :db.cardinality/many}
:db.install/valueType {:db/valueType :db.type/ref
:db/cardinality :db.cardinality/many}
:db.install/attribute {:db/valueType :db.type/ref
:db/cardinality :db.cardinality/many}
;; TODO: support user-specified functions in the future.
;; :db.install/function {:db/valueType :db.type/ref
;; :db/cardinality :db.cardinality/many}
:db/txInstant {:db/valueType :db.type/long
:db/cardinality :db.cardinality/one
:db/index true}
:db/valueType {:db/valueType :db.type/ref
:db/cardinality :db.cardinality/one}
:db/cardinality {:db/valueType :db.type/ref
:db/cardinality :db.cardinality/one}
:db/doc {:db/valueType :db.type/string
:db/cardinality :db.cardinality/one}
:db/unique {:db/valueType :db.type/ref
:db/cardinality :db.cardinality/one}
:db/isComponent {:db/valueType :db.type/boolean
:db/cardinality :db.cardinality/one}
:db/index {:db/valueType :db.type/boolean
:db/cardinality :db.cardinality/one}
:db/fulltext {:db/valueType :db.type/boolean
:db/cardinality :db.cardinality/one}
:db/noHistory {:db/valueType :db.type/boolean
:db/cardinality :db.cardinality/one}}"#;
edn::parse::value(s)
.map_err(|_| ErrorKind::BadBootstrapDefinition("Unable to parse V1_SYMBOLIC_SCHEMA".into()))
.unwrap()
};
static ref V2_SYMBOLIC_SCHEMA: Value = {
let s = r#"
{:db.alter/attribute {:db/valueType :db.type/ref
:db/cardinality :db.cardinality/many}
:db.schema/version {:db/valueType :db.type/long
:db/cardinality :db.cardinality/one}
;; unique-value because an attribute can only belong to a single
;; schema fragment.
:db.schema/attribute {:db/valueType :db.type/ref
:db/unique :db.unique/value
:db/cardinality :db.cardinality/many}}"#;
let right = edn::parse::value(s)
.map_err(|_| ErrorKind::BadBootstrapDefinition("Unable to parse V2_SYMBOLIC_SCHEMA".into()))
.unwrap();
edn::utils::merge(&V1_SYMBOLIC_SCHEMA, &right)
.ok_or(ErrorKind::BadBootstrapDefinition("Unable to parse V2_SYMBOLIC_SCHEMA".into()))
.unwrap()
};
}
/// Convert (ident, entid) pairs into [:db/add IDENT :db/ident IDENT] `Value` instances.
fn idents_to_assertions(idents: &[(&str, i64)]) -> Vec<Value> {
idents
.into_iter()
.map(|&(ident, _)| {
let value = Value::NamespacedKeyword(to_namespaced_keyword(&ident).unwrap());
Value::Vector(vec![values::DB_ADD.clone(), value.clone(), values::DB_IDENT.clone(), value.clone()])
})
.collect()
}
/// Convert {:ident {:key :value ...} ...} to vec![(String(:ident), String(:key), TypedValue(:value)), ...].
///
/// Such triples are closer to what the transactor will produce when processing
/// :db.install/attribute assertions.
fn symbolic_schema_to_triples(ident_map: &IdentMap, symbolic_schema: &Value) -> Result<Vec<(String, String, TypedValue)>> {
// Failure here is a coding error, not a runtime error.
let mut triples: Vec<(String, String, TypedValue)> = vec![];
// TODO: Consider `flat_map` and `map` rather than loop.
match *symbolic_schema {
Value::Map(ref m) => {
for (ident, mp) in m {
let ident = match ident {
&Value::NamespacedKeyword(ref ident) => ident.to_string(),
_ => bail!(ErrorKind::BadBootstrapDefinition(format!("Expected namespaced keyword for ident but got '{:?}'", ident)))
};
match *mp {
Value::Map(ref mpp) => {
for (attr, value) in mpp {
let attr = match attr {
&Value::NamespacedKeyword(ref attr) => attr.to_string(),
_ => bail!(ErrorKind::BadBootstrapDefinition(format!("Expected namespaced keyword for attr but got '{:?}'", attr)))
};
// We have symbolic idents but the transactor handles entids. Ad-hoc
// convert right here. This is a fundamental limitation on the
// bootstrap symbolic schema format; we can't represent "real" keywords
// at this time.
//
// TODO: remove this limitation, perhaps by including a type tag in the
// bootstrap symbolic schema, or by representing the initial bootstrap
// schema directly as Rust data.
let typed_value = match TypedValue::from_edn_value(value) {
Some(TypedValue::Keyword(ref s)) => TypedValue::Ref(*ident_map.get(s).ok_or(ErrorKind::UnrecognizedIdent(s.clone()))?),
Some(v) => v,
_ => bail!(ErrorKind::BadBootstrapDefinition(format!("Expected Mentat typed value for value but got '{:?}'", value)))
};
triples.push((ident.clone(),
attr.clone(),
typed_value));
}
},
_ => bail!(ErrorKind::BadBootstrapDefinition("Expected {:db/ident {:db/attr value ...} ...}".into()))
}
}
},
_ => bail!(ErrorKind::BadBootstrapDefinition("Expected {...}".into()))
}
Ok(triples)
}
/// Convert {IDENT {:key :value ...} ...} to [[:db/add IDENT :key :value] ...].
/// In addition, add [:db.add :db.part/db :db.install/attribute IDENT] installation assertions.
fn symbolic_schema_to_assertions(symbolic_schema: &Value) -> Result<Vec<Value>> {
// Failure here is a coding error, not a runtime error.
let mut assertions: Vec<Value> = vec![];
match *symbolic_schema {
Value::Map(ref m) => {
for (ident, mp) in m {
assertions.push(Value::Vector(vec![values::DB_ADD.clone(),
values::DB_PART_DB.clone(),
values::DB_INSTALL_ATTRIBUTE.clone(),
ident.clone()]));
match *mp {
Value::Map(ref mpp) => {
for (attr, value) in mpp {
assertions.push(Value::Vector(vec![values::DB_ADD.clone(),
ident.clone(),
attr.clone(),
value.clone()]));
}
},
_ => bail!(ErrorKind::BadBootstrapDefinition("Expected {:db/ident {:db/attr value ...} ...}".into()))
}
}
},
_ => bail!(ErrorKind::BadBootstrapDefinition("Expected {...}".into()))
}
Ok(assertions)
}
pub fn bootstrap_partition_map() -> PartitionMap {
V2_PARTS[..].iter()
.map(|&(part, start, index)| (part.to_string(), Partition::new(start, index)))
.collect()
}
pub fn bootstrap_ident_map() -> IdentMap {
V2_IDENTS[..].iter()
.map(|&(ident, entid)| (ident.to_string(), entid))
.collect()
}
pub fn bootstrap_schema() -> Schema {
let ident_map = bootstrap_ident_map();
let bootstrap_triples = symbolic_schema_to_triples(&ident_map, &V2_SYMBOLIC_SCHEMA).unwrap();
Schema::from_ident_map_and_triples(ident_map, bootstrap_triples).unwrap()
}
pub fn bootstrap_entities() -> Vec<Entity> {
let bootstrap_assertions: Value = Value::Vector([
symbolic_schema_to_assertions(&V2_SYMBOLIC_SCHEMA).unwrap(),
idents_to_assertions(&V2_IDENTS[..]),
].concat());
// Failure here is a coding error (since the inputs are fixed), not a runtime error.
// TODO: represent these bootstrap data errors rather than just panicing.
let bootstrap_entities: Vec<Entity> = mentat_tx_parser::Tx::parse(&[bootstrap_assertions][..]).unwrap();
return bootstrap_entities;
}

516
db/src/db.rs Normal file
View file

@ -0,0 +1,516 @@
// Copyright 2016 Mozilla
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use
// this file except in compliance with the License. You may obtain a copy of the
// License at http://www.apache.org/licenses/LICENSE-2.0
// Unless required by applicable law or agreed to in writing, software distributed
// under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
// CONDITIONS OF ANY KIND, either express or implied. See the License for the
// specific language governing permissions and limitations under the License.
#![allow(dead_code)]
use rusqlite;
use rusqlite::types::{ToSql, ToSqlOutput};
use bootstrap;
use edn::types::Value;
use errors::*;
use mentat_tx::entities as entmod;
use mentat_tx::entities::Entity;
use types::*;
pub fn new_connection() -> rusqlite::Connection {
return rusqlite::Connection::open_in_memory().unwrap();
}
/// Version history:
///
/// 1: initial schema.
/// 2: added :db.schema/version and /attribute in bootstrap; assigned idents 36 and 37, so we bump
/// the part range here; tie bootstrapping to the SQLite user_version.
pub const CURRENT_VERSION: i32 = 2;
const TRUE: &'static bool = &true;
const FALSE: &'static bool = &false;
/// Turn an owned bool into a static reference to a bool.
///
/// `rusqlite` is designed around references to values; this lets us use computed bools easily.
#[inline(always)]
fn to_bool_ref(x: bool) -> &'static bool {
if x { TRUE } else { FALSE }
}
lazy_static! {
/// SQL statements to be executed, in order, to create the Mentat SQL schema (version 2).
#[cfg_attr(rustfmt, rustfmt_skip)]
static ref V2_STATEMENTS: Vec<&'static str> = { vec![
r#"CREATE TABLE datoms (e INTEGER NOT NULL, a SMALLINT NOT NULL, v BLOB NOT NULL, tx INTEGER NOT NULL,
value_type_tag SMALLINT NOT NULL,
index_avet TINYINT NOT NULL DEFAULT 0, index_vaet TINYINT NOT NULL DEFAULT 0,
index_fulltext TINYINT NOT NULL DEFAULT 0,
unique_value TINYINT NOT NULL DEFAULT 0)"#,
r#"CREATE UNIQUE INDEX idx_datoms_eavt ON datoms (e, a, value_type_tag, v)"#,
r#"CREATE UNIQUE INDEX idx_datoms_aevt ON datoms (a, e, value_type_tag, v)"#,
// Opt-in index: only if a has :db/index true.
r#"CREATE UNIQUE INDEX idx_datoms_avet ON datoms (a, value_type_tag, v, e) WHERE index_avet IS NOT 0"#,
// Opt-in index: only if a has :db/valueType :db.type/ref. No need for tag here since all
// indexed elements are refs.
r#"CREATE UNIQUE INDEX idx_datoms_vaet ON datoms (v, a, e) WHERE index_vaet IS NOT 0"#,
// Opt-in index: only if a has :db/fulltext true; thus, it has :db/valueType :db.type/string,
// which is not :db/valueType :db.type/ref. That is, index_vaet and index_fulltext are mutually
// exclusive.
r#"CREATE INDEX idx_datoms_fulltext ON datoms (value_type_tag, v, a, e) WHERE index_fulltext IS NOT 0"#,
// TODO: possibly remove this index. :db.unique/{value,identity} should be asserted by the
// transactor in all cases, but the index may speed up some of SQLite's query planning. For now,
// it serves to validate the transactor implementation. Note that tag is needed here to
// differentiate, e.g., keywords and strings.
r#"CREATE UNIQUE INDEX idx_datoms_unique_value ON datoms (a, value_type_tag, v) WHERE unique_value IS NOT 0"#,
r#"CREATE TABLE transactions (e INTEGER NOT NULL, a SMALLINT NOT NULL, v BLOB NOT NULL, tx INTEGER NOT NULL, added TINYINT NOT NULL DEFAULT 1, value_type_tag SMALLINT NOT NULL)"#,
r#"CREATE INDEX idx_transactions_tx ON transactions (tx, added)"#,
// Fulltext indexing.
// A fulltext indexed value v is an integer rowid referencing fulltext_values.
// Optional settings:
// tokenize="porter"#,
// prefix='2,3'
// By default we use Unicode-aware tokenizing (particularly for case folding), but preserve
// diacritics.
r#"CREATE VIRTUAL TABLE fulltext_values
USING FTS4 (text NOT NULL, searchid INT, tokenize=unicode61 "remove_diacritics=0")"#,
// This combination of view and triggers allows you to transparently
// update-or-insert into FTS. Just INSERT INTO fulltext_values_view (text, searchid).
r#"CREATE VIEW fulltext_values_view AS SELECT * FROM fulltext_values"#,
r#"CREATE TRIGGER replace_fulltext_searchid
INSTEAD OF INSERT ON fulltext_values_view
WHEN EXISTS (SELECT 1 FROM fulltext_values WHERE text = new.text)
BEGIN
UPDATE fulltext_values SET searchid = new.searchid WHERE text = new.text;
END"#,
r#"CREATE TRIGGER insert_fulltext_searchid
INSTEAD OF INSERT ON fulltext_values_view
WHEN NOT EXISTS (SELECT 1 FROM fulltext_values WHERE text = new.text)
BEGIN
INSERT INTO fulltext_values (text, searchid) VALUES (new.text, new.searchid);
END"#,
// A view transparently interpolating fulltext indexed values into the datom structure.
r#"CREATE VIEW fulltext_datoms AS
SELECT e, a, fulltext_values.text AS v, tx, value_type_tag, index_avet, index_vaet, index_fulltext, unique_value
FROM datoms, fulltext_values
WHERE datoms.index_fulltext IS NOT 0 AND datoms.v = fulltext_values.rowid"#,
// A view transparently interpolating all entities (fulltext and non-fulltext) into the datom structure.
r#"CREATE VIEW all_datoms AS
SELECT e, a, v, tx, value_type_tag, index_avet, index_vaet, index_fulltext, unique_value
FROM datoms
WHERE index_fulltext IS 0
UNION ALL
SELECT e, a, v, tx, value_type_tag, index_avet, index_vaet, index_fulltext, unique_value
FROM fulltext_datoms"#,
// Materialized views of the schema.
r#"CREATE TABLE idents (ident TEXT NOT NULL PRIMARY KEY, entid INTEGER UNIQUE NOT NULL)"#,
r#"CREATE TABLE schema (ident TEXT NOT NULL, attr TEXT NOT NULL, value BLOB NOT NULL, value_type_tag SMALLINT NOT NULL,
FOREIGN KEY (ident) REFERENCES idents (ident))"#,
r#"CREATE INDEX idx_schema_unique ON schema (ident, attr, value, value_type_tag)"#,
r#"CREATE TABLE parts (part TEXT NOT NULL PRIMARY KEY, start INTEGER NOT NULL, idx INTEGER NOT NULL)"#,
]
};
}
/// Set the SQLite user version.
///
/// Mentat manages its own SQL schema version using the user version. See the [SQLite
/// documentation](https://www.sqlite.org/pragma.html#pragma_user_version).
fn set_user_version(conn: &rusqlite::Connection, version: i32) -> Result<()> {
conn.execute(&format!("PRAGMA user_version = {}", version), &[])
.chain_err(|| "Could not set_user_version")
.map(|_| ())
}
/// Get the SQLite user version.
///
/// Mentat manages its own SQL schema version using the user version. See the [SQLite
/// documentation](https://www.sqlite.org/pragma.html#pragma_user_version).
fn get_user_version(conn: &rusqlite::Connection) -> Result<i32> {
conn.query_row("PRAGMA user_version", &[], |row| {
row.get(0)
})
.chain_err(|| "Could not get_user_version")
}
// TODO: rename "SQL" functions to align with "datoms" functions.
pub fn create_current_version(conn: &mut rusqlite::Connection) -> Result<i32> {
let tx = conn.transaction()?;
for statement in (&V2_STATEMENTS).iter() {
tx.execute(statement, &[])?;
}
let bootstrap_partition_map = bootstrap::bootstrap_partition_map();
// TODO: think more carefully about allocating new parts and bitmasking part ranges.
// TODO: install these using bootstrap assertions. It's tricky because the part ranges are implicit.
// TODO: one insert, chunk into 999/3 sections, for safety.
for (part, partition) in bootstrap_partition_map.iter() {
// TODO: Convert "keyword" part to SQL using Value conversion.
tx.execute("INSERT INTO parts VALUES (?, ?, ?)", &[part, &partition.start, &partition.index])?;
}
let bootstrap_db = DB::new(bootstrap_partition_map, bootstrap::bootstrap_schema());
bootstrap_db.transact_internal(&tx, &bootstrap::bootstrap_entities()[..])?;
set_user_version(&tx, CURRENT_VERSION)?;
let user_version = get_user_version(&tx)?;
// TODO: use the drop semantics to do this automagically?
tx.commit()?;
Ok(user_version)
}
// (def v2-statements v1-statements)
// (defn create-temp-tx-lookup-statement [table-name]
// // n.b., v0/value_type_tag0 can be NULL, in which case we look up v from datoms;
// // and the datom columns are NULL into the LEFT JOIN fills them in.
// // The table-name is not escaped in any way, in order to allow r#"temp.dotted" names.
// // TODO: update comment about sv.
// [(str r#"CREATE TABLE IF NOT EXISTS r#" table-name
// r#" (e0 INTEGER NOT NULL, a0 SMALLINT NOT NULL, v0 BLOB NOT NULL, tx0 INTEGER NOT NULL, added0 TINYINT NOT NULL,
// value_type_tag0 SMALLINT NOT NULL,
// index_avet0 TINYINT, index_vaet0 TINYINT,
// index_fulltext0 TINYINT,
// unique_value0 TINYINT,
// sv BLOB,
// svalue_type_tag SMALLINT,
// rid INTEGER,
// e INTEGER, a SMALLINT, v BLOB, tx INTEGER, value_type_tag SMALLINT)")])
// (defn create-temp-tx-lookup-eavt-statement [idx-name table-name]
// // Note that the consuming code creates and drops the indexes
// // manually, which makes insertion slightly faster.
// // This index prevents overlapping transactions.
// // The idx-name and table-name are not escaped in any way, in order
// // to allow r#"temp.dotted" names.
// // TODO: drop added0?
// [(str r#"CREATE UNIQUE INDEX IF NOT EXISTS r#"#,
// idx-name
// r#" ON r#"#,
// table-name
// r#" (e0, a0, v0, added0, value_type_tag0) WHERE sv IS NOT NULL")])
// (defn <create-current-version
// [db bootstrapper]
// (println r#"Creating database at" current-version)
// (s/in-transaction!
// db
// #(go-pair
// (doseq [statement v2-statements]
// (try
// (<? (s/execute! db [statement]))
// (catch #?(:clj Throwable :cljs js/Error) e
// (throw (ex-info r#"Failed to execute statement" {:statement statement} e)))))
// (<? (bootstrapper db 0))
// (<? (s/set-user-version db current-version))
// [0 (<? (s/get-user-version db))])))
// (defn <update-from-version
// [db from-version bootstrapper]
// {:pre [(> from-version 0)]} // Or we'd create-current-version instead.
// {:pre [(< from-version current-version)]} // Or we wouldn't need to update-from-version.
// (println r#"Upgrading database from" from-version r#"to" current-version)
// (s/in-transaction!
// db
// #(go-pair
// // We must only be migrating from v1 to v2.
// (let [statement r#"UPDATE parts SET idx = idx + 2 WHERE part = ?"]
// (try
// (<? (s/execute!
// db
// [statement :db.part/db]))
// (catch #?(:clj Throwable :cljs js/Error) e
// (throw (ex-info r#"Failed to execute statement" {:statement statement} e)))))
// (<? (bootstrapper db from-version))
// (<? (s/set-user-version db current-version))
// [from-version (<? (s/get-user-version db))])))
// (defn <ensure-current-version
// r#"Returns a pair: [previous-version current-version]."#,
// [db bootstrapper]
// (go-pair
// (let [v (<? (s/get-user-version db))]
// (cond
// (= v current-version)
// [v v]
// (= v 0)
// (<? (<create-current-version db bootstrapper))
// (< v current-version)
// (<? (<update-from-version db v bootstrapper))))))
// */
pub fn update_from_version(conn: &mut rusqlite::Connection, current_version: i32) -> Result<i32> {
if current_version < 0 || CURRENT_VERSION <= current_version {
bail!(ErrorKind::BadSQLiteStoreVersion(current_version))
}
let tx = conn.transaction()?;
// TODO: actually implement upgrade.
set_user_version(&tx, CURRENT_VERSION)?;
let user_version = get_user_version(&tx)?;
// TODO: use the drop semantics to do this automagically?
tx.commit()?;
Ok(user_version)
}
pub fn ensure_current_version(conn: &mut rusqlite::Connection) -> Result<i32> {
let user_version = get_user_version(&conn)?;
match user_version {
CURRENT_VERSION => Ok(user_version),
0 => create_current_version(conn),
v => update_from_version(conn, v),
}
}
impl TypedValue {
/// Given a SQLite `value` and a `value_type_tag`, return the corresponding `TypedValue`.
pub fn from_sql_value_pair(value: rusqlite::types::Value, value_type_tag: &i32) -> Result<TypedValue> {
match (*value_type_tag, value) {
(0, rusqlite::types::Value::Integer(x)) => Ok(TypedValue::Ref(x)),
(1, rusqlite::types::Value::Integer(x)) => Ok(TypedValue::Boolean(0 != x)),
// SQLite distinguishes integral from decimal types, allowing long and double to
// share a tag.
(5, rusqlite::types::Value::Integer(x)) => Ok(TypedValue::Long(x)),
(5, rusqlite::types::Value::Real(x)) => Ok(TypedValue::Double(x.into())),
(10, rusqlite::types::Value::Text(x)) => Ok(TypedValue::String(x)),
(13, rusqlite::types::Value::Text(x)) => Ok(TypedValue::Keyword(x)),
(_, value) => bail!(ErrorKind::BadSQLValuePair(value, *value_type_tag)),
}
}
/// Given an EDN `value`, return a corresponding Mentat `TypedValue`.
///
/// An EDN `Value` does not encode a unique Mentat `ValueType`, so the composition
/// `from_edn_value(first(to_edn_value_pair(...)))` loses information. Additionally, there are
/// EDN values which are not Mentat typed values.
///
/// This function is deterministic.
pub fn from_edn_value(value: &Value) -> Option<TypedValue> {
match value {
&Value::Boolean(x) => Some(TypedValue::Boolean(x)),
&Value::Integer(x) => Some(TypedValue::Long(x)),
&Value::Float(ref x) => Some(TypedValue::Double(x.clone())),
&Value::Text(ref x) => Some(TypedValue::String(x.clone())),
&Value::NamespacedKeyword(ref x) => Some(TypedValue::Keyword(x.to_string())),
_ => None
}
}
/// Return the corresponding SQLite `value` and `value_type_tag` pair.
pub fn to_sql_value_pair<'a>(&'a self) -> (ToSqlOutput<'a>, i32) {
match self {
&TypedValue::Ref(x) => (rusqlite::types::Value::Integer(x).into(), 0),
&TypedValue::Boolean(x) => (rusqlite::types::Value::Integer(if x { 1 } else { 0 }).into(), 1),
// SQLite distinguishes integral from decimal types, allowing long and double to share a tag.
&TypedValue::Long(x) => (rusqlite::types::Value::Integer(x).into(), 5),
&TypedValue::Double(x) => (rusqlite::types::Value::Real(x.into_inner()).into(), 5),
&TypedValue::String(ref x) => (rusqlite::types::ValueRef::Text(x.as_str()).into(), 10),
&TypedValue::Keyword(ref x) => (rusqlite::types::ValueRef::Text(x.as_str()).into(), 13),
}
}
/// Return the corresponding EDN `value` and `value_type` pair.
pub fn to_edn_value_pair(&self) -> (Value, ValueType) {
match self {
&TypedValue::Ref(x) => (Value::Integer(x), ValueType::Ref),
&TypedValue::Boolean(x) => (Value::Boolean(x), ValueType::Boolean),
&TypedValue::Long(x) => (Value::Integer(x), ValueType::Long),
&TypedValue::Double(x) => (Value::Float(x), ValueType::Double),
&TypedValue::String(ref x) => (Value::Text(x.clone()), ValueType::String),
&TypedValue::Keyword(ref x) => (Value::Text(x.clone()), ValueType::Keyword),
}
}
}
/// Read the ident map materialized view from the given SQL store.
pub fn read_ident_map(conn: &rusqlite::Connection) -> Result<IdentMap> {
let mut stmt: rusqlite::Statement = conn.prepare("SELECT ident, entid FROM idents")?;
let m = stmt.query_and_then(&[], |row| -> Result<(String, Entid)> {
Ok((row.get(0), row.get(1)))
})?.collect();
m
}
/// Read the partition map materialized view from the given SQL store.
pub fn read_partition_map(conn: &rusqlite::Connection) -> Result<PartitionMap> {
let mut stmt: rusqlite::Statement = conn.prepare("SELECT part, start, idx FROM parts")?;
let m = stmt.query_and_then(&[], |row| -> Result<(String, Partition)> {
Ok((row.get_checked(0)?, Partition::new(row.get_checked(1)?, row.get_checked(2)?)))
})?.collect();
m
}
/// Read the schema materialized view from the given SQL store.
pub fn read_schema(conn: &rusqlite::Connection, ident_map: &IdentMap) -> Result<Schema> {
let mut stmt: rusqlite::Statement = conn.prepare("SELECT ident, attr, value, value_type_tag FROM schema")?;
let r: Result<Vec<(String, String, TypedValue)>> = stmt.query_and_then(&[], |row| {
// Each row looks like :db/index|:db/valueType|28|0. Observe that 28|0 represents a
// :db.type/ref to entid 28, which needs to be converted to a TypedValue.
// TODO: don't use textual ident and attr; just use entids directly.
let symbolic_ident: String = row.get_checked(0)?;
let symbolic_attr: String = row.get_checked(1)?;
let v: rusqlite::types::Value = row.get_checked(2)?;
let value_type_tag: i32 = row.get_checked(3)?;
let typed_value = TypedValue::from_sql_value_pair(v, &value_type_tag)?;
Ok((symbolic_ident, symbolic_attr, typed_value))
})?.collect();
r.and_then(|triples| Schema::from_ident_map_and_triples(ident_map.clone(), triples))
}
/// Read the materialized views from the given SQL store and return a Mentat `DB` for querying and
/// applying transactions.
pub fn read_db(conn: &rusqlite::Connection) -> Result<DB> {
let partition_map = read_partition_map(conn)?;
let ident_map = read_ident_map(conn)?;
let schema = read_schema(conn, &ident_map)?;
Ok(DB::new(partition_map, schema))
}
impl DB {
/// Do schema-aware typechecking and coercion.
///
/// Either assert that the given value is in the attribute's value set, or (in limited cases)
/// coerce the given value into the attribute's value set.
pub fn to_typed_value(&self, value: &Value, attribute: &Attribute) -> Result<TypedValue> {
// TODO: encapsulate entid-ident-attribute for better error messages.
match TypedValue::from_edn_value(value) {
// We don't recognize this EDN at all. Get out!
None => bail!(ErrorKind::BadEDNValuePair(value.clone(), attribute.value_type.clone())),
Some(typed_value) => match (&attribute.value_type, typed_value) {
// Most types don't coerce at all.
(&ValueType::Boolean, tv @ TypedValue::Boolean(_)) => Ok(tv),
(&ValueType::Long, tv @ TypedValue::Long(_)) => Ok(tv),
(&ValueType::Double, tv @ TypedValue::Double(_)) => Ok(tv),
(&ValueType::String, tv @ TypedValue::String(_)) => Ok(tv),
(&ValueType::Keyword, tv @ TypedValue::Keyword(_)) => Ok(tv),
// Ref coerces a little: we interpret some things depending on the schema as a Ref.
(&ValueType::Ref, TypedValue::Long(x)) => Ok(TypedValue::Ref(x)),
(&ValueType::Ref, TypedValue::Keyword(ref x)) => self.schema.require_entid(&x.to_string()).map(|&entid| TypedValue::Ref(entid)),
// Otherwise, we have a type mismatch.
(value_type, _) => bail!(ErrorKind::BadEDNValuePair(value.clone(), value_type.clone())),
}
}
}
// TODO: move this to the transactor layer.
pub fn transact_internal(&self, conn: &rusqlite::Connection, entities: &[Entity]) -> Result<()>{
// TODO: manage :db/tx, write :db/txInstant.
let tx = 1;
let r: Vec<Result<()>> = entities.into_iter().map(|entity: &Entity| -> Result<()> {
match *entity {
Entity::Add {
e: entmod::EntidOrLookupRef::Entid(entmod::Entid::Ident(ref e_)),
a: entmod::Entid::Ident(ref a_),
v: entmod::ValueOrLookupRef::Value(ref v_),
tx: _ } => {
// TODO: prepare and cache all these statements outside the transaction loop.
// XXX: Error types.
let mut stmt: rusqlite::Statement = conn.prepare("INSERT INTO datoms(e, a, v, tx, value_type_tag, index_avet, index_vaet, index_fulltext, unique_value) VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?)")?;
let e: i64 = *self.schema.require_entid(&e_.to_string())?;
let a: i64 = *self.schema.require_entid(&a_.to_string())?;
let attribute: &Attribute = self.schema.require_attribute_for_entid(&a)?;
// This is our chance to do schema-aware typechecking: to either assert that the
// given value is in the attribute's value set, or (in limited cases) to coerce
// the value into the attribute's value set.
let typed_value: TypedValue = self.to_typed_value(v_, &attribute)?;
// Now we can represent the typed value as an SQL value.
let (value, value_type_tag): (ToSqlOutput, i32) = typed_value.to_sql_value_pair();
// Fun times, type signatures.
let values: [&ToSql; 9] = [&e, &a, &value, &tx, &value_type_tag, &attribute.index, to_bool_ref(attribute.value_type == ValueType::Ref), &attribute.fulltext, &attribute.unique_value];
stmt.insert(&values[..])?;
Ok(())
},
// TODO: find a better error type for this.
_ => panic!(format!("Transacting entity not yet supported: {:?}", entity))
}
}).collect();
let x: Result<Vec<()>> = r.into_iter().collect();
x.map(|_| ())
}
}
#[cfg(test)]
mod tests {
use super::*;
use bootstrap;
use debug;
use rusqlite;
use types::*;
#[test]
fn test_open_current_version() {
// TODO: figure out how to reference the fixtures directory for real. For now, assume we're
// executing `cargo test` in `db/`.
let conn = rusqlite::Connection::open("../fixtures/v2empty.db").unwrap();
// assert_eq!(ensure_current_version(&mut conn).unwrap(), CURRENT_VERSION);
// TODO: write :db/txInstant, bump :db.part/tx.
// let partition_map = read_partition_map(&conn).unwrap();
// assert_eq!(partition_map, bootstrap::bootstrap_partition_map());
let ident_map = read_ident_map(&conn).unwrap();
assert_eq!(ident_map, bootstrap::bootstrap_ident_map());
let schema = read_schema(&conn, &ident_map).unwrap();
assert_eq!(schema, bootstrap::bootstrap_schema()); // Schema::default());
let db = read_db(&conn).unwrap();
let datoms = debug::datoms_after(&conn, &db, &0).unwrap();
assert_eq!(datoms.len(), 89); // The 89th is the :db/txInstant value.
// // TODO: fewer magic numbers!
// assert_eq!(debug::datoms_after(&conn, &db, &0x10000001).unwrap(), vec![]);
}
#[test]
fn test_create_current_version() {
// // assert_eq!(bootstrap_schema().unwrap(), Schema::default());
// // Ignore result.
// use std::fs;
// let _ = fs::remove_file("/Users/nalexander/Mozilla/mentat/test.db");
// let mut conn = rusqlite::Connection::open("file:///Users/nalexander/Mozilla/mentat/test.db").unwrap();
let mut conn = new_connection();
assert_eq!(ensure_current_version(&mut conn).unwrap(), CURRENT_VERSION);
let bootstrap_db = DB::new(bootstrap::bootstrap_partition_map(), bootstrap::bootstrap_schema());
// TODO: write materialized view of bootstrapped schema to SQL store.
// let db = read_db(&conn).unwrap();
// assert_eq!(db, bootstrap_db);
let datoms = debug::datoms_after(&conn, &bootstrap_db, &0).unwrap();
assert_eq!(datoms.len(), 88);
}
}

66
db/src/debug.rs Normal file
View file

@ -0,0 +1,66 @@
// Copyright 2016 Mozilla
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use
// this file except in compliance with the License. You may obtain a copy of the
// License at http://www.apache.org/licenses/LICENSE-2.0
// Unless required by applicable law or agreed to in writing, software distributed
// under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
// CONDITIONS OF ANY KIND, either express or implied. See the License for the
// specific language governing permissions and limitations under the License.
#![allow(dead_code)]
/// Low-level functions for testing.
use rusqlite;
use {to_namespaced_keyword};
use edn::types::{Value};
use mentat_tx::entities::{Entid};
use types::{DB, TypedValue};
use errors::Result;
/// Represents an assertion (*datom*) in the store.
#[derive(Clone,Debug,Eq,Hash,Ord,PartialOrd,PartialEq)]
pub struct Datom {
// TODO: generalize this.
e: Entid,
a: Entid,
v: Value,
tx: Option<i64>,
}
/// Return the complete set of datoms in the store, ordered by (e, a, v).
pub fn datoms(conn: &rusqlite::Connection, db: &DB) -> Result<Vec<Datom>> {
// TODO: fewer magic numbers!
datoms_after(conn, db, &0x10000000)
}
/// Return the set of datoms in the store with transaction ID strictly
/// greater than the given `tx`, ordered by (tx, e, a, v).
pub fn datoms_after(conn: &rusqlite::Connection, db: &DB, tx: &i32) -> Result<Vec<Datom>> {
let mut stmt: rusqlite::Statement = conn.prepare("SELECT e, a, v, value_type_tag FROM datoms WHERE tx > ? ORDER BY tx, e, a, v")?;
// Convert numeric entid to entity Entid.
let to_entid = |x| {
db.schema.get_ident(&x).and_then(|y| to_namespaced_keyword(&y)).map(Entid::Ident).unwrap_or(Entid::Entid(x))
};
let datoms = stmt.query_and_then(&[tx], |row| {
let e: i64 = row.get_checked(0)?;
let a: i64 = row.get_checked(1)?;
let v: rusqlite::types::Value = row.get_checked(2)?;
let value_type_tag: i32 = row.get_checked(3)?;
let typed_value = TypedValue::from_sql_value_pair(v, &value_type_tag)?;
let (value, _) = typed_value.to_edn_value_pair();
Ok(Datom {
e: to_entid(e),
a: to_entid(a),
v: value,
tx: None,
})
})?.collect();
datoms
}

58
db/src/entids.rs Normal file
View file

@ -0,0 +1,58 @@
// Copyright 2016 Mozilla
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use
// this file except in compliance with the License. You may obtain a copy of the
// License at http://www.apache.org/licenses/LICENSE-2.0
// Unless required by applicable law or agreed to in writing, software distributed
// under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
// CONDITIONS OF ANY KIND, either express or implied. See the License for the
// specific language governing permissions and limitations under the License.
#![allow(dead_code)]
/// Literal `Entid` values in the the "db" namespace.
///
/// Used through-out the transactor to match core DB constructs.
use types::{Entid};
// Added in SQL schema v1.
pub const DB_IDENT: Entid = 1;
pub const DB_PART_DB: Entid = 2;
pub const DB_TX_INSTANT: Entid = 3;
pub const DB_INSTALL_PARTITION: Entid = 4;
pub const DB_INSTALL_VALUETYPE: Entid = 5;
pub const DB_INSTALL_ATTRIBUTE: Entid = 6;
pub const DB_VALUE_TYPE: Entid = 7;
pub const DB_CARDINALITY: Entid = 8;
pub const DB_UNIQUE: Entid = 9;
pub const DB_IS_COMPONENT: Entid = 10;
pub const DB_INDEX: Entid = 11;
pub const DB_FULLTEXT: Entid = 12;
pub const DB_NO_HISTORY: Entid = 13;
pub const DB_ADD: Entid = 14;
pub const DB_RETRACT: Entid = 15;
pub const DB_PART_USER: Entid = 16;
pub const DB_PART_TX: Entid = 17;
pub const DB_EXCISE: Entid = 18;
pub const DB_EXCISE_ATTRS: Entid = 19;
pub const DB_EXCISE_BEFORE_T: Entid = 20;
pub const DB_EXCISE_BEFORE: Entid = 21;
pub const DB_ALTER_ATTRIBUTE: Entid = 22;
pub const DB_TYPE_REF: Entid = 23;
pub const DB_TYPE_KEYWORD: Entid = 24;
pub const DB_TYPE_LONG: Entid = 25;
pub const DB_TYPE_DOUBLE: Entid = 26;
pub const DB_TYPE_STRING: Entid = 27;
pub const DB_TYPE_BOOLEAN: Entid = 28;
pub const DB_TYPE_INSTANT: Entid = 29;
pub const DB_TYPE_BYTES: Entid = 30;
pub const DB_CARDINALITY_ONE: Entid = 31;
pub const DB_CARDINALITY_MANY: Entid = 32;
pub const DB_UNIQUE_VALUE: Entid = 33;
pub const DB_UNIQUE_IDENTITY: Entid = 34;
pub const DB_DOC: Entid = 35;
// Added in SQL schema v2.
pub const DB_SCHEMA_VERSION: Entid = 36;
pub const DB_SCHEMA_ATTRIBUTE: Entid = 37;

79
db/src/errors.rs Normal file
View file

@ -0,0 +1,79 @@
// Copyright 2016 Mozilla
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use
// this file except in compliance with the License. You may obtain a copy of the
// License at http://www.apache.org/licenses/LICENSE-2.0
// Unless required by applicable law or agreed to in writing, software distributed
// under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
// CONDITIONS OF ANY KIND, either express or implied. See the License for the
// specific language governing permissions and limitations under the License.
#![allow(dead_code)]
use edn;
use rusqlite;
use types::{Entid, ValueType};
error_chain! {
types {
Error, ErrorKind, ResultExt, Result;
}
foreign_links {
Rusqlite(rusqlite::Error);
}
errors {
/// We're just not done yet. Message that the feature is recognized but not yet
/// implemented.
NotYetImplemented(t: String) {
description("not yet implemented")
display("not yet implemented: {}", t)
}
/// We've been given an EDN value that isn't the correct Mentat type.
BadEDNValuePair(value: edn::types::Value, value_type: ValueType) {
description("EDN value is not the expected Mentat value type")
display("EDN value '{:?}' is not the expected Mentat value type {:?}", value, value_type)
}
/// We've got corrupt data in the SQL store: a value and value_type_tag don't line up.
BadSQLValuePair(value: rusqlite::types::Value, value_type_tag: i32) {
description("bad SQL (value_type_tag, value) pair")
display("bad SQL (value_type_tag, value) pair: ({}, {:?})", value_type_tag, value.data_type())
}
/// The SQLite store user_version isn't recognized. This could be an old version of Mentat
/// trying to open a newer version SQLite store; or it could be a corrupt file; or ...
BadSQLiteStoreVersion(version: i32) {
description("bad SQL store user_version")
display("bad SQL store user_version: {}", version)
}
/// A bootstrap definition couldn't be parsed or installed. This is a programmer error, not
/// a runtime error.
BadBootstrapDefinition(t: String) {
description("bad bootstrap definition")
display("bad bootstrap definition: '{}'", t)
}
/// A schema assertion couldn't be parsed.
BadSchemaAssertion(t: String) {
description("bad schema assertion")
display("bad schema assertion: '{}'", t)
}
/// An ident->entid mapping failed.
UnrecognizedIdent(ident: String) {
description("no entid found for ident")
display("no entid found for ident: '{}'", ident)
}
/// An entid->ident mapping failed.
UnrecognizedEntid(entid: Entid) {
description("no ident found for entid")
display("no ident found for entid: '{}'", entid)
}
}
}

45
db/src/lib.rs Normal file
View file

@ -0,0 +1,45 @@
// Copyright 2016 Mozilla
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use
// this file except in compliance with the License. You may obtain a copy of the
// License at http://www.apache.org/licenses/LICENSE-2.0
// Unless required by applicable law or agreed to in writing, software distributed
// under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
// CONDITIONS OF ANY KIND, either express or implied. See the License for the
// specific language governing permissions and limitations under the License.
#[macro_use]
extern crate error_chain;
#[macro_use]
extern crate lazy_static;
extern crate num;
extern crate ordered_float;
extern crate rusqlite;
extern crate edn;
extern crate mentat_tx;
extern crate mentat_tx_parser;
pub use errors::*;
pub use schema::*;
pub use types::*;
pub mod db;
mod bootstrap;
mod debug;
mod entids;
mod errors;
mod schema;
mod types;
mod values;
use edn::symbols;
pub fn to_namespaced_keyword(s: &str) -> Option<symbols::NamespacedKeyword> {
let splits = [':', '/'];
let mut i = s.split(&splits[..]);
match (i.next(), i.next(), i.next(), i.next()) {
(Some(""), Some(namespace), Some(name), None) => Some(symbols::NamespacedKeyword::new(namespace, name)),
_ => None
}
}

164
db/src/schema.rs Normal file
View file

@ -0,0 +1,164 @@
// Copyright 2016 Mozilla
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use
// this file except in compliance with the License. You may obtain a copy of the
// License at http://www.apache.org/licenses/LICENSE-2.0
// Unless required by applicable law or agreed to in writing, software distributed
// under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
// CONDITIONS OF ANY KIND, either express or implied. See the License for the
// specific language governing permissions and limitations under the License.
#![allow(dead_code)]
use entids;
use errors::*;
use types::{Attribute, Entid, EntidMap, IdentMap, Schema, SchemaMap, TypedValue, ValueType};
/// Return `Ok(())` if `schema_map` defines a valid Mentat schema.
fn validate_schema_map(entid_map: &EntidMap, schema_map: &SchemaMap) -> Result<()> {
for (entid, attribute) in schema_map {
let ident = entid_map.get(entid).ok_or(ErrorKind::BadSchemaAssertion(format!("Could not get ident for entid: {}", entid)))?;
if attribute.unique_identity && !attribute.unique_value {
bail!(ErrorKind::BadSchemaAssertion(format!(":db/unique :db/unique_identity without :db/unique :db/unique_value for entid: {}", ident)))
}
if attribute.fulltext && attribute.value_type != ValueType::String {
bail!(ErrorKind::BadSchemaAssertion(format!(":db/fulltext true without :db/valueType :db.type/string for entid: {}", ident)))
}
if attribute.component && attribute.value_type != ValueType::Ref {
bail!(ErrorKind::BadSchemaAssertion(format!(":db/isComponent true without :db/valueType :db.type/ref for entid: {}", ident)))
}
// TODO: consider warning if we have :db/index true for :db/valueType :db.type/string,
// since this may be inefficient. More generally, we should try to drive complex
// :db/valueType (string, uri, json in the future) users to opt-in to some hash-indexing
// scheme, as discussed in https://github.com/mozilla/mentat/issues/69.
}
Ok(())
}
impl Schema {
pub fn get_ident(&self, x: &Entid) -> Option<&String> {
self.entid_map.get(x)
}
pub fn get_entid(&self, x: &String) -> Option<&Entid> {
self.ident_map.get(x)
}
pub fn attribute_for_entid(&self, x: &Entid) -> Option<&Attribute> {
self.schema_map.get(x)
}
pub fn require_ident(&self, entid: &Entid) -> Result<&String> {
self.get_ident(&entid).ok_or(ErrorKind::UnrecognizedEntid(*entid).into())
}
pub fn require_entid(&self, ident: &String) -> Result<&Entid> {
self.get_entid(&ident).ok_or(ErrorKind::UnrecognizedIdent(ident.clone()).into())
}
pub fn require_attribute_for_entid(&self, entid: &Entid) -> Result<&Attribute> {
self.attribute_for_entid(entid).ok_or(ErrorKind::UnrecognizedEntid(*entid).into())
}
/// Create a valid `Schema` from the constituent maps.
pub fn from(ident_map: IdentMap, schema_map: SchemaMap) -> Result<Schema> {
let entid_map: EntidMap = ident_map.iter().map(|(k, v)| (v.clone(), k.clone())).collect();
validate_schema_map(&entid_map, &schema_map)?;
Ok(Schema {
ident_map: ident_map,
entid_map: entid_map,
schema_map: schema_map,
})
}
/// Turn vec![(String(:ident), String(:key), TypedValue(:value)), ...] into a Mentat `Schema`.
pub fn from_ident_map_and_triples<U>(ident_map: IdentMap, assertions: U) -> Result<Schema>
where U: IntoIterator<Item=(String, String, TypedValue)>{
let mut schema_map = SchemaMap::new();
for (ref symbolic_ident, ref symbolic_attr, ref value) in assertions.into_iter() {
let ident: i64 = *ident_map.get(symbolic_ident).ok_or(ErrorKind::UnrecognizedIdent(symbolic_ident.clone()))?;
let attr: i64 = *ident_map.get(symbolic_attr).ok_or(ErrorKind::UnrecognizedIdent(symbolic_attr.clone()))?;
let attributes = schema_map.entry(ident).or_insert(Attribute::default());
// TODO: improve error messages throughout.
match attr {
entids::DB_VALUE_TYPE => {
match *value {
TypedValue::Ref(entids::DB_TYPE_REF) => { attributes.value_type = ValueType::Ref; },
TypedValue::Ref(entids::DB_TYPE_BOOLEAN) => { attributes.value_type = ValueType::Boolean; },
TypedValue::Ref(entids::DB_TYPE_LONG) => { attributes.value_type = ValueType::Long; },
TypedValue::Ref(entids::DB_TYPE_STRING) => { attributes.value_type = ValueType::String; },
TypedValue::Ref(entids::DB_TYPE_KEYWORD) => { attributes.value_type = ValueType::Keyword; },
_ => bail!(ErrorKind::BadSchemaAssertion(format!("Expected [... :db/valueType :db.type/*] but got [... :db/valueType {:?}] for ident '{}' and attribute '{}'", value, ident, attr)))
}
},
entids::DB_CARDINALITY => {
match *value {
TypedValue::Ref(entids::DB_CARDINALITY_MANY) => { attributes.multival = true; },
TypedValue::Ref(entids::DB_CARDINALITY_ONE) => { attributes.multival = false; },
_ => bail!(ErrorKind::BadSchemaAssertion(format!("Expected [... :db/cardinality :db.cardinality/many|:db.cardinality/one] but got [... :db/cardinality {:?}]", value)))
}
},
entids::DB_UNIQUE => {
match *value {
TypedValue::Ref(entids::DB_UNIQUE_VALUE) => { attributes.unique_value = true; },
TypedValue::Ref(entids::DB_UNIQUE_IDENTITY) => {
attributes.unique_value = true;
attributes.unique_identity = true;
},
_ => bail!(ErrorKind::BadSchemaAssertion(format!("Expected [... :db/unique :db.unique/value|:db.unique/identity] but got [... :db/unique {:?}]", value)))
}
},
entids::DB_INDEX => {
match *value {
TypedValue::Boolean(x) => { attributes.index = x },
_ => bail!(ErrorKind::BadSchemaAssertion(format!("Expected [... :db/index true|false] but got [... :db/index {:?}]", value)))
}
},
entids::DB_FULLTEXT => {
match *value {
TypedValue::Boolean(x) => {
attributes.fulltext = x;
if attributes.fulltext {
attributes.index = true;
}
},
_ => bail!(ErrorKind::BadSchemaAssertion(format!("Expected [... :db/fulltext true|false] but got [... :db/fulltext {:?}]", value)))
}
},
entids::DB_IS_COMPONENT => {
match *value {
TypedValue::Boolean(x) => { attributes.component = x },
_ => bail!(ErrorKind::BadSchemaAssertion(format!("Expected [... :db/isComponent true|false] but got [... :db/isComponent {:?}]", value)))
}
},
entids::DB_DOC => {
// Nothing for now.
},
entids::DB_IDENT => {
// Nothing for now.
},
entids::DB_INSTALL_ATTRIBUTE => {
// Nothing for now.
},
_ => {
bail!(ErrorKind::BadSchemaAssertion(format!("Do not recognize attribute '{}' for ident '{}'", attr, ident)))
}
}
};
Schema::from(ident_map.clone(), schema_map)
}
}

200
db/src/types.rs Normal file
View file

@ -0,0 +1,200 @@
// Copyright 2016 Mozilla
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use
// this file except in compliance with the License. You may obtain a copy of the
// License at http://www.apache.org/licenses/LICENSE-2.0
// Unless required by applicable law or agreed to in writing, software distributed
// under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
// CONDITIONS OF ANY KIND, either express or implied. See the License for the
// specific language governing permissions and limitations under the License.
#![allow(dead_code)]
use std::collections::{BTreeMap};
use ordered_float::{OrderedFloat};
/// Core types defining a Mentat knowledge base.
/// Represents one entid in the entid space.
///
/// Per https://www.sqlite.org/datatype3.html (see also http://stackoverflow.com/a/8499544), SQLite
/// stores signed integers up to 64 bits in size. Since u32 is not appropriate for our use case, we
/// use i64 rather than manually truncating u64 to u63 and casting to i64 throughout the codebase.
pub type Entid = i64;
/// The attribute of each Mentat assertion has a :db/valueType constraining the value to a
/// particular set. Mentat recognizes the following :db/valueType values.
#[derive(Clone,Debug,Eq,Hash,Ord,PartialOrd,PartialEq)]
pub enum ValueType {
Ref,
Boolean,
Instant,
Long,
Double,
String,
Keyword,
}
/// Represents a Mentat value in a particular value set.
// TODO: expand to include :db.type/{instant,url,uuid}.
#[derive(Clone,Debug,Eq,Hash,Ord,PartialOrd,PartialEq)]
pub enum TypedValue {
Ref(Entid),
Boolean(bool),
Long(i64),
Double(OrderedFloat<f64>),
// TODO: &str throughout?
String(String),
Keyword(String),
}
impl TypedValue {
pub fn value_type(&self) -> ValueType {
match self {
&TypedValue::Ref(_) => ValueType::Ref,
&TypedValue::Boolean(_) => ValueType::Boolean,
&TypedValue::Long(_) => ValueType::Long,
&TypedValue::Double(_) => ValueType::Double,
&TypedValue::String(_) => ValueType::String,
&TypedValue::Keyword(_) => ValueType::Keyword,
}
}
}
/// Represents one partition of the entid space.
#[derive(Clone,Debug,Eq,Hash,Ord,PartialOrd,PartialEq)]
pub struct Partition {
/// The first entid in the partition.
pub start: i64,
/// The next entid to be allocated in the partition.
pub index: i64,
}
impl Partition {
pub fn new(start: i64, next: i64) -> Partition {
assert!(start <= next, "A partition represents a monotonic increasing sequence of entids.");
Partition { start: start, index: next }
}
}
/// Map partition names to `Partition` instances.
pub type PartitionMap = BTreeMap<String, Partition>;
/// A Mentat schema attribute has a value type and several other flags determining how assertions
/// with the attribute are interpreted.
///
/// TODO: consider packing this into a bitfield or similar.
#[derive(Clone,Debug,Eq,Hash,Ord,PartialOrd,PartialEq)]
pub struct Attribute {
/// The associated value type, i.e., `:db/valueType`?
pub value_type: ValueType,
/// `true` if this attribute is multi-valued, i.e., it is `:db/cardinality
/// :db.cardinality/many`. `false` if this attribute is single-valued (the default), i.e., it
/// is `:db/cardinality :db.cardinality/one`.
pub multival: bool,
/// `true` if this attribute is unique-value, i.e., it is `:db/unique :db.unique/value`.
///
/// *Unique-value* means that there is at most one assertion with the attribute and a
/// particular value in the datom store.
pub unique_value: bool,
/// `true` if this attribute is unique-identity, i.e., it is `:db/unique :db.unique/identity`.
///
/// Unique-identity attributes always have value type `Ref`.
///
/// *Unique-identity* means that the attribute is *unique-value* and that they can be used in
/// lookup-refs and will automatically upsert where appropriate.
pub unique_identity: bool,
/// `true` if this attribute is automatically indexed, i.e., it is `:db/indexing true`.
pub index: bool,
/// `true` if this attribute is automatically fulltext indexed, i.e., it is `:db/fulltext true`.
///
/// Fulltext attributes always have string values.
pub fulltext: bool,
/// `true` if this attribute is a component, i.e., it is `:db/isComponent true`.
///
/// Component attributes always have value type `Ref`.
///
/// They are used to compose entities from component sub-entities: they are fetched recursively
/// by pull expressions, and they are automatically recursively deleted where appropriate.
pub component: bool,
}
impl Default for Attribute {
fn default() -> Attribute {
Attribute {
// There's no particular reason to favour one value type, so Ref it is.
value_type: ValueType::Ref,
fulltext: false,
index: false,
multival: false,
unique_value: false,
unique_identity: false,
component: false,
}
}
}
/// Map `String` idents (`:db/ident`) to positive integer entids (`1`).
pub type IdentMap = BTreeMap<String, Entid>;
/// Map positive integer entids (`1`) to `String` idents (`:db/ident`).
pub type EntidMap = BTreeMap<Entid, String>;
/// Map attribute entids to `Attribute` instances.
pub type SchemaMap = BTreeMap<i64, Attribute>;
/// Represents a Mentat schema.
///
/// Maintains the mapping between string idents and positive integer entids; and exposes the schema
/// flags associated to a given entid (equivalently, ident).
///
/// TODO: consider a single bi-directional map instead of separate ident->entid and entid->ident
/// maps.
#[derive(Clone,Debug,Default,Eq,Hash,Ord,PartialOrd,PartialEq)]
pub struct Schema {
/// Map entid->ident.
///
/// Invariant: is the inverse map of `ident_map`.
pub entid_map: EntidMap,
/// Map ident->entid.
///
/// Invariant: is the inverse map of `entid_map`.
pub ident_map: IdentMap,
/// Map entid->attribute flags.
///
/// Invariant: key-set is the same as the key-set of `entid_map` (equivalently, the value-set of
/// `ident_map`).
pub schema_map: SchemaMap,
}
/// Represents the metadata required to query from, or apply transactions to, a Mentat store.
///
/// See https://github.com/mozilla/mentat/wiki/Thoughts:-modeling-db-conn-in-Rust.
#[derive(Clone,Debug,Default,Eq,Hash,Ord,PartialOrd,PartialEq)]
pub struct DB {
/// Map partition name->`Partition`.
///
/// TODO: represent partitions as entids.
pub partition_map: PartitionMap,
/// The schema of the store.
pub schema: Schema,
}
impl DB {
pub fn new(partition_map: PartitionMap, schema: Schema) -> DB {
DB {
partition_map: partition_map,
schema: schema
}
}
}

61
db/src/values.rs Normal file
View file

@ -0,0 +1,61 @@
// Copyright 2016 Mozilla
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use
// this file except in compliance with the License. You may obtain a copy of the
// License at http://www.apache.org/licenses/LICENSE-2.0
// Unless required by applicable law or agreed to in writing, software distributed
// under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
// CONDITIONS OF ANY KIND, either express or implied. See the License for the
// specific language governing permissions and limitations under the License.
#![allow(dead_code)]
/// Literal `Value` instances in the the "db" namespace.
///
/// Used through-out the transactor to match core DB constructs.
use edn::types::Value;
use edn::symbols;
/// Declare a lazy static `ident` of type `Value::NamespacedKeyword` with the given `namespace` and
/// `name`.
///
/// It may look surprising that we declare a new `lazy_static!` block rather than including
/// invocations inside an existing `lazy_static!` block. The latter cannot be done, since macros
/// are expanded outside-in. Looking at the `lazy_static!` source suggests that there is no harm in
/// repeating that macro, since internally a multi-`static` block is expanded into many
/// single-`static` blocks.
///
/// TODO: take just ":db.part/db" and define DB_PART_DB using "db.part" and "db".
macro_rules! lazy_static_namespaced_keyword_value (
($tag:ident, $namespace:expr, $name:expr) => (
lazy_static! {
pub static ref $tag: Value = {
Value::NamespacedKeyword(symbols::NamespacedKeyword::new($namespace, $name))
};
}
)
);
lazy_static_namespaced_keyword_value!(DB_ADD, "db", "add");
lazy_static_namespaced_keyword_value!(DB_ALTER_ATTRIBUTE, "db.alter", "attribute");
lazy_static_namespaced_keyword_value!(DB_CARDINALITY, "db", "cardinality");
lazy_static_namespaced_keyword_value!(DB_CARDINALITY_MANY, "db.cardinality", "many");
lazy_static_namespaced_keyword_value!(DB_CARDINALITY_ONE, "db.cardinality", "one");
lazy_static_namespaced_keyword_value!(DB_IDENT, "db", "ident");
lazy_static_namespaced_keyword_value!(DB_INSTALL_ATTRIBUTE, "db.install", "attribute");
lazy_static_namespaced_keyword_value!(DB_PART_DB, "db.part", "db");
lazy_static_namespaced_keyword_value!(DB_RETRACT, "db", "retract");
lazy_static_namespaced_keyword_value!(DB_TYPE_BOOLEAN, "db.type", "boolean");
lazy_static_namespaced_keyword_value!(DB_TYPE_DOUBLE, "db.type", "double");
lazy_static_namespaced_keyword_value!(DB_TYPE_INSTANT, "db.type", "instant");
lazy_static_namespaced_keyword_value!(DB_TYPE_KEYWORD, "db.type", "keyword");
lazy_static_namespaced_keyword_value!(DB_TYPE_LONG, "db.type", "long");
lazy_static_namespaced_keyword_value!(DB_TYPE_REF, "db.type", "ref");
lazy_static_namespaced_keyword_value!(DB_TYPE_STRING, "db.type", "string");
lazy_static_namespaced_keyword_value!(DB_TYPE_URI, "db.type", "uri");
lazy_static_namespaced_keyword_value!(DB_TYPE_UUID, "db.type", "uuid");
lazy_static_namespaced_keyword_value!(DB_UNIQUE, "db", "unique");
lazy_static_namespaced_keyword_value!(DB_UNIQUE_IDENTITY, "db.unique", "identity");
lazy_static_namespaced_keyword_value!(DB_UNIQUE_VALUE, "db.unique", "value");
lazy_static_namespaced_keyword_value!(DB_VALUE_TYPE, "db", "valueType");

View file

@ -15,6 +15,7 @@ extern crate num;
pub mod symbols;
pub mod types;
pub mod utils;
pub mod parse {
include!(concat!(env!("OUT_DIR"), "/edn.rs"));

31
edn/src/utils.rs Normal file
View file

@ -0,0 +1,31 @@
// Copyright 2016 Mozilla
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use
// this file except in compliance with the License. You may obtain a copy of the
// License at http://www.apache.org/licenses/LICENSE-2.0
// Unless required by applicable law or agreed to in writing, software distributed
// under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR
// CONDITIONS OF ANY KIND, either express or implied. See the License for the
// specific language governing permissions and limitations under the License.
#![allow(dead_code)]
use types::Value;
/// Merge the EDN `Value::Map` instance `right` into `left`. Returns `None` if either `left` or
/// `right` is not a `Value::Map`.
///
/// Keys present in `right` overwrite keys present in `left`. See also
/// https://clojuredocs.org/clojure.core/merge.
///
/// TODO: implement `merge` for [Value], following the `concat`/`SliceConcatExt` pattern.
pub fn merge(left: &Value, right: &Value) -> Option<Value> {
match (left, right) {
(&Value::Map(ref l), &Value::Map(ref r)) => {
let mut result = l.clone();
result.extend(r.clone().into_iter());
Some(Value::Map(result))
}
_ => None
}
}

View file

@ -21,6 +21,7 @@ use edn::symbols;
use edn::types::Value;
use edn::types::Value::*;
use edn::parse::*;
use edn::utils;
// Helper for making wrapped keywords with a namespace.
fn k_ns(ns: &str, name: &str) -> Value {
@ -850,6 +851,42 @@ fn test_spurious_commas() {
assert_eq!(value("[3,,]"), result);
}
#[test]
fn test_utils_merge() {
// Take BTreeMap instances, wrap into Value::Map instances.
let test = |left: &BTreeMap<Value, Value>, right: &BTreeMap<Value, Value>, expected: &BTreeMap<Value, Value>| {
let l = Value::Map(left.clone());
let r = Value::Map(right.clone());
let result = utils::merge(&l, &r).unwrap();
let e = Value::Map(expected.clone());
assert_eq!(result, e);
};
let mut left = BTreeMap::new();
left.insert(Value::Integer(1), Value::Integer(1));
left.insert(Value::Text("a".into()), Value::Text("a".into()));
let mut right = BTreeMap::new();
right.insert(Value::Integer(2), Value::Integer(2));
right.insert(Value::Text("a".into()), Value::Text("b".into()));
let mut expected = BTreeMap::new();
expected.insert(Value::Integer(1), Value::Integer(1));
expected.insert(Value::Integer(2), Value::Integer(2));
expected.insert(Value::Text("a".into()), Value::Text("b".into()));
let mut expected = BTreeMap::new();
expected.insert(Value::Integer(1), Value::Integer(1));
expected.insert(Value::Integer(2), Value::Integer(2));
expected.insert(Value::Text("a".into()), Value::Text("b".into()));
test(&left, &right, &expected);
let mut expected = BTreeMap::new();
expected.insert(Value::Integer(1), Value::Integer(1));
expected.insert(Value::Integer(2), Value::Integer(2));
expected.insert(Value::Text("a".into()), Value::Text("a".into()));
test(&right, &left, &expected);
}
/*
// Handy templates for creating test cases follow:

View file

@ -15,7 +15,7 @@ extern crate edn;
use self::edn::types::Value;
use self::edn::symbols::NamespacedKeyword;
#[derive(Clone, Debug, PartialEq)]
#[derive(Clone,Debug,Eq,Hash,Ord,PartialOrd,PartialEq)]
pub enum Entid {
Entid(i64),
Ident(NamespacedKeyword),