3
0
mirror of https://github.com/Qortal/altcoinj.git synced 2025-01-30 23:02:15 +00:00

Introduce a new MemoryPool abstraction and use it for tracking peer confidences. This replaces the previous, less explicit system which had each peer try and track all transactions it'd seen in a per-peer pool.

This makes experimental usage of weak references and a reference queue to make the MemoryPool only bother tracking transactions that were retained by some other part of the program, ie, because they were relevant to the wallet. This stops the memory usage from bloating due to lots of large transactions flying around the network whilst still letting us intelligently monitor how many peers announced transactions without tight coupling between components.

Note that we still need the cap on the pool size because otherwise you could DoS a bitcoinj based program by sending it lots of fake transactions it found interesting. The DoS potential still exists but is a bit different now.
This commit is contained in:
Mike Hearn 2012-04-04 21:35:10 +02:00
parent dc42630526
commit d82f102b4b
6 changed files with 361 additions and 101 deletions

View File

@ -0,0 +1,237 @@
/*
* Copyright 2012 Google Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.google.bitcoin.core;
import com.google.common.base.Preconditions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.lang.ref.Reference;
import java.lang.ref.ReferenceQueue;
import java.lang.ref.WeakReference;
import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.Map;
import java.util.Set;
/**
* Tracks transactions that are being announced across the network. Typically one is created for you by a PeerGroup
* and then given to each Peer to update. The current purpose is to let Peers update the confidence (number of peers
* broadcasting). It helps address an attack scenario in which a malicious remote peer (or several) feeds you
* invalid transactions, eg, ones that spend coins which don't exist. If you don't see most of the peers announce the
* transaction within a reasonable time, it may be that the TX is not valid. Alternatively, an attacker may control
* your entire internet connection: in this scenario counting broadcasting peers does not help you.<p>
*
* It is <b>not</b> at this time directly equivalent to the Satoshi clients memory pool, which tracks
* all transactions not currently included in the best chain - it's simply a cache.
*/
public class MemoryPool {
private static final Logger log = LoggerFactory.getLogger(MemoryPool.class);
// For each transaction we may have seen:
// - only its hash in an inv packet
// - the full transaction itself, if we asked for it to be sent to us (or a peer sent it regardless)
//
// Before we see the full transaction, we need to track how many peers advertised it, so we can estimate its
// confidence pre-chain inclusion assuming an un-tampered with network connection. After we see the full transaction
// we need to switch from tracking that data in the Entry to tracking it in the TransactionConfidence object itself.
private class WeakTransactionReference extends WeakReference<Transaction> {
public Sha256Hash hash;
public WeakTransactionReference(Transaction tx, ReferenceQueue<Transaction> queue) {
super(tx, queue);
hash = tx.getHash();
}
};
private class Entry {
// Invariants: one of the two fields must be null, to indicate which is used.
Set<PeerAddress> addresses;
// We keep a weak reference to the transaction. This means that if no other bit of code finds the transaction
// worth keeping around it will drop out of memory and we will, at some point, forget about it, which means
// both addresses and tx.get() will be null. When this happens the WeakTransactionReference appears in the queue
// allowing us to delete the associated entry (the tx itself has already gone away).
WeakTransactionReference tx;
}
private LinkedHashMap<Sha256Hash, Entry> memoryPool;
// This ReferenceQueue gets entries added to it when they are only weakly reachable, ie, the MemoryPool is the
// only thing that is tracking the transaction anymore. We check it from time to time and delete memoryPool entries
// corresponding to expired transactions. In this way memory usage of the system is in line with however many
// transactions you actually care to track the confidence of. We can still end up with lots of hashes being stored
// if our peers flood us with invs but the MAX_SIZE param caps this.
private ReferenceQueue<Transaction> referenceQueue;
/** The max size of a memory pool created with the no-args constructor. */
public static final int MAX_SIZE = 1000;
/**
* Creates a memory pool that will track at most the given number of transactions (allowing you to bound memory
* usage).
* @param size Max number of transactions to track. The pool will fill up to this size then stop growing.
*/
public MemoryPool(final int size) {
memoryPool = new LinkedHashMap<Sha256Hash, Entry>() {
@Override
protected boolean removeEldestEntry(Map.Entry<Sha256Hash, Entry> entry) {
// An arbitrary choice to stop the memory used by tracked transactions getting too huge in the event
// of some kind of DoS attack.
return size() > size;
}
};
referenceQueue = new ReferenceQueue<Transaction>();
}
/**
* Creates a memory pool that will track at most {@link MemoryPool#MAX_SIZE} entries. You should normally use
* this constructor.
*/
public MemoryPool() {
this(MAX_SIZE);
}
/**
* If any transactions have expired due to being only weakly reachable through us, go ahead and delete their
* memoryPool entries - it means we downloaded the transaction and sent it to various event listeners, none of
* which bothered to keep a reference. Typically, this is because the transaction does not involve any keys that
* are relevant to any of our wallets.
*/
private synchronized void cleanPool() {
Reference<? extends Transaction> ref;
while ((ref = referenceQueue.poll()) != null) {
// Find which transaction got deleted by the GC.
WeakTransactionReference txRef = (WeakTransactionReference) ref;
// And remove the associated map entry so the other bits of memory can also be reclaimed.
log.info("Cleaned up tx {}", txRef.hash);
memoryPool.remove(txRef.hash);
}
}
/**
* Returns the number of peers that have seen the given hash recently.
*/
public synchronized int numBroadcastPeers(Sha256Hash txHash) {
cleanPool();
Entry entry = memoryPool.get(txHash);
if (entry == null) {
// No such TX known.
return 0;
} else if (entry.tx == null) {
// We've seen at least one peer announce with an inv.
Preconditions.checkNotNull(entry.addresses);
return entry.addresses.size();
} else if (entry.tx.get() == null) {
// We previously downloaded this transaction, but nothing cared about it so the garbage collector threw
// it away. We also deleted the set that tracked which peers had seen it. Treat this case as a zero and
// just delete it from the map.
memoryPool.remove(txHash);
return 0;
} else {
Preconditions.checkState(entry.addresses == null);
return entry.tx.get().getConfidence().numBroadcastPeers();
}
}
/**
* Called by peers when they receive a "tx" message containing a valid serialized transaction.
* @param tx The TX deserialized from the wire.
* @param byPeer The Peer that received it.
* @return An object that is semantically the same TX but may be a different object instance.
*/
public synchronized Transaction seen(Transaction tx, PeerAddress byPeer) {
cleanPool();
Entry entry = memoryPool.get(tx.getHash());
if (entry != null) {
// This TX or its hash have been previously announced.
if (entry.tx != null) {
// We already downloaded it.
Preconditions.checkState(entry.addresses == null);
// We only want one canonical object instance for a transaction no matter how many times it is
// deserialized.
if (entry.tx.get() == null) {
// We previously downloaded this transaction, but the garbage collector threw it away because
// no other part of the system cared enough to keep it around (it's not relevant to us).
// Given the lack of interest last time we probably don't need to track it this time either.
log.info("{}: Provided with a transaction that we previously threw away: {}", byPeer, tx.getHash());
} else {
// We saw it before and kept it around. Hand back the canonical copy.
tx = entry.tx.get();
log.info("{}: Provided with a transaction downloaded before: [{}] {}",
new Object[] { byPeer, tx.getConfidence().numBroadcastPeers(), tx.getHash() });
}
tx.getConfidence().markBroadcastBy(byPeer);
return tx;
} else {
// We received a transaction that we have previously seen announced but not downloaded until now.
Preconditions.checkNotNull(entry.addresses);
entry.tx = new WeakTransactionReference(tx, referenceQueue);
// Copy the previously announced peers into the confidence and then clear it out.
TransactionConfidence confidence = tx.getConfidence();
for (PeerAddress a : entry.addresses) {
confidence.markBroadcastBy(a);
}
entry.addresses = null;
log.info("{}: Adding tx [{}] {} to the memory pool",
new Object[] { byPeer, confidence.numBroadcastPeers(), tx.getHashAsString() });
return tx;
}
} else {
log.info("{}: Provided with a downloaded transaction we didn't see announced yet: {}",
byPeer, tx.getHashAsString());
entry = new Entry();
entry.tx = new WeakTransactionReference(tx, referenceQueue);
memoryPool.put(tx.getHash(), entry);
tx.getConfidence().markBroadcastBy(byPeer);
return tx;
}
}
/**
* Called by peers when they see a transaction advertised in an "inv" message. It either will increase the
* confidence of the pre-existing transaction or will just keep a record of the address for future usage.
*/
public synchronized void seen(Sha256Hash hash, PeerAddress byPeer) {
cleanPool();
Entry entry = memoryPool.get(hash);
if (entry != null) {
// This TX or its hash have been previously announced.
if (entry.tx != null) {
Preconditions.checkState(entry.addresses == null);
if (entry.tx.get() != null) {
Transaction tx = entry.tx.get();
tx.getConfidence().markBroadcastBy(byPeer);
log.info("{}: Announced transaction we have seen before [{}] {}",
new Object[] { byPeer, tx.getConfidence().numBroadcastPeers(), tx.getHashAsString() });
} else {
// The inv is telling us about a transaction that we previously downloaded, and threw away because
// nothing found it interesting enough to keep around. So do nothing.
}
} else {
Preconditions.checkNotNull(entry.addresses);
entry.addresses.add(byPeer);
log.info("{}: Announced transaction we have seen announced before [{}] {}",
new Object[] { byPeer, entry.addresses.size(), hash });
}
} else {
// This TX has never been seen before.
entry = new Entry();
// TODO: Using hashsets here is inefficient compared to just having an array.
entry.addresses = new HashSet<PeerAddress>();
entry.addresses.add(byPeer);
memoryPool.put(hash, entry);
log.info("{}: Announced new transaction [1] {}", byPeer, hash);
}
}
}

