Refactor channel states:

- persisted states are saved
 - state transitions are checked
 - transient states are stored in channel.peer_state
 - new channel states: 'PREOPENING', 'FUNDED' and 'REDEEMED'
 - upgrade storage to version 21
This commit is contained in:
ThomasV 2019-10-29 08:02:14 +01:00
parent c31fa059fe
commit 61dfcba092
11 changed files with 151 additions and 79 deletions

View file

@ -136,10 +136,11 @@ class ChannelDetailsPopup(Popup):
def details(self): def details(self):
chan = self.chan chan = self.chan
status = self.app.wallet.lnworker.get_channel_status(chan)
return { return {
_('Short Chan ID'): format_short_channel_id(chan.short_channel_id), _('Short Chan ID'): format_short_channel_id(chan.short_channel_id),
_('Initiator'): 'Local' if chan.constraints.is_initiator else 'Remote', _('Initiator'): 'Local' if chan.constraints.is_initiator else 'Remote',
_('State'): chan.get_state(), _('State'): status,
_('Local CTN'): chan.get_latest_ctn(LOCAL), _('Local CTN'): chan.get_latest_ctn(LOCAL),
_('Remote CTN'): chan.get_latest_ctn(REMOTE), _('Remote CTN'): chan.get_latest_ctn(REMOTE),
_('Capacity'): self.app.format_amount_and_units(chan.constraints.capacity), _('Capacity'): self.app.format_amount_and_units(chan.constraints.capacity),
@ -181,7 +182,7 @@ class ChannelDetailsPopup(Popup):
def _force_close(self, b): def _force_close(self, b):
if not b: if not b:
return return
if self.chan.get_state() == 'CLOSED': if self.chan.is_closed():
self.app.show_error(_('Channel already closed')) self.app.show_error(_('Channel already closed'))
return return
loop = self.app.wallet.network.asyncio_loop loop = self.app.wallet.network.asyncio_loop
@ -223,7 +224,7 @@ class LightningChannelsDialog(Factory.Popup):
def update_item(self, item): def update_item(self, item):
chan = item._chan chan = item._chan
item.status = chan.get_state() item.status = self.app.wallet.lnworker.get_channel_status(chan)
item.short_channel_id = format_short_channel_id(chan.short_channel_id) item.short_channel_id = format_short_channel_id(chan.short_channel_id)
l, r = self.format_fields(chan) l, r = self.format_fields(chan)
item.local_balance = _('Local') + ':' + l item.local_balance = _('Local') + ':' + l

View file

@ -60,12 +60,13 @@ class ChannelsList(MyTreeView):
if bal_other != bal_minus_htlcs_other: if bal_other != bal_minus_htlcs_other:
label += ' (+' + self.parent.format_amount(bal_other - bal_minus_htlcs_other) + ')' label += ' (+' + self.parent.format_amount(bal_other - bal_minus_htlcs_other) + ')'
labels[subject] = label labels[subject] = label
status = self.lnworker.get_channel_status(chan)
return [ return [
format_short_channel_id(chan.short_channel_id), format_short_channel_id(chan.short_channel_id),
bh2u(chan.node_id), bh2u(chan.node_id),
labels[LOCAL], labels[LOCAL],
labels[REMOTE], labels[REMOTE],
chan.get_state() status
] ]
def on_success(self, txid): def on_success(self, txid):

View file

@ -435,7 +435,7 @@ class ElectrumWindow(QMainWindow, MessageBoxMixin, Logger):
wallet.thread = TaskThread(self, self.on_error) wallet.thread = TaskThread(self, self.on_error)
self.update_recently_visited(wallet.storage.path) self.update_recently_visited(wallet.storage.path)
if wallet.lnworker: if wallet.lnworker:
wallet.lnworker.on_channels_updated() wallet.network.trigger_callback('channels_updated', wallet)
self.need_update.set() self.need_update.set()
# Once GUI has been initialized check if we want to announce something since the callback has been called before the GUI was initialized # Once GUI has been initialized check if we want to announce something since the callback has been called before the GUI was initialized
# update menus # update menus

View file

@ -40,7 +40,7 @@ from .logging import Logger
OLD_SEED_VERSION = 4 # electrum versions < 2.0 OLD_SEED_VERSION = 4 # electrum versions < 2.0
NEW_SEED_VERSION = 11 # electrum versions >= 2.0 NEW_SEED_VERSION = 11 # electrum versions >= 2.0
FINAL_SEED_VERSION = 20 # electrum >= 2.7 will set this to prevent FINAL_SEED_VERSION = 21 # electrum >= 2.7 will set this to prevent
# old versions from overwriting new format # old versions from overwriting new format
@ -214,6 +214,7 @@ class JsonDB(Logger):
self._convert_version_18() self._convert_version_18()
self._convert_version_19() self._convert_version_19()
self._convert_version_20() self._convert_version_20()
self._convert_version_21()
self.put('seed_version', FINAL_SEED_VERSION) # just to be sure self.put('seed_version', FINAL_SEED_VERSION) # just to be sure
self._after_upgrade_tasks() self._after_upgrade_tasks()
@ -485,6 +486,16 @@ class JsonDB(Logger):
self.put('seed_version', 20) self.put('seed_version', 20)
def _convert_version_21(self):
if not self._is_upgrade_method_needed(20, 20):
return
channels = self.get('channels')
if channels:
for channel in channels:
channel['state'] = 'OPENING'
self.put('channels', channels)
self.put('seed_version', 21)
def _convert_imported(self): def _convert_imported(self):
if not self._is_upgrade_method_needed(0, 13): if not self._is_upgrade_method_needed(0, 13):
return return

View file

@ -26,7 +26,7 @@ import os
from collections import namedtuple, defaultdict from collections import namedtuple, defaultdict
import binascii import binascii
import json import json
from enum import Enum, auto from enum import IntEnum
from typing import Optional, Dict, List, Tuple, NamedTuple, Set, Callable, Iterable, Sequence, TYPE_CHECKING from typing import Optional, Dict, List, Tuple, NamedTuple, Set, Callable, Iterable, Sequence, TYPE_CHECKING
import time import time
@ -55,6 +55,42 @@ if TYPE_CHECKING:
from .lnworker import LNWallet from .lnworker import LNWallet
# lightning channel states
class channel_states(IntEnum):
PREOPENING = 0 # negociating
OPENING = 1 # awaiting funding tx
FUNDED = 2 # funded (requires min_depth and tx verification)
OPEN = 3 # both parties have sent funding_locked
FORCE_CLOSING = 4 # force-close tx has been broadcast
CLOSING = 5 # closing negociation
CLOSED = 6 # funding txo has been spent
REDEEMED = 7 # we can stop watching
class peer_states(IntEnum):
DISCONNECTED = 0
REESTABLISHING = 1
GOOD = 2
cs = channel_states
state_transitions = [
(cs.PREOPENING, cs.OPENING),
(cs.OPENING, cs.FUNDED),
(cs.FUNDED, cs.OPEN),
(cs.OPENING, cs.CLOSING),
(cs.FUNDED, cs.CLOSING),
(cs.OPEN, cs.CLOSING),
(cs.OPENING, cs.FORCE_CLOSING),
(cs.FUNDED, cs.FORCE_CLOSING),
(cs.OPEN, cs.FORCE_CLOSING),
(cs.CLOSING, cs.FORCE_CLOSING),
(cs.OPENING, cs.CLOSED),
(cs.FUNDED, cs.CLOSED),
(cs.OPEN, cs.CLOSED),
(cs.CLOSING, cs.CLOSED),
(cs.FORCE_CLOSING, cs.CLOSED),
(cs.CLOSED, cs.REDEEMED),
]
class ChannelJsonEncoder(json.JSONEncoder): class ChannelJsonEncoder(json.JSONEncoder):
def default(self, o): def default(self, o):
if isinstance(o, bytes): if isinstance(o, bytes):
@ -136,18 +172,13 @@ class Channel(Logger):
self.short_channel_id = ShortChannelID.normalize(state["short_channel_id"]) self.short_channel_id = ShortChannelID.normalize(state["short_channel_id"])
self.short_channel_id_predicted = self.short_channel_id self.short_channel_id_predicted = self.short_channel_id
self.onion_keys = str_bytes_dict_from_save(state.get('onion_keys', {})) self.onion_keys = str_bytes_dict_from_save(state.get('onion_keys', {}))
self.force_closed = state.get('force_closed')
self.data_loss_protect_remote_pcp = str_bytes_dict_from_save(state.get('data_loss_protect_remote_pcp', {})) self.data_loss_protect_remote_pcp = str_bytes_dict_from_save(state.get('data_loss_protect_remote_pcp', {}))
self.remote_update = bfh(state.get('remote_update')) if state.get('remote_update') else None self.remote_update = bfh(state.get('remote_update')) if state.get('remote_update') else None
log = state.get('log') log = state.get('log')
self.hm = HTLCManager(log=log, self.hm = HTLCManager(log=log, initial_feerate=initial_feerate)
initial_feerate=initial_feerate) self._state = channel_states[state['state']]
self.peer_state = peer_states.DISCONNECTED
self._is_funding_txo_spent = None # "don't know"
self._state = None
self.set_state('DISCONNECTED')
self.sweep_info = {} # type: Dict[str, Dict[str, SweepInfo]] self.sweep_info = {} # type: Dict[str, Dict[str, SweepInfo]]
self._outgoing_channel_update = None # type: Optional[bytes] self._outgoing_channel_update = None # type: Optional[bytes]
@ -184,26 +215,31 @@ class Channel(Logger):
next_per_commitment_point=None) next_per_commitment_point=None)
self.config[LOCAL] = self.config[LOCAL]._replace(current_commitment_signature=remote_sig) self.config[LOCAL] = self.config[LOCAL]._replace(current_commitment_signature=remote_sig)
self.hm.channel_open_finished() self.hm.channel_open_finished()
self.set_state('OPENING') self.peer_state = peer_states.GOOD
self.set_state(channel_states.OPENING)
def set_force_closed(self): def set_state(self, state):
self.force_closed = True """ set on-chain state """
if (self._state, state) not in state_transitions:
def set_state(self, state: str): raise Exception(f"Transition not allowed: {self._state.name} -> {state.name}")
self._state = state self._state = state
self.logger.debug(f'Setting channel state: {state.name}')
if self.lnworker:
self.lnworker.save_channel(self)
self.lnworker.network.trigger_callback('channel', self)
def get_state(self): def get_state(self):
return self._state return self._state
def is_closed(self): def is_closed(self):
return self.force_closed or self.get_state() in ['CLOSED', 'CLOSING'] return self.get_state() > channel_states.OPEN
def _check_can_pay(self, amount_msat: int) -> None: def _check_can_pay(self, amount_msat: int) -> None:
# TODO check if this method uses correct ctns (should use "latest" + 1) # TODO check if this method uses correct ctns (should use "latest" + 1)
if self.is_closed(): if self.is_closed():
raise PaymentFailure('Channel closed') raise PaymentFailure('Channel closed')
if self.get_state() != 'OPEN': if self.get_state() != channel_states.OPEN:
raise PaymentFailure('Channel not open') raise PaymentFailure('Channel not open', self.get_state())
if self.available_to_spend(LOCAL) < amount_msat: if self.available_to_spend(LOCAL) < amount_msat:
raise PaymentFailure(f'Not enough local balance. Have: {self.available_to_spend(LOCAL)}, Need: {amount_msat}') raise PaymentFailure(f'Not enough local balance. Have: {self.available_to_spend(LOCAL)}, Need: {amount_msat}')
if len(self.hm.htlcs(LOCAL)) + 1 > self.config[REMOTE].max_accepted_htlcs: if len(self.hm.htlcs(LOCAL)) + 1 > self.config[REMOTE].max_accepted_htlcs:
@ -222,12 +258,8 @@ class Channel(Logger):
return False return False
return True return True
def set_funding_txo_spentness(self, is_spent: bool):
assert isinstance(is_spent, bool)
self._is_funding_txo_spent = is_spent
def should_try_to_reestablish_peer(self) -> bool: def should_try_to_reestablish_peer(self) -> bool:
return self._is_funding_txo_spent is False and self._state == 'DISCONNECTED' return self._state < channel_states.CLOSED and self.peer_state == peer_states.DISCONNECTED
def get_funding_address(self): def get_funding_address(self):
script = funding_output_script(self.config[LOCAL], self.config[REMOTE]) script = funding_output_script(self.config[LOCAL], self.config[REMOTE])
@ -624,7 +656,7 @@ class Channel(Logger):
"node_id": self.node_id, "node_id": self.node_id,
"log": self.hm.to_save(), "log": self.hm.to_save(),
"onion_keys": str_bytes_dict_to_save(self.onion_keys), "onion_keys": str_bytes_dict_to_save(self.onion_keys),
"force_closed": self.force_closed, "state": self._state.name,
"data_loss_protect_remote_pcp": str_bytes_dict_to_save(self.data_loss_protect_remote_pcp), "data_loss_protect_remote_pcp": str_bytes_dict_to_save(self.data_loss_protect_remote_pcp),
"remote_update": self.remote_update.hex() if self.remote_update else None "remote_update": self.remote_update.hex() if self.remote_update else None
} }

