Ensure that `mark_ephemeral_address_as_mined` correctly handles indices

within the gap. Also support paging for `get_known_ephemeral_addresses`.

Co-authored-by: Jack Grigg <jack@electriccoin.co>
Signed-off-by: Daira-Emma Hopwood <daira@jacaranda.org>
This commit is contained in:
Daira-Emma Hopwood 2024-07-04 05:22:18 +01:00
parent e97da43409
commit a01588bc65
13 changed files with 397 additions and 189 deletions

1
Cargo.lock generated
View File

@ -3121,6 +3121,7 @@ dependencies = [
"schemer-rusqlite",
"secrecy",
"shardtree",
"static_assertions",
"subtle",
"tempfile",
"time",

View File

@ -103,8 +103,9 @@ tonic-build = { version = "0.11", default-features = false }
secrecy = "0.8"
subtle = "2.2.3"
# Static constants
# Static constants and assertions
lazy_static = "1"
static_assertions = "1"
# Tests and benchmarks
assert_matches = "1.5"

View File

@ -97,7 +97,10 @@ use zcash_primitives::{
};
#[cfg(feature = "transparent-inputs")]
use {crate::wallet::TransparentAddressMetadata, zcash_primitives::legacy::TransparentAddress};
use {
crate::wallet::TransparentAddressMetadata, std::ops::Range,
zcash_primitives::legacy::TransparentAddress,
};
#[cfg(any(test, feature = "test-dependencies"))]
use zcash_primitives::consensus::NetworkUpgrade;
@ -929,10 +932,11 @@ pub trait WalletRead {
/// if let Some(result) = self.get_transparent_receivers(account)?.get(address) {
/// return Ok(result.clone());
/// }
/// if let Some(result) = self.get_known_ephemeral_addresses(account, false)?.get(address) {
/// return Ok(Some(result.clone()));
/// }
/// Ok(None)
/// Ok(self
/// .get_known_ephemeral_addresses(account, None)?
/// .into_iter()
/// .find(|(known_addr, _)| known_addr == address)
/// .map(|(_, metadata)| metadata))
/// ```
///
/// Returns `Ok(None)` if the address is not recognized, or we do not have metadata for it.
@ -947,28 +951,24 @@ pub trait WalletRead {
if let Some(result) = self.get_transparent_receivers(account)?.get(address) {
return Ok(result.clone());
}
if let Some(result) = self
.get_known_ephemeral_addresses(account, false)?
.get(address)
{
return Ok(Some(result.clone()));
}
Ok(None)
Ok(self
.get_known_ephemeral_addresses(account, None)?
.into_iter()
.find(|(known_addr, _)| known_addr == address)
.map(|(_, metadata)| metadata))
}
/// Returns a set of ephemeral transparent addresses associated with the given
/// account controlled by this wallet, along with their metadata.
/// Returns a vector of ephemeral transparent addresses associated with the given
/// account controlled by this wallet, along with their metadata. The result includes
/// reserved addresses, and addresses for `GAP_LIMIT` additional indices (capped to
/// the maximum index).
///
/// If `for_detection` is false, the set only includes addresses reserved by
/// `reserve_next_n_ephemeral_addresses`. If `for_detection` is true, it includes
/// those addresses and also the ones that will be reserved next, for an additional
/// `GAP_LIMIT` indices (up to and including the maximum index given by
/// `NonHardenedChildIndex::from_index(i32::MAX as u32)`).
/// If `index_range` is some `Range`, it limits the result to addresses with indices
/// in that range.
///
/// Wallets should scan the chain for UTXOs sent to the ephemeral transparent
/// receivers obtained with `for_detection` set to `true`, but do not need to do
/// so regularly. Under expected usage, outputs would only be detected with these
/// receivers in the following situations:
/// Wallets should scan the chain for UTXOs sent to these ephemeral transparent
/// receivers, but do not need to do so regularly. Under expected usage, outputs
/// would only be detected with these receivers in the following situations:
///
/// - This wallet created a payment to a ZIP 320 (TEX) address, but the second
/// transaction (that spent the output sent to the ephemeral address) did not get
@ -994,9 +994,9 @@ pub trait WalletRead {
fn get_known_ephemeral_addresses(
&self,
_account: Self::AccountId,
_for_detection: bool,
) -> Result<HashMap<TransparentAddress, TransparentAddressMetadata>, Self::Error> {
Ok(HashMap::new())
_index_range: Option<Range<u32>>,
) -> Result<Vec<(TransparentAddress, TransparentAddressMetadata)>, Self::Error> {
Ok(vec![])
}
/// If a given transparent address has been reserved, i.e. would be included in
@ -1007,7 +1007,11 @@ pub trait WalletRead {
/// This is equivalent to (but may be implemented more efficiently than):
/// ```compile_fail
/// for account_id in self.get_account_ids()? {
/// if self.get_known_ephemeral_addresses(account_id, false)?.contains_key(address)? {
/// if self
/// .get_known_ephemeral_addresses(account_id, None)?
/// .into_iter()
/// .any(|(known_addr, _)| &known_addr == address)
/// {
/// return Ok(Some(account_id));
/// }
/// }
@ -1020,8 +1024,9 @@ pub trait WalletRead {
) -> Result<Option<Self::AccountId>, Self::Error> {
for account_id in self.get_account_ids()? {
if self
.get_known_ephemeral_addresses(account_id, false)?
.contains_key(address)
.get_known_ephemeral_addresses(account_id, None)?
.into_iter()
.any(|(known_addr, _)| &known_addr == address)
{
return Ok(Some(account_id));
}
@ -1672,10 +1677,6 @@ pub trait WalletWrite: WalletRead {
/// the given number of addresses, or if the account identifier does not correspond
/// to a known account.
///
/// # Panics
///
/// Panics if the precondition `n < 0x80000000` does not hold.
///
/// [BIP 44]: https://github.com/bitcoin/bips/blob/master/bip-0044.mediawiki#user-content-Address_gap_limit
#[cfg(feature = "transparent-inputs")]
fn reserve_next_n_ephemeral_addresses(
@ -1787,7 +1788,10 @@ pub mod testing {
};
#[cfg(feature = "transparent-inputs")]
use {crate::wallet::TransparentAddressMetadata, zcash_primitives::legacy::TransparentAddress};
use {
crate::wallet::TransparentAddressMetadata, std::ops::Range,
zcash_primitives::legacy::TransparentAddress,
};
#[cfg(feature = "orchard")]
use super::ORCHARD_SHARD_HEIGHT;
@ -2024,9 +2028,9 @@ pub mod testing {
fn get_known_ephemeral_addresses(
&self,
_account: Self::AccountId,
_for_detection: bool,
) -> Result<HashMap<TransparentAddress, TransparentAddressMetadata>, Self::Error> {
Ok(HashMap::new())
_index_range: Option<Range<u32>>,
) -> Result<Vec<(TransparentAddress, TransparentAddressMetadata)>, Self::Error> {
Ok(vec![])
}
#[cfg(feature = "transparent-inputs")]
@ -2103,9 +2107,8 @@ pub mod testing {
fn reserve_next_n_ephemeral_addresses(
&mut self,
_account_id: Self::AccountId,
n: u32,
_n: u32,
) -> Result<Vec<(TransparentAddress, TransparentAddressMetadata)>, Self::Error> {
assert!(n < 0x80000000);
Err(())
}
}

View File

@ -1151,14 +1151,13 @@ where
change_value.is_ephemeral() && change_value.output_pool() == PoolType::Transparent
})
.collect();
if ephemeral_outputs.len() > i32::MAX as usize {
return Err(Error::ProposalNotSupported);
}
let n = ephemeral_outputs
.len()
.try_into()
.map_err(|_| Error::ProposalNotSupported)?;
let addresses_and_metadata = wallet_db
.reserve_next_n_ephemeral_addresses(
account_id,
ephemeral_outputs.len().try_into().unwrap(),
)
.reserve_next_n_ephemeral_addresses(account_id, n)
.map_err(Error::DataSource)?;
assert_eq!(addresses_and_metadata.len(), ephemeral_outputs.len());

View File

@ -54,6 +54,9 @@ jubjub.workspace = true
secrecy.workspace = true
subtle.workspace = true
# - Static assertions
static_assertions.workspace = true
# - Shielded protocols
orchard = { workspace = true, optional = true }
sapling.workspace = true

View File

@ -302,7 +302,7 @@ impl<C: Borrow<rusqlite::Connection>, P: consensus::Parameters> WalletRead for W
type Account = wallet::Account;
fn get_account_ids(&self) -> Result<Vec<AccountId>, Self::Error> {
wallet::get_account_ids(self.conn.borrow())
Ok(wallet::get_account_ids(self.conn.borrow())?)
}
fn get_account(
@ -553,13 +553,13 @@ impl<C: Borrow<rusqlite::Connection>, P: consensus::Parameters> WalletRead for W
fn get_known_ephemeral_addresses(
&self,
account: Self::AccountId,
for_detection: bool,
) -> Result<HashMap<TransparentAddress, TransparentAddressMetadata>, Self::Error> {
index_range: Option<Range<u32>>,
) -> Result<Vec<(TransparentAddress, TransparentAddressMetadata)>, Self::Error> {
wallet::transparent::ephemeral::get_known_ephemeral_addresses(
self.conn.borrow(),
&self.params,
account,
for_detection,
index_range,
)
}

View File

@ -445,7 +445,7 @@ pub(crate) fn send_multi_step_proposed_transfer<T: ShieldedPoolTester>() {
// Each transfer should use a different ephemeral address.
let (ephemeral0, _) = run_test(&mut st, 0);
let (ephemeral1, _) = run_test(&mut st, 1);
assert!(ephemeral0 != ephemeral1);
assert_ne!(ephemeral0, ephemeral1);
add_funds(&mut st, value);

View File

@ -133,6 +133,10 @@ pub(crate) mod transparent;
pub(crate) const BLOCK_SAPLING_FRONTIER_ABSENT: &[u8] = &[0x0];
/// The number of ephemeral addresses that can be safely reserved without observing any
/// of them to be mined. This is the same as the gap limit in Bitcoin.
pub(crate) const GAP_LIMIT: u32 = 20;
fn parse_account_source(
account_kind: u32,
hd_seed_fingerprint: Option<[u8; 32]>,
@ -509,6 +513,10 @@ pub(crate) fn add_account<P: consensus::Parameters>(
let (address, d_idx) = account.default_address(DEFAULT_UA_REQUEST)?;
insert_address(conn, params, account_id, d_idx, &address)?;
// Initialize the `ephemeral_addresses` table.
#[cfg(feature = "transparent-inputs")]
transparent::ephemeral::init_account(conn, params, account_id)?;
Ok(account_id)
}
@ -1914,9 +1922,11 @@ pub(crate) fn truncate_to_height<P: consensus::Parameters>(
}
/// Returns a vector with the IDs of all accounts known to this wallet.
///
/// Note that this is called from db migration code.
pub(crate) fn get_account_ids(
conn: &rusqlite::Connection,
) -> Result<Vec<AccountId>, SqliteClientError> {
) -> Result<Vec<AccountId>, rusqlite::Error> {
let mut stmt = conn.prepare("SELECT id FROM accounts")?;
let mut rows = stmt.query([])?;
let mut result = Vec::new();

View File

@ -13,10 +13,12 @@
// from showing up in `cargo doc --document-private-items`.
#![allow(dead_code)]
use static_assertions::const_assert_eq;
use zcash_client_backend::data_api::scanning::ScanPriority;
use zcash_protocol::consensus::{NetworkUpgrade, Parameters};
use crate::wallet::scanning::priority_code;
use crate::wallet::{scanning::priority_code, GAP_LIMIT};
/// Stores information about the accounts that the wallet is tracking.
pub(super) const TABLE_ACCOUNTS: &str = r#"
@ -76,12 +78,17 @@ CREATE INDEX "addresses_accounts" ON "addresses" (
"account_id" ASC
)"#;
/// Stores ephemeral transparent addresses used for ZIP 320. For each account, these addresses are
/// allocated sequentially by address index under custom scope 2 at the "change" level of the BIP 32
/// address hierarchy. Only "reserved" ephemeral addresses, that is addresses that have been allocated
/// for use in a ZIP 320 transaction proposal, are stored in the table. Addresses are never removed.
/// New ones should only be reserved via the `WalletWrite::reserve_next_n_ephemeral_addresses` API.
/// All of the addresses in the table should be scanned for incoming funds.
/// Stores ephemeral transparent addresses used for ZIP 320.
///
/// For each account, these addresses are allocated sequentially by address index under scope 2
/// (`TransparentKeyScope::EPHEMERAL`) at the "change" level of the BIP 32 address hierarchy.
/// The ephemeral addresses stored in the table are exactly the "reserved" ephemeral addresses
/// (that is addresses that have been allocated for use in a ZIP 320 transaction proposal), plus
/// the addresses at the next `GAP_LIMIT` indices.
///
/// Addresses are never removed. New ones should only be reserved via the
/// `WalletWrite::reserve_next_n_ephemeral_addresses` API. All of the addresses in the table
/// should be scanned for incoming funds.
///
/// ### Columns
/// - `address` contains the string (Base58Check) encoding of a transparent P2PKH address.
@ -96,22 +103,35 @@ CREATE INDEX "addresses_accounts" ON "addresses" (
/// a debugging aid (although the latter allows us to account for whether the referenced transaction
/// is unmined). We only really care which addresses have been used, and whether we can allocate a
/// new address within the gap limit.
///
/// It is an external invariant that within each account:
/// - the address indices are contiguous and start from 0;
/// - the last `GAP_LIMIT` addresses have `used_in_tx` and `mined_in_tx` both NULL.
///
/// All but the last `GAP_LIMIT` addresses are defined to be "reserved" addresses. Since the next
/// index to reserve is determined by dead reckoning from the last stored address, we use dummy
/// entries after the maximum valid index in order to allow the last `GAP_LIMIT` addresses at the
/// end of the index range to be used.
pub(super) const TABLE_EPHEMERAL_ADDRESSES: &str = r#"
CREATE TABLE ephemeral_addresses (
account_id INTEGER NOT NULL,
address_index INTEGER NOT NULL,
address TEXT NOT NULL,
address TEXT,
used_in_tx INTEGER,
mined_in_tx INTEGER,
FOREIGN KEY (account_id) REFERENCES accounts(id),
FOREIGN KEY (used_in_tx) REFERENCES transactions(id_tx),
FOREIGN KEY (mined_in_tx) REFERENCES transactions(id_tx),
PRIMARY KEY (account_id, address_index),
CONSTRAINT address_index_in_range CHECK (address_index >= 0 AND address_index <= 0x7FFFFFFF)
CONSTRAINT index_range_and_address_nullity CHECK (
(address_index BETWEEN 0 AND 0x7FFFFFFF AND address IS NOT NULL) OR
(address_index BETWEEN 0x80000000 AND 0x7FFFFFFF + 20 AND address IS NULL AND used_in_tx IS NULL AND mined_in_tx IS NULL)
)
) WITHOUT ROWID"#;
// Hexadecimal integer literals were added in SQLite version 3.8.6 (2014-08-15).
// libsqlite3-sys requires at least version 3.14.0.
// "WITHOUT ROWID" tells SQLite to use a clustered index on the (composite) primary key.
const_assert_eq!(GAP_LIMIT, 20);
pub(super) const INDEX_EPHEMERAL_ADDRESSES_ADDRESS: &str = r#"
CREATE INDEX ephemeral_addresses_address ON ephemeral_addresses (
address ASC

View File

@ -119,7 +119,9 @@ pub(super) fn all_migrations<P: consensus::Parameters + 'static>(
params: params.clone(),
}),
Box::new(utxos_to_txos::Migration),
Box::new(ephemeral_addresses::Migration),
Box::new(ephemeral_addresses::Migration {
params: params.clone(),
}),
]
}

View File

@ -5,22 +5,31 @@ use rusqlite;
use schemer;
use schemer_rusqlite::RusqliteMigration;
use uuid::Uuid;
use zcash_protocol::consensus;
use crate::wallet::init::WalletMigrationError;
#[cfg(feature = "transparent-inputs")]
use crate::wallet::{self, init, transparent::ephemeral};
use super::utxos_to_txos;
pub(super) const MIGRATION_ID: Uuid = Uuid::from_u128(0x0e1d4274_1f8e_44e2_909d_689a4bc2967b);
pub(super) struct Migration;
const DEPENDENCIES: [Uuid; 1] = [utxos_to_txos::MIGRATION_ID];
impl schemer::Migration for Migration {
#[allow(dead_code)]
pub(super) struct Migration<P> {
pub(super) params: P,
}
impl<P> schemer::Migration for Migration<P> {
fn id(&self) -> Uuid {
MIGRATION_ID
}
fn dependencies(&self) -> HashSet<Uuid> {
[utxos_to_txos::MIGRATION_ID].into_iter().collect()
DEPENDENCIES.into_iter().collect()
}
fn description(&self) -> &'static str {
@ -28,7 +37,7 @@ impl schemer::Migration for Migration {
}
}
impl RusqliteMigration for Migration {
impl<P: consensus::Parameters> RusqliteMigration for Migration<P> {
type Error = WalletMigrationError;
fn up(&self, transaction: &rusqlite::Transaction) -> Result<(), WalletMigrationError> {
@ -36,19 +45,30 @@ impl RusqliteMigration for Migration {
"CREATE TABLE ephemeral_addresses (
account_id INTEGER NOT NULL,
address_index INTEGER NOT NULL,
address TEXT NOT NULL,
address TEXT,
used_in_tx INTEGER,
mined_in_tx INTEGER,
FOREIGN KEY (account_id) REFERENCES accounts(id),
FOREIGN KEY (used_in_tx) REFERENCES transactions(id_tx),
FOREIGN KEY (mined_in_tx) REFERENCES transactions(id_tx),
PRIMARY KEY (account_id, address_index),
CONSTRAINT address_index_in_range CHECK (address_index >= 0 AND address_index <= 0x7FFFFFFF)
CONSTRAINT index_range_and_address_nullity CHECK (
(address_index BETWEEN 0 AND 0x7FFFFFFF AND address IS NOT NULL) OR
(address_index BETWEEN 0x80000000 AND 0x7FFFFFFF + 20 AND address IS NULL AND used_in_tx IS NULL AND mined_in_tx IS NULL)
)
) WITHOUT ROWID;
CREATE INDEX ephemeral_addresses_address ON ephemeral_addresses (
address ASC
);",
)?;
// Make sure that at least `GAP_LIMIT` ephemeral transparent addresses are
// stored in each account.
#[cfg(feature = "transparent-inputs")]
for account_id in wallet::get_account_ids(transaction)? {
ephemeral::init_account(transaction, &self.params, account_id)
.map_err(init::sqlite_client_error_to_wallet_migration_error)?;
}
Ok(())
}
@ -65,4 +85,115 @@ mod tests {
fn migrate() {
test_migrate(&[super::MIGRATION_ID]);
}
#[test]
#[cfg(feature = "transparent-inputs")]
fn initialize_table() {
use rusqlite::named_params;
use secrecy::Secret;
use tempfile::NamedTempFile;
use zcash_client_backend::{
data_api::{AccountBirthday, AccountSource, WalletWrite},
wallet::TransparentAddressMetadata,
};
use zcash_keys::keys::UnifiedSpendingKey;
use zcash_primitives::{block::BlockHash, legacy::keys::NonHardenedChildIndex};
use zcash_protocol::consensus::Network;
use zip32::{fingerprint::SeedFingerprint, AccountId as Zip32AccountId};
use crate::{
error::SqliteClientError,
wallet::{
account_kind_code, init::init_wallet_db_internal, transparent::ephemeral, GAP_LIMIT,
},
WalletDb,
};
let network = Network::TestNetwork;
let data_file = NamedTempFile::new().unwrap();
let mut db_data = WalletDb::for_path(data_file.path(), network).unwrap();
let seed0 = vec![0x00; 32];
init_wallet_db_internal(
&mut db_data,
Some(Secret::new(seed0.clone())),
&super::DEPENDENCIES,
false,
)
.unwrap();
let birthday = AccountBirthday::from_sapling_activation(&network, BlockHash([0; 32]));
// Simulate creating an account prior to this migration.
let account0_index = Zip32AccountId::ZERO;
let account0_seed_fp = [0u8; 32];
let account0_kind = account_kind_code(AccountSource::Derived {
seed_fingerprint: SeedFingerprint::from_seed(&account0_seed_fp).unwrap(),
account_index: account0_index,
});
assert_eq!(u32::from(account0_index), 0);
let account0_id = crate::AccountId(0);
let usk0 = UnifiedSpendingKey::from_seed(&network, &seed0, account0_index).unwrap();
let ufvk0 = usk0.to_unified_full_viewing_key();
let uivk0 = ufvk0.to_unified_incoming_viewing_key();
db_data
.conn
.execute(
"INSERT INTO accounts (id, account_kind, hd_seed_fingerprint, hd_account_index, ufvk, uivk, birthday_height)
VALUES (:id, :account_kind, :hd_seed_fingerprint, :hd_account_index, :ufvk, :uivk, :birthday_height)",
named_params![
":id": account0_id.0,
":account_kind": account0_kind,
":hd_seed_fingerprint": account0_seed_fp,
":hd_account_index": u32::from(account0_index),
":ufvk": ufvk0.encode(&network),
":uivk": uivk0.encode(&network),
":birthday_height": u32::from(birthday.height()),
],
)
.unwrap();
// The `ephemeral_addresses` table is expected not to exist before migration.
assert_matches!(
ephemeral::first_unstored_index(&db_data.conn, account0_id),
Err(SqliteClientError::DbError(_))
);
let check = |db: &WalletDb<_, _>, account_id| {
eprintln!("checking {account_id:?}");
assert_matches!(ephemeral::first_unstored_index(&db.conn, account_id), Ok(addr_index) if addr_index == GAP_LIMIT);
assert_matches!(ephemeral::first_unreserved_index(&db.conn, account_id), Ok(addr_index) if addr_index == 0);
let known_addrs =
ephemeral::get_known_ephemeral_addresses(&db.conn, &db.params, account_id, None)
.unwrap();
let expected_metadata: Vec<TransparentAddressMetadata> = (0..GAP_LIMIT)
.map(|i| ephemeral::metadata(NonHardenedChildIndex::from_index(i).unwrap()))
.collect();
let actual_metadata: Vec<TransparentAddressMetadata> =
known_addrs.into_iter().map(|(_, meta)| meta).collect();
assert_eq!(actual_metadata, expected_metadata);
};
// The migration should initialize `ephemeral_addresses`.
init_wallet_db_internal(
&mut db_data,
Some(Secret::new(seed0)),
&[super::MIGRATION_ID],
false,
)
.unwrap();
check(&db_data, account0_id);
// Creating a new account should initialize `ephemeral_addresses` for that account.
let seed1 = vec![0x01; 32];
let (account1_id, _usk) = db_data
.create_account(&Secret::new(seed1), &birthday)
.unwrap();
assert_ne!(account0_id, account1_id);
check(&db_data, account1_id);
}
}

View File

@ -492,7 +492,7 @@ pub(crate) fn get_transparent_address_metadata<P: consensus::Parameters>(
}
}
// Search ephemeral addresses that have already been reserved.
// Search known ephemeral addresses.
if let Some(raw_index) =
ephemeral::find_index_for_ephemeral_address_str(conn, account_id, &address_str)?
{
@ -500,9 +500,6 @@ pub(crate) fn get_transparent_address_metadata<P: consensus::Parameters>(
return Ok(Some(ephemeral::metadata(address_index)));
}
// We intentionally don't check for unreserved ephemeral addresses within the gap
// limit here. It's unnecessary to look up metadata for addresses from which we
// can spend.
Ok(None)
}
@ -534,7 +531,7 @@ pub(crate) fn find_account_for_transparent_output<P: consensus::Parameters>(
return Ok(Some(account_id));
}
// Search ephemeral addresses that have already been reserved.
// Search known ephemeral addresses.
if let Some(account_id) = ephemeral::find_account_for_ephemeral_address_str(conn, &address_str)?
{
return Ok(Some(account_id));
@ -556,20 +553,6 @@ pub(crate) fn find_account_for_transparent_output<P: consensus::Parameters>(
}
}
// Finally we check for ephemeral addresses within the gap limit.
for account_id in account_ids {
let ephemeral_ivk = ephemeral::get_ephemeral_ivk(conn, params, account_id)?;
let last_reserved_index = ephemeral::last_reserved_index(conn, account_id)?;
for raw_index in ephemeral::range_after(last_reserved_index, ephemeral::GAP_LIMIT) {
let address_index = NonHardenedChildIndex::from_index(raw_index).unwrap();
if &ephemeral_ivk.derive_ephemeral_address(address_index)? == output.recipient_address()
{
return Ok(Some(account_id));
}
}
}
Ok(None)
}

View File

@ -1,7 +1,6 @@
//! Functions for wallet support of ephemeral transparent addresses.
use std::cmp::max;
use std::collections::HashMap;
use std::ops::RangeInclusive;
use std::cmp::{max, min};
use std::ops::Range;
use rusqlite::{named_params, OptionalExtension};
@ -16,11 +15,11 @@ use zcash_primitives::{
};
use zcash_protocol::consensus;
use crate::{error::SqliteClientError, wallet::get_account, AccountId, SqlTransaction, WalletDb};
/// The number of ephemeral addresses that can be safely reserved without observing any
/// of them to be mined. This is the same as the gap limit in Bitcoin.
pub(crate) const GAP_LIMIT: i32 = 20;
use crate::{
error::SqliteClientError,
wallet::{get_account, GAP_LIMIT},
AccountId, SqlTransaction, WalletDb,
};
// Returns `TransparentAddressMetadata` in the ephemeral scope for the
// given address index.
@ -28,12 +27,11 @@ pub(crate) fn metadata(address_index: NonHardenedChildIndex) -> TransparentAddre
TransparentAddressMetadata::new(TransparentKeyScope::EPHEMERAL, address_index)
}
/// Returns the last reserved ephemeral address index in the given account,
/// or -1 if the account has no reserved ephemeral addresses.
pub(crate) fn last_reserved_index(
/// Returns the first unstored ephemeral address index in the given account.
pub(crate) fn first_unstored_index(
conn: &rusqlite::Connection,
account_id: AccountId,
) -> Result<i32, SqliteClientError> {
) -> Result<u32, SqliteClientError> {
match conn
.query_row(
"SELECT address_index FROM ephemeral_addresses
@ -41,19 +39,33 @@ pub(crate) fn last_reserved_index(
ORDER BY address_index DESC
LIMIT 1",
named_params![":account_id": account_id.0],
|row| row.get::<_, i32>(0),
|row| row.get::<_, u32>(0),
)
.optional()?
{
Some(i) if i < 0 => unreachable!("violates constraint address_index_in_range"),
Some(i) => Ok(i),
None => Ok(-1),
Some(i) if i >= (1 << 31) + GAP_LIMIT => {
unreachable!("violates constraint index_range_and_address_nullity")
}
Some(i) => Ok(i.checked_add(1).unwrap()),
None => Ok(0),
}
}
/// Returns the last ephemeral address index in the given account that
/// would not violate the gap invariant if used.
pub(crate) fn last_safe_index(
/// Returns the first unreserved ephemeral address index in the given account.
pub(crate) fn first_unreserved_index(
conn: &rusqlite::Connection,
account_id: AccountId,
) -> Result<u32, SqliteClientError> {
first_unstored_index(conn, account_id)?
.checked_sub(GAP_LIMIT)
.ok_or(SqliteClientError::CorruptedData(
"ephemeral_addresses table has not been initialized".to_owned(),
))
}
/// Returns the first ephemeral address index in the given account that
/// would violate the gap invariant if used.
pub(crate) fn first_unsafe_index(
conn: &rusqlite::Connection,
account_id: AccountId,
) -> Result<u32, SqliteClientError> {
@ -62,7 +74,7 @@ pub(crate) fn last_safe_index(
// to have been mined in a transaction that we currently see as unmined.
// This is conservative in terms of avoiding violation of the gap
// invariant: it can only cause us to get to the end of the gap sooner.
let last_mined_index: i32 = match conn
let first_unmined_index: u32 = match conn
.query_row(
"SELECT address_index FROM ephemeral_addresses
JOIN transactions t ON t.id_tx = mined_in_tx
@ -70,30 +82,30 @@ pub(crate) fn last_safe_index(
ORDER BY address_index DESC
LIMIT 1",
named_params![":account_id": account_id.0],
|row| row.get::<_, i32>(0),
|row| row.get::<_, u32>(0),
)
.optional()?
{
Some(i) if i < 0 => unreachable!("violates constraint address_index_in_range"),
Some(i) => i,
None => -1,
Some(i) if i >= 1 << 31 => {
unreachable!("violates constraint index_range_and_address_nullity")
}
Some(i) => i.checked_add(1).unwrap(),
None => 0,
};
Ok(u32::try_from(last_mined_index.saturating_add(GAP_LIMIT)).unwrap())
Ok(min(
1 << 31,
first_unmined_index.checked_add(GAP_LIMIT).unwrap(),
))
}
/// Utility function to return an `InclusiveRange<u32>` that starts at `i + 1`
/// and is of length up to `n`. The range is truncated if necessary to end at
/// the maximum valid address index, `i32::MAX`.
///
/// # Panics
///
/// Panics if the precondition `i >= -1 and n > 0` does not hold.
pub(crate) fn range_after(i: i32, n: i32) -> RangeInclusive<u32> {
assert!(i >= -1);
assert!(n > 0);
let first = u32::try_from(i64::from(i) + 1).unwrap();
let last = u32::try_from(i.saturating_add(n)).unwrap();
first..=last
/// Utility function to return an `Range<u32>` that starts at `i`
/// and is of length up to `n`. The range is truncated if necessary
/// so that it contains no elements beyond the maximum valid address
/// index, `(1 << 31) - 1`.
pub(crate) fn range_from(i: u32, n: u32) -> Range<u32> {
let first = min(1 << 31, i);
let last = min(1 << 31, i.saturating_add(n));
first..last
}
/// Returns the ephemeral transparent IVK for a given account ID.
@ -110,47 +122,40 @@ pub(crate) fn get_ephemeral_ivk<P: consensus::Parameters>(
.derive_ephemeral_ivk()?)
}
/// Returns a mapping of ephemeral transparent addresses potentially belonging
/// to this wallet to their metadata.
/// Returns a vector of ephemeral transparent addresses associated with the given
/// account controlled by this wallet, along with their metadata. The result includes
/// reserved addresses, and addresses for `GAP_LIMIT` additional indices (capped to
/// the maximum index).
///
/// If `for_detection` is false, the result only includes reserved addresses.
/// If `for_detection` is true, it includes addresses for an additional
/// `GAP_LIMIT` indices, up to the limit.
/// If `index_range` is some `Range`, it limits the result to addresses with indices
/// in that range.
pub(crate) fn get_known_ephemeral_addresses<P: consensus::Parameters>(
conn: &rusqlite::Connection,
params: &P,
account_id: AccountId,
for_detection: bool,
) -> Result<HashMap<TransparentAddress, TransparentAddressMetadata>, SqliteClientError> {
let mut stmt = conn.prepare(
"SELECT address, address_index FROM ephemeral_addresses WHERE account_id = :account ORDER BY address_index",
)?;
let mut rows = stmt.query(named_params! { ":account": account_id.0 })?;
index_range: Option<Range<u32>>,
) -> Result<Vec<(TransparentAddress, TransparentAddressMetadata)>, SqliteClientError> {
let index_range = index_range.unwrap_or(0..(1 << 31));
let mut result = HashMap::new();
let mut first_unused_index: Option<i32> = Some(0);
let mut stmt = conn.prepare(
"SELECT address, address_index FROM ephemeral_addresses
WHERE account_id = :account AND address_index >= :start AND address_index < :end
ORDER BY address_index",
)?;
let mut rows = stmt.query(named_params![
":account": account_id.0,
":start": index_range.start,
":end": min(1 << 31, index_range.end),
])?;
let mut result = vec![];
while let Some(row) = rows.next()? {
let addr_str: String = row.get(0)?;
let raw_index: u32 = row.get(1)?;
first_unused_index = i32::try_from(raw_index)
.map_err(|e| SqliteClientError::CorruptedData(e.to_string()))?
.checked_add(1);
let address_index = NonHardenedChildIndex::from_index(raw_index).unwrap();
let address = TransparentAddress::decode(params, &addr_str)?;
result.insert(address, metadata(address_index));
}
if for_detection {
if let Some(first) = first_unused_index {
let ephemeral_ivk = get_ephemeral_ivk(conn, params, account_id)?;
for raw_index in range_after(first, GAP_LIMIT) {
let address_index = NonHardenedChildIndex::from_index(raw_index).unwrap();
let address = ephemeral_ivk.derive_ephemeral_address(address_index)?;
result.insert(address, metadata(address_index));
}
}
result.push((address, metadata(address_index)));
}
Ok(result)
}
@ -189,10 +194,6 @@ pub(crate) fn find_index_for_ephemeral_address_str(
/// Returns a vector with the next `n` previously unreserved ephemeral addresses for
/// the given account.
///
/// # Panics
///
/// Panics if the precondition `n < 0x80000000` does not hold.
///
/// # Errors
///
/// * `SqliteClientError::AccountUnknown`, if there is no account with the given id.
@ -211,43 +212,81 @@ pub(crate) fn reserve_next_n_ephemeral_addresses<P: consensus::Parameters>(
if n == 0 {
return Ok(vec![]);
}
assert!(n > 0);
let n = i32::try_from(n).expect("precondition violated");
let ephemeral_ivk = get_ephemeral_ivk(wdb.conn.0, &wdb.params, account_id)?;
let last_reserved_index = last_reserved_index(wdb.conn.0, account_id)?;
let last_safe_index = last_safe_index(wdb.conn.0, account_id)?;
let allocation = range_after(last_reserved_index, n);
let first_unreserved = first_unreserved_index(wdb.conn.0, account_id)?;
let first_unsafe = first_unsafe_index(wdb.conn.0, account_id)?;
let allocation = range_from(first_unreserved, n);
if allocation.clone().count() < n.try_into().unwrap() {
return Err(SqliteClientError::AddressGeneration(
AddressGenerationError::DiversifierSpaceExhausted,
if allocation.len() < n.try_into().unwrap() {
return Err(AddressGenerationError::DiversifierSpaceExhausted.into());
}
if allocation.end > first_unsafe {
return Err(SqliteClientError::ReachedGapLimit(
account_id,
max(first_unreserved, first_unsafe),
));
}
if *allocation.end() > last_safe_index {
let unsafe_index = max(*allocation.start(), last_safe_index.saturating_add(1));
return Err(SqliteClientError::ReachedGapLimit(account_id, unsafe_index));
reserve_until(wdb.conn.0, &wdb.params, account_id, allocation.end)?;
get_known_ephemeral_addresses(wdb.conn.0, &wdb.params, account_id, Some(allocation))
}
/// Initialize the `ephemeral_addresses` table. This must be called when
/// creating or migrating an account.
pub(crate) fn init_account<P: consensus::Parameters>(
conn: &rusqlite::Transaction,
params: &P,
account_id: AccountId,
) -> Result<(), SqliteClientError> {
reserve_until(conn, params, account_id, 0)
}
/// Extend the range of stored addresses in an account if necessary so that the
/// index of the next address to reserve will be *at least* `next_to_reserve`.
/// If it would already have been at least `next_to_reserve`, then do nothing.
///
/// Note that this is called from db migration code.
///
/// # Panics
///
/// Panics if `next_to_reserve > (1 << 31)`.
fn reserve_until<P: consensus::Parameters>(
conn: &rusqlite::Transaction,
params: &P,
account_id: AccountId,
next_to_reserve: u32,
) -> Result<(), SqliteClientError> {
assert!(next_to_reserve <= 1 << 31);
let first_unstored = first_unstored_index(conn, account_id)?;
let range_to_store = first_unstored..(next_to_reserve.checked_add(GAP_LIMIT).unwrap());
if range_to_store.is_empty() {
return Ok(());
}
let ephemeral_ivk = get_ephemeral_ivk(conn, params, account_id)?;
// used_in_tx and mined_in_tx are initially NULL
let mut stmt_insert_ephemeral_address = wdb.conn.0.prepare_cached(
let mut stmt_insert_ephemeral_address = conn.prepare_cached(
"INSERT INTO ephemeral_addresses (account_id, address_index, address)
VALUES (:account_id, :address_index, :address)",
)?;
allocation
.map(|raw_index| {
let address_index = NonHardenedChildIndex::from_index(raw_index).unwrap();
let address = ephemeral_ivk.derive_ephemeral_address(address_index)?;
stmt_insert_ephemeral_address.execute(named_params![
":account_id": account_id.0,
":address_index": raw_index,
":address": address.encode(&wdb.params),
])?;
Ok((address, metadata(address_index)))
})
.collect()
for raw_index in range_to_store {
let address_str_opt = match NonHardenedChildIndex::from_index(raw_index) {
Some(address_index) => Some(
ephemeral_ivk
.derive_ephemeral_address(address_index)?
.encode(params),
),
None => None,
};
stmt_insert_ephemeral_address.execute(named_params![
":account_id": account_id.0,
":address_index": raw_index,
":address": address_str_opt,
])?;
}
Ok(())
}
/// Returns a `SqliteClientError::EphemeralAddressReuse` error if the address was
@ -352,9 +391,25 @@ pub(crate) fn mark_ephemeral_address_as_mined<P: consensus::Parameters>(
|row| row.get::<_, i64>(0),
)?;
wdb.conn.0.execute(
"UPDATE ephemeral_addresses SET mined_in_tx = :mined_in_tx WHERE address = :address",
named_params![":mined_in_tx": &earlier_ref, ":address": address_str],
)?;
let mined_ephemeral = wdb
.conn
.0
.query_row(
"UPDATE ephemeral_addresses
SET mined_in_tx = :mined_in_tx
WHERE address = :address
RETURNING (account_id, address_index)",
named_params![":mined_in_tx": &earlier_ref, ":address": address_str],
|row| Ok((AccountId(row.get::<_, u32>(0)?), row.get::<_, u32>(1)?)),
)
.optional()?;
// If this is a known ephemeral address for an account in this wallet, we might need
// to extend the indices stored for that account to maintain the invariant that the
// last `GAP_LIMIT` addresses are unused and unmined.
if let Some((account_id, address_index)) = mined_ephemeral {
let next_to_reserve = min(1 << 31, address_index.saturating_add(1));
reserve_until(wdb.conn.0, &wdb.params, account_id, next_to_reserve)?;
}
Ok(())
}