channeld: pass htlcs to master daemon in batches.

When adding their HTLCs, it needs all the information.  When failing,
it needs the id as key and the failure reason.  When fulfilling, it
needs the id and payment preimage.

It also needs to know when we have received an revoke_and_ack or a
commitment_signed, to place in the database.

Signed-off-by: Rusty Russell <rusty@rustcorp.com.au>
This commit is contained in:
Rusty Russell 2017-06-20 15:20:03 +09:30
parent dcb3af7d7c
commit 4a161b710d
10 changed files with 890 additions and 288 deletions

View File

@ -373,6 +373,7 @@ enum channel_add_err channel_add_htlc(struct channel *channel,
if (!blocks_to_abs_locktime(cltv_expiry, &htlc->expiry))
return CHANNEL_ERR_INVALID_EXPIRY;
htlc->rhash = *payment_hash;
htlc->fail = NULL;
htlc->r = NULL;
/* BOLT #2:

View File

@ -113,6 +113,16 @@ struct peer {
};
static u8 *create_channel_announcement(const tal_t *ctx, struct peer *peer);
static void start_commit_timer(struct peer *peer);
/* Returns a pointer to the new end */
static void *tal_arr_append_(void **p, size_t size)
{
size_t n = tal_len(*p) / size;
tal_resize_(p, size, n+1, false);
return (char *)(*p) + n * size;
}
#define tal_arr_append(p) tal_arr_append_((void **)(p), sizeof(**(p)))
static struct io_plan *gossip_client_recv(struct io_conn *conn,
struct daemon_conn *dc)
@ -360,10 +370,41 @@ static struct io_plan *handle_peer_add_htlc(struct io_conn *conn,
return peer_read_message(conn, &peer->pcs, peer_in);
}
static void send_commit(struct peer *peer)
static struct changed_htlc *changed_htlc_arr(const tal_t *ctx,
const struct htlc **changed_htlcs)
{
tal_t *tmpctx = tal_tmpctx(peer);
struct changed_htlc *changed;
size_t i;
changed = tal_arr(ctx, struct changed_htlc, tal_count(changed_htlcs));
for (i = 0; i < tal_count(changed_htlcs); i++) {
changed[i].id = changed_htlcs[i]->id;
changed[i].newstate = changed_htlcs[i]->state;
}
return changed;
}
static u8 *sending_commitsig_msg(const tal_t *ctx,
u64 remote_commit_index,
const struct htlc **changed_htlcs)
{
const tal_t *tmpctx = tal_tmpctx(ctx);
struct changed_htlc *changed;
u8 *msg;
/* We tell master what (of our) HTLCs peer will now be
* committed to. */
changed = changed_htlc_arr(tmpctx, changed_htlcs);
msg = towire_channel_sending_commitsig(ctx, remote_commit_index,
changed);
tal_free(tmpctx);
return msg;
}
/* Master has acknowledged that we're sending commitment, so send it. */
static void handle_sending_commitsig_reply(struct peer *peer, const u8 *msg)
{
const tal_t *tmpctx = tal_tmpctx(peer);
secp256k1_ecdsa_signature commit_sig, *htlc_sigs;
size_t i;
struct bitcoin_tx **txs;
@ -372,28 +413,6 @@ static void send_commit(struct peer *peer)
struct pubkey localkey;
struct privkey local_secretkey;
/* Timer has expired. */
peer->commit_timer = NULL;
/* FIXME: Document this requirement in BOLT 2! */
/* We can't send two commits in a row. */
if (channel_awaiting_revoke_and_ack(peer->channel)) {
status_trace("Can't send commit: waiting for revoke_and_ack");
tal_free(tmpctx);
return;
}
/* BOLT #2:
*
* A node MUST NOT send a `commitment_signed` message which does not
* include any updates.
*/
if (!channel_sending_commit(peer->channel, NULL)) {
status_trace("Can't send commit: nothing to send");
tal_free(tmpctx);
return;
}
if (!derive_simple_privkey(&peer->our_secrets.payment_basepoint_secret,
&peer->channel->basepoints[LOCAL].payment,
&peer->current_per_commit[REMOTE],
@ -423,6 +442,7 @@ static void send_commit(struct peer *peer)
tal_hex(trc, wscripts[0]),
type_to_string(trc, struct pubkey,
&peer->channel->funding_pubkey[LOCAL]));
dump_htlcs(peer->channel, "Sending commit_sig");
/* BOLT #2:
*
@ -450,28 +470,199 @@ static void send_commit(struct peer *peer)
}
status_trace("Sending commit_sig with %zu htlc sigs",
tal_count(htlc_sigs));
msg = towire_commitment_signed(tmpctx, &peer->channel_id,
&commit_sig, htlc_sigs);
msg_enqueue(&peer->peer_out, take(msg));
/* Timer now considered expired, you can add a new one. */
peer->commit_timer = NULL;
/* FIXME: In case we had outstanding commits, restart timer */
start_commit_timer(peer);
tal_free(tmpctx);
}
/* This blocks other traffic from the master until we get reply. */
static void master_sync_reply(struct peer *peer, const u8 *msg,
enum channel_wire_type replytype,
void (*handle)(struct peer *peer, const u8 *msg))
{
assert(!peer->handle_master_reply);
peer->handle_master_reply = handle;
peer->master_reply_type = replytype;
daemon_conn_send(&peer->master, msg);
}
static void send_commit(struct peer *peer)
{
tal_t *tmpctx = tal_tmpctx(peer);
u8 *msg;
const struct htlc **changed_htlcs;
/* FIXME: Document this requirement in BOLT 2! */
/* We can't send two commits in a row. */
if (channel_awaiting_revoke_and_ack(peer->channel)
|| peer->handle_master_reply) {
status_trace("Can't send commit: waiting for revoke_and_ack %s",
peer->handle_master_reply ? "processing" : "reply");
/* Mark this as done and try again. */
peer->commit_timer = NULL;
start_commit_timer(peer);
tal_free(tmpctx);
return;
}
/* BOLT #2:
*
* A node MUST NOT send a `commitment_signed` message which does not
* include any updates.
*/
changed_htlcs = tal_arr(tmpctx, const struct htlc *, 0);
if (!channel_sending_commit(peer->channel, &changed_htlcs)) {
status_trace("Can't send commit: nothing to send");
peer->commit_timer = NULL;
tal_free(tmpctx);
return;
}
status_trace("Telling master we're about to commit...");
/* Tell master to save to database, then wait for reply. */
msg = sending_commitsig_msg(tmpctx, peer->commit_index[REMOTE],
changed_htlcs);
master_sync_reply(peer, take(msg),
WIRE_CHANNEL_SENDING_COMMITSIG_REPLY,
handle_sending_commitsig_reply);
tal_free(tmpctx);
}
static void start_commit_timer(struct peer *peer)
{
/* Already armed? */
if (peer->commit_timer)
if (peer->commit_timer) {
status_trace("Commit timer already running...");
return;
}
peer->commit_timer = new_reltimer(&peer->timers, peer,
time_from_msec(peer->commit_msec),
send_commit, peer);
}
/* We come back here once master has acked the commit_sig we received */
static struct io_plan *send_revocation(struct io_conn *conn, struct peer *peer)
{
struct pubkey oldpoint = peer->old_per_commit[LOCAL], test;
struct sha256 old_commit_secret;
u8 *msg;
peer->old_per_commit[LOCAL] = peer->current_per_commit[LOCAL];
if (!next_per_commit_point(&peer->shaseed, &old_commit_secret,
&peer->current_per_commit[LOCAL],
peer->commit_index[LOCAL]))
status_failed(WIRE_CHANNEL_CRYPTO_FAILED,
"Deriving next commit_point");
pubkey_from_privkey((struct privkey *)&old_commit_secret, &test);
if (!pubkey_eq(&test, &oldpoint))
status_failed(WIRE_CHANNEL_CRYPTO_FAILED,
"Invalid secret %s for commit_point",
tal_hexstr(trc, &old_commit_secret,
sizeof(old_commit_secret)));
peer->commit_index[LOCAL]++;
/* If this queues more changes on the other end, send commit. */
if (channel_sending_revoke_and_ack(peer->channel)) {
status_trace("revoke_and_ack made pending: commit timer");
start_commit_timer(peer);
}
msg = towire_revoke_and_ack(peer, &peer->channel_id, &old_commit_secret,
&peer->current_per_commit[LOCAL]);
msg_enqueue(&peer->peer_out, take(msg));
return peer_read_message(conn, &peer->pcs, peer_in);
}
static u8 *got_commitsig_msg(const tal_t *ctx,
u64 local_commit_index,
const secp256k1_ecdsa_signature *commit_sig,
const secp256k1_ecdsa_signature *htlc_sigs,
const struct htlc **changed_htlcs)
{
const tal_t *tmpctx = tal_tmpctx(ctx);
struct changed_htlc *changed;
struct fulfilled_htlc *fulfilled;
struct failed_htlc *failed;
struct added_htlc *added;
u8 *msg;
changed = tal_arr(tmpctx, struct changed_htlc, 0);
added = tal_arr(tmpctx, struct added_htlc, 0);
failed = tal_arr(tmpctx, struct failed_htlc, 0);
fulfilled = tal_arr(tmpctx, struct fulfilled_htlc, 0);
for (size_t i = 0; i < tal_count(changed_htlcs); i++) {
const struct htlc *htlc = changed_htlcs[i];
if (htlc->state == RCVD_ADD_COMMIT) {
struct added_htlc *a = tal_arr_append(&added);
a->id = htlc->id;
a->amount_msat = htlc->msatoshi;
a->payment_hash = htlc->rhash;
a->cltv_expiry = abs_locktime_to_blocks(&htlc->expiry);
memcpy(a->onion_routing_packet,
htlc->routing,
sizeof(a->onion_routing_packet));
} else if (htlc->state == RCVD_REMOVE_COMMIT) {
if (htlc->r) {
struct fulfilled_htlc *f;
assert(!htlc->fail);
f = tal_arr_append(&fulfilled);
f->id = htlc->id;
f->payment_preimage = *htlc->r;
} else {
struct failed_htlc *f;
assert(htlc->fail);
f = tal_arr_append(&failed);
f->id = htlc->id;
f->failreason = cast_const(u8 *, htlc->fail);
}
} else {
struct changed_htlc *c = tal_arr_append(&changed);
assert(htlc->state == RCVD_REMOVE_ACK_COMMIT
|| htlc->state == RCVD_ADD_ACK_COMMIT);
c->id = htlc->id;
c->newstate = htlc->state;
}
}
msg = towire_channel_got_commitsig(ctx, local_commit_index,
commit_sig,
htlc_sigs,
added,
fulfilled,
failed,
changed);
tal_free(tmpctx);
return msg;
}
/* Tell peer to continue now master has replied. */
static void handle_reply_wake_peer(struct peer *peer, const u8 *msg)
{
io_wake(peer);
}
static struct io_plan *handle_peer_commit_sig(struct io_conn *conn,
struct peer *peer, const u8 *msg)
{
tal_t *tmpctx = tal_tmpctx(peer);
struct sha256 old_commit_secret;
const tal_t *tmpctx = tal_tmpctx(peer);
struct channel_id channel_id;
secp256k1_ecdsa_signature commit_sig, *htlc_sigs;
struct pubkey remotekey;
@ -518,7 +709,8 @@ static struct io_plan *handle_peer_commit_sig(struct io_conn *conn,
* applied.
*/
if (!check_tx_sig(txs[0], 0, NULL, wscripts[0],
&peer->channel->funding_pubkey[REMOTE], &commit_sig))
&peer->channel->funding_pubkey[REMOTE], &commit_sig)) {
dump_htlcs(peer->channel, "receiving commit_sig");
peer_failed(io_conn_fd(peer->peer_conn),
&peer->pcs.cs,
&peer->channel_id,
@ -531,6 +723,7 @@ static struct io_plan *handle_peer_commit_sig(struct io_conn *conn,
type_to_string(msg, struct pubkey,
&peer->channel->funding_pubkey
[REMOTE]));
}
/* BOLT #2:
*
@ -569,62 +762,38 @@ static struct io_plan *handle_peer_commit_sig(struct io_conn *conn,
status_trace("Received commit_sig with %zu htlc sigs",
tal_count(htlc_sigs));
struct pubkey oldpoint = peer->old_per_commit[LOCAL], test;
status_trace("Sending secret for point %"PRIu64" %s",
peer->commit_index[LOCAL]-1,
type_to_string(trc, struct pubkey,
&peer->old_per_commit[LOCAL]));
/* Tell master daemon, then wait for ack. */
msg = got_commitsig_msg(tmpctx, peer->commit_index[LOCAL], &commit_sig,
htlc_sigs, changed_htlcs);
peer->old_per_commit[LOCAL] = peer->current_per_commit[LOCAL];
if (!next_per_commit_point(&peer->shaseed, &old_commit_secret,
&peer->current_per_commit[LOCAL],
peer->commit_index[LOCAL]))
status_failed(WIRE_CHANNEL_CRYPTO_FAILED,
"Deriving next commit_point");
master_sync_reply(peer, take(msg),
WIRE_CHANNEL_GOT_COMMITSIG_REPLY,
handle_reply_wake_peer);
pubkey_from_privkey((struct privkey *)&old_commit_secret, &test);
if (!pubkey_eq(&test, &oldpoint))
status_failed(WIRE_CHANNEL_CRYPTO_FAILED,
"Invalid secret %s for commit_point",
tal_hexstr(msg, &old_commit_secret,
sizeof(old_commit_secret)));
peer->commit_index[LOCAL]++;
/* If this queues more changes on the other end, send commit. */
if (channel_sending_revoke_and_ack(peer->channel)) {
status_trace("revoke_and_ack made pending: commit timer");
start_commit_timer(peer);
}
msg = towire_revoke_and_ack(msg, &channel_id, &old_commit_secret,
&peer->current_per_commit[LOCAL]);
msg_enqueue(&peer->peer_out, take(msg));
tal_free(tmpctx);
return peer_read_message(conn, &peer->pcs, peer_in);
/* And peer waits for reply. */
return io_wait(conn, peer, send_revocation, peer);
}
static void their_htlc_locked(const struct htlc *htlc, struct peer *peer)
static void add_htlc_with_ss(u64 **added_ids, struct secret **shared_secrets,
const struct htlc *htlc)
{
tal_t *tmpctx = tal_tmpctx(peer);
u8 *msg;
struct onionpacket *op;
struct sha256 bad_onion_sha;
struct secret ss;
enum onion_type failcode;
enum channel_remove_err rerr;
tal_t *tmpctx = tal_tmpctx(*added_ids);
struct pubkey ephemeral;
struct onionpacket *op;
u8 *msg;
struct secret *ss = tal_arr_append(shared_secrets);
u64 *id = tal_arr_append(added_ids);
status_trace("their htlc %"PRIu64" locked", htlc->id);
*id = htlc->id;
/* We unwrap the onion now. */
/* FIXME: We could do this earlier and call HSM async, for speed. */
op = parse_onionpacket(tmpctx, htlc->routing, TOTAL_PACKET_SIZE);
if (!op) {
/* FIXME: could be bad version, bad key. */
failcode = WIRE_INVALID_ONION_VERSION;
goto bad_onion;
/* Return an invalid shared secret. */
memset(ss, 0, sizeof(*ss));
tal_free(tmpctx);
return;
}
/* Because wire takes struct pubkey. */
@ -633,42 +802,50 @@ static void their_htlc_locked(const struct htlc *htlc, struct peer *peer)
if (!wire_sync_write(HSM_FD, msg))
status_failed(WIRE_CHANNEL_HSM_FAILED, "Writing ecdh req");
msg = wire_sync_read(tmpctx, HSM_FD);
if (!msg || !fromwire_hsm_ecdh_resp(msg, NULL, &ss))
/* Gives all-zero shares_secret if it was invalid. */
if (!msg || !fromwire_hsm_ecdh_resp(msg, NULL, ss))
status_failed(WIRE_CHANNEL_HSM_FAILED, "Reading ecdh response");
tal_free(tmpctx);
}
if (memeqzero(&ss, sizeof(ss))) {
failcode = WIRE_INVALID_ONION_KEY;
goto bad_onion;
static u8 *got_revoke_msg(const tal_t *ctx, u64 revoke_num,
const struct sha256 *per_commitment_secret,
const struct htlc **changed_htlcs)
{
tal_t *tmpctx = tal_tmpctx(ctx);
u8 *msg;
u64 *added_ids = tal_arr(tmpctx, u64, 0);
struct secret *shared_secrets = tal_arr(tmpctx, struct secret, 0);
struct changed_htlc *changed = tal_arr(tmpctx, struct changed_htlc, 0);
for (size_t i = 0; i < tal_count(changed_htlcs); i++) {
const struct htlc *htlc = changed_htlcs[i];
status_trace("HTLC %"PRIu64"[%s] => %s",
htlc->id, side_to_str(htlc_owner(htlc)),
htlc_state_name(htlc->state));
/* We've both committed to their htlc now. */
if (htlc->state == RCVD_ADD_ACK_REVOCATION) {
add_htlc_with_ss(&added_ids, &shared_secrets, htlc);
} else {
struct changed_htlc *c = tal_arr_append(&changed);
c->id = changed_htlcs[i]->id;
c->newstate = changed_htlcs[i]->state;
}
}
/* Tell master to deal with it. */
msg = towire_channel_accepted_htlc(tmpctx, htlc->id, htlc->msatoshi,
abs_locktime_to_blocks(&htlc->expiry),
&htlc->rhash,
&ss,
htlc->routing);
daemon_conn_send(&peer->master, take(msg));
msg = towire_channel_got_revoke(ctx, revoke_num, per_commitment_secret,
added_ids, shared_secrets, changed);
tal_free(tmpctx);
return;
return msg;
}
bad_onion:
sha256(&bad_onion_sha, htlc->routing, TOTAL_PACKET_SIZE);
msg = towire_update_fail_malformed_htlc(tmpctx, &peer->channel_id,
htlc->id, &bad_onion_sha,
failcode);
msg_enqueue(&peer->peer_out, take(msg));
status_trace("htlc %"PRIu64" %s", htlc->id, onion_type_name(failcode));
rerr = channel_fail_htlc(peer->channel, REMOTE, htlc->id);
if (rerr != CHANNEL_ERR_REMOVE_OK)
peer_failed(io_conn_fd(peer->peer_conn),
&peer->pcs.cs,
&peer->channel_id,
WIRE_CHANNEL_INTERNAL_ERROR,
"Could not fail malformed htlc %"PRIu64": %u",
htlc->id, rerr);
/* We come back here once master has acked the revoke_and_ack we received */
static struct io_plan *accepted_revocation(struct io_conn *conn,
struct peer *peer)
{
start_commit_timer(peer);
tal_free(tmpctx);
return peer_read_message(conn, &peer->pcs, peer_in);
}
static struct io_plan *handle_peer_revoke_and_ack(struct io_conn *conn,
@ -732,9 +909,6 @@ static struct io_plan *handle_peer_revoke_and_ack(struct io_conn *conn,
peer->commit_index[REMOTE],
type_to_string(msg, struct privkey, &privkey));
}
peer->commit_index[REMOTE]++;
peer->old_per_commit[REMOTE] = peer->current_per_commit[REMOTE];
peer->current_per_commit[REMOTE] = next_per_commit;
/* We start timer even if this returns false: we might have delayed
* commit because we were waiting for this! */
@ -743,15 +917,19 @@ static struct io_plan *handle_peer_revoke_and_ack(struct io_conn *conn,
else
status_trace("No commits outstanding after recv revoke_and_ack");
/* Tell master about locked-in htlcs. */
for (size_t i = 0; i < tal_count(changed_htlcs); i++) {
if (changed_htlcs[i]->state == RCVD_ADD_ACK_REVOCATION) {
their_htlc_locked(changed_htlcs[i], peer);
}
}
peer->commit_index[REMOTE]++;
peer->old_per_commit[REMOTE] = peer->current_per_commit[REMOTE];
peer->current_per_commit[REMOTE] = next_per_commit;
start_commit_timer(peer);
return peer_read_message(conn, &peer->pcs, peer_in);
/* Tell master about things this locks in, wait for response */
msg = got_revoke_msg(msg, peer->commit_index[REMOTE],
&old_commit_secret, changed_htlcs);
master_sync_reply(peer, take(msg),
WIRE_CHANNEL_GOT_REVOKE_REPLY,
handle_reply_wake_peer);
/* And peer waits for reply. */
return io_wait(conn, peer, accepted_revocation, peer);
}
static struct io_plan *handle_peer_fulfill_htlc(struct io_conn *conn,
@ -774,8 +952,7 @@ static struct io_plan *handle_peer_fulfill_htlc(struct io_conn *conn,
e = channel_fulfill_htlc(peer->channel, LOCAL, id, &preimage);
switch (e) {
case CHANNEL_ERR_REMOVE_OK:
msg = towire_channel_fulfilled_htlc(msg, id, &preimage);
daemon_conn_send(&peer->master, take(msg));
/* FIXME: We could send preimages to master immediately. */
start_commit_timer(peer);
return peer_read_message(conn, &peer->pcs, peer_in);
/* These shouldn't happen, because any offered HTLC (which would give
@ -803,6 +980,7 @@ static struct io_plan *handle_peer_fail_htlc(struct io_conn *conn,
u64 id;
enum channel_remove_err e;
u8 *reason;
struct htlc *htlc;
if (!fromwire_update_fail_htlc(msg, msg, NULL,
&channel_id, &id, &reason)) {
@ -816,8 +994,9 @@ static struct io_plan *handle_peer_fail_htlc(struct io_conn *conn,
e = channel_fail_htlc(peer->channel, LOCAL, id);
switch (e) {
case CHANNEL_ERR_REMOVE_OK:
msg = towire_channel_failed_htlc(msg, id, reason);
daemon_conn_send(&peer->master, take(msg));
/* Save reason for when we tell master. */
htlc = channel_get_htlc(peer->channel, LOCAL, id);
htlc->fail = tal_steal(htlc, reason);
start_commit_timer(peer);
return peer_read_message(conn, &peer->pcs, peer_in);
case CHANNEL_ERR_NO_SUCH_ID:
@ -843,10 +1022,13 @@ static struct io_plan *handle_peer_fail_malformed_htlc(struct io_conn *conn,
u64 id;
enum channel_remove_err e;
struct sha256 sha256_of_onion;
u16 failcode;
u16 failure_code;
struct htlc *htlc;
u8 *fail;
if (!fromwire_update_fail_malformed_htlc(msg, NULL, &channel_id, &id,
&sha256_of_onion, &failcode)) {
&sha256_of_onion,
&failure_code)) {
peer_failed(io_conn_fd(peer->peer_conn),
&peer->pcs.cs,
&peer->channel_id,
@ -855,12 +1037,46 @@ static struct io_plan *handle_peer_fail_malformed_htlc(struct io_conn *conn,
tal_hex(msg, msg));
}
/* BOLT #2:
*
* A receiving node MUST fail the channel if the `BADONION` bit in
* `failure_code` is not set for `update_fail_malformed_htlc`.
*/
if (!(failure_code & BADONION)) {
peer_failed(io_conn_fd(peer->peer_conn),
&peer->pcs.cs,
&peer->channel_id,
WIRE_CHANNEL_PEER_BAD_MESSAGE,
"Bad update_fail_malformed_htlc failure code %u",
failure_code);
}
e = channel_fail_htlc(peer->channel, LOCAL, id);
switch (e) {
case CHANNEL_ERR_REMOVE_OK:
msg = towire_channel_malformed_htlc(msg, id, &sha256_of_onion,
failcode);
daemon_conn_send(&peer->master, take(msg));
htlc = channel_get_htlc(peer->channel, LOCAL, id);
/* FIXME: Do this! */
/* BOLT #2:
*
* A receiving node MAY check the `sha256_of_onion`
* in `update_fail_malformed_htlc` and MAY retry or choose an
* alternate error response if it does not match the onion it
* sent.
*/
/* BOLT #2:
*
* Otherwise, a receiving node which has an outgoing HTLC
* canceled by `update_fail_malformed_htlc` MUST return an
* error in the `update_fail_htlc` sent to the link which
* originally sent the HTLC using the `failure_code` given and
* setting the data to `sha256_of_onion`.
*/
fail = tal_arr(htlc, u8, 0);
towire_u16(&fail, failure_code);
towire_sha256(&fail, &sha256_of_onion);
/* FIXME: Make htlc->fail a u8 *! */
htlc->fail = fail;
start_commit_timer(peer);
return peer_read_message(conn, &peer->pcs, peer_in);
case CHANNEL_ERR_NO_SUCH_ID:
@ -1350,13 +1566,15 @@ static struct io_plan *req_in(struct io_conn *conn, struct daemon_conn *master)
case WIRE_CHANNEL_NORMAL_OPERATION:
case WIRE_CHANNEL_INIT:
case WIRE_CHANNEL_OFFER_HTLC_REPLY:
case WIRE_CHANNEL_ACCEPTED_HTLC:
case WIRE_CHANNEL_FULFILLED_HTLC:
case WIRE_CHANNEL_FAILED_HTLC:
case WIRE_CHANNEL_MALFORMED_HTLC:
case WIRE_CHANNEL_PING_REPLY:
case WIRE_CHANNEL_PEER_BAD_MESSAGE:
case WIRE_CHANNEL_ANNOUNCED:
case WIRE_CHANNEL_SENDING_COMMITSIG:
case WIRE_CHANNEL_GOT_COMMITSIG:
case WIRE_CHANNEL_GOT_REVOKE:
case WIRE_CHANNEL_SENDING_COMMITSIG_REPLY:
case WIRE_CHANNEL_GOT_COMMITSIG_REPLY:
case WIRE_CHANNEL_GOT_REVOKE_REPLY:
break;
}
status_failed(WIRE_CHANNEL_BAD_COMMAND, "%u %s", t,
@ -1418,6 +1636,7 @@ int main(int argc, char *argv[])
peer->commit_index[LOCAL] = peer->commit_index[REMOTE] = 0;
peer->have_sigs[LOCAL] = peer->have_sigs[REMOTE] = false;
peer->handle_master_reply = NULL;
peer->master_reply_type = 0;
msg_queue_init(&peer->master_deferred, peer);
/* We send these to HSM to get real signatures; don't have valgrind

View File

@ -78,35 +78,6 @@ channel_fail_htlc,,id,8
channel_fail_htlc,,len,2
channel_fail_htlc,,error_pkt,len*u8
# Peer and I are irrevocably committed to this HTLC.
# channelds get shared secret from HSM to save master daemon work.
channel_accepted_htlc,7
channel_accepted_htlc,,id,8
channel_accepted_htlc,,amount_msat,8
channel_accepted_htlc,,cltv_expiry,4
channel_accepted_htlc,,payment_hash,32
channel_accepted_htlc,,shared_secret,struct secret
channel_accepted_htlc,,onion,1366*u8
# FIXME: Add code to commit current channel state!
# The HTLC preimage was given.
channel_fulfilled_htlc,8
channel_fulfilled_htlc,,id,8
channel_fulfilled_htlc,,payment_preimage,struct preimage
# This HTLC failed
channel_failed_htlc,9
channel_failed_htlc,,id,8
channel_failed_htlc,,len,2
channel_failed_htlc,,reason,len*u8
# This HTLC was returned malformed
channel_malformed_htlc,10
channel_malformed_htlc,,id,8
channel_malformed_htlc,,sha256_of_onion,32
channel_malformed_htlc,,failure_code,2
# Ping/pong test.
channel_ping,11
channel_ping,,num_pong_bytes,u16
@ -117,3 +88,50 @@ channel_ping_reply,,totlen,u16
# Channeld tells the master that the channel has been announced
channel_announced,12
# When we send a commitment_signed message, tell master.
channel_sending_commitsig,20
channel_sending_commitsig,,commitnum,u64
# SENT_ADD_COMMIT, SENT_REMOVE_ACK_COMMIT, SENT_ADD_ACK_COMMIT, SENT_REMOVE_COMMIT
channel_sending_commitsig,,num_changed,u16
channel_sending_commitsig,,changed,num_changed*struct changed_htlc
# Wait for reply, to make sure it's on disk before we send commit.
channel_sending_commitsig_reply,120
# When we have a commitment_signed message, tell master to remember.
channel_got_commitsig,21
channel_got_commitsig,,commitnum,u64
channel_got_commitsig,,signature,secp256k1_ecdsa_signature
channel_got_commitsig,,num_htlcs,u16
channel_got_commitsig,,htlc_signature,num_htlcs*secp256k1_ecdsa_signature
# RCVD_ADD_COMMIT: we're now committed to their new offered HTLCs.
channel_got_commitsig,,num_added,u16
channel_got_commitsig,,added,num_added*struct added_htlc
# RCVD_REMOVE_COMMIT: we're now no longer committed to these HTLCs.
channel_got_commitsig,,num_fulfilled,u16
channel_got_commitsig,,fulfilled,num_fulfilled*struct fulfilled_htlc
channel_got_commitsig,,num_failed,u16
channel_got_commitsig,,failed,num_failed*struct failed_htlc
# RCVD_ADD_ACK_COMMIT, RCVD_REMOVE_ACK_COMMIT
channel_got_commitsig,,num_changed,u16
channel_got_commitsig,,changed,num_changed*struct changed_htlc
# Wait for reply, to make sure it's on disk before we send revocation.
channel_got_commitsig_reply,121
#include <lightningd/htlc_wire.h>
channel_got_revoke,22
channel_got_revoke,,revokenum,u64
channel_got_revoke,,per_commitment_secret,struct sha256
# RCVD_ADD_ACK_REVOCATION
channel_got_revoke,,num_added,u16
channel_got_revoke,,added_ids,num_added*u64
channel_got_revoke,,shared_secret,num_added*struct secret
# RCVD_REMOVE_ACK_REVOCATION, RCVD_ADD_REVOCATION, RCVD_REMOVE_REVOCATION
channel_got_revoke,,num_changed,u16
channel_got_revoke,,changed,num_changed*struct changed_htlc
# Wait for reply, to make sure it's on disk before we continue
# (eg. if we sent another commitment_signed, that would implicitly ack).
channel_got_revoke_reply,122

1 # Shouldn't happen
78 channel_accepted_htlc,,onion,1366*u8 # When we send a commitment_signed message, tell master.
79 # FIXME: Add code to commit current channel state! channel_sending_commitsig,20
80 # The HTLC preimage was given. channel_sending_commitsig,,commitnum,u64
channel_fulfilled_htlc,8
channel_fulfilled_htlc,,id,8
channel_fulfilled_htlc,,payment_preimage,struct preimage
# This HTLC failed
channel_failed_htlc,9
channel_failed_htlc,,id,8
channel_failed_htlc,,len,2
channel_failed_htlc,,reason,len*u8
# This HTLC was returned malformed
channel_malformed_htlc,10
channel_malformed_htlc,,id,8
channel_malformed_htlc,,sha256_of_onion,32
channel_malformed_htlc,,failure_code,2
# Ping/pong test.
channel_ping,11
channel_ping,,num_pong_bytes,u16
channel_ping,,len,u16
channel_ping_reply,111
channel_ping_reply,,totlen,u16
# Channeld tells the master that the channel has been announced
channel_announced,12
81 # SENT_ADD_COMMIT, SENT_REMOVE_ACK_COMMIT, SENT_ADD_ACK_COMMIT, SENT_REMOVE_COMMIT
82 channel_sending_commitsig,,num_changed,u16
83 channel_sending_commitsig,,changed,num_changed*struct changed_htlc
88 channel_got_commitsig,,commitnum,u64
89 channel_got_commitsig,,signature,secp256k1_ecdsa_signature
90 channel_got_commitsig,,num_htlcs,u16
91 channel_got_commitsig,,htlc_signature,num_htlcs*secp256k1_ecdsa_signature
92 # RCVD_ADD_COMMIT: we're now committed to their new offered HTLCs.
93 channel_got_commitsig,,num_added,u16
94 channel_got_commitsig,,added,num_added*struct added_htlc
95 # RCVD_REMOVE_COMMIT: we're now no longer committed to these HTLCs.
96 channel_got_commitsig,,num_fulfilled,u16
97 channel_got_commitsig,,fulfilled,num_fulfilled*struct fulfilled_htlc
98 channel_got_commitsig,,num_failed,u16
99 channel_got_commitsig,,failed,num_failed*struct failed_htlc
100 # RCVD_ADD_ACK_COMMIT, RCVD_REMOVE_ACK_COMMIT
101 channel_got_commitsig,,num_changed,u16
102 channel_got_commitsig,,changed,num_changed*struct changed_htlc
103 # Wait for reply, to make sure it's on disk before we send revocation.
104 channel_got_commitsig_reply,121
105 #include <lightningd/htlc_wire.h>
106 channel_got_revoke,22
107 channel_got_revoke,,revokenum,u64
108 channel_got_revoke,,per_commitment_secret,struct sha256
109 # RCVD_ADD_ACK_REVOCATION
110 channel_got_revoke,,num_added,u16
111 channel_got_revoke,,added_ids,num_added*u64
112 channel_got_revoke,,shared_secret,num_added*struct secret
113 # RCVD_REMOVE_ACK_REVOCATION, RCVD_ADD_REVOCATION, RCVD_REMOVE_REVOCATION
114 channel_got_revoke,,num_changed,u16
115 channel_got_revoke,,changed,num_changed*struct changed_htlc
116 # Wait for reply, to make sure it's on disk before we continue
117 # (eg. if we sent another commitment_signed, that would implicitly ack).
118 channel_got_revoke_reply,122
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137

View File

@ -135,6 +135,7 @@ static void json_dev_newhtlc(struct command *cmd,
hend = tal(cmd, struct htlc_end);
hend->which_end = HTLC_DST;
hend->hstate = SENT_ADD_HTLC;
hend->peer = peer;
hend->msatoshis = msatoshi;
hend->other_end = NULL;

View File

@ -3,6 +3,7 @@
#include "config.h"
#include <ccan/htable/htable_type.h>
#include <ccan/short_types/short_types.h>
#include <daemon/htlc_state.h>
#include <lightningd/sphinx.h>
/* A HTLC has a source and destination: if other is NULL, it's this node.
@ -21,6 +22,9 @@ struct htlc_end {
/* If this is driven by a command. */
struct pay_command *pay_command;
/* FIXME: We really only need this in the database. */
enum htlc_state hstate;
/* Temporary information, while we resolve the next hop */
u8 *next_onion;
struct short_channel_id next_channel;
@ -29,9 +33,12 @@ struct htlc_end {
u32 cltv_expiry;
struct sha256 payment_hash;
/* If we failed HTLC, here's the message. */
/* If they failed HTLC, here's the message. */
const u8 *fail_msg;
/* If they succeeded, here's the preimage. */
struct sha256 *preimage;
/* If we are forwarding, remember the shared secret for an
* eventual reply */
struct secret *shared_secret;

View File

@ -341,7 +341,9 @@ static void json_sendpay(struct command *cmd,
pc->out = tal(pc, struct htlc_end);
pc->out->which_end = HTLC_DST;
pc->out->hstate = SENT_ADD_HTLC;
pc->out->peer = peer;
pc->out->fail_msg = NULL;
pc->out->msatoshis = amount;
pc->out->other_end = NULL;
pc->out->pay_command = pc;

View File

@ -857,14 +857,12 @@ static int channel_msg(struct subd *sd, const u8 *msg, const int *unused)
peer_set_condition(sd->peer,
CHANNELD_AWAITING_LOCKIN, CHANNELD_NORMAL);
break;
case WIRE_CHANNEL_ACCEPTED_HTLC:
return peer_accepted_htlc(sd->peer, msg);
case WIRE_CHANNEL_FULFILLED_HTLC:
return peer_fulfilled_htlc(sd->peer, msg);
case WIRE_CHANNEL_FAILED_HTLC:
return peer_failed_htlc(sd->peer, msg);
case WIRE_CHANNEL_MALFORMED_HTLC:
return peer_failed_malformed_htlc(sd->peer, msg);
case WIRE_CHANNEL_SENDING_COMMITSIG:
return peer_sending_commitsig(sd->peer, msg);
case WIRE_CHANNEL_GOT_COMMITSIG:
return peer_got_commitsig(sd->peer, msg);
case WIRE_CHANNEL_GOT_REVOKE:
return peer_got_revoke(sd->peer, msg);
case WIRE_CHANNEL_ANNOUNCED:
peer_channel_announced(sd->peer, msg);
break;
@ -886,6 +884,9 @@ static int channel_msg(struct subd *sd, const u8 *msg, const int *unused)
case WIRE_CHANNEL_FULFILL_HTLC:
case WIRE_CHANNEL_FAIL_HTLC:
case WIRE_CHANNEL_PING:
case WIRE_CHANNEL_GOT_COMMITSIG_REPLY:
case WIRE_CHANNEL_GOT_REVOKE_REPLY:
case WIRE_CHANNEL_SENDING_COMMITSIG_REPLY:
/* Replies go to requests. */
case WIRE_CHANNEL_OFFER_HTLC_REPLY:
case WIRE_CHANNEL_PING_REPLY:

View File

@ -1,9 +1,12 @@
#include <ccan/build_assert/build_assert.h>
#include <ccan/mem/mem.h>
#include <daemon/chaintopology.h>
#include <daemon/invoice.h>
#include <daemon/log.h>
#include <lightningd/channel/gen_channel_wire.h>
#include <lightningd/gossip/gen_gossip_wire.h>
#include <lightningd/htlc_end.h>
#include <lightningd/htlc_wire.h>
#include <lightningd/lightningd.h>
#include <lightningd/pay.h>
#include <lightningd/peer_control.h>
@ -13,6 +16,17 @@
#include <overflows.h>
#include <wire/gen_onion_wire.h>
/* This is where we write to the database the minimal HTLC info
* required to do penalty transaction */
static void save_htlc_stub(struct lightningd *ld,
struct peer *peer,
enum htlc_end_type htlc_end_type,
u32 cltv_value,
const struct sha256 *payment_hash)
{
/* FIXME: remember peer, direction, cltv and RIPEMD160(hash) */
}
/* This obfuscates the message, whether local or forwarded. */
static void relay_htlc_failmsg(struct htlc_end *hend)
{
@ -246,8 +260,6 @@ static void handle_localpay(struct htlc_end *hend,
goto fail;
}
connect_htlc_end(&hend->peer->ld->htlc_ends, hend);
log_info(hend->peer->ld->log, "Resolving invoice '%s' with HTLC %"PRIu64,
invoice->label, hend->htlc_id);
fulfill_htlc(hend, &invoice->r);
@ -272,6 +284,8 @@ static void hend_subd_died(struct htlc_end *hend)
fail_htlc(hend->other_end, WIRE_TEMPORARY_CHANNEL_FAILURE, NULL);
}
/* This is where channeld gives us the HTLC id, and also reports if it
* failed immediately. */
static bool rcvd_htlc_reply(struct subd *subd, const u8 *msg, const int *fds,
struct htlc_end *hend)
{
@ -297,10 +311,10 @@ static bool rcvd_htlc_reply(struct subd *subd, const u8 *msg, const int *fds,
return true;
}
tal_del_destructor(hend, hend_subd_died);
/* Add it to lookup table now we know id. */
connect_htlc_end(&subd->ld->htlc_ends, hend);
/* Add it to lookup table. */
connect_htlc_end(&hend->peer->ld->htlc_ends, hend);
/* When channeld includes it in commitment, we'll make it persistent. */
return true;
}
@ -324,11 +338,19 @@ static void forward_htlc(struct htlc_end *hend,
}
if (!peer_can_add_htlc(next)) {
log_info(next->log, "Attempt to forward HTLC but not ready");
log_info(next->log, "Attempt to forward HTLC but not ready (%s)",
peer_state_name(next->state));
failcode = WIRE_UNKNOWN_NEXT_PEER;
goto fail;
}
if (!next->owner) {
log_info(next->log, "Attempt to forward HTLC but unowned (%s)",
peer_state_name(next->state));
failcode = WIRE_TEMPORARY_CHANNEL_FAILURE;
goto fail;
}
/* BOLT #7:
*
* The node creating `channel_update` SHOULD accept HTLCs which pay a
@ -376,11 +398,14 @@ static void forward_htlc(struct htlc_end *hend,
/* Make sure daemon owns it, in case it fails. */
hend->other_end = tal(next->owner, struct htlc_end);
hend->other_end->hstate = SENT_ADD_HTLC;
hend->other_end->which_end = HTLC_DST;
hend->other_end->peer = next;
hend->other_end->other_end = hend;
hend->other_end->pay_command = NULL;
hend->other_end->msatoshis = amt_to_forward;
hend->other_end->outgoing_cltv_value = outgoing_cltv_value;
hend->other_end->payment_hash = hend->payment_hash;
tal_add_destructor(hend->other_end, hend_subd_died);
msg = towire_channel_offer_htlc(next, amt_to_forward,
@ -432,57 +457,98 @@ static bool channel_resolve_reply(struct subd *gossip, const u8 *msg,
return true;
}
int peer_accepted_htlc(struct peer *peer, const u8 *msg)
static bool hend_update_state(struct peer *peer,
struct htlc_end *hend,
enum htlc_state newstate)
{
enum htlc_state expected = hend->hstate + 1;
/* We never get told about RCVD_REMOVE_HTLC or SENT_REMOVE_HTLC, so
* skip over those (we initialize in SENT_ADD_HTLC / RCVD_ADD_COMMIT, so
* those work). */
if (expected == RCVD_REMOVE_HTLC)
expected = RCVD_REMOVE_COMMIT;
else if (expected == SENT_REMOVE_HTLC)
expected = SENT_REMOVE_COMMIT;
if (newstate != expected) {
log_broken(peer->log, "HTLC %"PRIu64" invalid update %s->%s",
hend->htlc_id,
htlc_state_name(hend->hstate),
htlc_state_name(newstate));
return false;
}
log_debug(peer->log, "%s HTLC %"PRIu64" %s->%s",
hend->which_end == HTLC_SRC ? "Their" : "Our",
hend->htlc_id,
htlc_state_name(hend->hstate),
htlc_state_name(newstate));
/* FIXME: db commit */
hend->hstate = newstate;
return true;
}
/* Everyone is committed to this htlc of theirs */
static bool peer_accepted_htlc(struct peer *peer,
u64 id,
const struct secret *shared_secret,
enum onion_type *failcode)
{
struct htlc_end *hend;
u8 *req;
u8 onion[TOTAL_PACKET_SIZE];
struct onionpacket *op;
struct route_step *rs;
struct sha256 bad_onion_sha;
struct onionpacket *op;
const tal_t *tmpctx = tal_tmpctx(peer);
hend = tal(msg, struct htlc_end);
hend->shared_secret = tal(hend, struct secret);
if (!fromwire_channel_accepted_htlc(msg, NULL,
&hend->htlc_id, &hend->msatoshis,
&hend->cltv_expiry,
&hend->payment_hash,
hend->shared_secret,
onion)) {
log_broken(peer->log, "bad fromwire_channel_accepted_htlc %s",
tal_hex(peer, msg));
return -1;
hend = find_htlc_end(&peer->ld->htlc_ends, peer, id, HTLC_SRC);
if (!hend) {
log_broken(peer->log,
"peer_got_revoke unknown htlc %"PRIu64, id);
return false;
}
/* channeld tests this, so we shouldn't see it! */
op = parse_onionpacket(msg, onion, TOTAL_PACKET_SIZE);
if (!hend_update_state(peer, hend, RCVD_ADD_ACK_REVOCATION))
return false;
/* channeld tests this, so it should have set ss to zeroes. */
op = parse_onionpacket(tmpctx, hend->next_onion,
tal_len(hend->next_onion));
if (!op) {
log_broken(peer->log, "bad onion in fromwire_channel_accepted_htlc %s",
tal_hex(peer, msg));
return -1;
if (!memeqzero(shared_secret, sizeof(*shared_secret))) {
log_broken(peer->log,
"bad onion in got_revoke: %s",
tal_hex(peer, hend->next_onion));
tal_free(tmpctx);
return false;
}
/* FIXME: could be bad version, bad key. */
*failcode = WIRE_INVALID_ONION_VERSION;
goto out;
}
tal_steal(peer, hend);
hend->which_end = HTLC_SRC;
hend->peer = peer;
hend->other_end = NULL;
hend->pay_command = NULL;
hend->fail_msg = NULL;
/* Channeld sets this to zero if HSM won't ecdh it */
if (memeqzero(shared_secret, sizeof(*shared_secret))) {
*failcode = WIRE_INVALID_ONION_KEY;
goto out;
}
/* If it's crap, not their fault, just fail it */
rs = process_onionpacket(msg, op, hend->shared_secret->data,
hend->shared_secret = tal_dup(hend, struct secret, shared_secret);
/* If it's crap, not channeld's fault, just fail it */
rs = process_onionpacket(tmpctx, op, hend->shared_secret->data,
hend->payment_hash.u.u8,
sizeof(hend->payment_hash));
if (!rs) {
sha256(&bad_onion_sha, onion, sizeof(onion));
fail_htlc(hend, WIRE_INVALID_ONION_HMAC, &bad_onion_sha);
return 0;
*failcode = WIRE_INVALID_ONION_HMAC;
goto out;
}
/* Unknown realm isn't a bad onion, it's a normal failure. */
if (rs->hop_data.realm != 0) {
fail_htlc(hend, WIRE_INVALID_REALM, NULL);
return 0;
*failcode = WIRE_INVALID_REALM;
goto out;
}
hend->amt_to_forward = rs->hop_data.amt_forward;
@ -491,129 +557,417 @@ int peer_accepted_htlc(struct peer *peer, const u8 *msg)
if (rs->nextcase == ONION_FORWARD) {
hend->next_onion = serialize_onionpacket(hend, rs->next);
req = towire_gossip_resolve_channel_request(msg, &hend->next_channel);
log_broken(peer->log, "Asking gossip to resolve channel %d/%d/%d", hend->next_channel.blocknum, hend->next_channel.txnum, hend->next_channel.outnum);
subd_req(hend, peer->ld->gossip, req, -1, 0, channel_resolve_reply, hend);
req = towire_gossip_resolve_channel_request(tmpctx,
&hend->next_channel);
log_debug(peer->log, "Asking gossip to resolve channel %s",
type_to_string(tmpctx, struct short_channel_id,
&hend->next_channel));
subd_req(hend, peer->ld->gossip, req, -1, 0,
channel_resolve_reply, hend);
/* FIXME(cdecker) Stuff all this info into hend */
} else
handle_localpay(hend, hend->cltv_expiry, &hend->payment_hash,
hend->amt_to_forward, hend->outgoing_cltv_value);
return 0;
*failcode = 0;
out:
log_debug(peer->log, "their htlc %"PRIu64" %s",
id, *failcode ? onion_type_name(*failcode) : "locked");
tal_free(tmpctx);
return true;
}
int peer_fulfilled_htlc(struct peer *peer, const u8 *msg)
static bool peer_fulfilled_our_htlc(struct peer *peer,
const struct fulfilled_htlc *fulfilled)
{
u64 id;
struct preimage preimage;
struct htlc_end *hend;
if (!fromwire_channel_fulfilled_htlc(msg, NULL, &id, &preimage)) {
log_broken(peer->log, "bad fromwire_channel_fulfilled_htlc %s",
tal_hex(peer, msg));
return -1;
}
hend = find_htlc_end(&peer->ld->htlc_ends, peer, id, HTLC_DST);
hend = find_htlc_end(&peer->ld->htlc_ends, peer, fulfilled->id,
HTLC_DST);
if (!hend) {
log_broken(peer->log,
"channel_fulfilled_htlc unknown htlc %"PRIu64,
id);
return -1;
"fulfilled_our_htlc unknown htlc %"PRIu64,
fulfilled->id);
return false;
}
/* They fulfilled our HTLC. Credit them, forward as required. */
if (!hend_update_state(peer, hend, RCVD_REMOVE_COMMIT))
return false;
/* FIXME: Type mismatch. */
hend->preimage = tal(hend, struct sha256);
memcpy(hend->preimage, &fulfilled->payment_preimage,
sizeof(fulfilled->payment_preimage));
BUILD_ASSERT(sizeof(*hend->preimage)
== sizeof(fulfilled->payment_preimage));
/* FIXME: Save to db */
/* They fulfilled our HTLC. Credit them, forward immediately. */
peer->balance[REMOTE] += hend->msatoshis;
peer->balance[LOCAL] -= hend->msatoshis;
if (hend->other_end)
fulfill_htlc(hend->other_end, &preimage);
fulfill_htlc(hend->other_end, &fulfilled->payment_preimage);
else
payment_succeeded(peer->ld, hend, &preimage);
tal_free(hend);
return 0;
payment_succeeded(peer->ld, hend, &fulfilled->payment_preimage);
return true;
}
int peer_failed_htlc(struct peer *peer, const u8 *msg)
static bool peer_failed_our_htlc(struct peer *peer,
const struct failed_htlc *failed)
{
u64 id;
u8 *reason;
struct htlc_end *hend;
enum onion_type failcode;
struct onionreply *reply;
if (!fromwire_channel_failed_htlc(msg, msg, NULL, &id, &reason)) {
log_broken(peer->log, "bad fromwire_channel_failed_htlc %s",
tal_hex(peer, msg));
return -1;
}
hend = find_htlc_end(&peer->ld->htlc_ends, peer, id, HTLC_DST);
hend = find_htlc_end(&peer->ld->htlc_ends, peer, failed->id, HTLC_DST);
if (!hend) {
log_broken(peer->log,
"channel_failed_htlc unknown htlc %"PRIu64,
id);
return -1;
"failed_our_htlc unknown htlc %"PRIu64,
failed->id);
return false;
}
if (!hend_update_state(peer, hend, RCVD_REMOVE_COMMIT))
return false;
log_debug(peer->log, "Our HTLC %"PRIu64" failed", failed->id);
hend->fail_msg = tal_dup_arr(hend, u8, failed->failreason,
tal_len(failed->failreason), 0);
return true;
}
static void remove_hend(struct peer *peer, struct htlc_end *hend)
{
log_debug(peer->log, "Removing %s hend %"PRIu64" state %s",
hend->which_end == HTLC_DST ? "outgoing" : "incoming",
hend->htlc_id,
htlc_state_name(hend->hstate));
/* If it's failed, now we can forward since it's completely locked-in */
if (hend->fail_msg && hend->which_end == HTLC_DST) {
if (hend->other_end) {
hend->other_end->fail_msg = tal_steal(hend->other_end, reason);
hend->other_end->fail_msg
= tal_dup_arr(hend->other_end, u8,
hend->fail_msg,
tal_len(hend->fail_msg), 0);
relay_htlc_failmsg(hend->other_end);
} else {
/* FIXME: Avoid copy here! */
enum onion_type failcode;
struct onionreply *reply;
size_t numhops = tal_count(hend->path_secrets);
struct secret *shared_secrets = tal_arr(hend, struct secret, numhops);
for (size_t i=0; i<numhops; i++) {
shared_secrets[i] = hend->path_secrets[i];
}
reply = unwrap_onionreply(msg, shared_secrets, numhops, reason);
reply = unwrap_onionreply(hend, shared_secrets, numhops,
hend->fail_msg);
if (!reply) {
log_info(peer->log, "htlc %"PRIu64" failed with bad reply (%s)",
id, tal_hex(msg, msg));
log_info(peer->log, "htlc %"PRIu64
" failed with bad reply (%s)",
hend->htlc_id,
tal_hex(hend, hend->fail_msg));
failcode = WIRE_PERMANENT_NODE_FAILURE;
} else {
failcode = fromwire_peektype(reply->msg);
log_info(peer->log, "htlc %"PRIu64" failed with code 0x%04x (%s)",
id, failcode, onion_type_name(failcode));
log_info(peer->log, "htlc %"PRIu64
" failed from %ith node with code 0x%04x (%s)",
hend->htlc_id,
reply->origin_index, failcode,
onion_type_name(failcode));
}
/* FIXME: Apply update if it contains it, etc */
payment_failed(peer->ld, hend, NULL, failcode);
}
}
return 0;
tal_free(hend);
}
int peer_failed_malformed_htlc(struct peer *peer, const u8 *msg)
static bool changed_htlc(struct peer *peer,
const struct changed_htlc *changed_htlc)
{
u64 id;
struct htlc_end *hend;
struct sha256 sha256_of_onion;
u16 failcode;
enum htlc_end_type end;
if (!fromwire_channel_malformed_htlc(msg, NULL, &id,
&sha256_of_onion, &failcode)) {
log_broken(peer->log, "bad fromwire_channel_malformed_htlc %s",
if (htlc_state_owner(changed_htlc->newstate) == LOCAL)
end = HTLC_DST;
else
end = HTLC_SRC;
hend = find_htlc_end(&peer->ld->htlc_ends, peer, changed_htlc->id, end);
if (!hend) {
log_broken(peer->log,
"Can't find %s HTLC %"PRIu64,
side_to_str(htlc_state_owner(changed_htlc->newstate)),
changed_htlc->id);
return false;
}
if (!hend_update_state(peer, hend, changed_htlc->newstate))
return false;
/* First transition into commitment; now it outlives peer. */
if (changed_htlc->newstate == SENT_ADD_COMMIT) {
tal_del_destructor(hend, hend_subd_died);
tal_steal(peer->ld, hend);
/* From now onwards, penalty tx might need this */
save_htlc_stub(peer->ld, peer, end, hend->outgoing_cltv_value,
&hend->payment_hash);
} else if (changed_htlc->newstate == RCVD_REMOVE_ACK_REVOCATION
|| changed_htlc->newstate == SENT_REMOVE_ACK_REVOCATION) {
remove_hend(peer, hend);
}
return true;
}
int peer_sending_commitsig(struct peer *peer, const u8 *msg)
{
u64 commitnum;
struct changed_htlc *changed_htlcs;
size_t i;
if (!fromwire_channel_sending_commitsig(msg, msg, NULL,
&commitnum,
&changed_htlcs)) {
log_broken(peer->log, "bad channel_sending_commitsig %s",
tal_hex(peer, msg));
return -1;
}
hend = find_htlc_end(&peer->ld->htlc_ends, peer, id, HTLC_DST);
if (!hend) {
for (i = 0; i < tal_count(changed_htlcs); i++) {
if (!changed_htlc(peer, changed_htlcs + i)) {
log_broken(peer->log,
"channel_malformed_htlc unknown htlc %"PRIu64,
id);
"channel_sending_commitsig: update failed");
return -1;
}
if (hend->other_end) {
/* Not really a local failure, but since the failing
* peer could not derive its shared secret it cannot
* create a valid HMAC, so we do it on his behalf */
fail_htlc(hend->other_end, failcode, &sha256_of_onion);
} else {
payment_failed(peer->ld, hend, NULL, failcode);
}
/* Tell it we've got it, and to go ahead with commitment_signed. */
subd_send_msg(peer->owner,
take(towire_channel_sending_commitsig_reply(msg)));
return 0;
}
static void added_their_htlc(struct peer *peer, const struct added_htlc *added)
{
struct htlc_end *hend;
/* This stays around even if we fail it immediately: it *is*
* part of the current commitment. */
hend = tal(peer, struct htlc_end);
hend->which_end = HTLC_SRC;
hend->hstate = RCVD_ADD_COMMIT;
hend->peer = peer;
hend->other_end = NULL;
hend->pay_command = NULL;
hend->fail_msg = NULL;
hend->htlc_id = added->id;
hend->msatoshis = added->amount_msat;
hend->payment_hash = added->payment_hash;
hend->cltv_expiry = added->cltv_expiry;
hend->next_onion = tal_dup_arr(hend, u8, added->onion_routing_packet,
sizeof(added->onion_routing_packet),
0);
/* FIXME: Save to db */
log_debug(peer->log, "Adding their HTLC %"PRIu64, added->id);
connect_htlc_end(&peer->ld->htlc_ends, hend);
/* Technically this can't be needed for a penalty transaction until
* after we send revoke_and_ack, then commit, then receive their
* revoke_and_ack. But might as well record it while we have it:
* a few extra entries won't hurt */
save_htlc_stub(peer->ld, peer, HTLC_SRC, hend->cltv_expiry,
&hend->payment_hash);
}
static bool update_by_id(struct peer *peer, u64 id, enum htlc_end_type end,
enum htlc_state newstate)
{
struct htlc_end *hend;
hend = find_htlc_end(&peer->ld->htlc_ends, peer, id, end);
if (!hend) {
log_broken(peer->log, "Could not find id %"PRIu64
" to update to %s", id, htlc_state_name(newstate));
return false;
}
return hend_update_state(peer, hend, newstate);
}
/* The peer doesn't tell us this separately, but logically it's a separate
* step to receiving commitsig */
static bool peer_sending_revocation(struct peer *peer,
struct added_htlc *added,
struct fulfilled_htlc *fulfilled,
struct failed_htlc *failed,
struct changed_htlc *changed)
{
size_t i;
for (i = 0; i < tal_count(added); i++) {
if (!update_by_id(peer, added[i].id, HTLC_SRC,
SENT_ADD_REVOCATION))
return false;
}
for (i = 0; i < tal_count(fulfilled); i++) {
if (!update_by_id(peer, fulfilled[i].id, HTLC_DST,
SENT_REMOVE_REVOCATION))
return false;
}
for (i = 0; i < tal_count(failed); i++) {
if (!update_by_id(peer, failed[i].id, HTLC_DST,
SENT_REMOVE_REVOCATION))
return false;
}
for (i = 0; i < tal_count(changed); i++) {
if (changed[i].newstate == RCVD_ADD_ACK_COMMIT) {
if (!update_by_id(peer, changed[i].id, HTLC_DST,
SENT_ADD_ACK_REVOCATION))
return false;
} else {
if (!update_by_id(peer, changed[i].id, HTLC_SRC,
SENT_REMOVE_ACK_REVOCATION))
return false;
}
}
return true;
}
/* This also implies we're sending revocation */
int peer_got_commitsig(struct peer *peer, const u8 *msg)
{
u64 commitnum;
secp256k1_ecdsa_signature commit_sig;
secp256k1_ecdsa_signature *htlc_sigs;
struct added_htlc *added;
struct fulfilled_htlc *fulfilled;
struct failed_htlc *failed;
struct changed_htlc *changed;
size_t i;
if (!fromwire_channel_got_commitsig(msg, msg, NULL,
&commitnum,
&commit_sig,
&htlc_sigs,
&added,
&fulfilled,
&failed,
&changed)) {
log_broken(peer->log, "bad fromwire_channel_got_commitsig %s",
tal_hex(peer, msg));
return -1;
}
log_debug(peer->log,
"got commitsig %"PRIu64
": %zu added, %zu fulfilled, %zu failed, %zu changed",
commitnum, tal_count(added), tal_count(fulfilled),
tal_count(failed), tal_count(changed));
/* FIXME: store commit & htlc signature information. */
/* New HTLCs */
for (i = 0; i < tal_count(added); i++)
added_their_htlc(peer, &added[i]);
/* Save information now for fulfilled & failed HTLCs */
for (i = 0; i < tal_count(fulfilled); i++) {
if (!peer_fulfilled_our_htlc(peer, &fulfilled[i]))
return -1;
}
for (i = 0; i < tal_count(failed); i++) {
if (!peer_failed_our_htlc(peer, &failed[i]))
return -1;
}
for (i = 0; i < tal_count(changed); i++) {
if (!changed_htlc(peer, &changed[i])) {
log_broken(peer->log,
"got_commitsig: update failed");
return -1;
}
}
/* Since we're about to send revoke, bump state again. */
if (!peer_sending_revocation(peer, added, fulfilled, failed, changed))
return -1;
/* Tell it we've committed, and to go ahead with revoke. */
msg = towire_channel_got_commitsig_reply(msg);
subd_send_msg(peer->owner, take(msg));
return 0;
}
int peer_got_revoke(struct peer *peer, const u8 *msg)
{
u64 revokenum;
struct sha256 per_commitment_secret;
u64 *added_ids;
struct secret *shared_secret;
struct changed_htlc *changed;
enum onion_type *failcodes;
size_t i;
if (!fromwire_channel_got_revoke(msg, msg, NULL,
&revokenum, &per_commitment_secret,
&added_ids, &shared_secret,
&changed)) {
log_broken(peer->log, "bad fromwire_channel_got_revoke %s",
tal_hex(peer, msg));
return -1;
}
log_debug(peer->log,
"got revoke %"PRIu64": %zu changed, %zu incoming locked in",
revokenum,
tal_count(changed), tal_count(added_ids));
/* Save any immediate failures for after we reply. */
failcodes = tal_arr(msg, enum onion_type, tal_count(added_ids));
for (i = 0; i < tal_count(added_ids); i++) {
if (!peer_accepted_htlc(peer, added_ids[i], &shared_secret[i],
&failcodes[i]))
return -1;
}
for (i = 0; i < tal_count(changed); i++) {
if (!changed_htlc(peer, &changed[i])) {
log_broken(peer->log, "got_revoke: update failed");
return -1;
}
}
/* FIXME: Save per-commit-secret! */
/* Tell it we've committed, and to go ahead with revoke. */
msg = towire_channel_got_revoke_reply(msg);
subd_send_msg(peer->owner, take(msg));
/* Now, any HTLCs we need to immediately fail? */
for (i = 0; i < tal_count(added_ids); i++) {
struct sha256 bad_onion_sha;
struct htlc_end *hend;
if (!failcodes[i])
continue;
hend = find_htlc_end(&peer->ld->htlc_ends, peer, added_ids[i],
HTLC_SRC);
sha256(&bad_onion_sha, hend->next_onion,
tal_len(hend->next_onion));
fail_htlc(hend, failcodes[i], &bad_onion_sha);
}
return 0;
}

View File

@ -4,9 +4,8 @@
#include "config.h"
#include <ccan/short_types/short_types.h>
int peer_accepted_htlc(struct peer *peer, const u8 *msg);
int peer_fulfilled_htlc(struct peer *peer, const u8 *msg);
int peer_failed_htlc(struct peer *peer, const u8 *msg);
int peer_failed_malformed_htlc(struct peer *peer, const u8 *msg);
int peer_sending_commitsig(struct peer *peer, const u8 *msg);
int peer_got_commitsig(struct peer *peer, const u8 *msg);
int peer_got_revoke(struct peer *peer, const u8 *msg);
#endif /* LIGHTNING_LIGHTNINGD_PEER_HTLCS_H */

View File

@ -219,7 +219,7 @@ class LightningDTests(BaseLightningDTests):
l1.daemon.wait_for_log('Sending commit_sig with 0 htlc sigs')
l2.daemon.wait_for_log('their htlc 0 locked')
l2.daemon.wait_for_log('failed htlc 0 code 0x400f')
l1.daemon.wait_for_log('htlc 0 failed with code 0x400f')
l1.daemon.wait_for_log('htlc 0 failed from 0th node with code 0x400f')
# Set up invoice (non-dust, just to test), and pay it.
# This one isn't dust.