View file

@ -29,7 +29,7 @@ from .logging import Logger
from .lnonion import (new_onion_packet, decode_onion_error, OnionFailureCode, calc_hops_data_for_payment, from .lnonion import (new_onion_packet, decode_onion_error, OnionFailureCode, calc_hops_data_for_payment,
process_onion_packet, OnionPacket, construct_onion_error, OnionRoutingFailureMessage, process_onion_packet, OnionPacket, construct_onion_error, OnionRoutingFailureMessage,
ProcessedOnionPacket) ProcessedOnionPacket)
from .lnchannel import Channel, RevokeAndAck, htlcsum, RemoteCtnTooFarInFuture from .lnchannel import Channel, RevokeAndAck, htlcsum, RemoteCtnTooFarInFuture, channel_states, peer_states
from . import lnutil from . import lnutil
from .lnutil import (Outpoint, LocalConfig, RECEIVED, UpdateAddHtlc, from .lnutil import (Outpoint, LocalConfig, RECEIVED, UpdateAddHtlc,
RemoteConfig, OnlyPubkeyKeypair, ChannelConstraints, RevocationStore, RemoteConfig, OnlyPubkeyKeypair, ChannelConstraints, RevocationStore,
@ -579,6 +579,7 @@ class Peer(Logger):
"local_config": local_config, "local_config": local_config,
"constraints": ChannelConstraints(capacity=funding_sat, is_initiator=True, funding_txn_minimum_depth=funding_txn_minimum_depth), "constraints": ChannelConstraints(capacity=funding_sat, is_initiator=True, funding_txn_minimum_depth=funding_txn_minimum_depth),
"remote_update": None, "remote_update": None,
"state": channel_states.PREOPENING.name,
} }
chan = Channel(chan_dict, chan = Channel(chan_dict,
sweep_address=self.lnworker.sweep_address, sweep_address=self.lnworker.sweep_address,
@ -594,10 +595,10 @@ class Peer(Logger):
self.logger.info('received funding_signed') self.logger.info('received funding_signed')
remote_sig = payload['signature'] remote_sig = payload['signature']
chan.receive_new_commitment(remote_sig, []) chan.receive_new_commitment(remote_sig, [])
chan.open_with_first_pcp(remote_per_commitment_point, remote_sig)
# broadcast funding tx # broadcast funding tx
# TODO make more robust (timeout low? server returns error?) # TODO make more robust (timeout low? server returns error?)
await asyncio.wait_for(self.network.broadcast_transaction(funding_tx), LN_P2P_NETWORK_TIMEOUT) await asyncio.wait_for(self.network.broadcast_transaction(funding_tx), LN_P2P_NETWORK_TIMEOUT)
chan.open_with_first_pcp(remote_per_commitment_point, remote_sig)
return chan return chan
async def on_open_channel(self, payload): async def on_open_channel(self, payload):
@ -664,6 +665,7 @@ class Peer(Logger):
"local_config": local_config, "local_config": local_config,
"constraints": ChannelConstraints(capacity=funding_sat, is_initiator=False, funding_txn_minimum_depth=min_depth), "constraints": ChannelConstraints(capacity=funding_sat, is_initiator=False, funding_txn_minimum_depth=min_depth),
"remote_update": None, "remote_update": None,
"state": channel_states.PREOPENING.name,
} }
chan = Channel(chan_dict, chan = Channel(chan_dict,
sweep_address=self.lnworker.sweep_address, sweep_address=self.lnworker.sweep_address,
@ -709,11 +711,11 @@ class Peer(Logger):
async def reestablish_channel(self, chan: Channel): async def reestablish_channel(self, chan: Channel):
await self.initialized.wait() await self.initialized.wait()
chan_id = chan.channel_id chan_id = chan.channel_id
if chan.get_state() != 'DISCONNECTED': if chan.peer_state != peer_states.DISCONNECTED:
self.logger.info('reestablish_channel was called but channel {} already in state {}' self.logger.info('reestablish_channel was called but channel {} already in state {}'
.format(chan_id, chan.get_state())) .format(chan_id, chan.get_state()))
return return
chan.set_state('REESTABLISHING') chan.peer_state = peer_states.REESTABLISHING
self.network.trigger_callback('channel', chan) self.network.trigger_callback('channel', chan)
# BOLT-02: "A node [...] upon disconnection [...] MUST reverse any uncommitted updates sent by the other side" # BOLT-02: "A node [...] upon disconnection [...] MUST reverse any uncommitted updates sent by the other side"
chan.hm.discard_unsigned_remote_updates() chan.hm.discard_unsigned_remote_updates()
@ -856,11 +858,11 @@ class Peer(Logger):
await self.lnworker.force_close_channel(chan_id) await self.lnworker.force_close_channel(chan_id)
return return
chan.peer_state = peer_states.GOOD
# note: chan.short_channel_id being set implies the funding txn is already at sufficient depth # note: chan.short_channel_id being set implies the funding txn is already at sufficient depth
if their_next_local_ctn == next_local_ctn == 1 and chan.short_channel_id: if their_next_local_ctn == next_local_ctn == 1 and chan.short_channel_id:
self.send_funding_locked(chan) self.send_funding_locked(chan)
# checks done # checks done
chan.set_state('OPENING')
if chan.config[LOCAL].funding_locked_received and chan.short_channel_id: if chan.config[LOCAL].funding_locked_received and chan.short_channel_id:
self.mark_open(chan) self.mark_open(chan)
self.network.trigger_callback('channel', chan) self.network.trigger_callback('channel', chan)
@ -949,11 +951,12 @@ class Peer(Logger):
def mark_open(self, chan: Channel): def mark_open(self, chan: Channel):
assert chan.short_channel_id is not None assert chan.short_channel_id is not None
scid = chan.short_channel_id scid = chan.short_channel_id
# only allow state transition to "OPEN" from "OPENING" # only allow state transition from "FUNDED" to "OPEN"
if chan.get_state() != "OPENING": if chan.get_state() != channel_states.FUNDED:
self.logger.info(f"cannot mark open, {chan.get_state()}")
return return
assert chan.config[LOCAL].funding_locked_received assert chan.config[LOCAL].funding_locked_received
chan.set_state("OPEN") chan.set_state(channel_states.OPEN)
self.network.trigger_callback('channel', chan) self.network.trigger_callback('channel', chan)
self.add_own_channel(chan) self.add_own_channel(chan)
self.logger.info(f"CHANNEL OPENING COMPLETED for {scid}") self.logger.info(f"CHANNEL OPENING COMPLETED for {scid}")
@ -1114,7 +1117,8 @@ class Peer(Logger):
async def pay(self, route: List['RouteEdge'], chan: Channel, amount_msat: int, async def pay(self, route: List['RouteEdge'], chan: Channel, amount_msat: int,
payment_hash: bytes, min_final_cltv_expiry: int) -> UpdateAddHtlc: payment_hash: bytes, min_final_cltv_expiry: int) -> UpdateAddHtlc:
assert chan.get_state() == "OPEN", chan.get_state() if chan.get_state() != channel_states.OPEN:
raise PaymentFailure('Channel not open')
assert amount_msat > 0, "amount_msat is not greater zero" assert amount_msat > 0, "amount_msat is not greater zero"
# create onion packet # create onion packet
final_cltv = self.network.get_local_height() + min_final_cltv_expiry final_cltv = self.network.get_local_height() + min_final_cltv_expiry
@ -1200,7 +1204,7 @@ class Peer(Logger):
amount_msat_htlc = int.from_bytes(payload["amount_msat"], 'big') amount_msat_htlc = int.from_bytes(payload["amount_msat"], 'big')
onion_packet = OnionPacket.from_bytes(payload["onion_routing_packet"]) onion_packet = OnionPacket.from_bytes(payload["onion_routing_packet"])
processed_onion = process_onion_packet(onion_packet, associated_data=payment_hash, our_onion_private_key=self.privkey) processed_onion = process_onion_packet(onion_packet, associated_data=payment_hash, our_onion_private_key=self.privkey)
if chan.get_state() != "OPEN": if chan.get_state() != channel_states.OPEN:
raise RemoteMisbehaving(f"received update_add_htlc while chan.get_state() != OPEN. state was {chan.get_state()}") raise RemoteMisbehaving(f"received update_add_htlc while chan.get_state() != OPEN. state was {chan.get_state()}")
if cltv_expiry >= 500_000_000: if cltv_expiry >= 500_000_000:
asyncio.ensure_future(self.lnworker.force_close_channel(channel_id)) asyncio.ensure_future(self.lnworker.force_close_channel(channel_id))
@ -1255,7 +1259,7 @@ class Peer(Logger):
return return
outgoing_chan_upd = self.get_outgoing_gossip_channel_update_for_chan(next_chan)[2:] outgoing_chan_upd = self.get_outgoing_gossip_channel_update_for_chan(next_chan)[2:]
outgoing_chan_upd_len = len(outgoing_chan_upd).to_bytes(2, byteorder="big") outgoing_chan_upd_len = len(outgoing_chan_upd).to_bytes(2, byteorder="big")
if next_chan.get_state() != 'OPEN': if next_chan.get_state() != channel_states.OPEN:
self.logger.info(f"cannot forward htlc. next_chan not OPEN: {next_chan_scid} in state {next_chan.get_state()}") self.logger.info(f"cannot forward htlc. next_chan not OPEN: {next_chan_scid} in state {next_chan.get_state()}")
reason = OnionRoutingFailureMessage(code=OnionFailureCode.TEMPORARY_CHANNEL_FAILURE, reason = OnionRoutingFailureMessage(code=OnionFailureCode.TEMPORARY_CHANNEL_FAILURE,
data=outgoing_chan_upd_len+outgoing_chan_upd) data=outgoing_chan_upd_len+outgoing_chan_upd)
@ -1453,7 +1457,7 @@ class Peer(Logger):
@log_exceptions @log_exceptions
async def _shutdown(self, chan: Channel, payload, is_local): async def _shutdown(self, chan: Channel, payload, is_local):
# set state so that we stop accepting HTLCs # set state so that we stop accepting HTLCs
chan.set_state('CLOSING') chan.set_state(channel_states.CLOSING)
# wait until no HTLCs remain in either commitment transaction # wait until no HTLCs remain in either commitment transaction
while len(chan.hm.htlcs(LOCAL)) + len(chan.hm.htlcs(REMOTE)) > 0: while len(chan.hm.htlcs(LOCAL)) + len(chan.hm.htlcs(REMOTE)) > 0:
self.logger.info('waiting for htlcs to settle...') self.logger.info('waiting for htlcs to settle...')

View file

@ -190,7 +190,7 @@ class LNWatcher(AddressSynchronizer):
return return
self.network.trigger_callback('update_closed_channel', funding_outpoint, spenders, self.network.trigger_callback('update_closed_channel', funding_outpoint, spenders,
funding_txid, funding_height, closing_txid, funding_txid, funding_height, closing_txid,
closing_height, closing_tx) # FIXME sooo many args.. closing_height, closing_tx, keep_watching) # FIXME sooo many args..
# TODO: add tests for local_watchtower # TODO: add tests for local_watchtower
await self.do_breach_remedy(funding_outpoint, spenders) await self.do_breach_remedy(funding_outpoint, spenders)
if not keep_watching: if not keep_watching:

View file

@ -40,6 +40,7 @@ from .lnaddr import lnencode, LnAddr, lndecode
from .ecc import der_sig_from_sig_string from .ecc import der_sig_from_sig_string
from .ecc_fast import is_using_fast_ecc from .ecc_fast import is_using_fast_ecc
from .lnchannel import Channel, ChannelJsonEncoder from .lnchannel import Channel, ChannelJsonEncoder
from .lnchannel import channel_states, peer_states
from . import lnutil from . import lnutil
from .lnutil import funding_output_script from .lnutil import funding_output_script
from .bitcoin import redeem_script_to_address from .bitcoin import redeem_script_to_address
@ -420,10 +421,21 @@ class LNWallet(LNWorker):
def peer_closed(self, peer): def peer_closed(self, peer):
for chan in self.channels_for_peer(peer.pubkey).values(): for chan in self.channels_for_peer(peer.pubkey).values():
chan.set_state('DISCONNECTED') chan.peer_state = peer_states.DISCONNECTED
self.network.trigger_callback('channel', chan) self.network.trigger_callback('channel', chan)
self.peers.pop(peer.pubkey) self.peers.pop(peer.pubkey)
def get_channel_status(self, chan):
# status displayed in the GUI
cs = chan.get_state()
if chan.is_closed():
return cs.name
peer = self.peers.get(chan.node_id)
ps = chan.peer_state
if ps != peer_states.GOOD:
return ps.name
return cs.name
def payment_completed(self, chan: Channel, direction: Direction, def payment_completed(self, chan: Channel, direction: Direction,
htlc: UpdateAddHtlc): htlc: UpdateAddHtlc):
chan_id = chan.channel_id chan_id = chan.channel_id
@ -629,7 +641,6 @@ class LNWallet(LNWorker):
block_height, tx_pos, chan.funding_outpoint.output_index) block_height, tx_pos, chan.funding_outpoint.output_index)
self.logger.info(f"save_short_channel_id: {chan.short_channel_id}") self.logger.info(f"save_short_channel_id: {chan.short_channel_id}")
self.save_channel(chan) self.save_channel(chan)
self.on_channels_updated()
def channel_by_txo(self, txo): def channel_by_txo(self, txo):
with self.lock: with self.lock:
@ -644,23 +655,25 @@ class LNWallet(LNWorker):
chan = self.channel_by_txo(funding_outpoint) chan = self.channel_by_txo(funding_outpoint)
if not chan: if not chan:
return return
#self.logger.debug(f'on_channel_open {funding_outpoint}') if chan.get_state() == channel_states.OPEN and self.should_channel_be_closed_due_to_expiring_htlcs(chan):
self.channel_timestamps[bh2u(chan.channel_id)] = funding_txid, funding_height.height, funding_height.timestamp, None, None, None
self.storage.put('lightning_channel_timestamps', self.channel_timestamps)
chan.set_funding_txo_spentness(False)
# send event to GUI
self.network.trigger_callback('channel', chan)
if self.should_channel_be_closed_due_to_expiring_htlcs(chan):
self.logger.info(f"force-closing due to expiring htlcs") self.logger.info(f"force-closing due to expiring htlcs")
await self.force_close_channel(chan.channel_id) await self.force_close_channel(chan.channel_id)
return return
if chan.get_state() == channel_states.OPENING:
if chan.short_channel_id is None: if chan.short_channel_id is None:
self.save_short_chan_id(chan) self.save_short_chan_id(chan)
if chan.get_state() == "OPENING" and chan.short_channel_id: if chan.short_channel_id:
peer = self.peers[chan.node_id] chan.set_state(channel_states.FUNDED)
self.channel_timestamps[bh2u(chan.channel_id)] = chan.funding_outpoint.txid, funding_height.height, funding_height.timestamp, None, None, None
self.storage.put('lightning_channel_timestamps', self.channel_timestamps)
if chan.get_state() == channel_states.FUNDED:
peer = self.peers.get(chan.node_id)
if peer:
peer.send_funding_locked(chan) peer.send_funding_locked(chan)
elif chan.get_state() == "OPEN":
elif chan.get_state() == channel_states.OPEN:
peer = self.peers.get(chan.node_id) peer = self.peers.get(chan.node_id)
if peer is None: if peer is None:
self.logger.info("peer not found for {}".format(bh2u(chan.node_id))) self.logger.info("peer not found for {}".format(bh2u(chan.node_id)))
@ -669,7 +682,8 @@ class LNWallet(LNWorker):
await peer.bitcoin_fee_update(chan) await peer.bitcoin_fee_update(chan)
conf = self.lnwatcher.get_tx_height(chan.funding_outpoint.txid).conf conf = self.lnwatcher.get_tx_height(chan.funding_outpoint.txid).conf
peer.on_network_update(chan, conf) peer.on_network_update(chan, conf)
elif chan.force_closed and chan.get_state() != 'CLOSED':
elif chan.get_state() == channel_states.FORCE_CLOSING:
txid = chan.force_close_tx().txid() txid = chan.force_close_tx().txid()
height = self.lnwatcher.get_tx_height(txid).height height = self.lnwatcher.get_tx_height(txid).height
self.logger.info(f"force closing tx {txid}, height {height}") self.logger.info(f"force closing tx {txid}, height {height}")
@ -677,21 +691,27 @@ class LNWallet(LNWorker):
self.logger.info('REBROADCASTING CLOSING TX') self.logger.info('REBROADCASTING CLOSING TX')
await self.force_close_channel(chan.channel_id) await self.force_close_channel(chan.channel_id)
@ignore_exceptions
@log_exceptions @log_exceptions
async def on_update_closed_channel(self, event, funding_outpoint, spenders, funding_txid, funding_height, closing_txid, closing_height, closing_tx): async def on_update_closed_channel(self, event, funding_outpoint, spenders, funding_txid, funding_height, closing_txid, closing_height, closing_tx, keep_watching):
chan = self.channel_by_txo(funding_outpoint) chan = self.channel_by_txo(funding_outpoint)
if not chan: if not chan:
return return
#self.logger.debug(f'on_channel_closed {funding_outpoint}')
# fixme: this is wasteful
self.channel_timestamps[bh2u(chan.channel_id)] = funding_txid, funding_height.height, funding_height.timestamp, closing_txid, closing_height.height, closing_height.timestamp self.channel_timestamps[bh2u(chan.channel_id)] = funding_txid, funding_height.height, funding_height.timestamp, closing_txid, closing_height.height, closing_height.timestamp
self.storage.put('lightning_channel_timestamps', self.channel_timestamps) self.storage.put('lightning_channel_timestamps', self.channel_timestamps)
chan.set_funding_txo_spentness(True)
chan.set_state('CLOSED')
self.on_channels_updated()
self.network.trigger_callback('channel', chan)
# remove from channel_db # remove from channel_db
if chan.short_channel_id is not None: if chan.short_channel_id is not None:
self.channel_db.remove_channel(chan.short_channel_id) self.channel_db.remove_channel(chan.short_channel_id)
if chan.get_state() < channel_states.CLOSED:
chan.set_state(channel_states.CLOSED)
if chan.get_state() == channel_states.CLOSED and not keep_watching:
chan.set_state(channel_states.REDEEMED)
# detect who closed and set sweep_info # detect who closed and set sweep_info
sweep_info_dict = chan.sweep_ctx(closing_tx) sweep_info_dict = chan.sweep_ctx(closing_tx)
self.logger.info(f'sweep_info_dict length: {len(sweep_info_dict)}') self.logger.info(f'sweep_info_dict length: {len(sweep_info_dict)}')
@ -800,11 +820,8 @@ class LNWallet(LNWorker):
temp_channel_id=os.urandom(32)) temp_channel_id=os.urandom(32))
self.save_channel(chan) self.save_channel(chan)
self.lnwatcher.add_channel(chan.funding_outpoint.to_str(), chan.get_funding_address()) self.lnwatcher.add_channel(chan.funding_outpoint.to_str(), chan.get_funding_address())
self.on_channels_updated()
return chan
def on_channels_updated(self):
self.network.trigger_callback('channels_updated', self.wallet) self.network.trigger_callback('channels_updated', self.wallet)
return chan
@log_exceptions @log_exceptions
async def add_peer(self, connect_str: str) -> Peer: async def add_peer(self, connect_str: str) -> Peer:
@ -1149,7 +1166,7 @@ class LNWallet(LNWorker):
# note: currently we add *all* our channels; but this might be a privacy leak? # note: currently we add *all* our channels; but this might be a privacy leak?
for chan in channels: for chan in channels:
# check channel is open # check channel is open
if chan.get_state() != "OPEN": if chan.get_state() != channel_states.OPEN:
continue continue
# check channel has sufficient balance # check channel has sufficient balance
# FIXME because of on-chain fees of ctx, this check is insufficient # FIXME because of on-chain fees of ctx, this check is insufficient
@ -1206,7 +1223,7 @@ class LNWallet(LNWorker):
'channel_id': format_short_channel_id(chan.short_channel_id) if chan.short_channel_id else None, 'channel_id': format_short_channel_id(chan.short_channel_id) if chan.short_channel_id else None,
'full_channel_id': bh2u(chan.channel_id), 'full_channel_id': bh2u(chan.channel_id),
'channel_point': chan.funding_outpoint.to_str(), 'channel_point': chan.funding_outpoint.to_str(),
'state': chan.get_state(), 'state': chan.get_state().name,
'remote_pubkey': bh2u(chan.node_id), 'remote_pubkey': bh2u(chan.node_id),
'local_balance': chan.balance(LOCAL)//1000, 'local_balance': chan.balance(LOCAL)//1000,
'remote_balance': chan.balance(REMOTE)//1000, 'remote_balance': chan.balance(REMOTE)//1000,
@ -1220,9 +1237,7 @@ class LNWallet(LNWorker):
async def force_close_channel(self, chan_id): async def force_close_channel(self, chan_id):
chan = self.channels[chan_id] chan = self.channels[chan_id]
tx = chan.force_close_tx() tx = chan.force_close_tx()
chan.set_force_closed() chan.set_state(channel_states.FORCE_CLOSING)
self.save_channel(chan)
self.on_channels_updated()
try: try:
await self.network.broadcast_transaction(tx) await self.network.broadcast_transaction(tx)
except Exception as e: except Exception as e:
@ -1276,6 +1291,7 @@ class LNWallet(LNWorker):
if ratio < 0.5: if ratio < 0.5:
self.logger.warning(f"fee level for channel {bh2u(chan.channel_id)} is {chan_feerate} sat/kiloweight, " self.logger.warning(f"fee level for channel {bh2u(chan.channel_id)} is {chan_feerate} sat/kiloweight, "
f"current recommended feerate is {self.current_feerate_per_kw()} sat/kiloweight, consider force closing!") f"current recommended feerate is {self.current_feerate_per_kw()} sat/kiloweight, consider force closing!")
# reestablish
if not chan.should_try_to_reestablish_peer(): if not chan.should_try_to_reestablish_peer():
continue continue
peer = self.peers.get(chan.node_id, None) peer = self.peers.get(chan.node_id, None)

