mirror of
https://github.com/serai-dex/serai.git
synced 2025-01-03 09:29:46 +00:00
support input encoded data for bitcoin network (#486)
* add input script check * add test * optimizations * bug fix * fix pr comments * Test SegWit-encoded data using a single output (not two) * Remove TODO used as a question, document origins when SegWit encoding --------- Co-authored-by: Luke Parker <lukeparker5132@gmail.com>
This commit is contained in:
parent
c05c511938
commit
d88aa90ec2
3 changed files with 338 additions and 88 deletions
|
@ -21,7 +21,8 @@ use bitcoin_serai::{
|
|||
consensus::{Encodable, Decodable},
|
||||
script::Instruction,
|
||||
address::{NetworkChecked, Address as BAddress},
|
||||
Transaction, Block, Network as BNetwork,
|
||||
Transaction, Block, Network as BNetwork, ScriptBuf,
|
||||
opcodes::all::{OP_SHA256, OP_EQUALVERIFY},
|
||||
},
|
||||
wallet::{
|
||||
tweak_keys, address_payload, ReceivedOutput, Scanner, TransactionError,
|
||||
|
@ -35,10 +36,11 @@ use bitcoin_serai::bitcoin::{
|
|||
secp256k1::{SECP256K1, SecretKey, Message},
|
||||
PrivateKey, PublicKey,
|
||||
sighash::{EcdsaSighashType, SighashCache},
|
||||
script::{PushBytesBuf, Builder},
|
||||
script::PushBytesBuf,
|
||||
absolute::LockTime,
|
||||
Amount as BAmount, Sequence, Script, Witness, OutPoint, TxOut, TxIn,
|
||||
Amount as BAmount, Sequence, Script, Witness, OutPoint,
|
||||
transaction::Version,
|
||||
blockdata::transaction::{TxIn, TxOut},
|
||||
};
|
||||
|
||||
use serai_client::{
|
||||
|
@ -447,6 +449,92 @@ impl Bitcoin {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Expected script has to start with SHA256 PUSH MSG_HASH OP_EQUALVERIFY ..
|
||||
fn segwit_data_pattern(script: &ScriptBuf) -> Option<bool> {
|
||||
let mut ins = script.instructions();
|
||||
|
||||
// first item should be SHA256 code
|
||||
if ins.next()?.ok()?.opcode()? != OP_SHA256 {
|
||||
return Some(false);
|
||||
}
|
||||
|
||||
// next should be a data push
|
||||
ins.next()?.ok()?.push_bytes()?;
|
||||
|
||||
// next should be a equality check
|
||||
if ins.next()?.ok()?.opcode()? != OP_EQUALVERIFY {
|
||||
return Some(false);
|
||||
}
|
||||
|
||||
Some(true)
|
||||
}
|
||||
|
||||
fn extract_serai_data(tx: &Transaction) -> Vec<u8> {
|
||||
// check outputs
|
||||
let mut data = (|| {
|
||||
for output in &tx.output {
|
||||
if output.script_pubkey.is_op_return() {
|
||||
match output.script_pubkey.instructions_minimal().last() {
|
||||
Some(Ok(Instruction::PushBytes(data))) => return data.as_bytes().to_vec(),
|
||||
_ => continue,
|
||||
}
|
||||
}
|
||||
}
|
||||
vec![]
|
||||
})();
|
||||
|
||||
// check inputs
|
||||
if data.is_empty() {
|
||||
for input in &tx.input {
|
||||
let witness = input.witness.to_vec();
|
||||
// expected witness at least has to have 2 items, msg and the redeem script.
|
||||
if witness.len() >= 2 {
|
||||
let redeem_script = ScriptBuf::from_bytes(witness.last().unwrap().clone());
|
||||
if Self::segwit_data_pattern(&redeem_script) == Some(true) {
|
||||
data = witness[witness.len() - 2].clone(); // len() - 1 is the redeem_script
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
data.truncate(MAX_DATA_LEN.try_into().unwrap());
|
||||
data
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
pub fn sign_btc_input_for_p2pkh(
|
||||
tx: &Transaction,
|
||||
input_index: usize,
|
||||
private_key: &PrivateKey,
|
||||
) -> ScriptBuf {
|
||||
let public_key = PublicKey::from_private_key(SECP256K1, private_key);
|
||||
let main_addr = BAddress::p2pkh(&public_key, BNetwork::Regtest);
|
||||
|
||||
let mut der = SECP256K1
|
||||
.sign_ecdsa_low_r(
|
||||
&Message::from(
|
||||
SighashCache::new(tx)
|
||||
.legacy_signature_hash(
|
||||
input_index,
|
||||
&main_addr.script_pubkey(),
|
||||
EcdsaSighashType::All.to_u32(),
|
||||
)
|
||||
.unwrap()
|
||||
.to_raw_hash(),
|
||||
),
|
||||
&private_key.inner,
|
||||
)
|
||||
.serialize_der()
|
||||
.to_vec();
|
||||
der.push(1);
|
||||
|
||||
ScriptBuf::builder()
|
||||
.push_slice(PushBytesBuf::try_from(der).unwrap())
|
||||
.push_key(&public_key)
|
||||
.into_script()
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait]
|
||||
|
@ -571,28 +659,27 @@ impl Network for Bitcoin {
|
|||
let offset_repr_ref: &[u8] = offset_repr.as_ref();
|
||||
let kind = kinds[offset_repr_ref];
|
||||
|
||||
let mut data = if kind == OutputType::External {
|
||||
(|| {
|
||||
for output in &tx.output {
|
||||
if output.script_pubkey.is_op_return() {
|
||||
match output.script_pubkey.instructions_minimal().last() {
|
||||
Some(Ok(Instruction::PushBytes(data))) => return data.as_bytes().to_vec(),
|
||||
_ => continue,
|
||||
let output = Output { kind, presumed_origin: None, output, data: vec![] };
|
||||
assert_eq!(output.tx_id(), tx.id());
|
||||
outputs.push(output);
|
||||
}
|
||||
}
|
||||
}
|
||||
vec![]
|
||||
})()
|
||||
} else {
|
||||
vec![]
|
||||
};
|
||||
data.truncate(MAX_DATA_LEN.try_into().unwrap());
|
||||
|
||||
if outputs.is_empty() {
|
||||
continue;
|
||||
}
|
||||
|
||||
// populate the outputs with the origin and data
|
||||
let presumed_origin = {
|
||||
let spent_output = tx.input[0].previous_output;
|
||||
let mut spent_tx = spent_output.txid.as_raw_hash().to_byte_array();
|
||||
spent_tx.reverse();
|
||||
// This may identify the P2WSH output *embedding the InInstruction* as the origin, which
|
||||
// would be a bit trickier to spend that a traditional output...
|
||||
// There's no risk of the InInstruction going missing as it'd already be on-chain though
|
||||
// We *could* parse out the script *without the InInstruction prefix* and declare that the
|
||||
// origin
|
||||
// TODO
|
||||
let spent_output = {
|
||||
let input = &tx.input[0];
|
||||
let mut spent_tx = input.previous_output.txid.as_raw_hash().to_byte_array();
|
||||
spent_tx.reverse();
|
||||
let mut tx;
|
||||
while {
|
||||
tx = self.get_transaction(&spent_tx).await;
|
||||
|
@ -601,16 +688,18 @@ impl Network for Bitcoin {
|
|||
log::error!("couldn't get transaction from bitcoin node: {tx:?}");
|
||||
sleep(Duration::from_secs(5)).await;
|
||||
}
|
||||
tx.unwrap().output.swap_remove(usize::try_from(spent_output.vout).unwrap())
|
||||
tx.unwrap().output.swap_remove(usize::try_from(input.previous_output.vout).unwrap())
|
||||
};
|
||||
BAddress::from_script(&spent_output.script_pubkey, BNetwork::Bitcoin)
|
||||
.ok()
|
||||
.and_then(Address::new)
|
||||
};
|
||||
|
||||
let output = Output { kind, presumed_origin, output, data };
|
||||
assert_eq!(output.tx_id(), tx.id());
|
||||
outputs.push(output);
|
||||
let data = Self::extract_serai_data(tx);
|
||||
for output in &mut outputs {
|
||||
if output.kind == OutputType::External {
|
||||
output.data = data.clone();
|
||||
}
|
||||
output.presumed_origin = presumed_origin.clone();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -774,14 +863,10 @@ impl Network for Bitcoin {
|
|||
let new_block = self.get_latest_block_number().await.unwrap() + 1;
|
||||
self
|
||||
.rpc
|
||||
.rpc_call::<Vec<String>>("generatetoaddress", serde_json::json!([1, main_addr]))
|
||||
.rpc_call::<Vec<String>>("generatetoaddress", serde_json::json!([100, main_addr]))
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
for _ in 0 .. 100 {
|
||||
self.mine_block().await;
|
||||
}
|
||||
|
||||
let tx = self.get_block(new_block).await.unwrap().txdata.swap_remove(0);
|
||||
let mut tx = Transaction {
|
||||
version: Version(2),
|
||||
|
@ -797,24 +882,7 @@ impl Network for Bitcoin {
|
|||
script_pubkey: address.as_ref().script_pubkey(),
|
||||
}],
|
||||
};
|
||||
|
||||
let mut der = SECP256K1
|
||||
.sign_ecdsa_low_r(
|
||||
&Message::from(
|
||||
SighashCache::new(&tx)
|
||||
.legacy_signature_hash(0, &main_addr.script_pubkey(), EcdsaSighashType::All.to_u32())
|
||||
.unwrap()
|
||||
.to_raw_hash(),
|
||||
),
|
||||
&private_key.inner,
|
||||
)
|
||||
.serialize_der()
|
||||
.to_vec();
|
||||
der.push(1);
|
||||
tx.input[0].script_sig = Builder::new()
|
||||
.push_slice(PushBytesBuf::try_from(der).unwrap())
|
||||
.push_key(&public_key)
|
||||
.into_script();
|
||||
tx.input[0].script_sig = Self::sign_btc_input_for_p2pkh(&tx, 0, &private_key);
|
||||
|
||||
let block = self.get_latest_block_number().await.unwrap() + 1;
|
||||
self.rpc.send_raw_transaction(&tx).await.unwrap();
|
||||
|
|
|
@ -5,8 +5,42 @@ use dockertest::{
|
|||
|
||||
#[cfg(feature = "bitcoin")]
|
||||
mod bitcoin {
|
||||
use std::sync::Arc;
|
||||
|
||||
use rand_core::OsRng;
|
||||
|
||||
use frost::Participant;
|
||||
|
||||
use bitcoin_serai::bitcoin::{
|
||||
secp256k1::{SECP256K1, SecretKey, Message},
|
||||
PrivateKey, PublicKey,
|
||||
hashes::{HashEngine, Hash, sha256::Hash as Sha256},
|
||||
sighash::{SighashCache, EcdsaSighashType},
|
||||
absolute::LockTime,
|
||||
Amount as BAmount, Sequence, Script, Witness, OutPoint,
|
||||
address::Address as BAddress,
|
||||
transaction::{Version, Transaction, TxIn, TxOut},
|
||||
Network as BNetwork, ScriptBuf,
|
||||
opcodes::all::{OP_SHA256, OP_EQUALVERIFY},
|
||||
};
|
||||
|
||||
use scale::Encode;
|
||||
use sp_application_crypto::Pair;
|
||||
use serai_client::{in_instructions::primitives::Shorthand, primitives::insecure_pair_from_name};
|
||||
|
||||
use tokio::{
|
||||
time::{timeout, Duration},
|
||||
sync::Mutex,
|
||||
};
|
||||
|
||||
use serai_db::MemDb;
|
||||
|
||||
use super::*;
|
||||
use crate::networks::{Network, Bitcoin};
|
||||
use crate::{
|
||||
networks::{Network, Bitcoin, Output, OutputType, Block},
|
||||
tests::scanner::new_scanner,
|
||||
multisigs::scanner::ScannerEvent,
|
||||
};
|
||||
|
||||
#[test]
|
||||
fn test_dust_constant() {
|
||||
|
@ -19,6 +53,142 @@ mod bitcoin {
|
|||
check::<IsTrue<{ Bitcoin::DUST >= bitcoin_serai::wallet::DUST }>>();
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_receive_data_from_input() {
|
||||
let docker = spawn_bitcoin();
|
||||
docker.run(|ops| async move {
|
||||
let btc = bitcoin(&ops).await;
|
||||
|
||||
// generate a multisig address to receive the coins
|
||||
let mut keys = frost::tests::key_gen::<_, <Bitcoin as Network>::Curve>(&mut OsRng)
|
||||
.remove(&Participant::new(1).unwrap())
|
||||
.unwrap();
|
||||
<Bitcoin as Network>::tweak_keys(&mut keys);
|
||||
let group_key = keys.group_key();
|
||||
let serai_btc_address = <Bitcoin as Network>::external_address(group_key);
|
||||
|
||||
// btc key pair to send from
|
||||
let private_key = PrivateKey::new(SecretKey::new(&mut rand_core::OsRng), BNetwork::Regtest);
|
||||
let public_key = PublicKey::from_private_key(SECP256K1, &private_key);
|
||||
let main_addr = BAddress::p2pkh(&public_key, BNetwork::Regtest);
|
||||
|
||||
// get unlocked coins
|
||||
let new_block = btc.get_latest_block_number().await.unwrap() + 1;
|
||||
btc
|
||||
.rpc
|
||||
.rpc_call::<Vec<String>>("generatetoaddress", serde_json::json!([100, main_addr]))
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
// create a scanner
|
||||
let db = MemDb::new();
|
||||
let mut scanner = new_scanner(&btc, &db, group_key, &Arc::new(Mutex::new(true))).await;
|
||||
|
||||
// make a transfer instruction & hash it for script.
|
||||
let serai_address = insecure_pair_from_name("alice").public();
|
||||
let message = Shorthand::transfer(None, serai_address.into()).encode();
|
||||
let mut data = Sha256::engine();
|
||||
data.input(&message);
|
||||
|
||||
// make the output script => msg_script(OP_SHA256 PUSH MSG_HASH OP_EQUALVERIFY) + any_script
|
||||
let mut script = ScriptBuf::builder()
|
||||
.push_opcode(OP_SHA256)
|
||||
.push_slice(Sha256::from_engine(data).as_byte_array())
|
||||
.push_opcode(OP_EQUALVERIFY)
|
||||
.into_script();
|
||||
// append a regular spend script
|
||||
for i in main_addr.script_pubkey().instructions() {
|
||||
script.push_instruction(i.unwrap());
|
||||
}
|
||||
|
||||
// Create the first transaction
|
||||
let tx = btc.get_block(new_block).await.unwrap().txdata.swap_remove(0);
|
||||
let mut tx = Transaction {
|
||||
version: Version(2),
|
||||
lock_time: LockTime::ZERO,
|
||||
input: vec![TxIn {
|
||||
previous_output: OutPoint { txid: tx.txid(), vout: 0 },
|
||||
script_sig: Script::new().into(),
|
||||
sequence: Sequence(u32::MAX),
|
||||
witness: Witness::default(),
|
||||
}],
|
||||
output: vec![TxOut {
|
||||
value: tx.output[0].value - BAmount::from_sat(10000),
|
||||
script_pubkey: ScriptBuf::new_p2wsh(&script.wscript_hash()),
|
||||
}],
|
||||
};
|
||||
tx.input[0].script_sig = Bitcoin::sign_btc_input_for_p2pkh(&tx, 0, &private_key);
|
||||
let initial_output_value = tx.output[0].value;
|
||||
|
||||
// send it
|
||||
btc.rpc.send_raw_transaction(&tx).await.unwrap();
|
||||
|
||||
// Chain a transaction spending it with the InInstruction embedded in the input
|
||||
let mut tx = Transaction {
|
||||
version: Version(2),
|
||||
lock_time: LockTime::ZERO,
|
||||
input: vec![TxIn {
|
||||
previous_output: OutPoint { txid: tx.txid(), vout: 0 },
|
||||
script_sig: Script::new().into(),
|
||||
sequence: Sequence(u32::MAX),
|
||||
witness: Witness::new(),
|
||||
}],
|
||||
output: vec![TxOut {
|
||||
value: tx.output[0].value - BAmount::from_sat(10000),
|
||||
script_pubkey: serai_btc_address.as_ref().script_pubkey(),
|
||||
}],
|
||||
};
|
||||
|
||||
// add the witness script
|
||||
// This is the standard script with an extra argument of the InInstruction
|
||||
let mut sig = SECP256K1
|
||||
.sign_ecdsa_low_r(
|
||||
&Message::from(
|
||||
SighashCache::new(&tx)
|
||||
.p2wsh_signature_hash(0, &script, initial_output_value, EcdsaSighashType::All)
|
||||
.unwrap()
|
||||
.to_raw_hash(),
|
||||
),
|
||||
&private_key.inner,
|
||||
)
|
||||
.serialize_der()
|
||||
.to_vec();
|
||||
sig.push(1);
|
||||
tx.input[0].witness.push(sig);
|
||||
tx.input[0].witness.push(public_key.inner.serialize());
|
||||
tx.input[0].witness.push(message.clone());
|
||||
tx.input[0].witness.push(script);
|
||||
|
||||
// Send it immediately, as Bitcoin allows mempool chaining
|
||||
btc.rpc.send_raw_transaction(&tx).await.unwrap();
|
||||
|
||||
// Mine enough confirmations
|
||||
let block_number = btc.get_latest_block_number().await.unwrap() + 1;
|
||||
for _ in 0 .. <Bitcoin as Network>::CONFIRMATIONS {
|
||||
btc.mine_block().await;
|
||||
}
|
||||
let tx_block = btc.get_block(block_number).await.unwrap();
|
||||
|
||||
// verify that scanner picked up the output
|
||||
let outputs =
|
||||
match timeout(Duration::from_secs(30), scanner.events.recv()).await.unwrap().unwrap() {
|
||||
ScannerEvent::Block { is_retirement_block, block, outputs } => {
|
||||
scanner.multisig_completed.send(false).unwrap();
|
||||
assert!(!is_retirement_block);
|
||||
assert_eq!(block, tx_block.id());
|
||||
assert_eq!(outputs.len(), 1);
|
||||
assert_eq!(outputs[0].kind(), OutputType::External);
|
||||
outputs
|
||||
}
|
||||
_ => panic!("unexpectedly got eventuality completion"),
|
||||
};
|
||||
|
||||
// verify that the amount and message are correct
|
||||
assert_eq!(outputs[0].balance().amount.0, tx.output[0].value.to_sat());
|
||||
assert_eq!(outputs[0].data(), message);
|
||||
});
|
||||
}
|
||||
|
||||
fn spawn_bitcoin() -> DockerTest {
|
||||
serai_docker_tests::build("bitcoin".to_string());
|
||||
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
use core::time::Duration;
|
||||
use std::sync::Arc;
|
||||
|
||||
use ciphersuite::Ciphersuite;
|
||||
use rand_core::OsRng;
|
||||
|
||||
use frost::{Participant, tests::key_gen};
|
||||
|
@ -14,21 +15,13 @@ use crate::{
|
|||
multisigs::scanner::{ScannerEvent, Scanner, ScannerHandle},
|
||||
};
|
||||
|
||||
pub async fn test_scanner<N: Network>(network: N) {
|
||||
let mut keys =
|
||||
frost::tests::key_gen::<_, N::Curve>(&mut OsRng).remove(&Participant::new(1).unwrap()).unwrap();
|
||||
N::tweak_keys(&mut keys);
|
||||
let group_key = keys.group_key();
|
||||
|
||||
// Mine blocks so there's a confirmed block
|
||||
for _ in 0 .. N::CONFIRMATIONS {
|
||||
network.mine_block().await;
|
||||
}
|
||||
|
||||
let first = Arc::new(Mutex::new(true));
|
||||
pub async fn new_scanner<N: Network, D: Db>(
|
||||
network: &N,
|
||||
db: &D,
|
||||
group_key: <N::Curve as Ciphersuite>::G,
|
||||
first: &Arc<Mutex<bool>>,
|
||||
) -> ScannerHandle<N, D> {
|
||||
let activation_number = network.get_latest_block_number().await.unwrap();
|
||||
let db = MemDb::new();
|
||||
let new_scanner = || async {
|
||||
let mut db = db.clone();
|
||||
let (mut scanner, current_keys) = Scanner::new(network.clone(), db.clone());
|
||||
let mut first = first.lock().await;
|
||||
|
@ -45,8 +38,22 @@ pub async fn test_scanner<N: Network>(network: N) {
|
|||
assert_eq!(current_keys.len(), 1);
|
||||
}
|
||||
scanner
|
||||
};
|
||||
let scanner = new_scanner().await;
|
||||
}
|
||||
|
||||
pub async fn test_scanner<N: Network>(network: N) {
|
||||
let mut keys =
|
||||
frost::tests::key_gen::<_, N::Curve>(&mut OsRng).remove(&Participant::new(1).unwrap()).unwrap();
|
||||
N::tweak_keys(&mut keys);
|
||||
let group_key = keys.group_key();
|
||||
|
||||
// Mine blocks so there's a confirmed block
|
||||
for _ in 0 .. N::CONFIRMATIONS {
|
||||
network.mine_block().await;
|
||||
}
|
||||
|
||||
let db = MemDb::new();
|
||||
let first = Arc::new(Mutex::new(true));
|
||||
let scanner = new_scanner(&network, &db, group_key, &first).await;
|
||||
|
||||
// Receive funds
|
||||
let block = network.test_send(N::external_address(keys.group_key())).await;
|
||||
|
@ -73,7 +80,7 @@ pub async fn test_scanner<N: Network>(network: N) {
|
|||
let (mut scanner, outputs) = verify_event(scanner).await;
|
||||
|
||||
// Create a new scanner off the current DB and verify it re-emits the above events
|
||||
verify_event(new_scanner().await).await;
|
||||
verify_event(new_scanner(&network, &db, group_key, &first).await).await;
|
||||
|
||||
// Acknowledge the block
|
||||
let mut cloned_db = db.clone();
|
||||
|
@ -86,7 +93,12 @@ pub async fn test_scanner<N: Network>(network: N) {
|
|||
assert!(timeout(Duration::from_secs(30), scanner.events.recv()).await.is_err());
|
||||
|
||||
// Create a new scanner off the current DB and make sure it also does nothing
|
||||
assert!(timeout(Duration::from_secs(30), new_scanner().await.events.recv()).await.is_err());
|
||||
assert!(timeout(
|
||||
Duration::from_secs(30),
|
||||
new_scanner(&network, &db, group_key, &first).await.events.recv()
|
||||
)
|
||||
.await
|
||||
.is_err());
|
||||
}
|
||||
|
||||
pub async fn test_no_deadlock_in_multisig_completed<N: Network>(network: N) {
|
||||
|
|
Loading…
Reference in a new issue