implement the Storage<DB> change

This commit is contained in:
Niko Matsakis 2020-07-02 10:31:02 +00:00
parent 4bf46f5f24
commit 67687376ed
22 changed files with 152 additions and 254 deletions

View file

@ -13,6 +13,7 @@ pub(crate) fn database(args: TokenStream, input: TokenStream) -> TokenStream {
let query_groups = &args.query_groups;
let database_name = &input.ident;
let visibility = &input.vis;
let db_storage_field = quote! { storage };
let mut output = proc_macro2::TokenStream::new();
output.extend(quote! { #input });
@ -63,8 +64,7 @@ pub(crate) fn database(args: TokenStream, input: TokenStream) -> TokenStream {
has_group_impls.extend(quote! {
impl salsa::plumbing::HasQueryGroup<#group_path> for #database_name {
fn group_storage(db: &Self) -> &#group_storage {
let runtime = salsa::Database::salsa_runtime(db);
&runtime.storage().#group_name_snake
&db.#db_storage_field.query_store().#group_name_snake
}
}
});
@ -134,6 +134,14 @@ pub(crate) fn database(args: TokenStream, input: TokenStream) -> TokenStream {
}
output.extend(quote! {
impl salsa::plumbing::DatabaseOps for #database_name {
fn ops_salsa_runtime(&self) -> &salsa::Runtime {
self.#db_storage_field.salsa_runtime()
}
fn ops_salsa_runtime_mut(&mut self) -> &mut salsa::Runtime {
self.#db_storage_field.salsa_runtime_mut()
}
fn fmt_index(
&self,
input: salsa::DatabaseKeyIndex,

View file

@ -16,16 +16,8 @@ use crate::interner::InternerDatabase;
#[salsa::database(InternerDatabase, CompilerDatabase)]
#[derive(Default)]
pub struct DatabaseImpl {
runtime: salsa::Runtime<DatabaseImpl>,
storage: salsa::Storage<DatabaseImpl>,
}
/// This impl tells salsa where to find the salsa runtime.
impl salsa::Database for DatabaseImpl {
fn salsa_runtime(&self) -> &salsa::Runtime<Self> {
&self.runtime
}
fn salsa_runtime_mut(&mut self) -> &mut salsa::Runtime<Self> {
&mut self.runtime
}
}
impl salsa::Database for DatabaseImpl {}

View file

@ -77,19 +77,10 @@ fn length(db: &impl HelloWorld, (): ()) -> usize {
#[salsa::database(HelloWorldStorage)]
#[derive(Default)]
struct DatabaseStruct {
runtime: salsa::Runtime<DatabaseStruct>,
storage: salsa::Storage<Self>,
}
// Tell salsa where to find the runtime in your context.
impl salsa::Database for DatabaseStruct {
fn salsa_runtime(&self) -> &salsa::Runtime<Self> {
&self.runtime
}
fn salsa_runtime_mut(&mut self) -> &mut salsa::Runtime<Self> {
&mut self.runtime
}
}
impl salsa::Database for DatabaseStruct {}
// ANCHOR_END:database
// This shows how to use a query.

View file

@ -326,7 +326,7 @@ where
&self,
db: &DB,
state: StateGuard,
runtime: &Runtime<DB>,
runtime: &Runtime,
revision_now: Revision,
) -> ProbeState<StampedValue<Q::Value>, DatabaseKeyIndex, StateGuard>
where
@ -720,7 +720,7 @@ where
fn register_with_in_progress_thread(
&self,
_db: &DB,
runtime: &Runtime<DB>,
runtime: &Runtime,
other_id: RuntimeId,
waiting: &Mutex<SmallVec<[Promise<WaitResult<Q::Value, DatabaseKeyIndex>>; 2]>>,
) -> Result<BlockingFuture<WaitResult<Q::Value, DatabaseKeyIndex>>, CycleDetected> {
@ -772,7 +772,7 @@ where
database_key_index: DatabaseKeyIndex,
slot: &'me Slot<DB, Q, MP>,
memo: Option<Memo<DB, Q>>,
runtime: &'me Runtime<DB>,
runtime: &'me Runtime,
}
impl<'me, DB, Q, MP> PanicGuard<'me, DB, Q, MP>
@ -785,7 +785,7 @@ where
database_key_index: DatabaseKeyIndex,
slot: &'me Slot<DB, Q, MP>,
memo: Option<Memo<DB, Q>>,
runtime: &'me Runtime<DB>,
runtime: &'me Runtime,
) -> Self {
Self {
database_key_index,

View file

@ -7,6 +7,7 @@
//! re-execute the derived queries and it will try to re-use results
//! from previous invocations as appropriate.
mod blocking_future;
mod derived;
mod doctest;
mod durability;
@ -16,7 +17,7 @@ mod interned;
mod lru;
mod revision;
mod runtime;
mod blocking_future;
mod storage;
pub mod debug;
/// Items in this module are public for implementation reasons,
@ -39,17 +40,12 @@ pub use crate::intern_id::InternId;
pub use crate::interned::InternKey;
pub use crate::runtime::Runtime;
pub use crate::runtime::RuntimeId;
pub use crate::storage::Storage;
/// The base trait which your "query context" must implement. Gives
/// access to the salsa runtime, which you must embed into your query
/// context (along with whatever other state you may require).
pub trait Database: plumbing::DatabaseStorageTypes + plumbing::DatabaseOps {
/// Gives access to the underlying salsa runtime.
fn salsa_runtime(&self) -> &Runtime<Self>;
/// Gives access to the underlying salsa runtime.
fn salsa_runtime_mut(&mut self) -> &mut Runtime<Self>;
/// Iterates through all query storage and removes any values that
/// have not been used since the last revision was created. The
/// intended use-cycle is that you first execute all of your
@ -58,7 +54,12 @@ pub trait Database: plumbing::DatabaseStorageTypes + plumbing::DatabaseOps {
/// remove other values that were not needed for your main query
/// results.
fn sweep_all(&self, strategy: SweepStrategy) {
self.salsa_runtime().sweep_all(self, strategy);
// Note that we do not acquire the query lock (or any locks)
// here. Each table is capable of sweeping itself atomically
// and there is no need to bring things to a halt. That said,
// users may wish to guarantee atomicity.
self.for_each_query(|query_storage| query_storage.sweep(self, strategy));
}
/// Get access to extra methods pertaining to a given query. For
@ -123,6 +124,16 @@ pub trait Database: plumbing::DatabaseStorageTypes + plumbing::DatabaseOps {
fn on_propagated_panic(&self) -> ! {
panic!("concurrent salsa query panicked")
}
/// Gives access to the underlying salsa runtime.
fn salsa_runtime(&self) -> &Runtime {
self.ops_salsa_runtime()
}
/// Gives access to the underlying salsa runtime.
fn salsa_runtime_mut(&mut self) -> &mut Runtime {
self.ops_salsa_runtime_mut()
}
}
/// The `Event` struct identifies various notable things that can

View file

@ -17,7 +17,7 @@ pub use crate::derived::MemoizedStorage;
pub use crate::input::InputStorage;
pub use crate::interned::InternedStorage;
pub use crate::interned::LookupInternedStorage;
pub use crate::{revision::Revision, DatabaseKeyIndex};
pub use crate::{revision::Revision, DatabaseKeyIndex, Runtime};
#[derive(Clone, Debug)]
pub struct CycleDetected {
@ -37,6 +37,12 @@ pub trait DatabaseStorageTypes: Sized {
/// Internal operations that the runtime uses to operate on the database.
pub trait DatabaseOps: Sized {
/// Gives access to the underlying salsa runtime.
fn ops_salsa_runtime(&self) -> &Runtime;
/// Gives access to the underlying salsa runtime.
fn ops_salsa_runtime_mut(&mut self) -> &mut Runtime;
/// Formats a database key index in a human readable fashion.
fn fmt_index(
&self,

View file

@ -1,7 +1,7 @@
use crate::durability::Durability;
use crate::plumbing::CycleDetected;
use crate::revision::{AtomicRevision, Revision};
use crate::{CycleError, Database, DatabaseKeyIndex, Event, EventKind, SweepStrategy};
use crate::{CycleError, Database, DatabaseKeyIndex, Event, EventKind};
use log::debug;
use parking_lot::lock_api::{RawRwLock, RawRwLockRecursive};
use parking_lot::{Mutex, RwLock};
@ -24,26 +24,23 @@ use local_state::LocalState;
/// `Runtime::default`) will have an independent set of query storage
/// associated with it. Normally, therefore, you only do this once, at
/// the start of your application.
pub struct Runtime<DB: Database> {
pub struct Runtime {
/// Our unique runtime id.
id: RuntimeId,
/// If this is a "forked" runtime, then the `revision_guard` will
/// be `Some`; this guard holds a read-lock on the global query
/// lock.
revision_guard: Option<RevisionGuard<DB>>,
revision_guard: Option<RevisionGuard>,
/// Local state that is specific to this runtime (thread).
local_state: LocalState,
/// Shared state that is accessible via all runtimes.
shared_state: Arc<SharedState<DB>>,
shared_state: Arc<SharedState>,
}
impl<DB> Default for Runtime<DB>
where
DB: Database,
{
impl Default for Runtime {
fn default() -> Self {
Runtime {
id: RuntimeId { counter: 0 },
@ -54,10 +51,7 @@ where
}
}
impl<DB> std::fmt::Debug for Runtime<DB>
where
DB: Database,
{
impl std::fmt::Debug for Runtime {
fn fmt(&self, fmt: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
fmt.debug_struct("Runtime")
.field("id", &self.id())
@ -67,36 +61,15 @@ where
}
}
impl<DB> Runtime<DB>
where
DB: Database,
{
impl Runtime {
/// Create a new runtime; equivalent to `Self::default`. This is
/// used when creating a new database.
pub fn new() -> Self {
Self::default()
}
/// Returns the underlying storage, where the keys/values for all queries are kept.
pub fn storage(&self) -> &DB::DatabaseStorage {
&self.shared_state.storage
}
/// Returns a "forked" runtime, suitable for use in a forked
/// database. "Forked" runtimes hold a read-lock on the global
/// state, which means that any attempt to `set` an input will
/// block until the forked runtime is dropped. See
/// `ParallelDatabase::snapshot` for more information.
///
/// **Warning.** This second handle is intended to be used from a
/// separate thread. Using two database handles from the **same
/// thread** can lead to deadlock.
pub fn snapshot(&self, from_db: &DB) -> Self {
assert!(
Arc::ptr_eq(&self.shared_state, &from_db.salsa_runtime().shared_state),
"invoked `snapshot` with a non-matching database"
);
/// See [`crate::storage::Storage::snapshot`].
pub(crate) fn snapshot(&self) -> Self {
if self.local_state.query_in_progress() {
panic!("it is not legal to `snapshot` during a query (see salsa-rs/salsa#80)");
}
@ -146,16 +119,6 @@ where
self.with_incremented_revision(&mut |_next_revision| Some(durability));
}
/// Default implementation for `Database::sweep_all`.
pub fn sweep_all(&self, db: &DB, strategy: SweepStrategy) {
// Note that we do not acquire the query lock (or any locks)
// here. Each table is capable of sweeping itself atomically
// and there is no need to bring things to a halt. That said,
// users may wish to guarantee atomicity.
db.for_each_query(|query_storage| query_storage.sweep(db, strategy));
}
/// The unique identifier attached to this `SalsaRuntime`. Each
/// snapshotted runtime has a distinct identifier.
#[inline]
@ -336,12 +299,15 @@ where
self.revision_guard.is_none() && !self.local_state.query_in_progress()
}
pub(crate) fn execute_query_implementation<V>(
pub(crate) fn execute_query_implementation<DB, V>(
&self,
db: &DB,
database_key_index: DatabaseKeyIndex,
execute: impl FnOnce() -> V,
) -> ComputedQueryResult<V> {
) -> ComputedQueryResult<V>
where
DB: Database,
{
debug!(
"{:?}: execute_query_implementation invoked",
database_key_index
@ -534,9 +500,7 @@ where
}
/// State that will be common to all threads (when we support multiple threads)
struct SharedState<DB: Database> {
storage: DB::DatabaseStorage,
struct SharedState {
/// Stores the next id to use for a snapshotted runtime (starts at 1).
next_id: AtomicUsize,
@ -571,11 +535,10 @@ struct SharedState<DB: Database> {
dependency_graph: Mutex<DependencyGraph<DatabaseKeyIndex>>,
}
impl<DB: Database> SharedState<DB> {
impl SharedState {
fn with_durabilities(durabilities: usize) -> Self {
SharedState {
next_id: AtomicUsize::new(1),
storage: Default::default(),
query_lock: Default::default(),
revisions: (0..durabilities).map(|_| AtomicRevision::start()).collect(),
pending_revision: AtomicRevision::start(),
@ -584,23 +547,15 @@ impl<DB: Database> SharedState<DB> {
}
}
impl<DB> std::panic::RefUnwindSafe for SharedState<DB>
where
DB: Database,
DB::DatabaseStorage: std::panic::RefUnwindSafe,
{
}
impl std::panic::RefUnwindSafe for SharedState {}
impl<DB: Database> Default for SharedState<DB> {
impl Default for SharedState {
fn default() -> Self {
Self::with_durabilities(Durability::LEN)
}
}
impl<DB> std::fmt::Debug for SharedState<DB>
where
DB: Database,
{
impl std::fmt::Debug for SharedState {
fn fmt(&self, fmt: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
let query_lock = if self.query_lock.try_write().is_some() {
"<unlocked>"
@ -826,15 +781,12 @@ where
}
}
struct RevisionGuard<DB: Database> {
shared_state: Arc<SharedState<DB>>,
struct RevisionGuard {
shared_state: Arc<SharedState>,
}
impl<DB> RevisionGuard<DB>
where
DB: Database,
{
fn new(shared_state: &Arc<SharedState<DB>>) -> Self {
impl RevisionGuard {
fn new(shared_state: &Arc<SharedState>) -> Self {
// Subtle: we use a "recursive" lock here so that it is not an
// error to acquire a read-lock when one is already held (this
// happens when a query uses `snapshot` to spawn off parallel
@ -861,10 +813,7 @@ where
}
}
impl<DB> Drop for RevisionGuard<DB>
where
DB: Database,
{
impl Drop for RevisionGuard {
fn drop(&mut self) {
// Release our read-lock without using RAII. As documented in
// `Snapshot::new` above, this requires the unsafe keyword.

53
src/storage.rs Normal file
View file

@ -0,0 +1,53 @@
use crate::{Database, Runtime};
use std::sync::Arc;
/// Stores the cached results and dependency information for all the queries
/// defined on your salsa database. Also embeds a [`Runtime`] which is used to
/// manage query execution. Every database must include a `storage:
/// Storage<Self>` field.
pub struct Storage<DB: Database> {
query_store: Arc<DB::DatabaseStorage>,
runtime: Runtime,
}
impl<DB: Database> Default for Storage<DB> {
fn default() -> Self {
Self {
query_store: Default::default(),
runtime: Default::default(),
}
}
}
impl<DB: Database> Storage<DB> {
/// Gives access to the underlying salsa runtime.
pub fn salsa_runtime(&self) -> &Runtime {
&self.runtime
}
/// Gives access to the underlying salsa runtime.
pub fn salsa_runtime_mut(&mut self) -> &mut Runtime {
&mut self.runtime
}
/// Access the query storage tables. Not meant to be used directly by end
/// users.
pub fn query_store(&self) -> &DB::DatabaseStorage {
&self.query_store
}
/// Returns a "snapshotted" storage, suitable for use in a forked database.
/// This snapshot hold a read-lock on the global state, which means that any
/// attempt to `set` an input will block until the forked runtime is
/// dropped. See `ParallelDatabase::snapshot` for more information.
///
/// **Warning.** This second handle is intended to be used from a separate
/// thread. Using two database handles from the **same thread** can lead to
/// deadlock.
pub fn snapshot(&self) -> Self {
Storage {
query_store: self.query_store.clone(),
runtime: self.runtime.snapshot(),
}
}
}

View file

@ -8,23 +8,15 @@ struct Error {
#[salsa::database(GroupStruct)]
#[derive(Default)]
struct DatabaseImpl {
runtime: salsa::Runtime<DatabaseImpl>,
storage: salsa::Storage<Self>,
}
impl salsa::Database for DatabaseImpl {
fn salsa_runtime(&self) -> &salsa::Runtime<Self> {
&self.runtime
}
fn salsa_runtime_mut(&mut self) -> &mut salsa::Runtime<Self> {
&mut self.runtime
}
}
impl salsa::Database for DatabaseImpl {}
impl ParallelDatabase for DatabaseImpl {
fn snapshot(&self) -> Snapshot<Self> {
Snapshot::new(DatabaseImpl {
runtime: self.runtime.snapshot(self),
storage: self.storage.snapshot(),
})
}
}

View file

@ -3,18 +3,10 @@
#[salsa::database(DynTraitStorage)]
#[derive(Default)]
struct DynTraitDatabase {
runtime: salsa::Runtime<DynTraitDatabase>,
storage: salsa::Storage<Self>,
}
impl salsa::Database for DynTraitDatabase {
fn salsa_runtime(&self) -> &salsa::Runtime<Self> {
&self.runtime
}
fn salsa_runtime_mut(&mut self) -> &mut salsa::Runtime<Self> {
&mut self.runtime
}
}
impl salsa::Database for DynTraitDatabase {}
#[salsa::query_group(DynTraitStorage)]
trait DynTrait {

View file

@ -6,19 +6,11 @@ use crate::volatile_tests;
#[salsa::database(group::Gc, interned::Intern, volatile_tests::Volatile)]
#[derive(Default)]
pub(crate) struct DatabaseImpl {
runtime: salsa::Runtime<DatabaseImpl>,
storage: salsa::Storage<Self>,
log: Log,
}
impl salsa::Database for DatabaseImpl {
fn salsa_runtime(&self) -> &salsa::Runtime<DatabaseImpl> {
&self.runtime
}
fn salsa_runtime_mut(&mut self) -> &mut salsa::Runtime<DatabaseImpl> {
&mut self.runtime
}
}
impl salsa::Database for DatabaseImpl {}
impl DatabaseImpl {
pub(crate) fn clear_log(&self) {

View file

@ -18,7 +18,7 @@ pub(crate) trait TestContext: salsa::Database {
)]
#[derive(Default)]
pub(crate) struct TestContextImpl {
runtime: salsa::Runtime<TestContextImpl>,
storage: salsa::Storage<TestContextImpl>,
clock: Counter,
log: Log,
}
@ -54,12 +54,4 @@ impl TestContext for TestContextImpl {
}
}
impl salsa::Database for TestContextImpl {
fn salsa_runtime(&self) -> &salsa::Runtime<Self> {
&self.runtime
}
fn salsa_runtime_mut(&mut self) -> &mut salsa::Runtime<Self> {
&mut self.runtime
}
}
impl salsa::Database for TestContextImpl {}

View file

@ -5,23 +5,15 @@ use salsa::InternId;
#[salsa::database(InternStorage)]
#[derive(Default)]
struct Database {
runtime: salsa::Runtime<Database>,
storage: salsa::Storage<Self>,
}
impl salsa::Database for Database {
fn salsa_runtime(&self) -> &salsa::Runtime<Self> {
&self.runtime
}
fn salsa_runtime_mut(&mut self) -> &mut salsa::Runtime<Self> {
&mut self.runtime
}
}
impl salsa::Database for Database {}
impl salsa::ParallelDatabase for Database {
fn snapshot(&self) -> salsa::Snapshot<Self> {
salsa::Snapshot::new(Database {
runtime: self.runtime.snapshot(self),
storage: self.storage.snapshot(),
})
}
}

View file

@ -43,18 +43,10 @@ fn get_volatile(db: &impl QueryGroup, _x: u32) -> usize {
#[salsa::database(QueryGroupStorage)]
#[derive(Default)]
struct Database {
runtime: salsa::Runtime<Database>,
storage: salsa::Storage<Self>,
}
impl salsa::Database for Database {
fn salsa_runtime(&self) -> &salsa::Runtime<Self> {
&self.runtime
}
fn salsa_runtime_mut(&mut self) -> &mut salsa::Runtime<Self> {
&mut self.runtime
}
}
impl salsa::Database for Database {}
#[test]
fn lru_works() {

View file

@ -19,18 +19,10 @@ fn no_send_sync_key(_db: &impl NoSendSyncDatabase, key: Rc<bool>) -> bool {
#[salsa::database(NoSendSyncStorage)]
#[derive(Default)]
struct DatabaseImpl {
runtime: salsa::Runtime<DatabaseImpl>,
storage: salsa::Storage<Self>,
}
impl salsa::Database for DatabaseImpl {
fn salsa_runtime(&self) -> &salsa::Runtime<Self> {
&self.runtime
}
fn salsa_runtime_mut(&mut self) -> &mut salsa::Runtime<Self> {
&mut self.runtime
}
}
impl salsa::Database for DatabaseImpl {}
#[test]
fn no_send_sync() {

View file

@ -37,20 +37,12 @@ fn c(db: &impl QueryGroup, x: u32) -> u32 {
#[salsa::database(QueryGroupStorage)]
#[derive(Default)]
struct Database {
runtime: salsa::Runtime<Database>,
storage: salsa::Storage<Self>,
external_state: HashMap<u32, u32>,
on_event: Option<Box<dyn Fn(salsa::Event)>>,
}
impl salsa::Database for Database {
fn salsa_runtime(&self) -> &salsa::Runtime<Self> {
&self.runtime
}
fn salsa_runtime_mut(&mut self) -> &mut salsa::Runtime<Self> {
&mut self.runtime
}
fn salsa_event(&self, event_fn: impl Fn() -> salsa::Event) {
if let Some(cb) = &self.on_event {
cb(event_fn())

View file

@ -26,23 +26,15 @@ fn outer(db: &impl PanicSafelyDatabase) -> () {
#[salsa::database(PanicSafelyStruct)]
#[derive(Default)]
struct DatabaseStruct {
runtime: salsa::Runtime<DatabaseStruct>,
storage: salsa::Storage<Self>,
}
impl salsa::Database for DatabaseStruct {
fn salsa_runtime(&self) -> &salsa::Runtime<Self> {
&self.runtime
}
fn salsa_runtime_mut(&mut self) -> &mut salsa::Runtime<Self> {
&mut self.runtime
}
}
impl salsa::Database for DatabaseStruct {}
impl salsa::ParallelDatabase for DatabaseStruct {
fn snapshot(&self) -> Snapshot<Self> {
Snapshot::new(DatabaseStruct {
runtime: self.runtime.snapshot(self),
storage: self.storage.snapshot(),
})
}
}

View file

@ -187,19 +187,11 @@ fn snapshot_me(db: &impl ParDatabase) {
#[salsa::database(Par)]
#[derive(Default)]
pub(crate) struct ParDatabaseImpl {
runtime: salsa::Runtime<ParDatabaseImpl>,
storage: salsa::Storage<Self>,
knobs: KnobsStruct,
}
impl Database for ParDatabaseImpl {
fn salsa_runtime(&self) -> &salsa::Runtime<Self> {
&self.runtime
}
fn salsa_runtime_mut(&mut self) -> &mut salsa::Runtime<Self> {
&mut self.runtime
}
fn salsa_event(&self, event_fn: impl Fn() -> salsa::Event) {
let event = event_fn();
match event.kind {
@ -219,7 +211,7 @@ impl Database for ParDatabaseImpl {
impl ParallelDatabase for ParDatabaseImpl {
fn snapshot(&self) -> Snapshot<Self> {
Snapshot::new(ParDatabaseImpl {
runtime: self.runtime.snapshot(self),
storage: self.storage.snapshot(),
knobs: self.knobs.clone(),
})
}

View file

@ -38,23 +38,15 @@ fn c(db: &impl StressDatabase, key: usize) -> Cancelable<usize> {
#[salsa::database(Stress)]
#[derive(Default)]
struct StressDatabaseImpl {
runtime: salsa::Runtime<StressDatabaseImpl>,
storage: salsa::Storage<Self>,
}
impl salsa::Database for StressDatabaseImpl {
fn salsa_runtime(&self) -> &salsa::Runtime<StressDatabaseImpl> {
&self.runtime
}
fn salsa_runtime_mut(&mut self) -> &mut salsa::Runtime<StressDatabaseImpl> {
&mut self.runtime
}
}
impl salsa::Database for StressDatabaseImpl {}
impl salsa::ParallelDatabase for StressDatabaseImpl {
fn snapshot(&self) -> Snapshot<StressDatabaseImpl> {
Snapshot::new(StressDatabaseImpl {
runtime: self.runtime.snapshot(self),
storage: self.storage.snapshot(),
})
}
}

View file

@ -4,7 +4,7 @@ use std::cell::Cell;
#[salsa::database(queries::GroupStruct)]
#[derive(Default)]
pub(crate) struct DatabaseImpl {
runtime: salsa::Runtime<DatabaseImpl>,
storage: salsa::Storage<Self>,
counter: Cell<usize>,
}
@ -16,12 +16,4 @@ impl queries::Counter for DatabaseImpl {
}
}
impl salsa::Database for DatabaseImpl {
fn salsa_runtime(&self) -> &salsa::Runtime<DatabaseImpl> {
&self.runtime
}
fn salsa_runtime_mut(&mut self) -> &mut salsa::Runtime<DatabaseImpl> {
&mut self.runtime
}
}
impl salsa::Database for DatabaseImpl {}

View file

@ -20,18 +20,10 @@ fn get(db: &impl QueryGroup, x: u32) -> u32 {
#[salsa::database(QueryGroupStorage)]
#[derive(Default)]
struct Database {
runtime: salsa::Runtime<Database>,
storage: salsa::Storage<Self>,
}
impl salsa::Database for Database {
fn salsa_runtime(&self) -> &salsa::Runtime<Self> {
&self.runtime
}
fn salsa_runtime_mut(&mut self) -> &mut salsa::Runtime<Self> {
&mut self.runtime
}
}
impl salsa::Database for Database {}
#[test]
fn transparent_queries_work() {

View file

@ -31,18 +31,10 @@ fn trailing(_db: &impl HelloWorldDatabase, a: u32, b: u32) -> u32 {
#[salsa::database(HelloWorld)]
#[derive(Default)]
struct DatabaseStruct {
runtime: salsa::Runtime<DatabaseStruct>,
storage: salsa::Storage<Self>,
}
impl salsa::Database for DatabaseStruct {
fn salsa_runtime(&self) -> &salsa::Runtime<Self> {
&self.runtime
}
fn salsa_runtime_mut(&mut self) -> &mut salsa::Runtime<Self> {
&mut self.runtime
}
}
impl salsa::Database for DatabaseStruct {}
#[test]
fn execute() {