* Increase turbine propagation const
Value is used as a delay threshold for issuing shred repairs and analysis is showing we are overly aggressive in requesting repairs. Shreds show up via turbine before the repair completes the vast majority of the time
* Use Duration type for MAX_TURBINE_PROPAGATION
Store non-vote transaction counts that are now recorded by the banks
into the `blockstore`.
`SamplePerformanceService` now populates `PerfSampleV2` with counts from
the banks.
{verify,sign}_shreds_gpu need to point to offsets within the packets for
the signed data. For merkle shreds this signed data is the merkle root
of the erasure batch and this would necessitate embedding the merkle
roots in the shreds payload.
However this is wasteful and reduces shreds capacity to store data
because the merkle root can already be recovered from the encoded merkle
proof.
Instead of pointing to offsets within the shreds payload, this commit
recovers merkle roots from the merkle proofs and stores them in an
allocated buffer. {verify,sign}_shreds_gpu would then point to offsets
within this new buffer for the respective signed data.
This would unblock us from removing merkle roots from shreds payload
which would save capacity to send more data with each shred.
The commit adds an associated SignedData type to Shred trait so that
merkle and legacy shreds can return different types for signed_data
method.
This would allow legacy shreds to point to a section of the shred
payload, whereas merkle shreds would compute and return the merkle root.
Ultimately this would allow to remove the merkle root from the shreds
binary.
Merkle shreds within the same erasure batch have the same merkle root.
The root of the merkle tree is signed. So either the signatures match
or one fails sigverify, and the comparison of merkle roots is redundant.
If data is empty, make_shreds_from_data will now return one data shred
with empty data. This preserves invariants verified in tests regardless
of data size.
We currently use the is_connected field to be able to signal to
ReplayStage that a slot has replayable updates. It was discovered that
this functionality is effectively broken, and that is_connected is never
true. In order to convey this information to ReplayStage more
effectively, we need extra state information so this PR changes the
existing bool to bitflags with two bits.
From a compatibility standpoint, the is_connected bool was already
occupying one byte in the serialized SlotMeta in blockstore. Thus, the
change from a bool to bitflags still "fits" in that one byte allotment.
In consideration of a case where a client may wish to downgrade software
and use the same ledger, deserializing the bitflags into a bool could
fail if the new bit is set. As such, this PR introduces the second bit
field, but does not set it anywhere. Once clusters have mass adopted a
software version with this PR, a subsequent change to actually set and
use the new field can be introduced.
The num_repair field is only blockstore insertion metric being updated
outside of Blockstore::insert() call chain; move the update to insert()
with the rest of the fields in BlockstoreInsertionMetrics struct.
* support CpiGuard and PermanentDelegate extensions in transaction-status and account-decoder
* update transaction-status and account-decoder to new ConfidentialTransfer interfaces
* Update cost model to use requested_cu instead of estimated cu #27608
* remove CostUpdate and CostModel from replay/tvu
* revive cost update service to send cost tracker stats
* CostModel is now static
* remove unused package
Co-authored-by: Tao Zhu <tao@solana.com>
The manual Blockstore compaction that was being initiated from
LedgerCleanupService has been disabled for quite some time in favor of
several optimizations.
Co-authored-by: Ryo Onodera <ryoqun@gmail.com>
PR #28317 previously attempted to fix a case where blockstore processing
would create children banks for slots past the halt_at_slot.
However, the previous fix didn't handle the case where a slot could be
strictly less than the halt_at_slot, but have children that were greater
than the halt_at_slot. For example, this could happen if a child of slot
S is S+n where n > 1.
Thus, this change covers our processing logic to cover this second case
as well.
load_frozen_forks() finds new slots to process by creating new Banks for
the children of the current slot in process_next_slots(). Prior to this
change, we would then immediately check if we had reached the
halt_at_slot and correctly halt processing when appropriate. As such, it
would be possible for Banks to be created for slots beyond the
halt_at_slot.
While a potential child slot that is past halt_at_slot wouldn't be
replayed, the Bank being created still alters some universal state in
AccountsDb. So, this change moves the halt_at_slot check before we
create children Banks in process_next_slots().
A fifo rocksdb instance must be opened with max size parameter on the
fifo columns. To support this, we previously plumbed a constant up to
callers that provided a default if unbounded growth desired.
This change attempts to be more rusty by exposing an option for this
value, and converting the option to a constant at the lowest level
possible.
#### Summary of Changes
Removes the constant default for ShredStorageType::RocksFifo
as the shred storage size is either user-specified or derived
from --limit-ledger-size in #27459.
### Problem
When FIFO compaction is used while --rocksdb_fifo_shred_storage_size
is unspecified, the FIFO shred storage size is set to a const default based
on the default `--limit-ledger-size`.
### Summary of the Change
When --rocksdb_fifo_shred_storage_size is unspecified, it is now
derived from `--limit-ledger-size` by reserving 1500 bytes for each
shred.
### Problem
The documentation of each column family is missing
### Summary
The goal is to create a comment block that will essentially include a high-level
concept on what each column family is about and what are their key/value formats.
This PR is the first cut that includes the key/value format of each column family.
This should at least provide an easy pointer for readers to understand what this
column family stores by searching its value type and how to access the data based
on the key type.
#### Problem
The current implementation of get_slots_since() invokes multiple rocksdb::get().
As a result, each get() operation may end up requiring one disk read. This leads
to poor performance of get_slots_since described in #24878.
#### Summary of Changes
This PR makes get_slots_since() use the batched version of multi_get() instead,
which allows multiple get operations to be processed in batch so that they can
be answered with fewer disk reads.
Several of the get() methods return a deserialized object (as opposed to
a Vec<u8>) by first getting a byte array out of Rocks, and then using
bincode::deserialize() to get the underlying type. However,
deserialize() only requires a u8 slice, not an owned Vec<u8>. So, we can
use get_pinned_cf() to reference memory owned by Rocks and avoid an
unnecessary copy.
#### Problem
Previously before #26651, our LedgerCleanupService needs RocksDB background
compactions to reclaim ledger disk space via our custom CompactionFilter.
However, since RocksDB's compaction isn't smart enough to know which file to pick,
we rely on the 1-day compaction period so that each file will be forced to be compacted
once a day so that we can reclaim ledger disk space in time. The downside of this is
each ledger file will be rewritten once per day.
#### Summary of Changes
As #26651 makes LedgerCleanupService actively delete those files whose entire slot-range
is older than both --limit-ledger-size and the current root, we can remove the 1-day compaction
period and get rid of the daily ledger file rewrite.
The results on mainnet-beta shows that this PR reduces ~20% write-bytes-per-second
and reduces ~50% read-bytes-per-second on ledger disk.
#### Problem
Blockstore operations such as get_slots_since() issues multiple rocksdb::get()
at once which is not optimal for performance.
#### Summary of Changes
This PR adds LedgerColumn::multi_get() based on rocksdb::batched_multi_get(),
the optimized version of multi_get() where get requests are processed in batch
to minimize read I/O.
These methods are only used in tests but invoked on a merkle shred they
will always invalidate the shred because the merkle proof will no longer
verify. As a result the shred will not sanitize and blockstore will
avoid inserting them. Their use in tests will result in spurious test
coverage because the shreds will not be ingested.
The commit removes implementation of these methods for merkle shreds.
Follow up commits will entirely remove these methods from shreds api.
Add ledger-tool command print-file-metadata
#### Summary of Changes
This PR adds a ledger tool subcommand print-file-metadata.
```
USAGE:
solana-ledger-tool print-file-metadata [FLAGS] [OPTIONS] [SST_FILE_NAME]
Prints the metadata of the specified ledger-store file.
If no file name is unspecified, then it will print the metadata of all ledger files
```
#### Summary of Changes
Add code comments for lowest_cleanup_slot related functions to improve
the code readability for the consistency between blockstore purge logic
and the read side.
#### Problem
RocksDB's delete_range applies to [from, to) while delete_file_in_range
applies to [from, to] by default, and the rust-rocksdb api does not include
the option to make delete_file_in_range apply to [from, to). Such inconsistency
might cause `blockstore::run_purge` to produce an inconsistent result as it
invokes both delete_range and delete_file_in_range.
#### Summary of Changes
This PR makes all our purge / delete related functions to be inclusive
on both starting and ending slots.
Tenets:
1. Limit thread names to 15 characters
2. Prefix all Solana-controlled threads with "sol"
3. Use Camel case. It's more character dense than Snake or Kebab case
The commit
* Identifies Merkle shreds when recovering from erasure codes and
dispatches specialized code to reconstruct shreds.
* Coding shred headers are added to recovered erasure shards.
* Merkle tree is reconstructed for the erasure batch and added to
recovered shreds.
* The common signature (for the root of Merkle tree) is attached to all
recovered shreds.
#### Problem
get_entries_in_data_block() panics when there's inconsistency between
slot_meta and data_shred.
However, as we don't lock on reads, reading across multiple column families is
not atomic (especially for older slots) and thus does not guarantee consistency
as the background cleanup service could purge the slot in the middle. Such
panic was reported in #26980 when the validator serves a high load of RPC calls.
#### Summary of Changes
This PR makes get_entries_in_data_block() panic only when the inconsistency
between slot-meta and data-shred happens on a slot older than lowest_cleanup_slot.
As a consequence of removing buffering when generating coding shreds:
https://github.com/solana-labs/solana/pull/25807
more coding shreds are generated than data shreds, and so
MAX_CODE_SHREDS_PER_SLOT needs to be adjusted accordingly.
The respective value is tied to ERASURE_BATCH_SIZE.
Given the 32:32 erasure recovery schema, current implementation requires
exactly 32 data shreds to generate coding shreds for the batch (except
for the final erasure batch in each slot).
As a result, when serializing ledger entries to data shreds, if the
number of data shreds is not a multiple of 32, the coding shreds for the
last batch cannot be generated until there are more data shreds to
complete the batch to 32 data shreds. This adds latency in generating
and broadcasting coding shreds.
In addition, with Merkle variants for shreds, data shreds cannot be
signed and broadcasted until coding shreds are also generated. As a
result *both* code and data shreds will be delayed before broadcast if
we still require exactly 32 data shreds for each batch.
This commit instead always generates and broadcast coding shreds as soon
as there any number of data shreds available. When serializing entries
to shreds:
* if the number of resulting data shreds is less than 32, then more
coding shreds will be generated so that the resulting erasure batch
has the same recovery probabilities as a 32:32 batch.
* if the number of data shreds is more than 32, then the data shreds are
split uniformly into erasure batches with _at least_ 32 data shreds in
each batch. Each erasure batch will have the same number of code and
data shreds.
For example:
* If there are 19 data shreds, 27 coding shreds are generated. The
resulting 19(data):27(code) erasure batch has the same recovery
probabilities as a 32:32 batch.
* If there are 107 data shreds, they are split into 3 batches of 36:36,
36:36 and 35:35 data:code shreds each.
A consequence of this change is that code and data shreds indices will
no longer align as there will be more coding shreds than data shreds
(not only in the last batch in each slot but also in the intermediate
ones);
#### Problem
LedgerCleanupService requires compactions to propagate & digest range-delete tombstones
to eventually reclaim disk space.
#### Summary of Changes
This PR makes LedgerCleanupService::cleanup_ledger delete any file whose slot-range is
older than the lowest_cleanup_slot. This allows us to reclaim disk space more often with
fewer IOps. Experimental results on mainnet validators show that the PR can effectively
reduce 33% to 40% ledger disk size.
Prior to this change, long running commands like `solana-ledger-tool
verify` would OOM due to AccountsDb cleanup not happening.
Co-authored-by: Michael Vines <mvines@gmail.com>
* Concurrent replay slots
* Split out concurrent and single bank replay paths
* Sub function processing of replay results for readability
* Add feature switch for concurrent replay
#### Problem
Ledger-tool doesn't support shred-compaction-type other than the default rocksdb level compaction.
#### Summary of Changes
This PR enables ledger-tool to automatically detect the shred-compaction-type of the specified ledger.
#### Test Plan
New ledger-tool tests are added for both level and fifo compactions.
* Adjusts test cases for stricter requirements.
* Removes account reset in deserialization test.
* Removes verify related test cases.
* Replicates account modification verification logic of PreAccount in BorrowedAccount.
* Adds TransactionContext::account_touched_flags.
* Adds account modification verification to the BPF ABIv0 and ABIv1 deserialization, CPI syscall and program-test.
* Replicates the total sum of all lamports verification of PreAccounts in InstructionContext
* Check that the callers instruction balance is maintained during a call / push.
* Replicates PreAccount statistics in TransactionContext.
* Disable verify() and verify_and_update() if the feature enable_early_verification_of_account_modifications is enabled.
* Moves Option<Rent> of enable_early_verification_of_account_modifications into TransactionContext::new().
* Relaxes AccountDataMeter related test cases.
* Don't touch the account if nothing changes.
* Adds two tests to trigger InstructionError::UnbalancedInstruction.
Co-authored-by: Justin Starry <justin@solana.com>
#### Summary of Changes
Blockstore::blockstore_directory() function takes a ShredStorageType and returns a path.
As it's a ShredStorageType related function, moving it under ShredStorageType as a
member function is a better fit.
In addition, as we start doing automatic shred compaction type selection under ledger-tool,
it becomes more convenient to reuse the function and const under blockstore_options
namespace instead of blockstore.
#### Summary of Changes
Define PERF_METRIC_OP_NAME_PUT and PERF_METRIC_OP_NAME_WRITE_BATCH
to replace repetitive / hard-coded operation names for report_rocksdb_write_perf.
#### Problem
report_rocksdb_read_perf() always uses the hard-coded operation name "get"
#### Summary of Changes
As we will add a new read operation -- multi_get(), report_rocksdb_read_perf()
needs to have an input parameter for operation name.
* slots_connected check used by ledger-tool should not require a full slot for snapshot slot
* Cleaner Result<Option<>> unwrap/default
* return false if no meta for starting slot
* Add clarifying comments
Fully deserializing shreds in window-service before sending them to
retransmit stage adds latency to shreds propagation.
This commit instead channels through the payload and relies on only
partial deserialization of a few required fields: slot, shred-index,
shred-type.
Shred slot and parent are not verified until window-service where
resources are already wasted to sig-verify and deserialize shreds.
This commit moves above verification to earlier in the pipeline in fetch
stage.
Following commits will skip shreds deserializaton before retransmit, and
so we will only have a ShredId and not a fully deserialized shred to
obtain the shuffling seed from.
* Make sure to root local slots even with hard fork
* Address review comments
* Cleanup a bit
* Further clean up
* Further clean up a bit
* Add comment
* Tweak hard fork reconciliation code placement
* Define shuffle to prep using same shuffle for multiple slices
* Determine transaction indexes and plumb to execute_batch
* Pair transaction_index with transaction in TransactionStatusService
* Add new ReplicaTransactionInfoVersion
* Plumb transaction_indexes through BankingStage
* Prepare BankingStage to receive transaction indexes from PohRecorder
* Determine transaction indexes in PohRecorder; add field to WorkingBank
* Add PohRecorder::record unit test
* Only pass starting_transaction_index around PohRecorder
* Add helper structs to simplify test DashMap
* Pass entry and starting-index into process_entries_with_callback together
* Add tx-index checks to test_rebatch_transactions
* Revert shuffle definition and use zip/unzip
* Only zip/unzip if randomize
* Add confirm_slot_entries test
* Review nits
* Add type alias to make sender docs more clear
In prepration of
https://github.com/solana-labs/solana/pull/25807
which reworks erasure batch sizes, this commit:
* adds a helper function mapping the number of data shreds to the
erasure batch size.
* adds ProcessShredsStats to Shredder::entries_to_shreds in order to
replace and remove entries_to_data_shreds from the public interface.
* working on local snapshot
* Parallelization for slot storage minimization
* Additional clean-up and fixes
* make --minimize an option of create-snapshot
* remove now unnecessary function
* Parallelize parts of minimized account set generation
* clippy fixes
* Add rent collection accounts and voting node_pubkeys
* Simplify programdata_accounts generation
* Loop over storages to get slot set
* Parallelize minimized slot set generation
* Parallelize adding owners and programdata_accounts
* Remove some now unncessary checks on the blockstore
* Add a warning for minimized snapshots across epoch boundary
* Simplify ledger-tool minimize
* Clarify names of bank's minimization helper functions
* Remove unnecesary funciton, fix line spacing
* Use DashSets instead of HashSets for minimized account and slot sets
* Filter storages uses all threads instead of thread_pool
* Add some additional comments on functions for minimization
* Moved more into bank and parallelized
* Update programs/bpf/Cargo.lock for dashmap in ledger
* Clippy fix
* ledger-tool: convert minimize_bank_for_snapshot Measure into measure!
* bank.rs: convert minimize_bank_for_snapshot Measure into measure!
* accounts_db.rs: convert minimize_accounts_db Measure into measure!
* accounts_db.rs: add comment about use of minimize_accounts_db
* ledger-tool: CLI argument clarification
* minimization functions: make infos unique
* bank.rs: Add test_get_rent_collection_accounts_between_slots
* bank.rs: Add test_minimization_add_vote_accounts
* bank.rs: Add test_minimization_add_stake_accounts
* bank.rs: Add test_minimization_add_owner_accounts
* bank.rs: Add test_minimization_add_programdata_accounts
* accounts_db.rs: Add test_minimize_accounts_db
* bank.rs: Add negative case and comments in test_get_rent_collection_accounts_between_slots
* bank.rs: Negative test in test_minimization_add_programdata_accounts
* use new static runtime and sdk ids
* bank comments to doc comments
* Only need to insert the maximum slot a key is found in
* rename remove_pubkeys to purge_pubkeys
* add comment on builtins::get_pubkeys
* prevent excessive logging of removed dead slots
* don't need to remove slot from shrink slot candidates
* blockstore.rs: get_accounts_used_in_range shouldn't return Result
* blockstore.rs: get_accounts_used_in_range: parallelize slot loop
* report filtering progress on time instead of count
* parallelize loop over snapshot storages
* WIP: move some bank minimization functionality into a new class
* WIP: move some accounts_db minimization functionality into SnapshotMinimizer
* WIP: Use new SnapshotMinimizer
* SnapshotMinimizer: fix use statements
* remove bank and accounts_db minimization code, where possible
* measure! doesn't take a closure
* fix use statement in blockstore
* log_dead_slots does not need pub(crate)
* get_unique_accounts_from_storages does not need pub(crate)
* different way to get stake accounts/nodes
* fix tests
* move rent collection account functionality to snapshot minimizer
* move accounts_db minimize behavior to snapshot minimizer
* clean up
* Use bank reference instead of Arc. Additional comments
* Add a comment to blockstore function
* Additional clarifying comments
* Moved all non-transaction account accumulation into the SnapshotMinimizer.
* transaction_account_set does not need to be mutable now
* Add comment about load_to_collect_rent_eagerly
* Update log_dead_slots comment
* remove duplicate measure/print of get_minimized_slot_set
Shred versions are not verified until window-service where resources are
already wasted to sig-verify and deserialize shreds.
The commit verifies shred-version earlier in the pipeline in fetch stage.
A slot may be purged from the blockstore with clear_unconfirmed_slot().
If the slot is added back, the slot should only exist once in its'
parent SlotMeta::next_slots Vec. Prior to this change, repeated clearing
and re-adding of a slot could result in the slot existing in parent's
next_slots multiple times. The result is that if the next time the
parent slot is processed (node restart or ledger-tool-replay), slot
could be added to the queue of slots to play multiple times.
Added test that failed before change and works now as well
#### Problem
Currently, the creation of ShredStorageType::RocksFifo is hard coded in validator/src/main.rs.
But this common code will also need to be used in other places like ledger-tool.
#### Summary of Changes
This PR creates a helper functionShredStorageType::rocks_fifo that takes a total shred_storage_size
and equally allocates to data-shred and coding-shred storage.
Coding shreds can only be signed once erasure codings are already
generated. Therefore coding shreds recovered from erasure codings lack
slot leader's signature and so cannot be retransmitted to the rest of
the cluster.
shred/merkle.rs implements a new shred variant where we generate merkle
tree for each erasure encoded batch and each shred includes:
* root of the merkle tree (Hash truncated to 20 bytes).
* slot leader's signature of the root of the merkle tree.
* merkle tree nodes along the branch the shred belongs to, where hashes
are trimmed to 20 bytes during tree construction.
This schema results in the same signature for all shreds within an
erasure batch.
When recovering shreds from erasure codes, we can reconstruct merkle
tree for the batch and for each recovered shred also recover respective
merkle tree branch; then snap the slot leader's signature from any of
the shreds received from turbine and retransmit all recovered code or
data shreds.
Backward compatibility is achieved by encoding shred variant at byte 65
of payload (previously shred-type at this position):
* 0b0101_1010 indicates a legacy coding shred, which is also equal to
ShredType::Code for backward compatibility.
* 0b1010_0101 indicates a legacy data shred, which is also equal to
ShredType::Data for backward compatibility.
* 0b0100_???? indicates a merkle coding shred with merkle branch size
indicated by the last 4 bits.
* 0b1000_???? indicates a merkle data shred with merkle branch size
indicated by the last 4 bits.
Merkle root and branch are encoded at the end of the shred payload.
#### Problem
When FIFO compaction is used, the size ratio between data shred and coding
shred is set to 1:1 based on the `--rocksdb_fifo_shred_storage_size` arg.
However, BlockstoreRocksFifoOptions::default() uses a slightly optimized
5:4 ratio instead, and the default() function is only used in benchmarks.
#### Summary of Changes
This PR makes both validator argument and BlockstoreRocksFifoOptions::default()
to use 1:1 ratio between data and coding shred size.