View file

@ -90,6 +90,10 @@ if [[ $1 == "init" ]]; then
new_blocks 1 new_blocks 1
fi fi
if [[ $1 == "new_block" ]]; then
new_blocks 1
fi
# start daemons. Bob is started first because he is listening # start daemons. Bob is started first because he is listening
if [[ $1 == "start" ]]; then if [[ $1 == "start" ]]; then
$bob daemon -d $bob daemon -d

View file

@ -34,6 +34,7 @@ from electrum.lnutil import SENT, LOCAL, REMOTE, RECEIVED
from electrum.lnutil import FeeUpdate from electrum.lnutil import FeeUpdate
from electrum.ecc import sig_string_from_der_sig from electrum.ecc import sig_string_from_der_sig
from electrum.logging import console_stderr_handler from electrum.logging import console_stderr_handler
from electrum.lnchannel import channel_states
from . import ElectrumTestCase from . import ElectrumTestCase
@ -94,6 +95,7 @@ def create_channel_state(funding_txid, funding_index, funding_sat, is_initiator,
), ),
"node_id":other_node_id, "node_id":other_node_id,
'onion_keys': {}, 'onion_keys': {},
'state': 'PREOPENING',
} }
def bip32(sequence): def bip32(sequence):
@ -136,8 +138,8 @@ def create_test_channels(feerate=6000, local=None, remote=None):
alice.hm.log[LOCAL]['ctn'] = 0 alice.hm.log[LOCAL]['ctn'] = 0
bob.hm.log[LOCAL]['ctn'] = 0 bob.hm.log[LOCAL]['ctn'] = 0
alice.set_state('OPEN') alice._state = channel_states.OPEN
bob.set_state('OPEN') bob._state = channel_states.OPEN
a_out = alice.get_latest_commitment(LOCAL).outputs() a_out = alice.get_latest_commitment(LOCAL).outputs()
b_out = bob.get_next_commitment(REMOTE).outputs() b_out = bob.get_next_commitment(REMOTE).outputs()