View File

@ -25,9 +25,7 @@ import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.ArrayList;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.*;
/**
@ -49,8 +47,6 @@ public class Peer {
// whilst waiting for the response. Synchronized on itself. Is not used for downloads Peer generates itself.
// TODO: Make this work for transactions as well.
private final List<GetDataFuture<Block>> pendingGetBlockFutures;
// Height of the chain advertised in the peers version message.
private int bestHeight;
private PeerAddress address;
private List<PeerEventListener> eventListeners;
// Whether to try and download blocks and transactions from this peer. Set to false by PeerGroup if not the
@ -60,26 +56,9 @@ public class Peer {
// The version data to announce to the other side of the connections we make: useful for setting our "user agent"
// equivalent and other things.
private VersionMessage versionMessage;
/**
* Size of the pending transactions pool. Override this to reduce memory usage on constrained platforms. The pool
* is used to keep track of how many peers announced a transaction. With an untampered-with internet connection,
* the more peers announce a transaction, the more confidence you can have that it's valid.
*/
public static int TRANSACTION_MEMORY_POOL_SIZE = 1000;
// Maps announced transaction hashes to the Transaction objects. If this is not a download peer, the Transaction
// objects must be provided from elsewhere (ie, a PeerGroup object). If the Transaction hasn't been downloaded or
// provided yet, the map value is null. This is somewhat equivalent to the reference implementations memory pool.
private LinkedHashMap<Sha256Hash, Transaction> announcedTransactionHashes = new LinkedHashMap<Sha256Hash, Transaction>() {
@Override
protected boolean removeEldestEntry(Map.Entry<Sha256Hash, Transaction> sha256HashTransactionEntry) {
// An arbitrary choice to stop the memory used by tracked transactions getting too huge. Mobile platforms
// may want to reduce this.
return size() > TRANSACTION_MEMORY_POOL_SIZE;
}
};
// A class that tracks recent transactions that have been broadcast across the network, counts how many
// peers announced them and updates the transaction confidence data. It is passed to each Peer.
private MemoryPool memoryPool;
// A time before which we only download block headers, after that point we download block bodies.
private long fastCatchupTimeSecs;
// Whether we are currently downloading headers only or block bodies. Defaults to true, if the fast catchup time
@ -138,6 +117,18 @@ public class Peer {
return eventListeners.remove(listener);
}
/**
* Tells the peer to insert received transactions/transaction announcements into the given {@link MemoryPool}.
* This is normally done for you by the {@link PeerGroup} so you don't have to think about it. Transactions stored
* in a memory pool will have their confidence levels updated when a peer announces it, to reflect the greater
* likelyhood that the transaction is valid.
*
* @param pool A new pool or null to unlink.
*/
public synchronized void setMemoryPool(MemoryPool pool) {
memoryPool = pool;
}
@Override
public String toString() {
if (address == null) {
@ -230,8 +221,8 @@ public class Peer {
disconnect();
throw new PeerException(e);
} catch (RuntimeException e) {
log.error("Unexpected exception in peer loop: ", e.getMessage());
disconnect();
log.error("unexpected exception in peer loop: ", e.getMessage());
throw e;
}
@ -314,6 +305,8 @@ public class Peer {
private void processTransaction(Transaction m) {
log.info("Received broadcast tx {}", m.getHashAsString());
if (memoryPool != null)
memoryPool.seen(m, getAddress());
for (PeerEventListener listener : eventListeners) {
synchronized (listener) {
listener.onTransaction(this, m);
@ -377,7 +370,8 @@ public class Peer {
private void processInv(InventoryMessage inv) throws IOException {
// This should be called in the network loop thread for this peer.
List<InventoryItem> items = inv.getItems();
updateTransactionConfidenceLevels(items);
if (memoryPool != null)
updateTransactionConfidenceLevels(items);
// If this peer isn't responsible for downloading stuff, don't go further.
if (!downloadData)
@ -404,57 +398,11 @@ public class Peer {
conn.writeMessage(getdata);
}
/**
* When a peer broadcasts an "inv" containing a transaction hash, it means the peer validated it and won't accept
* double spends of those coins. So by measuring what proportion of our total connected peers have seen a
* transaction we can make a guesstimate of how likely it is to be included in a block, assuming our internet
* connection is trustworthy.<p>
*
* This method keeps a map of transaction hashes to {@link Transaction} objects. It may not have the associated
* transaction objects available, if they weren't downloaded yet. Once a Transaction is downloaded, it's set as
* the value in the txSeen map. If this Peer isn't the download peer, the {@link PeerGroup} will manage distributing
* the Transaction objects to every peer, at which point the peer is expected to update the
* {@link TransactionConfidence} object itself.
*
* @param items Inventory items that were just announced.
*/
private void updateTransactionConfidenceLevels(List<InventoryItem> items) {
// Announced hashes may be updated by other threads in response to messages coming in from other peers.
synchronized (announcedTransactionHashes) {
for (InventoryItem item : items) {
if (item.type != InventoryItem.Type.Transaction) continue;
Transaction transaction = announcedTransactionHashes.get(item.hash);
if (transaction == null) {
// We didn't see this tx before.
log.debug("Newly announced undownloaded transaction ", item.hash);
announcedTransactionHashes.put(item.hash, null);
} else {
// It's been downloaded. Update the confidence levels. This may be called multiple times for
// the same transaction and the same peer, there is no obligation in the protocol to avoid
// redundant advertisements.
log.debug("Marking tx {} as seen by {}", item.hash, toString());
transaction.getConfidence().markBroadcastBy(address);
}
}
}
}
/**
* Called by {@link PeerGroup} to tell the Peer about a transaction that was just downloaded. If we have tracked
* the announcement, update the transactions confidence level at this time. Otherwise wait for it to appear.
*/
void trackTransaction(Transaction tx) {
// May run on arbitrary peer threads.
synchronized (announcedTransactionHashes) {
if (announcedTransactionHashes.containsKey(tx.getHash())) {
Transaction storedTx = announcedTransactionHashes.get(tx.getHash());
Preconditions.checkState(storedTx == tx || storedTx == null, "single Transaction instance");
log.debug("Provided with a downloaded transaction we have seen before: {}", tx.getHash());
tx.getConfidence().markBroadcastBy(address);
} else {
log.debug("Provided with a downloaded transaction we didn't see broadcast yet: {}", tx.getHash());
}
announcedTransactionHashes.put(tx.getHash(), tx);
Preconditions.checkNotNull(memoryPool);
for (InventoryItem item : items) {
if (item.type != InventoryItem.Type.Transaction) continue;
memoryPool.seen(item.hash, this.getAddress());
}
}
@ -743,8 +691,8 @@ public class Peer {
/**
* @return the height of the best chain as claimed by peer.
*/
public int getBestHeight() {
return bestHeight;
public long getBestHeight() {
return conn.getVersionMessage().bestHeight;
}
/**

View File

@ -20,6 +20,7 @@ package com.google.bitcoin.core;
import com.google.bitcoin.discovery.PeerDiscovery;
import com.google.bitcoin.discovery.PeerDiscoveryException;
import com.google.bitcoin.utils.EventListenerInvoker;
import com.google.common.base.Preconditions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -79,6 +80,9 @@ public class PeerGroup {
private Set<PeerDiscovery> peerDiscoverers;
// The version message to use for new connections.
private VersionMessage versionMessage;
// A class that tracks recent transactions that have been broadcast across the network, counts how many
// peers announced them and updates the transaction confidence data. It is passed to each Peer.
private MemoryPool memoryPool;
private NetworkParameters params;
private BlockChain chain;
@ -113,6 +117,8 @@ public class PeerGroup {
// this is.
this.versionMessage = new VersionMessage(params, chain.getBestChainHeight());
this.memoryPool = new MemoryPool();
inactives = new LinkedBlockingQueue<PeerAddress>();
// TODO: Remove usage of synchronized sets here in favor of simple coarse-grained locking.
peers = Collections.synchronizedSet(new HashSet<Peer>());
@ -128,13 +134,6 @@ public class PeerGroup {
// when we download a transaction, we can distribute it to each Peer in the pool so they can update the
// transactions confidence level if they've seen it be announced/when they see it be announced.
peerEventListeners = new ArrayList<PeerEventListener>();
addEventListener(new AbstractPeerEventListener() {
@Override
public void onTransaction(Peer peer, Transaction t) {
handleBroadcastTransaction(t);
}
});
// This event listener is added to every peer. It's here so when we announce transactions via an "inv", every
// peer can fetch them.
getDataListener = new AbstractPeerEventListener() {
@ -160,14 +159,6 @@ public class PeerGroup {
return new LinkedList<Message>(transactions.values());
}
private synchronized void handleBroadcastTransaction(Transaction tx) {
// Called on the download peer thread when we have downloaded an advertised Transaction. Distribute it to all
// the peers in the group so they can update the confidence if they saw it be advertised or when they do see it.
for (Peer p : peers) {
p.trackTransaction(tx);
}
}
/**
* Sets the {@link VersionMessage} that will be announced on newly created connections. A version message is
* primarily interesting because it lets you customize the "subVer" field which is used a bit like the User-Agent
@ -251,8 +242,7 @@ public class PeerGroup {
*/
public void addPeer(Peer peer) {
synchronized (this) {
if (!running)
throw new IllegalStateException("Must call start() before adding peers.");
Preconditions.checkState(running, "Must call start() before adding peers.");
log.info("Adding directly to group: {}", peer);
}
// This starts the peer thread running. Note: this is not synchronized. If it were, we could not
@ -648,7 +638,10 @@ public class PeerGroup {
}
protected synchronized void handleNewPeer(final Peer peer) {
log.info("Handling new {}", peer);
// Runs on a peer thread for every peer that is newly connected.
log.info("{}: New peer", peer);
// Link the peer to the memory pool so broadcast transactions have their confidence levels updated.
peer.setMemoryPool(memoryPool);
// If we want to download the chain, and we aren't currently doing so, do so now.
if (downloadListener != null && downloadPeer == null) {
log.info(" starting block chain download");
@ -658,10 +651,17 @@ public class PeerGroup {
} else {
peer.setDownloadData(false);
}
// Make sure the peer knows how to upload transactions that are requested from us.
peer.addEventListener(getDataListener);
// Now tell the peers about any transactions we have which didn't appear in the chain yet. These are not
// necessarily spends we created. They may also be transactions broadcast across the network that we saw,
// which are relevant to us, and which we therefore wish to help propagate (ie they send us coins).
peer.addEventListener(getDataListener);
//
// Note that this can cause a DoS attack against us if a malicious remote peer knows what keys we own, and
// then sends us fake relevant transactions. We'll attempt to relay the bad transactions, our badness score
// in the Satoshi client will increase and we'll get disconnected.
//
// TODO: Find a way to balance the desire to propagate useful transactions against obscure DoS attacks.
announcePendingWalletTransactions(wallets, Collections.singleton(peer));
EventListenerInvoker.invoke(peerEventListeners, new EventListenerInvoker<PeerEventListener>() {
@Override

View File

@ -18,6 +18,7 @@ package com.google.bitcoin.core;
import com.google.bitcoin.store.BlockStoreException;
import com.google.bitcoin.utils.EventListenerInvoker;
import com.google.common.base.Preconditions;
import java.io.Serializable;
import java.math.BigInteger;
@ -66,6 +67,7 @@ public class TransactionConfidence implements Serializable {
private Set<PeerAddress> broadcastBy;
/** The Transaction that this confidence object is associated with. */
private Transaction transaction;
// Lazily created listeners array.
private transient ArrayList<Listener> listeners;
// TODO: The advice below is a mess. There should be block chain listeners, see issue 94.
@ -80,16 +82,15 @@ public class TransactionConfidence implements Serializable {
* confidence object to determine the new depth.</p>
*/
public synchronized void addEventListener(Listener listener) {
if (listener == null)
throw new IllegalArgumentException("listener is null");
Preconditions.checkNotNull(listener);
if (listeners == null)
listeners = new ArrayList<Listener>(1);
listeners.add(listener);
}
public synchronized void removeEventListener(Listener listener) {
if (listener == null)
throw new IllegalArgumentException("listener is null");
Preconditions.checkNotNull(listener);
Preconditions.checkNotNull(listeners);
listeners.remove(listener);
}

View File

@ -39,6 +39,7 @@ public abstract class EventListenerInvoker<E> {
public static <E> void invoke(List<E> listeners,
EventListenerInvoker<E> invoker) {
if (listeners == null) return;
synchronized (listeners) {
for (int i = 0; i < listeners.size(); i++) {
E l = listeners.get(i);

View File

@ -0,0 +1,73 @@
/*
* Copyright 2012 Google Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.google.bitcoin.core;
import com.google.bitcoin.utils.BriefLogFormatter;
import org.junit.Before;
import org.junit.Test;
import java.net.InetAddress;
import static org.junit.Assert.assertEquals;
public class MemoryPoolTest {
private NetworkParameters params = NetworkParameters.unitTests();
private Transaction tx1, tx2;
private PeerAddress address1, address2, address3;
@Before
public void setup() throws Exception {
BriefLogFormatter.init();
tx1 = TestUtils.createFakeTx(params, Utils.toNanoCoins(1, 0), new ECKey().toAddress(params));
tx2 = new Transaction(params, tx1.bitcoinSerialize());
address1 = new PeerAddress(InetAddress.getByAddress(new byte[] { 127, 0, 0, 1 }));
address2 = new PeerAddress(InetAddress.getByAddress(new byte[] { 127, 0, 0, 2 }));
address3 = new PeerAddress(InetAddress.getByAddress(new byte[] { 127, 0, 0, 3 }));
}
@Test
public void canonicalInstance() throws Exception {
MemoryPool pool = new MemoryPool();
// Check that if we repeatedly send it the same transaction but with different objects, we get back the same
// canonical instance with the confidences update.
assertEquals(0, pool.numBroadcastPeers(tx1.getHash()));
assertEquals(tx1, pool.seen(tx1, address1));
assertEquals(1, tx1.getConfidence().numBroadcastPeers());
assertEquals(1, pool.numBroadcastPeers(tx1.getHash()));
assertEquals(tx1, pool.seen(tx2, address2));
assertEquals(2, tx1.getConfidence().numBroadcastPeers());
assertEquals(2, pool.numBroadcastPeers(tx1.getHash()));
}
@Test
public void invAndDownload() throws Exception {
MemoryPool pool = new MemoryPool();
// Base case: we see a transaction announced twice and then download it. The count is in the confidence object.
assertEquals(0, pool.numBroadcastPeers(tx1.getHash()));
pool.seen(tx1.getHash(), address1);
assertEquals(1, pool.numBroadcastPeers(tx1.getHash()));
pool.seen(tx1.getHash(), address2);
assertEquals(2, pool.numBroadcastPeers(tx1.getHash()));
Transaction t = pool.seen(tx1, address1);
assertEquals(2, t.getConfidence().numBroadcastPeers());
// And now we see another inv.
pool.seen(tx1.getHash(), address3);
assertEquals(3, t.getConfidence().numBroadcastPeers());
assertEquals(3, pool.numBroadcastPeers(tx1.getHash()));
}
}