Skip to content

Commit

Permalink
TxPool: refactoring for blob txs (#5953)
Browse files Browse the repository at this point in the history
* add extensions

* adjust filters

* add tests

* draft

* fix metrics

* fix tx type, add test for replacing

* adjust blob replacement comparer

* add blob tx db and metrics

* fix test

* resurrect old TxStorage class and adjust a bit for blobs

* pass blob storage to tx pool, adjust tests

* handle db writes/reads and add test

* recreate light collection after restart

* adjust broadcaster test

* cosmetic

* add broadcaster test

* add todos

* add MaxFeePerDataGas check when updating gas bottleneck

* regression test for MaxFeePerDataGas check

* cosmetic

* unify & simplify tests

* cosmetics in BlobTxStorage

* add AcceptTxResult dedicated for rejected replacements

* adjust tests, add test for blob tx replacements

* simplify TxPool

* refactor to reuse id of deprecated result

* simplify filters

* add configurable sizes of blob pool and blob cache

* optimize TxType filter

* add light tx as separate class to reuse

* dont return blob txs from broadcaster and other refactorings

* add test for broadcaster

* add test for announcing txs with MaxFeePerDataGas >= current

* add more tx pool metrics

* update tx pool report

* add metric to filter

* improve tests

* refactors

* clean

* add test

* fix TxPool

* fix test

* fix EthereumTests

* fix report

* fix whitespaces

* fix blob txs picking when building block

* fix test

* fix benchmarks solution

* add filtering for MaxPriorityFeePerGas < 1GWei for blob transactions

* adjust tests to min requirement of 1 gwei

* add test

* add more txpool config options

* adjust storage

* add non-persistent blob tx collection

* one more

* adjust tests, add new one

* post merge fixes

* cosmetics

* fix bug in broadcasting - use size of full blob tx in light tx

* cosmetics

* lower default PersistentBlobStorageSize

* Divide TxPoolTests

* cosmetic - usings

* more tests moved to TxPoolTests.Blobs

* refactor ParityRpcModuleTests to reuse setup code

* cosmetic

* refactor picking txs for rebroadcasting/reannouncing

* Add MaxBlobsPerBlock to Eip4844Constants and use in several places instead of calculating

* whitespace

* cosmetic in BlobTxStorage + add tests

* make tx comparisons more readable

* cosmetics in TxTypeTxFilter

* filter blob txs in FeeTooLowFilter just as other types

* adjust test

* fix whitespaces and file encodings

* simplification in TxPool

* another whitespace fix

* override base methods in blob collections instead of creating new ones + simplifications in TxPool

* cosmetics

* add BlobTxStorage parameterless constructor that uses MemDb and simplify tests

* remove unused things

* post-merge fix

* move capacity insurances to inner classes

* add test for pool capacity

* clean

* try with synchronization

* refactor block building

* fix whitespace

* simplify decoding in BlobTxStorage

* cosmetics

* simplify tests

* fix tx priority?

* improve readability in NonceGapFilter

* improve readability of TxPoolTxSource

* save sender to db

* fix recreating light collection

* adjust EthStats to show sum of blob and non-blob txs

* fix

* make blob support configurable

* optimize snapshots

* fix blob support disabled

* don't request blob txs if blob support disabled

* fix tests

* fix files encoding

* fix test

* cosmetics

* add MaxPendingTxsPerSenderFilter

* add test

* disable blob pool by default

* cosmetic refactoring

* revert previous change

* fix tests after making blob pool disabled by default

* useBlobDb -> useBlobsDb

* TxConfig descrptions fixes

* add timestamp to db, drop linq

* add missing metric

* adjust txpool report

* cosmetic

* fix broadcaster to check in pending before requesting

* fix

* add metric of received hashes

* improve txpool report

* fix naming

* make naming more relevant

* stopwatch

* add missing metric

* move to ColumnsDb

* optimize full blobs db by prefixing key with timestamp

* fix files encoding

* fix and improve blob storage

* fix tests

* adjust tests to require sender of full blob txs loaded from db

* refactor future nonce filter

* simplify and improve block building

* refactor TxPoolTxSource

* fix block building - move loading full tx from db after all filters

* refactor low fee filtering

* add new metric

* adjust TxPool filters pipeline and TxPoolReport

* change naming as requested in review

* add size estimates to descriptions in ITxPoolConfig

* rename AcceptTxResult PendingTxsOfOtherType -> PendingTxsOfConflictingType

* add more comments to TxPool

* add more comments to PersistentBlobTxDistinctSortedPool

* fix file encoding

* small refactors

* cosmetic 1.GWei()

* reduce spec lookup

---------

Co-authored-by: MarekM25 <marekm2504@gmail.com>
Co-authored-by: lukasz.rozmej <lukasz.rozmej@gmail.com>
  • Loading branch information
3 people authored and brbrr committed Oct 17, 2023
1 parent d0c44fb commit d8c124d
Show file tree
Hide file tree
Showing 70 changed files with 2,059 additions and 355 deletions.
2 changes: 0 additions & 2 deletions src/Nethermind/Ethereum.Test.Base/BlockchainTestBase.cs
Original file line number Diff line number Diff line change
Expand Up @@ -132,8 +132,6 @@ protected async Task<EthereumTestResult> RunTest(BlockchainTest test, Stopwatch?
IBlockTree blockTree = new BlockTree(new MemDb(), new MemDb(), blockInfoDb, new ChainLevelInfoRepository(blockInfoDb), specProvider, NullBloomStorage.Instance, _logManager);
ITransactionComparerProvider transactionComparerProvider = new TransactionComparerProvider(specProvider, blockTree);
IStateReader stateReader = new StateReader(trieStore, codeDb, _logManager);
IChainHeadInfoProvider chainHeadInfoProvider = new ChainHeadInfoProvider(specProvider, blockTree, stateReader);
ITxPool transactionPool = new TxPool(ecdsa, chainHeadInfoProvider, new TxPoolConfig(), new TxValidator(specProvider.ChainId), _logManager, transactionComparerProvider.GetDefaultComparer());

IReceiptStorage receiptStorage = NullReceiptStorage.Instance;
IBlockhashProvider blockhashProvider = new BlockhashProvider(blockTree, _logManager);
Expand Down
2 changes: 2 additions & 0 deletions src/Nethermind/Nethermind.Api/IApiWithStores.cs
Original file line number Diff line number Diff line change
Expand Up @@ -8,12 +8,14 @@
using Nethermind.Crypto;
using Nethermind.Db.Blooms;
using Nethermind.State.Repositories;
using Nethermind.TxPool;
using Nethermind.Wallet;

namespace Nethermind.Api
{
public interface IApiWithStores : IBasicApi
{
ITxStorage? BlobTxStorage { get; set; }
IBlockTree? BlockTree { get; set; }
IBloomStorage? BloomStorage { get; set; }
IChainLevelInfoRepository? ChainLevelInfoRepository { get; set; }
Expand Down
1 change: 1 addition & 0 deletions src/Nethermind/Nethermind.Api/NethermindApi.cs
Original file line number Diff line number Diff line change
Expand Up @@ -104,6 +104,7 @@ public IBlockchainBridge CreateBlockchainBridge()
}

public IAbiEncoder AbiEncoder { get; } = Nethermind.Abi.AbiEncoder.Instance;
public ITxStorage? BlobTxStorage { get; set; }
public IBlockchainProcessor? BlockchainProcessor { get; set; }
public CompositeBlockPreprocessorStep BlockPreprocessor { get; } = new();
public IBlockProcessingQueue? BlockProcessingQueue { get; set; }
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,7 @@ protected override async Task<IDbProvider> CreateDbProvider()
{
IDbProvider dbProvider = new DbProvider(DbModeHint.Persisted);
RocksDbFactory rocksDbFactory = new(new DbConfig(), LogManager, TempDirectory.Path);
StandardDbInitializer standardDbInitializer = new(dbProvider, rocksDbFactory, new MemDbFactory(), new FileSystem(), true);
StandardDbInitializer standardDbInitializer = new(dbProvider, rocksDbFactory, new MemDbFactory(), new FileSystem());
await standardDbInitializer.InitStandardDbsAsync(true);
return dbProvider;
}
Expand Down
1 change: 1 addition & 0 deletions src/Nethermind/Nethermind.Blockchain.Test/ReorgTests.cs
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,7 @@ public void Setup()
LimboLogs.Instance);
TxPool.TxPool txPool = new(
ecdsa,
new BlobTxStorage(),
new ChainHeadInfoProvider(specProvider, _blockTree, stateProvider),
new TxPoolConfig(),
new TxValidator(specProvider.ChainId),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
using NSubstitute;
using NUnit.Framework;
using Nethermind.Config;
using Nethermind.Core.Crypto;

namespace Nethermind.Blockchain.Test
{
Expand Down Expand Up @@ -229,13 +230,30 @@ void SetAccountStates(IEnumerable<Address> missingAddresses)
new(specProvider, blockTree);
IComparer<Transaction> defaultComparer = transactionComparerProvider.GetDefaultComparer();
IComparer<Transaction> comparer = CompareTxByNonce.Instance.ThenBy(defaultComparer);
Dictionary<Address, Transaction[]> transactions = testCase.Transactions
.Where(t => t.SenderAddress is not null)
.GroupBy(t => t.SenderAddress)
.ToDictionary(
g => g.Key!,
g => g.OrderBy(t => t, comparer).ToArray());

Dictionary<Address, Transaction[]> GroupTransactions(bool supportBlobs) =>
testCase.Transactions
.Where(t => t.SenderAddress is not null)
.Where(t => t.SupportsBlobs == supportBlobs)
.GroupBy(t => t.SenderAddress)
.ToDictionary(
g => g.Key!,
g => g.OrderBy(t => t, comparer).ToArray());

Dictionary<Address, Transaction[]> transactions = GroupTransactions(false);
Dictionary<Address, Transaction[]> blobTransactions = GroupTransactions(true);
transactionPool.GetPendingTransactionsBySender().Returns(transactions);
transactionPool.GetPendingLightBlobTransactionsBySender().Returns(blobTransactions);
foreach (Transaction blobTx in blobTransactions.SelectMany(kvp => kvp.Value))
{
transactionPool.TryGetPendingBlobTransaction(Arg.Is<Keccak>(h => h == blobTx.Hash),
out Arg.Any<Transaction?>()).Returns(x =>
{
x[1] = blobTx;
return true;
});
}

BlocksConfig blocksConfig = new() { MinGasPrice = testCase.MinGasPriceForMining };
ITxFilterPipeline txFilterPipeline = new TxFilterPipelineBuilder(LimboLogs.Instance)
.WithMinGasPriceFilter(blocksConfig, specProvider)
Expand All @@ -252,6 +270,7 @@ void SetAccountStates(IEnumerable<Address> missingAddresses)
{
parentHeader = parentHeader.WithExcessBlobGas(0);
}

IEnumerable<Transaction> selectedTransactions =
poolTxSource.GetTransactions(parentHeader.TestObject,
testCase.GasLimit);
Expand Down
7 changes: 7 additions & 0 deletions src/Nethermind/Nethermind.Blockchain/ChainHeadInfoProvider.cs
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@
using Nethermind.Blockchain.Spec;
using Nethermind.Core;
using Nethermind.Core.Specs;
using Nethermind.Evm;
using Nethermind.Int256;
using Nethermind.State;
using Nethermind.TxPool;
Expand Down Expand Up @@ -42,12 +43,18 @@ public ChainHeadInfoProvider(IChainHeadSpecProvider specProvider, IBlockTree blo

public UInt256 CurrentBaseFee { get; private set; }

public UInt256 CurrentPricePerBlobGas { get; internal set; }

public event EventHandler<BlockReplacementEventArgs>? HeadChanged;

private void OnHeadChanged(object? sender, BlockReplacementEventArgs e)
{
BlockGasLimit = e.Block!.GasLimit;
CurrentBaseFee = e.Block.Header.BaseFeePerGas;
CurrentPricePerBlobGas =
BlobGasCalculator.TryCalculateBlobGasPricePerUnit(e.Block.Header, out UInt256 currentPricePerBlobGas)
? currentPricePerBlobGas
: UInt256.Zero;
HeadChanged?.Invoke(sender, e);
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -107,7 +107,13 @@ public On CreateNode(PrivateKey privateKey, bool withGenesisAlreadyProcessed = f
ITransactionComparerProvider transactionComparerProvider =
new TransactionComparerProvider(specProvider, blockTree);

TxPool.TxPool txPool = new(_ethereumEcdsa, new ChainHeadInfoProvider(new FixedForkActivationChainHeadSpecProvider(GoerliSpecProvider.Instance), blockTree, stateProvider), new TxPoolConfig(), new TxValidator(goerliSpecProvider.ChainId), _logManager, transactionComparerProvider.GetDefaultComparer());
TxPool.TxPool txPool = new(_ethereumEcdsa,
new BlobTxStorage(),
new ChainHeadInfoProvider(new FixedForkActivationChainHeadSpecProvider(GoerliSpecProvider.Instance), blockTree, stateProvider),
new TxPoolConfig(),
new TxValidator(goerliSpecProvider.ChainId),
_logManager,
transactionComparerProvider.GetDefaultComparer());
_pools[privateKey] = txPool;

BlockhashProvider blockhashProvider = new(blockTree, LimboLogs.Instance);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -274,6 +274,7 @@ protected override TxPool.TxPool CreateTxPool()

return new TxPool.TxPool(
_api.EthereumEcdsa,
_api.BlobTxStorage ?? NullBlobTxStorage.Instance,
new ChainHeadInfoProvider(_api.SpecProvider, _api.BlockTree, _api.StateReader),
NethermindApi.Config<ITxPoolConfig>(),
_api.TxValidator,
Expand Down
163 changes: 121 additions & 42 deletions src/Nethermind/Nethermind.Consensus/Producers/TxPoolTxSource.cs
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,7 @@

using System;
using System.Collections.Generic;
using System.Diagnostics.CodeAnalysis;
using System.Linq;
using System.Runtime.CompilerServices;
using Nethermind.Consensus.Comparers;
Expand Down Expand Up @@ -46,26 +47,26 @@ public class TxPoolTxSource : ITxSource
public IEnumerable<Transaction> GetTransactions(BlockHeader parent, long gasLimit)
{
long blockNumber = parent.Number + 1;
IEip1559Spec specFor1559 = _specProvider.GetSpecFor1559(blockNumber);
UInt256 baseFee = BaseFeeCalculator.Calculate(parent, specFor1559);
IReleaseSpec spec = _specProvider.GetSpec(parent);
UInt256 baseFee = BaseFeeCalculator.Calculate(parent, spec);
IDictionary<Address, Transaction[]> pendingTransactions = _transactionPool.GetPendingTransactionsBySender();
IDictionary<Address, Transaction[]> pendingBlobTransactionsEquivalences = _transactionPool.GetPendingLightBlobTransactionsBySender();
IComparer<Transaction> comparer = GetComparer(parent, new BlockPreparationContext(baseFee, blockNumber))
.ThenBy(ByHashTxComparer.Instance); // in order to sort properly and not lose transactions we need to differentiate on their identity which provided comparer might not be doing

IEnumerable<Transaction> transactions = GetOrderedTransactions(pendingTransactions, comparer);
IEnumerable<Transaction> blobTransactions = GetOrderedTransactions(pendingBlobTransactionsEquivalences, comparer);
if (_logger.IsDebug) _logger.Debug($"Collecting pending transactions at block gas limit {gasLimit}.");

int checkedTransactions = 0;
int selectedTransactions = 0;
int i = 0;
using ArrayPoolList<Transaction> selectedBlobTxs = new(Eip4844Constants.MaxBlobsPerBlock);

// TODO: removing transactions from TX pool here seems to be a bad practice since they will
// not come back if the block is ignored?
int blobsCounter = 0;
UInt256 blobGasPrice = UInt256.Zero;
SelectBlobTransactions(blobTransactions, parent, spec, selectedBlobTxs);

foreach (Transaction tx in transactions)
{
i++;
checkedTransactions++;

if (tx.SenderAddress is null)
{
Expand All @@ -77,52 +78,130 @@ public IEnumerable<Transaction> GetTransactions(BlockHeader parent, long gasLimi
bool success = _txFilterPipeline.Execute(tx, parent);
if (!success) continue;

if (tx.SupportsBlobs)
foreach (Transaction blobTx in PickBlobTxsBetterThanCurrentTx(selectedBlobTxs, tx, comparer))
{
if (blobGasPrice.IsZero)
{
ulong? excessBlobGas = BlobGasCalculator.CalculateExcessBlobGas(parent, _specProvider.GetSpec(parent));
if (excessBlobGas is null)
{
if (_logger.IsTrace) _logger.Trace($"Declining {tx.ToShortString()}, the specification is not configured to handle shard blob transactions.");
continue;
}
if (!BlobGasCalculator.TryCalculateBlobGasPricePerUnit(excessBlobGas.Value, out blobGasPrice))
{
if (_logger.IsTrace) _logger.Trace($"Declining {tx.ToShortString()}, failed to calculate blob gas price.");
continue;
}
}
yield return blobTx;
}

int txAmountOfBlobs = tx.BlobVersionedHashes?.Length ?? 0;
if (_logger.IsTrace) _logger.Trace($"Selected {tx.ToShortString()} to be potentially included in block.");

if (blobGasPrice > tx.MaxFeePerBlobGas)
{
if (_logger.IsTrace) _logger.Trace($"Declining {tx.ToShortString()}, blob gas fee is too low.");
continue;
}
selectedTransactions++;
yield return tx;
}

if (BlobGasCalculator.CalculateBlobGas(blobsCounter + txAmountOfBlobs) >
Eip4844Constants.MaxBlobGasPerBlock)
{
if (_logger.IsTrace) _logger.Trace($"Declining {tx.ToShortString()}, no more blob space.");
continue;
}
if (selectedBlobTxs.Count > 0)
{
foreach (Transaction blobTx in selectedBlobTxs)
{
yield return blobTx;
}
}

if (_logger.IsDebug) _logger.Debug($"Potentially selected {selectedTransactions} out of {checkedTransactions} pending transactions checked.");
}

blobsCounter += txAmountOfBlobs;
if (_logger.IsTrace) _logger.Trace($"Selected shard blob tx {tx.ToShortString()} to be potentially included in block, total blobs included: {blobsCounter}.");
private IEnumerable<Transaction> PickBlobTxsBetterThanCurrentTx(ArrayPoolList<Transaction> selectedBlobTxs, Transaction tx, IComparer<Transaction> comparer)
{
while (selectedBlobTxs.Count > 0)
{
Transaction blobTx = selectedBlobTxs[0];
if (comparer.Compare(blobTx, tx) > 0)
{
yield return blobTx;
selectedBlobTxs.Remove(blobTx);
}
else
{
if (_logger.IsTrace)
_logger.Trace($"Selected {tx.ToShortString()} to be potentially included in block.");
break;
}
}
}

selectedTransactions++;
yield return tx;
private void SelectBlobTransactions(IEnumerable<Transaction> blobTransactions, BlockHeader parent, IReleaseSpec spec, ArrayPoolList<Transaction> selectedBlobTxs)
{
int checkedBlobTransactions = 0;
int selectedBlobTransactions = 0;
int blobsCounter = 0;
UInt256 blobGasPrice = UInt256.Zero;

foreach (Transaction blobTx in blobTransactions)
{
if (blobsCounter == Eip4844Constants.MaxBlobsPerBlock)
{
if (_logger.IsTrace) _logger.Trace($"Declining {blobTx.ToShortString()}, no more blob space. Block already have {blobsCounter} which is max value allowed.");
break;
}

checkedBlobTransactions++;

int txAmountOfBlobs = blobTx.BlobVersionedHashes?.Length ?? 0;
if (blobsCounter + txAmountOfBlobs > Eip4844Constants.MaxBlobsPerBlock)
{
if (_logger.IsTrace) _logger.Trace($"Declining {blobTx.ToShortString()}, not enough blob space.");
continue;
}

if (blobGasPrice.IsZero && !TryUpdateBlobGasPrice(blobTx, parent, spec, out blobGasPrice))
{
if (_logger.IsTrace) _logger.Trace($"Declining {blobTx.ToShortString()}, failed to get full version of this blob tx from TxPool.");
continue;
}

if (blobGasPrice > blobTx.MaxFeePerBlobGas)
{
if (_logger.IsTrace) _logger.Trace($"Declining {blobTx.ToShortString()}, data gas fee is too low.");
continue;
}

bool success = _txFilterPipeline.Execute(blobTx, parent);
if (!success) continue;

if (!TryGetFullBlobTx(blobTx, out Transaction fullBlobTx))
{
if (_logger.IsTrace) _logger.Trace($"Declining {blobTx.ToShortString()}, failed to get full version of this blob tx from TxPool.");
continue;
}

blobsCounter += txAmountOfBlobs;
if (_logger.IsTrace) _logger.Trace($"Selected shard blob tx {fullBlobTx.ToShortString()} to be potentially included in block, total blobs included: {blobsCounter}.");

selectedBlobTransactions++;
selectedBlobTxs.Add(fullBlobTx);
}

if (_logger.IsDebug) _logger.Debug($"Potentially selected {selectedBlobTransactions} out of {checkedBlobTransactions} pending blob transactions checked.");
}

private bool TryGetFullBlobTx(Transaction blobTx, [NotNullWhen(true)] out Transaction? fullBlobTx)
{
if (blobTx.NetworkWrapper is not null)
{
fullBlobTx = blobTx;
return true;
}

fullBlobTx = null;
return blobTx.Hash is not null && _transactionPool.TryGetPendingBlobTransaction(blobTx.Hash, out fullBlobTx);
}

private bool TryUpdateBlobGasPrice(Transaction lightBlobTx, BlockHeader parent, IReleaseSpec spec, out UInt256 blobGasPrice)
{
ulong? excessDataGas = BlobGasCalculator.CalculateExcessBlobGas(parent, spec);
if (excessDataGas is null)
{
if (_logger.IsTrace) _logger.Trace($"Declining {lightBlobTx.ToShortString()}, the specification is not configured to handle shard blob transactions.");
blobGasPrice = UInt256.Zero;
return false;
}

if (!BlobGasCalculator.TryCalculateBlobGasPricePerUnit(excessDataGas.Value, out blobGasPrice))
{
if (_logger.IsTrace) _logger.Trace($"Declining {lightBlobTx.ToShortString()}, failed to calculate data gas price.");
blobGasPrice = UInt256.Zero;
return false;
}

if (_logger.IsDebug) _logger.Debug($"Potentially selected {selectedTransactions} out of {i} pending transactions checked.");
return true;
}

protected virtual IEnumerable<Transaction> GetOrderedTransactions(IDictionary<Address, Transaction[]> pendingTransactions, IComparer<Transaction> comparer) =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -278,8 +278,9 @@ protected virtual IBlockProducer CreateTestBlockProducer(TxPoolTxSource txPoolTx
protected virtual TxPool.TxPool CreateTxPool() =>
new(
EthereumEcdsa,
new BlobTxStorage(),
new ChainHeadInfoProvider(new FixedForkActivationChainHeadSpecProvider(SpecProvider), BlockTree, ReadOnlyState),
new TxPoolConfig(),
new TxPoolConfig() { BlobSupportEnabled = true },
new TxValidator(SpecProvider.ChainId),
LogManager,
TransactionComparerProvider.GetDefaultComparer());
Expand Down
5 changes: 3 additions & 2 deletions src/Nethermind/Nethermind.Core/Eip4844Constants.cs
Original file line number Diff line number Diff line change
Expand Up @@ -8,10 +8,11 @@ namespace Nethermind.Core;
public class Eip4844Constants
{
public const int MinBlobsPerTransaction = 1;
public const int MaxBlobsPerBlock = 6;

public const ulong BlobGasPerBlob = 1 << 17;
public const ulong TargetBlobGasPerBlock = BlobGasPerBlob * 3;
public const ulong MaxBlobGasPerBlock = BlobGasPerBlob * 6;
public const ulong MaxBlobGasPerBlock = BlobGasPerBlob * MaxBlobsPerBlock;
public const ulong TargetBlobGasPerBlock = MaxBlobGasPerBlock / 2;
public const ulong MaxBlobGasPerTransaction = MaxBlobGasPerBlock;

public static readonly UInt256 BlobGasUpdateFraction = 3338477;
Expand Down
3 changes: 2 additions & 1 deletion src/Nethermind/Nethermind.Core/Transaction.cs
Original file line number Diff line number Diff line change
Expand Up @@ -145,7 +145,8 @@ private void ClearPreHashInternal()
/// <remarks>Used for sorting in edge cases.</remarks>
public ulong PoolIndex { get; set; }

private int? _size = null;
protected int? _size = null;

/// <summary>
/// Encoded transaction length
/// </summary>
Expand Down

0 comments on commit d8c124d

Please sign in to comment.