View file

@ -16,6 +16,7 @@ from electrum.lnpeer import Peer
from electrum.lnutil import LNPeerAddr, Keypair, privkey_to_pubkey from electrum.lnutil import LNPeerAddr, Keypair, privkey_to_pubkey
from electrum.lnutil import LightningPeerConnectionClosed, RemoteMisbehaving from electrum.lnutil import LightningPeerConnectionClosed, RemoteMisbehaving
from electrum.lnutil import PaymentFailure, LnLocalFeatures from electrum.lnutil import PaymentFailure, LnLocalFeatures
from electrum.lnchannel import channel_states
from electrum.lnrouter import LNPathFinder from electrum.lnrouter import LNPathFinder
from electrum.channel_db import ChannelDB from electrum.channel_db import ChannelDB
from electrum.lnworker import LNWallet, NoPathFound from electrum.lnworker import LNWallet, NoPathFound
@ -202,9 +203,9 @@ class TestPeer(ElectrumTestCase):
w1.peer = p1 w1.peer = p1
w2.peer = p2 w2.peer = p2
# mark_open won't work if state is already OPEN. # mark_open won't work if state is already OPEN.
# so set it to OPENING # so set it to FUNDED
self.alice_channel.set_state("OPENING") self.alice_channel._state = channel_states.FUNDED
self.bob_channel.set_state("OPENING") self.bob_channel._state = channel_states.FUNDED
# this populates the channel graph: # this populates the channel graph:
p1.mark_open(self.alice_channel) p1.mark_open(self.alice_channel)
p2.mark_open(self.bob_channel) p2.mark_open(self.bob_channel)