diff --git a/CHANGES.md b/CHANGES.md
index 63bb89be7f6..69ef465c94c 100644
--- a/CHANGES.md
+++ b/CHANGES.md
@@ -10,6 +10,13 @@ To be released.
### Backward-incompatible API changes
+ - Changed `BlockLocator.Create()` to take a single `BlockHash`. [[#3913]]
+ - Changed `BlockLocator()` to throw an `ArgumentException` when
+ the number of given `BlockHash`es is not 1. [[#3913]]
+ - Removed `threshold` parameter from `BlockChain.GetBlockLocator()`.
+ [[#3913]]
+ - (Libplanet.Net) Removed `SwarmOptions.BranchpointThreshold` property.
+ [[#3913]]
- (Libplanet.Store) Removed unused `HashNode.Serialize()` method.
[[#3922], [#3924]]
@@ -21,6 +28,8 @@ To be released.
### Behavioral changes
+ - Changed `BlockChain.FindBranchPoint()` to only check for the first
+ `BlockHash` in a given `BlockLocator`. [[#3913]]
- (Libplanet.Store) Optimized `HashNode.ToBencodex()` method.
[[#3922], [#3924]]
@@ -30,6 +39,7 @@ To be released.
### CLI tools
+[#3913]: https://github.com/planetarium/libplanet/pull/3913
[#3922]: https://github.com/planetarium/libplanet/issues/3922
[#3924]: https://github.com/planetarium/libplanet/pull/3924
diff --git a/src/Libplanet.Net/Options/SwarmOptions.cs b/src/Libplanet.Net/Options/SwarmOptions.cs
index 0b8c5f01941..3c84f703fd9 100644
--- a/src/Libplanet.Net/Options/SwarmOptions.cs
+++ b/src/Libplanet.Net/Options/SwarmOptions.cs
@@ -46,13 +46,6 @@ public class SwarmOptions
///
public TimeSpan StaticPeersMaintainPeriod { get; set; } = TimeSpan.FromSeconds(10);
- ///
- /// The threshold for detecting branchpoint when block synchronization.
- /// If the branch point is outside threshold from the ,
- /// using an approximated value.
- ///
- public int BranchpointThreshold { get; set; } = 10;
-
///
/// The minimum number to select from routing table when broadcast messages.
/// It is 10 by default.
diff --git a/src/Libplanet.Net/Swarm.BlockCandidate.cs b/src/Libplanet.Net/Swarm.BlockCandidate.cs
index 91493924d98..83bc085d4bf 100644
--- a/src/Libplanet.Net/Swarm.BlockCandidate.cs
+++ b/src/Libplanet.Net/Swarm.BlockCandidate.cs
@@ -437,7 +437,7 @@ private async Task BlockCandidateDownload(
{
var sessionRandom = new Random();
int subSessionId = sessionRandom.Next();
- BlockLocator locator = blockChain.GetBlockLocator(Options.BranchpointThreshold);
+ BlockLocator locator = blockChain.GetBlockLocator();
Block tip = blockChain.Tip;
IAsyncEnumerable> hashesAsync = GetBlockHashes(
diff --git a/src/Libplanet.Net/Swarm.cs b/src/Libplanet.Net/Swarm.cs
index 7c462333249..71cefc7001d 100644
--- a/src/Libplanet.Net/Swarm.cs
+++ b/src/Libplanet.Net/Swarm.cs
@@ -999,7 +999,7 @@ internal async IAsyncEnumerable GetTxsAsync(
[EnumeratorCancellation] CancellationToken cancellationToken = default
)
{
- BlockLocator locator = blockChain.GetBlockLocator(Options.BranchpointThreshold);
+ BlockLocator locator = blockChain.GetBlockLocator();
var exceptions = new List();
foreach ((BoundPeer peer, IBlockExcerpt excerpt) in peersWithExcerpts)
{
@@ -1118,32 +1118,9 @@ await GetBlockHashes(
);
}
- locator = BlockLocator.Create(
- startIndex: branchingIndex + downloaded.Count,
- idx =>
- {
- long arg = idx;
- if (idx <= branchingIndex)
- {
- return blockChain.Store.IndexBlockHash(blockChain.Id, idx);
- }
-
- int relIdx = (int)(idx - branchingIndex - 1);
-
- try
- {
- return downloaded[relIdx];
- }
- catch (ArgumentOutOfRangeException e)
- {
- const string msg =
- "Failed to look up a block hash by its index {Index} " +
- "(branching index: {BranchingIndex}; " +
- "downloaded: {Downloaded})";
- _logger.Error(e, msg, arg, branchingIndex, downloaded.Count);
- return null;
- }
- });
+ locator = downloaded.Count > 0
+ ? BlockLocator.Create(tipHash: downloaded.Last())
+ : locator;
}
while (downloaded.Count < chunkBlockHashesToDownload);
}
diff --git a/src/Libplanet/Blockchain/BlockChain.cs b/src/Libplanet/Blockchain/BlockChain.cs
index 5d7b3e1f484..4127ac57828 100644
--- a/src/Libplanet/Blockchain/BlockChain.cs
+++ b/src/Libplanet/Blockchain/BlockChain.cs
@@ -812,28 +812,18 @@ public BlockChain Fork(BlockHash point, bool inheritRenderers = true)
///
/// Returns a new from the tip of current chain.
///
- /// The amount of consequent blocks to include before sampling.
- ///
/// A instance of block locator.
- public BlockLocator GetBlockLocator(int threshold = 10)
+ public BlockLocator GetBlockLocator()
{
- long startIndex;
- Guid id;
_rwlock.EnterReadLock();
try
{
- startIndex = Tip.Index;
- id = Id;
+ return BlockLocator.Create(tipHash: Tip.Hash);
}
finally
{
_rwlock.ExitReadLock();
}
-
- return BlockLocator.Create(
- startIndex: startIndex,
- indexToBlockHash: idx => Store.IndexBlockHash(Id, idx),
- sampleAfter: threshold);
}
///
@@ -1293,12 +1283,12 @@ internal void AppendStateRootHashPreceded(
#pragma warning restore MEN003
///
- /// Find an approximate to the topmost common ancestor between this
+ /// Finds an approximate topmost common ancestor between this
/// and a given .
///
- /// A block locator that contains candidate common ancestors.
- /// An approximate to the topmost common ancestor. If it failed to find anything
- /// returns .
+ /// A block locator that contains common ancestor candidates.
+ /// An approximate to the topmost common ancestor if found, otherwise
+ /// .
internal BlockHash? FindBranchpoint(BlockLocator locator)
{
try
@@ -1308,19 +1298,16 @@ internal void AppendStateRootHashPreceded(
_logger.Debug(
"Finding a branchpoint with locator [{LocatorHead}, ...]",
locator.FirstOrDefault());
- foreach (BlockHash hash in locator)
+ BlockHash hash = locator.FirstOrDefault();
+ if (_blocks.ContainsKey(hash)
+ && _blocks[hash] is Block block
+ && hash.Equals(Store.IndexBlockHash(Id, block.Index)))
{
- if (_blocks.ContainsKey(hash)
- && _blocks[hash] is Block block
- && hash.Equals(Store.IndexBlockHash(Id, block.Index)))
- {
- _logger.Debug(
- "Found a branchpoint with locator [{LocatorHead}, ...]: {Hash}",
- locator.FirstOrDefault(),
- hash
- );
- return hash;
- }
+ _logger.Debug(
+ "Found a branchpoint with locator [{LocatorHead}, ...]: {Hash}",
+ locator.FirstOrDefault(),
+ hash);
+ return hash;
}
_logger.Debug(
diff --git a/src/Libplanet/Blockchain/BlockLocator.cs b/src/Libplanet/Blockchain/BlockLocator.cs
index abb66198cef..dff2ae49397 100644
--- a/src/Libplanet/Blockchain/BlockLocator.cs
+++ b/src/Libplanet/Blockchain/BlockLocator.cs
@@ -17,97 +17,29 @@ public class BlockLocator : IEnumerable
/// Initializes a new instance of from .
///
/// Enumerable of es to convert from.
- /// Thrown when is empty.
+ /// Thrown when
+ /// does not consist of a single element.
///
public BlockLocator(IEnumerable hashes)
{
- _impl = hashes.Any()
+ _impl = hashes.Count() == 1
? hashes.ToList()
: throw new ArgumentException(
- $"Given {nameof(hashes)} cannot be empty.", nameof(hashes));
+ $"Given {nameof(hashes)} must have exactly one element: {hashes.Count()}",
+ nameof(hashes));
}
///
///
- /// Creates a new instance of with an indexer
- /// function, sampling after number of
- /// s.
- ///
- ///
- /// This collects all es corresponding to indices inductively
- /// defined by:
- ///
- /// -
- /// i_0 = startIndex
- ///
- /// -
- /// i_k = max(i_(k - 1) - 1, 0) for 0 < k and k <= s
- ///
- /// -
- /// i_k = max(i_(k - 1) - 2^(k - 1 - s), 0) for 0 < k
- /// and s < k
- ///
- ///
- /// where s = max(sampleAfter, 0) and the sequence terminates after index i_k
- /// reaches zero or the returned by
- /// for i_k is ,
- /// in which case the corresponding to index 0
- /// (presumably a of the genesis )
- /// is added at the end.
+ /// Creates a new instance of .
///
///
- /// The starting index.
- /// The function that converts an index to a
- /// . This can be which indicates
- /// a missing at the index. Any value from 0 to
- /// may be used as an argument to call this function.
- /// The number of consecutive blocks to include before sampling.
- ///
- /// Thrown when
- /// is negative.
- /// Thrown when either returned
- /// by for index 0 is .
- ///
+ /// The of the tip.
///
/// An instance of created with given arguments.
///
- ///
- /// Returned created by this factory method is guaranteed
- /// to have the corresponding to index 0.
- ///
- public static BlockLocator Create(
- long startIndex,
- Func indexToBlockHash,
- long sampleAfter = 10)
- {
- if (startIndex < 0)
- {
- throw new ArgumentOutOfRangeException(
- nameof(startIndex),
- $"Given {nameof(startIndex)} cannot be negative: {startIndex}");
- }
-
- BlockHash genesisHash = indexToBlockHash(0) ??
- throw new ArgumentException(
- $"Given {nameof(indexToBlockHash)} should not be null at zero index.",
- nameof(indexToBlockHash));
- var hashes = new List();
-
- foreach (long index in GetEnumeratedIndices(startIndex, sampleAfter))
- {
- if (indexToBlockHash(index) is { } hash)
- {
- hashes.Add(hash);
- }
- else
- {
- hashes.Add(genesisHash);
- break;
- }
- }
-
- return new BlockLocator(hashes);
- }
+ public static BlockLocator Create(BlockHash tipHash) =>
+ new BlockLocator(new[] { tipHash });
///
/// Gets the enumerator.
@@ -122,19 +54,5 @@ IEnumerator IEnumerable.GetEnumerator()
{
return _impl.GetEnumerator();
}
-
- private static IEnumerable GetEnumeratedIndices(long startIndex, long sampleAfter)
- {
- long currentIndex = startIndex;
- long step = 1;
- while (currentIndex > 0)
- {
- yield return currentIndex;
- currentIndex = Math.Max(currentIndex - step, 0);
- step = startIndex - currentIndex <= sampleAfter ? step : step * 2;
- }
-
- yield return currentIndex;
- }
}
}
diff --git a/test/Libplanet.Net.Tests/SwarmTest.Broadcast.cs b/test/Libplanet.Net.Tests/SwarmTest.Broadcast.cs
index 77cb117613b..f40e95cd1c2 100644
--- a/test/Libplanet.Net.Tests/SwarmTest.Broadcast.cs
+++ b/test/Libplanet.Net.Tests/SwarmTest.Broadcast.cs
@@ -271,14 +271,12 @@ CancellationToken cancellationToken
await a.AddPeersAsync(new[] { b.AsPeer }, null);
var minerCanceller = new CancellationTokenSource();
- Task miningA = CreateMiner(minerA, a, chainA, 5000, minerCanceller.Token);
- Task miningB = CreateMiner(minerB, b, chainB, 8000, minerCanceller.Token);
+ Task miningA = CreateMiner(minerA, a, chainA, 4000, minerCanceller.Token);
await Task.Delay(10000);
minerCanceller.Cancel();
- await Task.WhenAll(miningA, miningB);
-
+ await miningA;
await Task.Delay(5000);
}
finally
@@ -652,13 +650,10 @@ public async Task CanBroadcastBlock()
Block block = chainA.ProposeBlock(
keyA, CreateBlockCommit(chainA.Tip));
chainA.Append(block, TestUtils.CreateBlockCommit(block));
- }
-
- foreach (int i in Enumerable.Range(0, 3))
- {
- Block block = chainB.ProposeBlock(
- keyB, CreateBlockCommit(chainB.Tip));
- chainB.Append(block, TestUtils.CreateBlockCommit(block));
+ if (i < 5)
+ {
+ chainB.Append(block, TestUtils.CreateBlockCommit(block));
+ }
}
try
@@ -670,7 +665,7 @@ public async Task CanBroadcastBlock()
await BootstrapAsync(swarmB, swarmA.AsPeer);
await BootstrapAsync(swarmC, swarmA.AsPeer);
- swarmB.BroadcastBlock(chainB[-1]);
+ swarmB.BroadcastBlock(chainB.Tip);
// chainA ignores block header received because its index is shorter.
await swarmA.BlockHeaderReceived.WaitAsync();
@@ -681,7 +676,7 @@ public async Task CanBroadcastBlock()
// than chainA
Assert.NotEqual(chainB, chainA);
- swarmA.BroadcastBlock(chainA[-1]);
+ swarmA.BroadcastBlock(chainA.Tip);
await swarmB.BlockAppended.WaitAsync();
await swarmC.BlockAppended.WaitAsync();
@@ -709,8 +704,7 @@ public async Task BroadcastBlockWithSkip()
var blockChain = MakeBlockChain(
policy, fx1.Store, fx1.StateStore, new SingleActionLoader(typeof(DumbAction)));
var privateKey = new PrivateKey();
- var minerSwarm =
- await CreateSwarm(blockChain, privateKey).ConfigureAwait(false);
+ var minerSwarm = await CreateSwarm(blockChain, privateKey).ConfigureAwait(false);
var fx2 = new MemoryStoreFixture();
var receiverRenderer = new RecordingActionRenderer();
var loggedRenderer = new LoggedActionRenderer(
@@ -902,29 +896,24 @@ public async Task PullBlocks()
BlockChain chainB = swarmB.BlockChain;
BlockChain chainC = swarmC.BlockChain;
- foreach (int i in Enumerable.Range(0, 10))
+ foreach (int i in Enumerable.Range(0, 5))
{
- Block block = chainA.ProposeBlock(
- keyA, CreateBlockCommit(chainA.Tip));
+ Block block = chainA.ProposeBlock(keyA, CreateBlockCommit(chainA.Tip));
chainA.Append(block, TestUtils.CreateBlockCommit(block));
+ if (i < 3)
+ {
+ chainC.Append(block, TestUtils.CreateBlockCommit(block));
+ }
}
Block chainATip = chainA.Tip;
- foreach (int i in Enumerable.Range(0, 5))
+ foreach (int i in Enumerable.Range(0, 10))
{
- Block block = chainB.ProposeBlock(
- keyB, CreateBlockCommit(chainB.Tip));
+ Block block = chainB.ProposeBlock(keyB, CreateBlockCommit(chainB.Tip));
chainB.Append(block, TestUtils.CreateBlockCommit(block));
}
- foreach (int i in Enumerable.Range(0, 3))
- {
- Block block = chainC.ProposeBlock(
- keyB, CreateBlockCommit(chainC.Tip));
- chainC.Append(block, TestUtils.CreateBlockCommit(block));
- }
-
try
{
await StartAsync(swarmA);
diff --git a/test/Libplanet.Net.Tests/SwarmTest.Preload.cs b/test/Libplanet.Net.Tests/SwarmTest.Preload.cs
index 5d1a8396813..aee8c9b96f1 100644
--- a/test/Libplanet.Net.Tests/SwarmTest.Preload.cs
+++ b/test/Libplanet.Net.Tests/SwarmTest.Preload.cs
@@ -1090,70 +1090,6 @@ await receiverSwarm.AddPeersAsync(
}
}
- [Fact(Timeout = Timeout)]
- public async Task ActionExecutionWithBranchpoint()
- {
- var policy = new BlockPolicy(
- new PolicyActionsRegistry(
- endBlockActions: ImmutableArray.Create(new MinerReward(1))));
- var fx1 = new MemoryStoreFixture(policy.PolicyActionsRegistry);
- var fx2 = new MemoryStoreFixture(policy.PolicyActionsRegistry);
- var seedChain = MakeBlockChain(
- policy, fx1.Store, fx1.StateStore, new SingleActionLoader(typeof(DumbAction)));
- var receiverChain = MakeBlockChain(
- policy, fx2.Store, fx2.StateStore, new SingleActionLoader(typeof(DumbAction)));
-
- var seedKey = new PrivateKey();
-
- Swarm seed =
- await CreateSwarm(seedChain, seedKey).ConfigureAwait(false);
- Swarm receiver =
- await CreateSwarm(receiverChain).ConfigureAwait(false);
-
- for (int i = 0; i < 10; i++)
- {
- var block = seedChain.ProposeBlock(
- seedKey, CreateBlockCommit(seedChain.Tip));
- seedChain.Append(block, TestUtils.CreateBlockCommit(block));
- receiverChain.Append(block, TestUtils.CreateBlockCommit(block));
- }
-
- var forked = seedChain.Fork(seedChain[5].Hash);
- for (int i = 0; i < 10; i++)
- {
- Block block = forked.ProposeBlock(
- seedKey, CreateBlockCommit(forked.Tip));
- forked.Append(block, TestUtils.CreateBlockCommit(block));
- }
-
- seedChain.Swap(forked, false);
- var actionExecutionCount = 0;
-
- var progress = new ActionProgress(state =>
- {
- if (state is ActionExecutionState)
- {
- actionExecutionCount++;
- }
- });
-
- try
- {
- await StartAsync(seed);
- await BootstrapAsync(receiver, seed.AsPeer);
- await receiver.PreloadAsync(progress: progress);
- await Task.Delay(500);
-
- Assert.Equal(seedChain.Tip, receiverChain.Tip);
- Assert.Equal(10, actionExecutionCount);
- }
- finally
- {
- CleaningSwarm(seed);
- CleaningSwarm(receiver);
- }
- }
-
[Fact(Timeout = Timeout)]
public async Task UpdateTxExecution()
{
diff --git a/test/Libplanet.Net.Tests/SwarmTest.cs b/test/Libplanet.Net.Tests/SwarmTest.cs
index 184fe796c81..058a4b38431 100644
--- a/test/Libplanet.Net.Tests/SwarmTest.cs
+++ b/test/Libplanet.Net.Tests/SwarmTest.cs
@@ -14,7 +14,6 @@
using Libplanet.Action.Tests.Common;
using Libplanet.Blockchain;
using Libplanet.Blockchain.Policies;
-using Libplanet.Blockchain.Renderers.Debug;
using Libplanet.Common;
using Libplanet.Crypto;
using Libplanet.Net.Consensus;
@@ -875,35 +874,25 @@ async Task MineAndBroadcast(CancellationToken cancellationToken)
}
[Fact(Timeout = Timeout)]
- public async Task RenderInFork()
+ public async Task CannotBlockSyncWithForkedChain()
{
- var policy = new BlockPolicy(
- new PolicyActionsRegistry(
- endBlockActions: ImmutableArray.Create(new MinerReward(1))));
- var renderer = new RecordingActionRenderer();
- var chain = MakeBlockChain(
+ var policy = new NullBlockPolicy();
+ var chain1 = MakeBlockChain(
policy,
new MemoryStore(),
new TrieStateStore(new MemoryKeyValueStore()),
- new SingleActionLoader(typeof(DumbAction)),
- renderers: new[] { renderer }
- );
+ new SingleActionLoader(typeof(DumbAction)));
+ var chain2 = MakeBlockChain(
+ policy,
+ new MemoryStore(),
+ new TrieStateStore(new MemoryKeyValueStore()),
+ new SingleActionLoader(typeof(DumbAction)));
var key1 = new PrivateKey();
var key2 = new PrivateKey();
- var miner1 = await CreateSwarm(chain, key1).ConfigureAwait(false);
- var miner2 = await CreateSwarm(
- MakeBlockChain(
- policy,
- new MemoryStore(),
- new TrieStateStore(new MemoryKeyValueStore()),
- new SingleActionLoader(typeof(DumbAction))
- ),
- key2
- ).ConfigureAwait(false);
-
- int renderCount = 0;
+ var miner1 = await CreateSwarm(chain1, key1).ConfigureAwait(false);
+ var miner2 = await CreateSwarm(chain2, key2).ConfigureAwait(false);
var privKey = new PrivateKey();
var addr = miner1.Address;
@@ -913,6 +902,7 @@ public async Task RenderInFork()
Block block1 = miner1.BlockChain.ProposeBlock(
key1, CreateBlockCommit(miner1.BlockChain.Tip));
miner1.BlockChain.Append(block1, TestUtils.CreateBlockCommit(block1));
+ var miner1TipHash = miner1.BlockChain.Tip.Hash;
miner2.BlockChain.MakeTransaction(privKey, new[] { DumbAction.Create((addr, item)) });
Block block2 = miner2.BlockChain.ProposeBlock(
@@ -924,9 +914,6 @@ public async Task RenderInFork()
key2, CreateBlockCommit(miner2.BlockChain.Tip));
miner2.BlockChain.Append(latest, TestUtils.CreateBlockCommit(latest));
- renderer.RenderEventHandler += (_, a) =>
- renderCount += IsDumbAction(a) ? 1 : 0;
-
await StartAsync(miner1);
await StartAsync(miner2);
@@ -934,205 +921,13 @@ public async Task RenderInFork()
miner2.BroadcastBlock(latest);
- await miner1.BlockReceived.WaitAsync();
- await miner1.BlockAppended.WaitAsync();
-
- Assert.Equal(miner1.BlockChain.Tip, miner2.BlockChain.Tip);
- Assert.Equal(miner1.BlockChain.Count, miner2.BlockChain.Count);
- Assert.Equal(2, renderCount);
+ await Task.Delay(5_000);
+ Assert.Equal(miner1TipHash, miner1.BlockChain.Tip.Hash);
CleaningSwarm(miner1);
CleaningSwarm(miner2);
}
- [Fact(Skip = "This should be fixed to work deterministically.")]
- public async Task HandleReorgInSynchronizing()
- {
- var policy = new BlockPolicy(
- new PolicyActionsRegistry(
- endBlockActions: ImmutableArray.Create(new MinerReward(1))));
-
- async Task MakeSwarm(PrivateKey key = null) =>
- await CreateSwarm(
- MakeBlockChain(
- policy,
- new MemoryStore(),
- new TrieStateStore(new MemoryKeyValueStore()),
- new SingleActionLoader(typeof(Sleep))
- ),
- key
- );
-
- var key1 = new PrivateKey();
- var key2 = new PrivateKey();
-
- var miner1 = await MakeSwarm(key1).ConfigureAwait(false);
- var miner2 = await MakeSwarm(key2).ConfigureAwait(false);
- var receiver = await MakeSwarm().ConfigureAwait(false);
-
- foreach (var i in Enumerable.Range(0, 8))
- {
- miner1.BlockChain.StageTransaction(
- Transaction.Create(
- 0,
- new PrivateKey(),
- miner1.BlockChain.Genesis.Hash,
- actions: new[] { new Sleep() }.ToPlainValues()
- )
- );
- var b = miner1.BlockChain.ProposeBlock(
- key1,
- CreateBlockCommit(
- miner1.BlockChain.Tip.Hash,
- miner1.BlockChain.Tip.Index,
- 0));
- miner1.BlockChain.Append(b, TestUtils.CreateBlockCommit(b));
- miner2.BlockChain.Append(b, TestUtils.CreateBlockCommit(b));
- }
-
- try
- {
- await StartAsync(miner1);
- await StartAsync(miner2);
-
- await BootstrapAsync(miner2, miner1.AsPeer);
- await BootstrapAsync(receiver, miner1.AsPeer);
-
- var t = receiver.PreloadAsync();
- Block block1 = miner1.BlockChain.ProposeBlock(key1);
- miner1.BlockChain.Append(block1, TestUtils.CreateBlockCommit(block1));
- Block block2 = miner2.BlockChain.ProposeBlock(key1);
- miner2.BlockChain.Append(block2, TestUtils.CreateBlockCommit(block2));
- Block latest = miner2.BlockChain.ProposeBlock(key2);
- miner2.BlockChain.Append(latest, TestUtils.CreateBlockCommit(latest));
- miner2.BroadcastBlock(latest);
- await t;
-
- Assert.Equal(miner1.BlockChain.Tip, miner2.BlockChain.Tip);
- Assert.Equal(miner1.BlockChain.Count, miner2.BlockChain.Count);
- Assert.Equal(miner1.BlockChain.Count, receiver.BlockChain.Count);
- Assert.Equal(miner1.BlockChain.Tip, receiver.BlockChain.Tip);
- }
- finally
- {
- CleaningSwarm(miner1);
- CleaningSwarm(miner2);
- CleaningSwarm(receiver);
- }
- }
-
- [Theory(Timeout = Timeout)]
- [InlineData(true)]
- [InlineData(false)]
- public async void RestageTransactionsOnceLocallyMinedAfterReorg(bool restage)
- {
- var keyA = new PrivateKey();
- var keyB = new PrivateKey();
-
- var minerA = await CreateSwarm(keyA).ConfigureAwait(false);
- var minerB = await CreateSwarm(keyB).ConfigureAwait(false);
-
- var privateKeyA = new PrivateKey();
- var privateKeyB = new PrivateKey();
-
- var targetAddress1 = new PrivateKey().Address;
- var targetAddress2 = new PrivateKey().Address;
-
- try
- {
- const string dumbItem = "item0.0";
- var txA = minerA.BlockChain.MakeTransaction(
- privateKeyA,
- new[] { DumbAction.Create((targetAddress1, dumbItem)), });
- var txB = minerB.BlockChain.MakeTransaction(
- privateKeyB,
- new[] { DumbAction.Create((targetAddress2, dumbItem)), });
-
- if (!restage)
- {
- minerB.BlockChain.StageTransaction(txA);
- }
-
- Log.Debug("Make minerB's chain longer than minerA's chain");
- Block blockA = minerA.BlockChain.ProposeBlock(
- keyA, CreateBlockCommit(minerA.BlockChain.Tip));
- minerA.BlockChain.Append(blockA, TestUtils.CreateBlockCommit(blockA));
- Block blockB = minerB.BlockChain.ProposeBlock(
- keyB, CreateBlockCommit(minerB.BlockChain.Tip));
- minerB.BlockChain.Append(blockB, TestUtils.CreateBlockCommit(blockB));
- Block blockC = minerB.BlockChain.ProposeBlock(
- keyB, CreateBlockCommit(minerB.BlockChain.Tip));
- minerB.BlockChain.Append(blockC, TestUtils.CreateBlockCommit(blockC));
-
- Assert.Equal(
- (Text)dumbItem,
- minerA.BlockChain
- .GetNextWorldState()
- .GetAccountState(ReservedAddresses.LegacyAccount)
- .GetState(targetAddress1));
- Assert.Equal(
- (Text)dumbItem,
- minerB.BlockChain
- .GetNextWorldState()
- .GetAccountState(ReservedAddresses.LegacyAccount)
- .GetState(targetAddress2));
-
- await StartAsync(minerA);
- await StartAsync(minerB);
-
- await BootstrapAsync(minerA, minerB.AsPeer);
-
- Log.Debug("Reorg occurs");
- minerB.BroadcastBlock(blockC);
- await minerA.BlockAppended.WaitAsync();
-
- Assert.Equal(minerA.BlockChain.Tip, minerB.BlockChain.Tip);
- Assert.Equal(3, minerA.BlockChain.Count);
- Assert.Equal(
- restage ? null : (Text?)dumbItem,
- minerA.BlockChain
- .GetNextWorldState()
- .GetAccountState(ReservedAddresses.LegacyAccount)
- .GetState(targetAddress1));
- Assert.Equal(
- (Text)dumbItem,
- minerA.BlockChain
- .GetNextWorldState()
- .GetAccountState(ReservedAddresses.LegacyAccount)
- .GetState(targetAddress2));
-
- Log.Debug("Check if txs in unrendered blocks staged again");
- Assert.Equal(
- restage,
- minerA.BlockChain.GetStagedTransactionIds().Contains(txA.Id));
-
- Block block = minerA.BlockChain.ProposeBlock(
- keyA, CreateBlockCommit(minerA.BlockChain.Tip));
- minerA.BlockChain.Append(block, TestUtils.CreateBlockCommit(block));
- minerA.BroadcastBlock(minerA.BlockChain.Tip);
- await minerB.BlockAppended.WaitAsync();
-
- Assert.Equal(minerA.BlockChain.Tip, minerB.BlockChain.Tip);
- Assert.Equal(
- (Text)dumbItem,
- minerA.BlockChain
- .GetNextWorldState()
- .GetAccountState(ReservedAddresses.LegacyAccount)
- .GetState(targetAddress1));
- Assert.Equal(
- (Text)dumbItem,
- minerA.BlockChain
- .GetNextWorldState()
- .GetAccountState(ReservedAddresses.LegacyAccount)
- .GetState(targetAddress2));
- }
- finally
- {
- CleaningSwarm(minerA);
- CleaningSwarm(minerB);
- }
- }
-
[Fact(Timeout = Timeout)]
public async Task UnstageInvalidTransaction()
{
@@ -1263,124 +1058,8 @@ TxPolicyViolationException IsSignerValid(
}
}
- // NOTE: Possibly not a valid test scenario.
- [Fact(Timeout = Timeout)]
- public async Task CreateNewChainWhenBranchPointNotExist()
- {
- // If the bucket stored peers are the same, the block may not propagate,
- // so specify private keys to make the buckets different.
- PrivateKey keyA = PrivateKey.FromString(
- "8568eb6f287afedece2c7b918471183db0451e1a61535bb0381cfdf95b85df20");
- PrivateKey keyB = PrivateKey.FromString(
- "c34f7498befcc39a14f03b37833f6c7bb78310f1243616524eda70e078b8313c");
- PrivateKey keyC = PrivateKey.FromString(
- "941bc2edfab840d79914d80fe3b30840628ac37a5d812d7f922b5d2405a223d3");
-
- var policy = new NullBlockPolicy();
- var policyA = new NullBlockPolicy();
- var policyB = new NullBlockPolicy();
- var fx = new DefaultStoreFixture();
- var aev = new ActionEvaluator(
- new PolicyActionsRegistry(),
- fx.StateStore,
- new SingleActionLoader(typeof(DumbAction)));
- var genesis = fx.GenesisBlock;
- var nextSrh = aev.Evaluate(genesis, genesis.StateRootHash).Last().OutputState;
- Block aBlock1 = ProposeNextBlock(
- genesis,
- keyA,
- stateRootHash: nextSrh);
- Block aBlock2 = ProposeNextBlock(
- aBlock1,
- keyA,
- stateRootHash: nextSrh,
- lastCommit: CreateBlockCommit(aBlock1));
- Block aBlock3 = ProposeNextBlock(
- aBlock2,
- keyA,
- stateRootHash: nextSrh,
- lastCommit: CreateBlockCommit(aBlock2));
- Block bBlock1 = ProposeNextBlock(
- genesis,
- keyB,
- stateRootHash: nextSrh);
- Block bBlock2 = ProposeNextBlock(
- bBlock1,
- keyB,
- stateRootHash: nextSrh,
- lastCommit: CreateBlockCommit(bBlock1));
-
- policyA.BlockedMiners.Add(keyB.Address);
- policyB.BlockedMiners.Add(keyA.Address);
-
- var minerSwarmA =
- await CreateSwarm(keyA, policy: policyA, genesis: genesis).ConfigureAwait(false);
- var minerSwarmB =
- await CreateSwarm(keyB, policy: policyB, genesis: genesis).ConfigureAwait(false);
- var receiverSwarm =
- await CreateSwarm(keyC, policy: policy, genesis: genesis).ConfigureAwait(false);
-
- BlockChain minerChainA = minerSwarmA.BlockChain;
- BlockChain minerChainB = minerSwarmB.BlockChain;
- BlockChain receiverChain = receiverSwarm.BlockChain;
-
- try
- {
- await StartAsync(minerSwarmA, 5000);
- await StartAsync(minerSwarmB, 5000);
- await StartAsync(receiverSwarm);
-
- await BootstrapAsync(minerSwarmA, receiverSwarm.AsPeer);
- await BootstrapAsync(minerSwarmB, receiverSwarm.AsPeer);
-
- // Broadcast SwarmA's first block.
- minerChainA.Append(aBlock1, TestUtils.CreateBlockCommit(aBlock1));
- await receiverSwarm.BlockAppended.WaitAsync();
- await AssertThatEventually(
- () => receiverChain.Tip.Equals(minerChainA.Tip),
- 15_000,
- output: _output,
- conditionLabel:
- $"{nameof(receiverChain)}'s tip being same to " +
- $"{nameof(minerChainA)}'s tip 1st"
- );
- minerChainB.Append(bBlock1, TestUtils.CreateBlockCommit(bBlock1));
-
- // Broadcast SwarmB's second block.
- minerChainB.Append(bBlock2, TestUtils.CreateBlockCommit(bBlock2));
- await receiverSwarm.BlockAppended.WaitAsync();
- await AssertThatEventually(
- () => receiverChain.Tip.Equals(minerChainB.Tip),
- 15_000,
- output: _output,
- conditionLabel:
- $"{nameof(receiverChain)}'s tip being same to " +
- $"{nameof(minerChainB)}'s tip 2nd"
- );
- minerChainA.Append(aBlock2, TestUtils.CreateBlockCommit(aBlock2));
-
- // Broadcast SwarmA's third block.
- minerChainA.Append(aBlock3, TestUtils.CreateBlockCommit(aBlock3));
- await receiverSwarm.BlockAppended.WaitAsync();
- await AssertThatEventually(
- () => receiverChain.Tip.Equals(minerChainA.Tip),
- 15_000,
- output: _output,
- conditionLabel:
- $"{nameof(receiverChain)}'s tip being same to " +
- $"{nameof(minerChainA)}'s tip 3rd"
- );
- }
- finally
- {
- CleaningSwarm(minerSwarmA);
- CleaningSwarm(minerSwarmB);
- CleaningSwarm(receiverSwarm);
- }
- }
-
[Fact(Timeout = Timeout)]
- public async Task DoNotReceiveBlockFromNodeHavingDifferenceGenesisBlock()
+ public async Task DoNotReceiveBlockFromNodeHavingDifferentGenesisBlock()
{
var keyA = ByteUtil.ParseHex(
"8568eb6f287afedece2c7b918471183db0451e1a61535bb0381cfdf95b85df20");
diff --git a/test/Libplanet.Tests/Blockchain/BlockChainTest.cs b/test/Libplanet.Tests/Blockchain/BlockChainTest.cs
index acccd29c3be..35c2393529f 100644
--- a/test/Libplanet.Tests/Blockchain/BlockChainTest.cs
+++ b/test/Libplanet.Tests/Blockchain/BlockChainTest.cs
@@ -461,7 +461,7 @@ public void FindNextHashes()
Assert.Equal(0, offsetIndex);
Assert.Equal(new[] { block0.Hash, block1.Hash, block2.Hash, block3.Hash }, hashes);
- _blockChain.FindNextHashes(new BlockLocator(new[] { block1.Hash, block0.Hash }))
+ _blockChain.FindNextHashes(new BlockLocator(new[] { block1.Hash }))
.Deconstruct(out offsetIndex, out hashes);
Assert.Equal(1, offsetIndex);
Assert.Equal(new[] { block1.Hash, block2.Hash, block3.Hash }, hashes);
@@ -499,8 +499,8 @@ public void FindNextHashesAfterFork()
forked.FindNextHashes(locator)
.Deconstruct(out long? offset, out IReadOnlyList hashes);
- Assert.Equal(forked[0].Index, offset);
- Assert.Equal(new[] { forked[0].Hash, forked[1].Hash }, hashes);
+ Assert.Null(offset);
+ Assert.Empty(hashes);
}
[SkippableFact]
@@ -803,17 +803,8 @@ public void GetBlockLocator()
blocks.Add(block);
}
- BlockLocator actual = _blockChain.GetBlockLocator(threshold: 3);
- BlockLocator expected = new BlockLocator(new[]
- {
- blocks[9].Hash,
- blocks[8].Hash,
- blocks[7].Hash,
- blocks[6].Hash,
- blocks[5].Hash,
- blocks[3].Hash,
- _blockChain.Genesis.Hash,
- });
+ BlockLocator actual = _blockChain.GetBlockLocator();
+ BlockLocator expected = new BlockLocator(new[] { blocks[9].Hash });
Assert.Equal(expected, actual);
}
@@ -1107,7 +1098,7 @@ public void CannotSwapForSameHeightTip(bool render)
[SkippableTheory]
[InlineData(true)]
[InlineData(false)]
- public void ReorgIsUnableToHeterogenousChain(bool render)
+ public void CannotSwapToChainWithDifferentGenesis(bool render)
{
using (var fx2 = new MemoryStoreFixture(_policy.PolicyActionsRegistry))
{
@@ -1462,8 +1453,7 @@ public void FindBranchPoint()
var emptyLocator = new BlockLocator(new[] { _blockChain.Genesis.Hash });
var invalidLocator = new BlockLocator(
new[] { new BlockHash(TestUtils.GetRandomBytes(BlockHash.Size)) });
- var locator = new BlockLocator(
- new[] { b4.Hash, b3.Hash, b1.Hash, _blockChain.Genesis.Hash });
+ var locator = new BlockLocator(new[] { b4.Hash });
using (var emptyFx = new MemoryStoreFixture(_policy.PolicyActionsRegistry))
using (var forkFx = new MemoryStoreFixture(_policy.PolicyActionsRegistry))
@@ -1496,18 +1486,18 @@ public void FindBranchPoint()
// Testing emptyChain
Assert.Equal(_blockChain.Genesis.Hash, emptyChain.FindBranchpoint(emptyLocator));
- Assert.Equal(_blockChain.Genesis.Hash, emptyChain.FindBranchpoint(locator));
Assert.Null(emptyChain.FindBranchpoint(invalidLocator));
+ Assert.Null(emptyChain.FindBranchpoint(locator));
// Testing _blockChain
Assert.Equal(_blockChain.Genesis.Hash, _blockChain.FindBranchpoint(emptyLocator));
- Assert.Equal(b4.Hash, _blockChain.FindBranchpoint(locator));
Assert.Null(_blockChain.FindBranchpoint(invalidLocator));
+ Assert.Equal(b4.Hash, _blockChain.FindBranchpoint(locator));
// Testing fork
Assert.Equal(_blockChain.Genesis.Hash, fork.FindBranchpoint(emptyLocator));
- Assert.Equal(b1.Hash, fork.FindBranchpoint(locator));
- Assert.Null(_blockChain.FindBranchpoint(invalidLocator));
+ Assert.Null(fork.FindBranchpoint(invalidLocator));
+ Assert.Null(fork.FindBranchpoint(locator));
}
}
diff --git a/test/Libplanet.Tests/Blockchain/BlockLocatorTest.cs b/test/Libplanet.Tests/Blockchain/BlockLocatorTest.cs
index c36dcd09808..e2452b6ef81 100644
--- a/test/Libplanet.Tests/Blockchain/BlockLocatorTest.cs
+++ b/test/Libplanet.Tests/Blockchain/BlockLocatorTest.cs
@@ -1,127 +1,18 @@
using System;
using System.Collections.Generic;
-using System.Linq;
using Libplanet.Blockchain;
using Libplanet.Types.Blocks;
using Xunit;
-using Xunit.Abstractions;
namespace Libplanet.Tests.Blockchain
{
public class BlockLocatorTest
{
- private readonly ITestOutputHelper _output;
-
- public BlockLocatorTest(ITestOutputHelper output)
- {
- _output = output;
- }
-
[Fact]
public void Constructor()
{
Assert.Throws(() =>
new BlockLocator(new List()));
}
-
- [Fact]
- public void Factory()
- {
- // Generate fixture block hashes looks like 0x00...00 0x00...01, 0x00...02, and so on,
- // for the sake of easier debugging.
- List indices = Enumerable.Range(0, 0x10).Select(i => (long)i).ToList();
- Func indexToBlockHash = i =>
- {
- byte[] bytes = GetBigEndianBytes(i);
- var buffer = new byte[32];
- bytes.CopyTo(buffer, buffer.Length - bytes.Length);
- return new BlockHash(buffer);
- };
- List hashes = indices.Select(i => indexToBlockHash(i)).ToList();
- Action printLocator = loc =>
- {
- _output.WriteLine(
- string.Join(", ", loc
- .Select(hash => hash.ToString())
- .Select(str => str.Substring(str.Length - 2))
- .Select(str => $"0x00..{str}")));
- };
-
- Assert.Throws(() =>
- BlockLocator.Create(
- startIndex: -1,
- indexToBlockHash: i => (BlockHash?)indexToBlockHash(i),
- sampleAfter: 0));
- Assert.Throws(() =>
- BlockLocator.Create(
- startIndex: 0,
- indexToBlockHash: i => i == 0 ? null : (BlockHash?)indexToBlockHash(i),
- sampleAfter: 0));
-
- var locator = BlockLocator.Create(
- startIndex: hashes.Count - 1,
- indexToBlockHash: i => (BlockHash?)indexToBlockHash(i),
- sampleAfter: 0);
- var expected = new BlockHash[]
- {
- hashes[0xf], // Sampling starts here.
- hashes[0xe],
- hashes[0xc],
- hashes[0x8],
- hashes[0x0],
- };
-
- printLocator(locator);
- Assert.Equal(expected, locator);
-
- locator = BlockLocator.Create(
- startIndex: hashes.Count - 1,
- indexToBlockHash: i => (BlockHash?)indexToBlockHash(i),
- sampleAfter: 1);
- expected = new BlockHash[]
- {
- hashes[0xf],
- hashes[0xe], // Sampling starts here.
- hashes[0xd],
- hashes[0xb],
- hashes[0x7],
- hashes[0x0],
- };
-
- printLocator(locator);
- Assert.Equal(expected, locator);
-
- locator = BlockLocator.Create(
- startIndex: hashes.Count - 1,
- indexToBlockHash: i => (BlockHash?)indexToBlockHash(i),
- sampleAfter: 5);
- expected = new BlockHash[]
- {
- hashes[0xf],
- hashes[0xe],
- hashes[0xd],
- hashes[0xc],
- hashes[0xb],
- hashes[0xa], // Sampling starts here.
- hashes[0x9],
- hashes[0x7],
- hashes[0x3],
- hashes[0x0],
- };
-
- printLocator(locator);
- Assert.Equal(expected, locator);
- }
-
- private static byte[] GetBigEndianBytes(long value)
- {
- byte[] bytes = BitConverter.GetBytes(value);
- if (BitConverter.IsLittleEndian)
- {
- Array.Reverse(bytes);
- }
-
- return bytes;
- }
}
}