#include <bitcoin/chainparams.h>
#include <ccan/array_size/array_size.h>
/*~ Welcome to the gossip daemon: keeper of maps!
 *
 * This is the last "global" daemon; it has three purposes.
 *
 * 1. To determine routes for payments when lightningd asks.
 * 2. The second purpose is to receive gossip from peers (via their
 *    per-peer daemons) and send it out to them.
 * 3. Talk to `connectd` to to answer address queries for nodes.
 *
 * The gossip protocol itself is fairly simple, but has some twists which
 * add complexity to this daemon.
 */
#include <ccan/asort/asort.h>
#include <ccan/bitmap/bitmap.h>
#include <ccan/build_assert/build_assert.h>
#include <ccan/cast/cast.h>
#include <ccan/container_of/container_of.h>
#include <ccan/crypto/hkdf_sha256/hkdf_sha256.h>
#include <ccan/crypto/siphash24/siphash24.h>
#include <ccan/endian/endian.h>
#include <ccan/fdpass/fdpass.h>
#include <ccan/io/fdpass/fdpass.h>
#include <ccan/io/io.h>
#include <ccan/list/list.h>
#include <ccan/mem/mem.h>
#include <ccan/noerr/noerr.h>
#include <ccan/take/take.h>
#include <ccan/tal/str/str.h>
#include <ccan/timer/timer.h>
#include <common/bech32.h>
#include <common/bech32_util.h>
#include <common/cryptomsg.h>
#include <common/daemon_conn.h>
#include <common/decode_short_channel_ids.h>
#include <common/features.h>
#include <common/memleak.h>
#include <common/ping.h>
#include <common/pseudorand.h>
#include <common/status.h>
#include <common/subdaemon.h>
#include <common/timeout.h>
#include <common/type_to_string.h>
#include <common/utils.h>
#include <common/version.h>
#include <common/wire_error.h>
#include <common/wireaddr.h>
#include <connectd/gen_connect_gossip_wire.h>
#include <errno.h>
#include <fcntl.h>
#include <gossipd/broadcast.h>
#include <gossipd/gen_gossip_peerd_wire.h>
#include <gossipd/gen_gossip_wire.h>
#include <gossipd/routing.h>
#include <hsmd/gen_hsm_wire.h>
#include <inttypes.h>
#include <lightningd/gossip_msg.h>
#include <netdb.h>
#include <netinet/in.h>
#include <secp256k1_ecdh.h>
#include <sodium/randombytes.h>
#include <sys/socket.h>
#include <sys/stat.h>
#include <sys/types.h>
#include <sys/un.h>
#include <unistd.h>
#include <wire/gen_peer_wire.h>
#include <wire/wire_io.h>
#include <wire/wire_sync.h>
#include <zlib.h>

/* We talk to `hsmd` to sign our gossip messages with the node key */
#define HSM_FD 3
/* connectd asks us for help finding nodes, and gossip fds for new peers */
#define CONNECTD_FD 4

/* In developer mode we provide hooks for whitebox testing */
#if DEVELOPER
static u32 max_scids_encode_bytes = -1U;
static bool suppress_gossip = false;
#endif

/*~ The core daemon structure: */
struct daemon {
	/* Who am I?  Helps us find ourself in the routing map. */
	struct node_id id;

	/* Peers we are gossiping to: id is unique */
	struct list_head peers;

	/* Connection to lightningd. */
	struct daemon_conn *master;

	/* Connection to connect daemon. */
	struct daemon_conn *connectd;

	/* Routing information */
	struct routing_state *rstate;

	/* chainhash for checking/making gossip msgs */
	struct bitcoin_blkid chain_hash;

	/* Timers: we batch gossip, and also refresh announcements */
	struct timers timers;

	/* Global features to list in node_announcement. */
	u8 *globalfeatures;

	/* Alias (not NUL terminated) and favorite color for node_announcement */
	u8 alias[32];
	u8 rgb[3];

	/* What addresses we can actually announce. */
	struct wireaddr *announcable;
};

/* This represents each peer we're gossiping with */
struct peer {
	/* daemon->peers */
	struct list_node list;

	/* parent pointer. */
	struct daemon *daemon;

	/* The ID of the peer (always unique) */
	struct node_id id;

	/* The two features gossip cares about (so far) */
	bool gossip_queries_feature, initial_routing_sync_feature;

	/* Are there outstanding queries on short_channel_ids? */
	const struct short_channel_id *scid_queries;
	size_t scid_query_idx;

	/* Are there outstanding node_announcements from scid_queries? */
	struct node_id *scid_query_nodes;
	size_t scid_query_nodes_idx;

	/* How many query responses are we expecting? */
	size_t num_scid_queries_outstanding;

	/* How many pongs are we expecting? */
	size_t num_pings_outstanding;

	/* Map of outstanding channel_range requests. */
	bitmap *query_channel_blocks;
	/* What we're querying: [range_first_blocknum, range_end_blocknum) */
	u32 range_first_blocknum, range_end_blocknum;
	u32 range_blocks_remaining;
	struct short_channel_id *query_channel_scids;

	/* The daemon_conn used to queue messages to/from the peer. */
	struct daemon_conn *dc;
};

/*~ A channel consists of a `struct half_chan` for each direction, each of
 * which has a `flags` word from the `channel_update`; bit 1 is
 * ROUTING_FLAGS_DISABLED in the `channel_update`.  But we also keep a local
 * whole-channel flag which indicates it's not available; we use this when a
 * peer disconnects, and generate a `channel_update` to tell the world lazily
 * when someone asks. */
static void peer_disable_channels(struct daemon *daemon, struct node *node)
{
	/* If this peer had a channel with us, mark it disabled. */
	struct chan_map_iter i;
	struct chan *c;

	for (c = first_chan(node, &i); c; c = next_chan(node, &i)) {
		if (node_id_eq(&other_node(node, c)->id, &daemon->id))
			local_disable_chan(daemon->rstate, c);
	}
}

/*~ Destroy a peer, usually because the per-peer daemon has exited.
 *
 * Were you wondering why we call this "destroy_peer" and not "peer_destroy"?
 * I thought not!  But while CCAN modules are required to keep to their own
 * prefix namespace, leading to unnatural word order, we couldn't stomach that
 * for our own internal use.  We use 'find_foo', 'destroy_foo' and 'new_foo'.
 */
static void destroy_peer(struct peer *peer)
{
	struct node *node;

	/* Remove it from the peers list */
	list_del_from(&peer->daemon->peers, &peer->list);

	/* If we have a channel with this peer, disable it. */
	node = get_node(peer->daemon->rstate, &peer->id);
	if (node)
		peer_disable_channels(peer->daemon, node);

	/* This is tricky: our lifetime is tied to the daemon_conn; it's our
	 * parent, so we are freed if it is, but we need to free it if we're
	 * freed manually.  tal_free() treats this as a noop if it's already
	 * being freed */
	tal_free(peer->dc);
}

/* Search for a peer. */
static struct peer *find_peer(struct daemon *daemon, const struct node_id *id)
{
	struct peer *peer;

	list_for_each(&daemon->peers, peer, list)
		if (node_id_eq(&peer->id, id))
			return peer;
	return NULL;
}

/* Queue a gossip message for the peer: the subdaemon on the other end simply
 * forwards it to the peer. */
static void queue_peer_msg(struct peer *peer, const u8 *msg TAKES)
{
	daemon_conn_send(peer->dc, msg);
}

/*~ We have a helper for messages from the store. */
static void queue_peer_from_store(struct peer *peer,
				  const struct broadcastable *bcast)
{
	struct gossip_store *gs = peer->daemon->rstate->gs;
	queue_peer_msg(peer, take(gossip_store_get(NULL, gs, bcast->index)));
}

/* BOLT #7:
 *
 * There are several messages which contain a long array of
 * `short_channel_id`s (called `encoded_short_ids`) so we utilize a
 * simple compression scheme: the first byte indicates the encoding, the
 * rest contains the data.
 */
static u8 *encode_short_channel_ids_start(const tal_t *ctx)
{
	u8 *encoded = tal_arr(ctx, u8, 0);
	towire_u8(&encoded, SHORTIDS_ZLIB);
	return encoded;
}

/* Marshal a single short_channel_id */
static void encode_add_short_channel_id(u8 **encoded,
					const struct short_channel_id *scid)
{
	towire_short_channel_id(encoded, scid);
}

/* Greg Maxwell asked me privately about using zlib for communicating a set,
 * and suggested that we'd be better off using Golomb-Rice coding a-la BIP
 * 158.  However, naively using Rice encoding isn't a win: we have to get
 * more complex and use separate streams.  The upside is that it's between
 * 2 and 5 times smaller (assuming optimal Rice encoding + gzip).  We can add
 * that later. */
static u8 *zencode_scids(const tal_t *ctx, const u8 *scids, size_t len)
{
	u8 *z;
	int err;
	unsigned long compressed_len = len;

	/* Prefer to fail if zlib makes it larger */
	z = tal_arr(ctx, u8, len);
	err = compress2(z, &compressed_len, scids, len, Z_BEST_COMPRESSION);
	if (err == Z_OK) {
		status_trace("short_ids compressed %zu into %lu",
			     len, compressed_len);
		tal_resize(&z, compressed_len);
		return z;
	}
	status_trace("short_ids compress %zu returned %i:"
		     " not compresssing", len, err);
	return NULL;
}

/* Once we've assembled */
static bool encode_short_channel_ids_end(u8 **encoded, size_t max_bytes)
{
	u8 *z;

	/* First byte says what encoding we want. */
	switch ((enum scid_encode_types)(*encoded)[0]) {
	case SHORTIDS_ZLIB:
		/* compress */
		z = zencode_scids(tmpctx, *encoded + 1, tal_count(*encoded) - 1);
		if (z) {
			/* If successful, copy over and trimp */
			tal_resize(encoded, 1 + tal_count(z));
			memcpy((*encoded) + 1, z, tal_count(z));
			goto check_length;
		}
		/* Otherwise, change first byte to 'uncompressed' */
		(*encoded)[0] = SHORTIDS_UNCOMPRESSED;
		/* Fall thru */
	case SHORTIDS_UNCOMPRESSED:
		goto check_length;
	}

	status_failed(STATUS_FAIL_INTERNAL_ERROR,
		      "Unknown short_ids encoding %u", (*encoded)[0]);

check_length:
#if DEVELOPER
	if (tal_count(*encoded) > max_scids_encode_bytes)
		return false;
#endif
	return tal_count(*encoded) <= max_bytes;
}

/* BOLT #7:
 *
 * A node:
 *   - if the `gossip_queries` feature is negotiated:
 * 	- MUST NOT relay any gossip messages unless explicitly requested.
 */
static void setup_gossip_range(struct peer *peer)
{
	u8 *msg;

	/*~ Without the `gossip_queries` feature, gossip flows automatically. */
	if (!peer->gossip_queries_feature)
		return;

	/*~ We need to ask for something to start the gossip flowing: we ask
	 * for everything from 1970 to 2106; this is horribly naive.  We
	 * should be much smarter about requesting only what we don't already
	 * have. */
	msg = towire_gossip_timestamp_filter(peer,
					     &peer->daemon->chain_hash,
					     0, UINT32_MAX);
	queue_peer_msg(peer, take(msg));
}

/* Create a node_announcement with the given signature. It may be NULL in the
 * case we need to create a provisional announcement for the HSM to sign.
 * This is called twice: once with the dummy signature to get it signed and a
 * second time to build the full packet with the signature. The timestamp is
 * handed in rather than using time_now() internally, since that could change
 * between the dummy creation and the call with a signature. */
static u8 *create_node_announcement(const tal_t *ctx, struct daemon *daemon,
				    secp256k1_ecdsa_signature *sig,
				    u32 timestamp)
{
	u8 *addresses = tal_arr(tmpctx, u8, 0);
	u8 *announcement;
	size_t i;
	if (!sig) {
		sig = tal(tmpctx, secp256k1_ecdsa_signature);
		memset(sig, 0, sizeof(*sig));
	}
	for (i = 0; i < tal_count(daemon->announcable); i++)
		towire_wireaddr(&addresses, &daemon->announcable[i]);

	announcement =
	    towire_node_announcement(ctx, sig, daemon->globalfeatures, timestamp,
				     &daemon->id, daemon->rgb, daemon->alias,
				     addresses);
	return announcement;
}

/*~ This routine created a `node_announcement` for our node, and hands it to
 * the routing.c code like any other `node_announcement`.  Such announcements
 * are only accepted if there is an announced channel associated with that node
 * (to prevent spam), so we only call this once we've announced a channel. */
static void send_node_announcement(struct daemon *daemon)
{
	u32 timestamp = time_now().ts.tv_sec;
	secp256k1_ecdsa_signature sig;
	u8 *msg, *nannounce, *err;
	struct node *self = get_node(daemon->rstate, &daemon->id);

	/* BOLT #7:
	 *
	 * The origin node:
	 *   - MUST set `timestamp` to be greater than that of any previous
	 *   `node_announcement` it has previously created.
	 */
	if (self && self->bcast.index && timestamp <= self->bcast.timestamp)
		timestamp = self->bcast.timestamp + 1;

	/* Get an unsigned one. */
	nannounce = create_node_announcement(tmpctx, daemon, NULL, timestamp);

	/* Ask hsmd to sign it (synchronous) */
	if (!wire_sync_write(HSM_FD, take(towire_hsm_node_announcement_sig_req(NULL, nannounce))))
		status_failed(STATUS_FAIL_MASTER_IO, "Could not write to HSM: %s", strerror(errno));

	msg = wire_sync_read(tmpctx, HSM_FD);
	if (!fromwire_hsm_node_announcement_sig_reply(msg, &sig))
		status_failed(STATUS_FAIL_MASTER_IO, "HSM returned an invalid node_announcement sig");

	/* We got the signature for out provisional node_announcement back
	 * from the HSM, create the real announcement and forward it to
	 * gossipd so it can take care of forwarding it. */
	nannounce = create_node_announcement(NULL, daemon, &sig, timestamp);

	/* This injects it into the routing code in routing.c; it should not
	 * reject it! */
	err = handle_node_announcement(daemon->rstate, take(nannounce));
	if (err)
		status_failed(STATUS_FAIL_INTERNAL_ERROR,
			      "rejected own node announcement: %s",
			      tal_hex(tmpctx, err));
}

/*~ We don't actually keep node_announcements in memory; we keep them in
 * a file called `gossip_store`.  If we need some node details, we reload
 * and reparse.  It's slow, but generally rare. */
static bool get_node_announcement(const tal_t *ctx,
				  struct daemon *daemon,
				  const struct node *n,
				  u8 rgb_color[3],
				  u8 alias[32],
				  u8 **features,
				  struct wireaddr **wireaddrs)
{
	const u8 *msg;
	struct node_id id;
	secp256k1_ecdsa_signature signature;
	u32 timestamp;
	u8 *addresses;

	if (!n->bcast.index)
		return false;

	msg = gossip_store_get(tmpctx, daemon->rstate->gs, n->bcast.index);

	/* Note: validity of node_id is already checked. */
	if (!fromwire_node_announcement(ctx, msg,
					&signature, features,
					&timestamp,
					&id, rgb_color, alias,
					&addresses)) {
		status_broken("Bad local node_announcement @%u: %s",
			      n->bcast.index, tal_hex(tmpctx, msg));
		return false;
	}
	assert(node_id_eq(&id, &n->id));
	assert(timestamp == n->bcast.timestamp);

	*wireaddrs = read_addresses(ctx, addresses);
	tal_free(addresses);
	return true;
}

/* Version which also does nodeid lookup */
static bool get_node_announcement_by_id(const tal_t *ctx,
					struct daemon *daemon,
					const struct node_id *node_id,
					u8 rgb_color[3],
					u8 alias[32],
					u8 **features,
					struct wireaddr **wireaddrs)
{
	struct node *n = get_node(daemon->rstate, node_id);
	if (!n)
		return false;

	return get_node_announcement(ctx, daemon, n, rgb_color, alias,
				     features, wireaddrs);
}


/* Return true if the only change would be the timestamp. */
static bool node_announcement_redundant(struct daemon *daemon)
{
	u8 rgb_color[3];
	u8 alias[32];
	u8 *features;
	struct wireaddr *wireaddrs;

	if (!get_node_announcement_by_id(tmpctx, daemon, &daemon->id,
					 rgb_color, alias, &features,
					 &wireaddrs))
		return false;

	if (tal_count(wireaddrs) != tal_count(daemon->announcable))
		return false;

	for (size_t i = 0; i < tal_count(wireaddrs); i++)
		if (!wireaddr_eq(&wireaddrs[i], &daemon->announcable[i]))
			return false;

	BUILD_ASSERT(ARRAY_SIZE(daemon->alias) == ARRAY_SIZE(alias));
	if (!memeq(daemon->alias, ARRAY_SIZE(daemon->alias),
		   alias, ARRAY_SIZE(alias)))
		return false;

	BUILD_ASSERT(ARRAY_SIZE(daemon->rgb) == ARRAY_SIZE(rgb_color));
	if (!memeq(daemon->rgb, ARRAY_SIZE(daemon->rgb),
		   rgb_color, ARRAY_SIZE(rgb_color)))
		return false;

	if (!memeq(daemon->globalfeatures, tal_count(daemon->globalfeatures),
		   features, tal_count(features)))
		return false;

	return true;
}

/* Should we announce our own node?  Called at strategic places. */
static void maybe_send_own_node_announce(struct daemon *daemon)
{
	/* We keep an internal flag in the routing code to say we've announced
	 * a local channel.  The alternative would be to have it make a
	 * callback, but when we start up we don't want to make multiple
	 * announcments, so we use this approach for now. */
	if (!daemon->rstate->local_channel_announced)
		return;

	if (node_announcement_redundant(daemon))
		return;

	send_node_announcement(daemon);
	daemon->rstate->local_channel_announced = false;
}

/*~Routines to handle gossip messages from peer, forwarded by subdaemons.
 *-----------------------------------------------------------------------
 *
 * It's not the subdaemon's fault if they're malformed or invalid; so these
 * all return an error packet which gets sent back to the subdaemon in that
 * case.
 */

/* The routing code checks that it's basically valid, returning an
 * error message for the peer or NULL.  NULL means it's OK, but the
 * message might be redundant, in which case scid is also NULL.
 * Otherwise `scid` gives us the short_channel_id claimed by the
 * message, and puts the announcemnt on an internal 'pending'
 * queue.  We'll send a request to lightningd to look it up, and continue
 * processing in `handle_txout_reply`. */
static const u8 *handle_channel_announcement_msg(struct peer *peer,
						 const u8 *msg)
{
	const struct short_channel_id *scid;
	const u8 *err;

	/* If it's OK, tells us the short_channel_id to lookup */
	err = handle_channel_announcement(peer->daemon->rstate, msg, &scid);
	if (err)
		return err;
	else if (scid)
		daemon_conn_send(peer->daemon->master,
				 take(towire_gossip_get_txout(NULL, scid)));
	return NULL;
}

static u8 *handle_channel_update_msg(struct peer *peer, const u8 *msg)
{
	/* Hand the channel_update to the routing code */
	u8 *err = handle_channel_update(peer->daemon->rstate, msg, "subdaemon");
	if (err)
		return err;

	/*~ As a nasty compromise in the spec, we only forward channel_announce
	 * once we have a channel_update; the channel isn't *usable* for
	 * routing until you have both anyway.  For this reason, we might have
	 * just sent out our own channel_announce, so we check if it's time to
	 * send a node_announcement too. */
	maybe_send_own_node_announce(peer->daemon);
	return NULL;
}

/*~ The peer can ask about an array of short channel ids: we don't assemble the
 * reply immediately but process them one at a time in dump_gossip which is
 * called when there's nothing more important to send. */
static const u8 *handle_query_short_channel_ids(struct peer *peer, const u8 *msg)
{
	struct bitcoin_blkid chain;
	u8 *encoded;
	struct short_channel_id *scids;

	if (!fromwire_query_short_channel_ids(tmpctx, msg, &chain, &encoded)) {
		return towire_errorfmt(peer, NULL,
				       "Bad query_short_channel_ids %s",
				       tal_hex(tmpctx, msg));
	}

	if (!bitcoin_blkid_eq(&peer->daemon->chain_hash, &chain)) {
		status_trace("%s sent query_short_channel_ids chainhash %s",
			     type_to_string(tmpctx, struct node_id, &peer->id),
			     type_to_string(tmpctx, struct bitcoin_blkid, &chain));
		return NULL;
	}

	/* BOLT #7:
	 *
	 * - if it has not sent `reply_short_channel_ids_end` to a
	 *   previously received `query_short_channel_ids` from this
         *   sender:
	 *    - MAY fail the connection.
	 */
	if (peer->scid_queries || peer->scid_query_nodes) {
		return towire_errorfmt(peer, NULL,
				       "Bad concurrent query_short_channel_ids");
	}

	scids = decode_short_ids(tmpctx, encoded);
	if (!scids) {
		return towire_errorfmt(peer, NULL,
				       "Bad query_short_channel_ids encoding %s",
				       tal_hex(tmpctx, encoded));
	}

	/* BOLT #7:
	 *
	 * - MUST respond to each known `short_channel_id` with a `channel_announcement`
	 *   and the latest `channel_update` for each end
	 *    - SHOULD NOT wait for the next outgoing gossip flush to send
	 *      these.
	 */
	peer->scid_queries = tal_steal(peer, scids);
	peer->scid_query_idx = 0;
	peer->scid_query_nodes = tal_arr(peer, struct node_id, 0);

	/* Notify the daemon_conn-write loop to invoke create_next_scid_reply */
	daemon_conn_wake(peer->dc);
	return NULL;
}

/*~ When we compact the gossip store, all the broadcast indexs move.
 * We simply offset everyone, which means in theory they could retransmit
 * some, but that's a lesser evil than skipping some. */
void update_peers_broadcast_index(struct list_head *peers, u32 offset)
{
	struct peer *peer, *next;

	list_for_each_safe(peers, peer, next, list) {
		int gs_fd;
		/*~ Since store has been compacted, they need a new fd for the
		 * new store.  We also tell them how much this is shrunk, so
		 * they can (approximately) tell where to start in the new store.
		 */
		gs_fd = gossip_store_readonly_fd(peer->daemon->rstate->gs);
		if (gs_fd < 0) {
			status_broken("Can't get read-only gossip store fd:"
				      " killing peer");
			tal_free(peer);
		} else {
			u8 *msg = towire_gossipd_new_store_fd(NULL, offset);
			daemon_conn_send(peer->dc, take(msg));
			daemon_conn_send_fd(peer->dc, gs_fd);
		}
	}
}

/*~ We can send multiple replies when the peer queries for all channels in
 * a given range of blocks; each one indicates the range of blocks it covers. */
static void reply_channel_range(struct peer *peer,
				u32 first_blocknum, u32 number_of_blocks,
				const u8 *encoded)
{
	/* BOLT #7:
	 *
	 * - For each `reply_channel_range`:
	 *   - MUST set with `chain_hash` equal to that of `query_channel_range`,
	 *   - MUST encode a `short_channel_id` for every open channel it
	 *     knows in blocks `first_blocknum` to `first_blocknum` plus
	 *     `number_of_blocks` minus one.
	 *   - MUST limit `number_of_blocks` to the maximum number of blocks
         *     whose results could fit in `encoded_short_ids`
	 *   - if does not maintain up-to-date channel information for
	 *     `chain_hash`:
	 *     - MUST set `complete` to 0.
	 *   - otherwise:
	 *     - SHOULD set `complete` to 1.
	 */
	u8 *msg = towire_reply_channel_range(NULL,
					     &peer->daemon->chain_hash,
					     first_blocknum,
					     number_of_blocks,
					     1, encoded);
	queue_peer_msg(peer, take(msg));
}

/*~ When we need to send an array of channels, it might go over our 64k packet
 * size.  If it doesn't, we recurse, splitting in two, etc.  Each message
 * indicates what blocks it contains, so the recipient knows when we're
 * finished.
 *
 * tail_blocks is the empty blocks at the end, in case they asked for all
 * blocks to 4 billion.
 */
static bool queue_channel_ranges(struct peer *peer,
				 u32 first_blocknum, u32 number_of_blocks,
				 u32 tail_blocks)
{
	struct routing_state *rstate = peer->daemon->rstate;
	u8 *encoded = encode_short_channel_ids_start(tmpctx);
	struct short_channel_id scid;
	bool scid_ok;

	/* BOLT #7:
	 *
	 * 1. type: 264 (`reply_channel_range`) (`gossip_queries`)
	 * 2. data:
	 *   * [`32`:`chain_hash`]
	 *   * [`4`:`first_blocknum`]
	 *   * [`4`:`number_of_blocks`]
	 *   * [`1`:`complete`]
	 *   * [`2`:`len`]
	 *   * [`len`:`encoded_short_ids`]
	 */
	const size_t reply_overhead = 32 + 4 + 4 + 1 + 2;
	const size_t max_encoded_bytes = 65535 - 2 - reply_overhead;

	/* Avoid underflow: we don't use block 0 anyway */
	if (first_blocknum == 0)
		scid_ok = mk_short_channel_id(&scid, 1, 0, 0);
	else
		scid_ok = mk_short_channel_id(&scid, first_blocknum, 0, 0);
	scid.u64--;
	if (!scid_ok)
		return false;

	/* We keep a `uintmap` of `short_channel_id` to `struct chan *`.
	 * Unlike a htable, it's efficient to iterate through, but it only
	 * works because each short_channel_id is basically a 64-bit unsigned
	 * integer.
	 *
	 * First we iteraate and gather all the short channel ids. */
	while (uintmap_after(&rstate->chanmap, &scid.u64)) {
		u32 blocknum = short_channel_id_blocknum(&scid);
		if (blocknum >= first_blocknum + number_of_blocks)
			break;

		encode_add_short_channel_id(&encoded, &scid);
	}

	/* If we can encode that, fine: send it */
	if (encode_short_channel_ids_end(&encoded, max_encoded_bytes)) {
		reply_channel_range(peer, first_blocknum,
				    number_of_blocks + tail_blocks,
				    encoded);
		return true;
	}

	/* It wouldn't all fit: divide in half */
	/* We assume we can always send one block! */
	if (number_of_blocks <= 1) {
		/* We always assume we can send 1 blocks worth */
		status_broken("Could not fit scids for single block %u",
			      first_blocknum);
		return false;
	}
	status_debug("queue_channel_ranges full: splitting %u+%u and %u+%u(+%u)",
		     first_blocknum,
		     number_of_blocks / 2,
		     first_blocknum + number_of_blocks / 2,
		     number_of_blocks - number_of_blocks / 2,
		     tail_blocks);
	return queue_channel_ranges(peer, first_blocknum, number_of_blocks / 2, 0)
		&& queue_channel_ranges(peer, first_blocknum + number_of_blocks / 2,
					number_of_blocks - number_of_blocks / 2,
					tail_blocks);
}

/*~ The peer can ask for all channels is a series of blocks.  We reply with one
 * or more messages containing the short_channel_ids. */
static u8 *handle_query_channel_range(struct peer *peer, const u8 *msg)
{
	struct routing_state *rstate = peer->daemon->rstate;
	struct bitcoin_blkid chain_hash;
	u32 first_blocknum, number_of_blocks, tail_blocks;
	struct short_channel_id last_scid;

	if (!fromwire_query_channel_range(msg, &chain_hash,
					  &first_blocknum, &number_of_blocks)) {
		return towire_errorfmt(peer, NULL,
				       "Bad query_channel_range %s",
				       tal_hex(tmpctx, msg));
	}

	/* FIXME: if they ask for the wrong chain, we should not ignore it,
	 * but give an empty response with the `complete` flag unset? */
	if (!bitcoin_blkid_eq(&peer->daemon->chain_hash, &chain_hash)) {
		status_trace("%s sent query_channel_range chainhash %s",
			     type_to_string(tmpctx, struct node_id, &peer->id),
			     type_to_string(tmpctx, struct bitcoin_blkid,
					    &chain_hash));
		return NULL;
	}

	/* If they ask for number_of_blocks UINTMAX, and we have to divide
	 * and conquer, we'll do a lot of unnecessary work.  Cap it at the
	 * last value we have, then send an empty reply. */
	if (uintmap_last(&rstate->chanmap, &last_scid.u64)) {
		u32 last_block = short_channel_id_blocknum(&last_scid);

		/* u64 here avoids overflow on number_of_blocks
		   UINTMAX for example */
		if ((u64)first_blocknum + number_of_blocks > last_block) {
			tail_blocks = first_blocknum + number_of_blocks
				- last_block - 1;
			number_of_blocks -= tail_blocks;
		} else
			tail_blocks = 0;
	} else
		tail_blocks = 0;

	if (!queue_channel_ranges(peer, first_blocknum, number_of_blocks,
				  tail_blocks))
		return towire_errorfmt(peer, NULL,
				       "Invalid query_channel_range %u+%u",
				       first_blocknum, number_of_blocks + tail_blocks);

	return NULL;
}

/*~ This is the reply we get when we send query_channel_range; we keep
 * expecting them until the entire range we asked for is covered. */
static const u8 *handle_reply_channel_range(struct peer *peer, const u8 *msg)
{
	struct bitcoin_blkid chain;
	u8 complete;
	u32 first_blocknum, number_of_blocks, start, end;
	u8 *encoded;
	struct short_channel_id *scids;
	size_t n;
	unsigned long b;

	if (!fromwire_reply_channel_range(tmpctx, msg, &chain, &first_blocknum,
					  &number_of_blocks, &complete,
					  &encoded)) {
		return towire_errorfmt(peer, NULL,
				       "Bad reply_channel_range %s",
				       tal_hex(tmpctx, msg));
	}

	if (!bitcoin_blkid_eq(&peer->daemon->chain_hash, &chain)) {
		return towire_errorfmt(peer, NULL,
				       "reply_channel_range for bad chain: %s",
				       tal_hex(tmpctx, msg));
	}

	if (!peer->query_channel_blocks) {
		return towire_errorfmt(peer, NULL,
				       "reply_channel_range without query: %s",
				       tal_hex(tmpctx, msg));
	}

	/* Beware overflow! */
	if (first_blocknum + number_of_blocks < first_blocknum) {
		return towire_errorfmt(peer, NULL,
				       "reply_channel_range invalid %u+%u",
				       first_blocknum, number_of_blocks);
	}

	scids = decode_short_ids(tmpctx, encoded);
	if (!scids) {
		return towire_errorfmt(peer, NULL,
				       "Bad reply_channel_range encoding %s",
				       tal_hex(tmpctx, encoded));
	}

	status_debug("peer %s reply_channel_range %u+%u (of %u+%u) %zu scids",
		     type_to_string(tmpctx, struct node_id, &peer->id),
		     first_blocknum, number_of_blocks,
		     peer->range_first_blocknum,
		     peer->range_end_blocknum - peer->range_first_blocknum,
		     tal_count(scids));

	/* BOLT #7:
	 *
	 * The receiver of `query_channel_range`:
	 *...
	 *  - MUST respond with one or more `reply_channel_range` whose
	 *    combined range cover the requested `first_blocknum` to
	 *    `first_blocknum` plus `number_of_blocks` minus one.
	 */
	/* ie. They can be outside range we asked, but they must overlap! */
	if (first_blocknum + number_of_blocks <= peer->range_first_blocknum
	    || first_blocknum >= peer->range_end_blocknum) {
		return towire_errorfmt(peer, NULL,
				       "reply_channel_range invalid %u+%u for query %u+%u",
				       first_blocknum, number_of_blocks,
				       peer->range_first_blocknum,
				       peer->range_end_blocknum
				       - peer->range_first_blocknum);
	}

	start = first_blocknum;
	end = first_blocknum + number_of_blocks;
	/* Trim to make it a subset of what we want. */
	if (start < peer->range_first_blocknum)
		start = peer->range_first_blocknum;
	if (end > peer->range_end_blocknum)
		end = peer->range_end_blocknum;

	/* We keep a bitmap of what blocks have been covered by replies: bit 0
	 * represents block peer->range_first_blocknum */
	b = bitmap_ffs(peer->query_channel_blocks,
		       start - peer->range_first_blocknum,
		       end - peer->range_first_blocknum);
	if (b != end - peer->range_first_blocknum) {
		return towire_errorfmt(peer, NULL,
				       "reply_channel_range %u+%u already have block %lu",
				       first_blocknum, number_of_blocks,
				       peer->range_first_blocknum + b);
	}

	/* Mark that short_channel_ids for this block have been received */
	bitmap_fill_range(peer->query_channel_blocks,
			  start - peer->range_first_blocknum,
			  end - peer->range_first_blocknum);
	peer->range_blocks_remaining -= end - start;

	/* Add scids */
	n = tal_count(peer->query_channel_scids);
	tal_resize(&peer->query_channel_scids, n + tal_count(scids));
	memcpy(peer->query_channel_scids + n, scids, tal_bytelen(scids));

	/* Still more to go? */
	if (peer->range_blocks_remaining)
		return NULL;

	/* All done, send reply to lightningd: that's currently the only thing
	 * which triggers this (for testing).  Eventually we might start probing
	 * for gossip information on our own. */
	msg = towire_gossip_query_channel_range_reply(NULL,
						      first_blocknum,
						      number_of_blocks,
						      complete,
						      peer->query_channel_scids);
	daemon_conn_send(peer->daemon->master, take(msg));
	peer->query_channel_scids = tal_free(peer->query_channel_scids);
	peer->query_channel_blocks = tal_free(peer->query_channel_blocks);
	return NULL;
}

/*~ For simplicity, all pings and pongs are forwarded to us here in gossipd. */
static u8 *handle_ping(struct peer *peer, const u8 *ping)
{
	u8 *pong;

	/* This checks the ping packet and makes a pong reply if needed; peer
	 * can specify it doesn't want a response, to simulate traffic. */
	if (!check_ping_make_pong(NULL, ping, &pong))
		return towire_errorfmt(peer, NULL, "Bad ping");

	if (pong)
		queue_peer_msg(peer, take(pong));
	return NULL;
}

/*~ When we get a pong, we tell lightningd about it (it's probably a response
 * to the `ping` JSON RPC command). */
static const u8 *handle_pong(struct peer *peer, const u8 *pong)
{
	const char *err = got_pong(pong, &peer->num_pings_outstanding);

	if (err)
		return towire_errorfmt(peer, NULL, "%s", err);

	daemon_conn_send(peer->daemon->master,
			 take(towire_gossip_ping_reply(NULL, &peer->id, true,
						       tal_count(pong))));
	return NULL;
}

/*~ When we ask about an array of short_channel_ids, we get all channel &
 * node announcements and channel updates which the peer knows.  There's an
 * explicit end packet; this is needed to differentiate between 'I'm slow'
 * and 'I don't know those channels'. */
static u8 *handle_reply_short_channel_ids_end(struct peer *peer, const u8 *msg)
{
	struct bitcoin_blkid chain;
	u8 complete;

	if (!fromwire_reply_short_channel_ids_end(msg, &chain, &complete)) {
		return towire_errorfmt(peer, NULL,
				       "Bad reply_short_channel_ids_end %s",
				       tal_hex(tmpctx, msg));
	}

	if (!bitcoin_blkid_eq(&peer->daemon->chain_hash, &chain)) {
		return towire_errorfmt(peer, NULL,
				       "reply_short_channel_ids_end for bad chain: %s",
				       tal_hex(tmpctx, msg));
	}

	if (peer->num_scid_queries_outstanding == 0) {
		return towire_errorfmt(peer, NULL,
				       "unexpected reply_short_channel_ids_end: %s",
				       tal_hex(tmpctx, msg));
	}

	peer->num_scid_queries_outstanding--;
	/* We tell lightningd: this is because we currently only ask for
	 * query_short_channel_ids when lightningd asks. */
	msg = towire_gossip_scids_reply(msg, true, complete);
	daemon_conn_send(peer->daemon->master, take(msg));
	return NULL;
}

/*~ Arbitrary ordering function of pubkeys.
 *
 * Note that we could use memcmp() here: even if they had somehow different
 * bitwise representations for the same key, we copied them all from struct
 * node which should make them unique.  Even if not (say, a node vanished
 * and reappeared) we'd just end up sending two node_announcement for the
 * same node.
 */
static int pubkey_order(const struct node_id *k1,
			const struct node_id *k2,
			void *unused UNUSED)
{
	return node_id_cmp(k1, k2);
}

static void uniquify_node_ids(struct node_id **ids)
{
	size_t dst, src;

	/* BOLT #7:
	 *
	 * - MUST follow with any `node_announcement`s for each
	 *   `channel_announcement`
	 *
	 *   - SHOULD avoid sending duplicate `node_announcements` in
	 *     response to a single `query_short_channel_ids`.
	 */
	/* ccan/asort is a typesafe qsort wrapper: like most ccan modules
	 * it eschews exposing 'void *' pointers and ensures that the
	 * callback function and its arguments match types correctly. */
	asort(*ids, tal_count(*ids), pubkey_order, NULL);

	/* Compact the array */
	for (dst = 0, src = 0; src < tal_count(*ids); src++) {
		if (dst && node_id_eq(&(*ids)[dst-1], &(*ids)[src]))
			continue;
		(*ids)[dst++] = (*ids)[src];
	}

	/* And trim to length, so tal_count() gives correct answer. */
	tal_resize(ids, dst);
}

/*~ We are fairly careful to avoid the peer DoSing us with channel queries:
 * this routine sends information about a single short_channel_id, unless
 * it's finished all of them. */
static void maybe_create_next_scid_reply(struct peer *peer)
{
	struct routing_state *rstate = peer->daemon->rstate;
	size_t i, num;
	bool sent = false;

	/* BOLT #7:
	 *
	 *   - MUST respond to each known `short_channel_id` with a
	 *     `channel_announcement` and the latest `channel_update` for each end
	 *     - SHOULD NOT wait for the next outgoing gossip flush
	 *       to send these.
	 */
	/* Search for next short_channel_id we know about. */
	num = tal_count(peer->scid_queries);
	for (i = peer->scid_query_idx; !sent && i < num; i++) {
		struct chan *chan;

		chan = get_channel(rstate, &peer->scid_queries[i]);
		if (!chan || !is_chan_public(chan))
			continue;

		queue_peer_from_store(peer, &chan->bcast);
		if (is_halfchan_defined(&chan->half[0]))
			queue_peer_from_store(peer, &chan->half[0].bcast);
		if (is_halfchan_defined(&chan->half[1]))
			queue_peer_from_store(peer, &chan->half[1].bcast);

		/* Record node ids for later transmission of node_announcement */
		tal_arr_expand(&peer->scid_query_nodes, chan->nodes[0]->id);
		tal_arr_expand(&peer->scid_query_nodes, chan->nodes[1]->id);
		sent = true;
	}

	/* Just finished channels?  Remove duplicate nodes. */
	if (peer->scid_query_idx != num && i == num)
		uniquify_node_ids(&peer->scid_query_nodes);

	/* Update index for next time we're called. */
	peer->scid_query_idx = i;

	/* BOLT #7:
	 *
	 *  - MUST follow with any `node_announcement`s for each
	 *   `channel_announcement`
	 *    - SHOULD avoid sending duplicate `node_announcements` in response
	 *     to a single `query_short_channel_ids`.
	 */
	/* If we haven't sent anything above, we look for the next
	 * node_announcement to send. */
	num = tal_count(peer->scid_query_nodes);
	for (i = peer->scid_query_nodes_idx; !sent && i < num; i++) {
		const struct node *n;

		/* Not every node announces itself (we know it exists because
		 * of a channel_announcement, however) */
		n = get_node(rstate, &peer->scid_query_nodes[i]);
		if (!n || !n->bcast.index)
			continue;

		queue_peer_from_store(peer, &n->bcast);
		sent = true;
	}
	peer->scid_query_nodes_idx = i;

	/* All finished? */
	if (peer->scid_queries && peer->scid_query_nodes_idx == num) {
		/* BOLT #7:
		 *
		 * - MUST follow these responses with
		 *   `reply_short_channel_ids_end`.
		 *   - if does not maintain up-to-date channel information for
		 *     `chain_hash`:
		 *      - MUST set `complete` to 0.
		 *   - otherwise:
		 *      - SHOULD set `complete` to 1.
		 */
		/* FIXME: We consider ourselves to have complete knowledge. */
		u8 *end = towire_reply_short_channel_ids_end(peer,
							     &peer->daemon->chain_hash,
							     true);
		queue_peer_msg(peer, take(end));

		/* We're done!  Clean up so we simply pass-through next time. */
		peer->scid_queries = tal_free(peer->scid_queries);
		peer->scid_query_idx = 0;
		peer->scid_query_nodes = tal_free(peer->scid_query_nodes);
		peer->scid_query_nodes_idx = 0;
	}
}

/*~ This is called when the outgoing queue is empty; gossip has lower priority
 * than just about anything else. */
static void dump_gossip(struct peer *peer)
{
	/* Do we have scid query replies to send? */
	maybe_create_next_scid_reply(peer);
}

/*~ This generates a `channel_update` message for one of our channels.  We do
 * this here, rather than in `channeld` because we (may) need to do it
 * ourselves anyway if channeld dies, or when we refresh it once a week. */
static void update_local_channel(struct daemon *daemon,
				 const struct chan *chan,
				 int direction,
				 bool disable,
				 u16 cltv_expiry_delta,
				 struct amount_msat htlc_minimum,
				 u32 fee_base_msat,
				 u32 fee_proportional_millionths,
				 struct amount_msat htlc_maximum,
				 const char *caller)
{
	secp256k1_ecdsa_signature dummy_sig;
	u8 *update, *msg;
	u32 timestamp = time_now().ts.tv_sec;
	u8 message_flags, channel_flags;

	/* So valgrind doesn't complain */
	memset(&dummy_sig, 0, sizeof(dummy_sig));

	/* BOLT #7:
	 *
	 * The origin node:
	 *...
	 *   - MUST set `timestamp` to greater than 0, AND to greater than any
	 *     previously-sent `channel_update` for this `short_channel_id`.
	 *     - SHOULD base `timestamp` on a UNIX timestamp.
	 */
	if (is_halfchan_defined(&chan->half[direction])
	    && timestamp == chan->half[direction].bcast.timestamp)
		timestamp++;

	/* BOLT #7:
	 *
	 * The `channel_flags` bitfield is used to indicate the direction of
	 * the channel: it identifies the node that this update originated
	 * from and signals various options concerning the channel. The
	 * following table specifies the meaning of its individual bits:
	 *
	 * | Bit Position  | Name        | Meaning                          |
	 * | ------------- | ----------- | -------------------------------- |
	 * | 0             | `direction` | Direction this update refers to. |
	 * | 1             | `disable`   | Disable the channel.             |
	 */
	channel_flags = direction;
	if (disable)
		channel_flags |= ROUTING_FLAGS_DISABLED;

	/* BOLT #7:
	 *
	 * The `message_flags` bitfield is used to indicate the presence of
	 * optional fields in the `channel_update` message:
	 *
	 *| Bit Position  | Name                      | Field                 |
	 *...
	 *| 0             | `option_channel_htlc_max` | `htlc_maximum_msat`   |
	 */
	message_flags = 0 | ROUTING_OPT_HTLC_MAX_MSAT;

	/* We create an update with a dummy signature, and hand to hsmd to get
	 * it signed. */
	update = towire_channel_update_option_channel_htlc_max(tmpctx, &dummy_sig,
				       &daemon->chain_hash,
				       &chan->scid,
				       timestamp,
				       message_flags, channel_flags,
				       cltv_expiry_delta,
				       htlc_minimum,
				       fee_base_msat,
				       fee_proportional_millionths,
				       htlc_maximum);

	/* Note that we treat the hsmd as synchronous.  This is simple (no
	 * callback hell)!, but may need to change to async if we ever want
	 * remote HSMs */
	if (!wire_sync_write(HSM_FD,
			     towire_hsm_cupdate_sig_req(tmpctx, update))) {
		status_failed(STATUS_FAIL_HSM_IO, "Writing cupdate_sig_req: %s",
			      strerror(errno));
	}

	msg = wire_sync_read(tmpctx, HSM_FD);
	if (!msg || !fromwire_hsm_cupdate_sig_reply(NULL, msg, &update)) {
		status_failed(STATUS_FAIL_HSM_IO,
			      "Reading cupdate_sig_req: %s",
			      strerror(errno));
	}

	/* BOLT #7:
	 *
	 * The origin node:
	 *  - MAY create a `channel_update` to communicate the channel parameters to the
	 *    channel peer, even though the channel has not yet been announced (i.e. the
	 *    `announce_channel` bit was not set).
	 */
	if (!is_chan_public(chan)) {
		/* handle_channel_update will not put private updates in the
		 * broadcast list, but we send it direct to the peer (if we
		 * have one connected) now */
		struct peer *peer = find_peer(daemon,
					      &chan->nodes[!direction]->id);
		if (peer)
			queue_peer_msg(peer, update);
	}

	/* We feed it into routing.c like any other channel_update; it may
	 * discard it (eg. non-public channel), but it should not complain
	 * about it being invalid! */
	msg = handle_channel_update(daemon->rstate, take(update), caller);
	if (msg)
		status_failed(STATUS_FAIL_INTERNAL_ERROR,
			      "%s: rejected local channel update %s: %s",
			      caller,
			      /* Normally we must not touch something taken()
			       * but we're in deep trouble anyway, and
			       * handle_channel_update only tal_steals onto
			       * tmpctx, so it's actually OK. */
			      tal_hex(tmpctx, update),
			      tal_hex(tmpctx, msg));
}

/*~ We generate local channel updates lazily; most of the time we simply
 * toggle the `local_disabled` flag so we don't use it to route.  We never
 * change anything else after startup (yet!) */
static void maybe_update_local_channel(struct daemon *daemon,
				       struct chan *chan, int direction)
{
	const struct half_chan *hc = &chan->half[direction];
	bool local_disabled;

	/* Don't generate a channel_update for an uninitialized channel. */
	if (!is_halfchan_defined(hc))
		return;

	/* Nothing to update? */
	local_disabled = is_chan_local_disabled(daemon->rstate, chan);
	/*~ Note the inversions here on both sides, which is cheap conversion to
	 * boolean for the RHS! */
	if (!local_disabled == !(hc->channel_flags & ROUTING_FLAGS_DISABLED))
		return;

	update_local_channel(daemon, chan, direction,
			     local_disabled,
			     hc->delay,
			     hc->htlc_minimum,
			     hc->base_fee,
			     hc->proportional_fee,
			     hc->htlc_maximum,
			     /* Note this magic C macro which expands to the
			      * function name, for debug messages */
			     __func__);
}

/*~ This helper figures out which direction of the channel is from-us; if
 * neither, it returns false.  This meets Linus' rule "Always return the error",
 * without doing some horrible 0/1/-1 return. */
static bool local_direction(struct daemon *daemon,
			    const struct chan *chan,
			    int *direction)
{
	for (*direction = 0; *direction < 2; (*direction)++) {
		if (node_id_eq(&chan->nodes[*direction]->id, &daemon->id))
			return true;
	}
	return false;
}

/*~ This is when channeld asks us for a channel_update for a local channel.
 * It does that to fill in the error field when lightningd fails an HTLC and
 * sets the UPDATE bit in the error type.  lightningd is too important to
 * fetch this itself, so channeld does it (channeld has to talk to us for
 * other things anyway, so why not?). */
static bool handle_get_update(struct peer *peer, const u8 *msg)
{
	struct short_channel_id scid;
	struct chan *chan;
	const u8 *update;
	struct routing_state *rstate = peer->daemon->rstate;
	int direction;

	if (!fromwire_gossipd_get_update(msg, &scid)) {
		status_broken("peer %s sent bad gossip_get_update %s",
			      type_to_string(tmpctx, struct node_id, &peer->id),
			      tal_hex(tmpctx, msg));
		return false;
	}

	/* It's possible that the channel has just closed (though v. unlikely) */
	chan = get_channel(rstate, &scid);
	if (!chan) {
		status_unusual("peer %s scid %s: unknown channel",
			       type_to_string(tmpctx, struct node_id, &peer->id),
			       type_to_string(tmpctx, struct short_channel_id,
					      &scid));
		update = NULL;
		goto out;
	}

	/* We want the update that comes from our end. */
	if (!local_direction(peer->daemon, chan, &direction)) {
		status_unusual("peer %s scid %s: not our channel?",
			       type_to_string(tmpctx, struct node_id, &peer->id),
			       type_to_string(tmpctx,
					      struct short_channel_id,
					      &scid));
		update = NULL;
		goto out;
	}

	/* Since we're going to send it out, make sure it's up-to-date. */
	maybe_update_local_channel(peer->daemon, chan, direction);

 	/* It's possible this is zero, if we've never sent a channel_update
	 * for that channel. */
	if (!is_halfchan_defined(&chan->half[direction]))
		update = NULL;
	else
		update = gossip_store_get(tmpctx, rstate->gs,
					  chan->half[direction].bcast.index);
out:
	status_trace("peer %s schanid %s: %s update",
		     type_to_string(tmpctx, struct node_id, &peer->id),
		     type_to_string(tmpctx, struct short_channel_id, &scid),
		     update ? "got" : "no");

	msg = towire_gossipd_get_update_reply(NULL, update);
	daemon_conn_send(peer->dc, take(msg));
	return true;
}

/*~ Return true if the channel information has changed.  This can only
* currently happen if the user restarts with different fee options, but we
* don't assume that. */
static bool halfchan_new_info(const struct half_chan *hc,
			      u16 cltv_delta, struct amount_msat htlc_minimum,
			      u32 fee_base_msat, u32 fee_proportional_millionths,
			      struct amount_msat htlc_maximum)
{
	if (!is_halfchan_defined(hc))
		return true;

	return hc->delay != cltv_delta
		|| !amount_msat_eq(hc->htlc_minimum, htlc_minimum)
		|| hc->base_fee != fee_base_msat
		|| hc->proportional_fee != fee_proportional_millionths
		|| !amount_msat_eq(hc->htlc_maximum, htlc_maximum);
}

/*~ channeld asks us to update the local channel. */
static bool handle_local_channel_update(struct peer *peer, const u8 *msg)
{
	struct chan *chan;
	struct short_channel_id scid;
	bool disable;
	u16 cltv_expiry_delta;
	struct amount_msat htlc_minimum, htlc_maximum;
	u32 fee_base_msat;
	u32 fee_proportional_millionths;
	int direction;

	/* FIXME: We should get scid from lightningd when setting up the
	 * connection, so no per-peer daemon can mess with channels other than
	 * its own! */
	if (!fromwire_gossipd_local_channel_update(msg,
						   &scid,
						   &disable,
						   &cltv_expiry_delta,
						   &htlc_minimum,
						   &fee_base_msat,
						   &fee_proportional_millionths,
						   &htlc_maximum)) {
		status_broken("peer %s bad local_channel_update %s",
			      type_to_string(tmpctx, struct node_id, &peer->id),
			      tal_hex(tmpctx, msg));
		return false;
	}

	/* Can theoretically happen if channel just closed. */
	chan = get_channel(peer->daemon->rstate, &scid);
	if (!chan) {
		status_trace("peer %s local_channel_update for unknown %s",
			      type_to_string(tmpctx, struct node_id, &peer->id),
			      type_to_string(tmpctx, struct short_channel_id,
					     &scid));
		return true;
	}

	/* You shouldn't be asking for a non-local channel though. */
	if (!local_direction(peer->daemon, chan, &direction)) {
		status_broken("peer %s bad local_channel_update for non-local %s",
			      type_to_string(tmpctx, struct node_id, &peer->id),
			      type_to_string(tmpctx, struct short_channel_id,
					     &scid));
		return false;
	}

	/* We could change configuration on restart; update immediately.
	 * Or, if we're *enabling* an announced-disabled channel.
	 * Or, if it's an unannounced channel (only sending to peer). */
	if (halfchan_new_info(&chan->half[direction],
			      cltv_expiry_delta, htlc_minimum,
			      fee_base_msat, fee_proportional_millionths,
			      htlc_maximum)
	    || ((chan->half[direction].channel_flags & ROUTING_FLAGS_DISABLED)
		&& !disable)
	    || !is_chan_public(chan)) {
		update_local_channel(peer->daemon, chan, direction,
				     disable,
				     cltv_expiry_delta,
				     htlc_minimum,
				     fee_base_msat,
				     fee_proportional_millionths,
				     htlc_maximum,
				     __func__);
	}

	/* Normal case: just toggle local_disabled, and generate broadcast in
	 * maybe_update_local_channel when/if someone asks about it. */
	if (disable)
		local_disable_chan(peer->daemon->rstate, chan);
	else
		local_enable_chan(peer->daemon->rstate, chan);

	return true;
}

/*~ This is where the per-peer daemons send us messages.  It's either forwarded
 * gossip, or a request for information.  We deliberately use non-overlapping
 * message types so we can distinguish them. */
static struct io_plan *peer_msg_in(struct io_conn *conn,
				    const u8 *msg,
				    struct peer *peer)
{
	const u8 *err;
	bool ok;

	/* These are messages relayed from peer */
	switch ((enum wire_type)fromwire_peektype(msg)) {
	case WIRE_CHANNEL_ANNOUNCEMENT:
		err = handle_channel_announcement_msg(peer, msg);
		goto handled_relay;
	case WIRE_CHANNEL_UPDATE:
		err = handle_channel_update_msg(peer, msg);
		goto handled_relay;
	case WIRE_NODE_ANNOUNCEMENT:
		err = handle_node_announcement(peer->daemon->rstate, msg);
		goto handled_relay;
	case WIRE_QUERY_CHANNEL_RANGE:
		err = handle_query_channel_range(peer, msg);
		goto handled_relay;
	case WIRE_REPLY_CHANNEL_RANGE:
		err = handle_reply_channel_range(peer, msg);
		goto handled_relay;
	case WIRE_QUERY_SHORT_CHANNEL_IDS:
		err = handle_query_short_channel_ids(peer, msg);
		goto handled_relay;
	case WIRE_REPLY_SHORT_CHANNEL_IDS_END:
		err = handle_reply_short_channel_ids_end(peer, msg);
		goto handled_relay;
	case WIRE_PING:
		err = handle_ping(peer, msg);
		goto handled_relay;
	case WIRE_PONG:
		err = handle_pong(peer, msg);
		goto handled_relay;

	/* These are non-gossip messages (!is_msg_for_gossipd()) */
	case WIRE_INIT:
	case WIRE_ERROR:
	case WIRE_OPEN_CHANNEL:
	case WIRE_ACCEPT_CHANNEL:
	case WIRE_FUNDING_CREATED:
	case WIRE_FUNDING_SIGNED:
	case WIRE_FUNDING_LOCKED:
	case WIRE_SHUTDOWN:
	case WIRE_CLOSING_SIGNED:
	case WIRE_UPDATE_ADD_HTLC:
	case WIRE_UPDATE_FULFILL_HTLC:
	case WIRE_UPDATE_FAIL_HTLC:
	case WIRE_UPDATE_FAIL_MALFORMED_HTLC:
	case WIRE_COMMITMENT_SIGNED:
	case WIRE_REVOKE_AND_ACK:
	case WIRE_UPDATE_FEE:
	case WIRE_CHANNEL_REESTABLISH:
	case WIRE_ANNOUNCEMENT_SIGNATURES:
	case WIRE_GOSSIP_TIMESTAMP_FILTER:
		status_broken("peer %s: relayed unexpected msg of type %s",
			      type_to_string(tmpctx, struct node_id, &peer->id),
			      wire_type_name(fromwire_peektype(msg)));
		return io_close(conn);
	}

	/* Must be a gossip_peerd_wire_type asking us to do something. */
	switch ((enum gossip_peerd_wire_type)fromwire_peektype(msg)) {
	case WIRE_GOSSIPD_GET_UPDATE:
		ok = handle_get_update(peer, msg);
		goto handled_cmd;
	case WIRE_GOSSIPD_LOCAL_ADD_CHANNEL:
		ok = handle_local_add_channel(peer->daemon->rstate, msg, 0);
		goto handled_cmd;
	case WIRE_GOSSIPD_LOCAL_CHANNEL_UPDATE:
		ok = handle_local_channel_update(peer, msg);
		goto handled_cmd;

	/* These are the ones we send, not them */
	case WIRE_GOSSIPD_GET_UPDATE_REPLY:
	case WIRE_GOSSIPD_NEW_STORE_FD:
		break;
	}

	/* Anything else should not have been sent to us: close on it */
	status_broken("peer %s: unexpected cmd of type %i %s",
		      type_to_string(tmpctx, struct node_id, &peer->id),
		      fromwire_peektype(msg),
		      gossip_peerd_wire_type_name(fromwire_peektype(msg)));
	return io_close(conn);

	/* Commands should always be OK. */
handled_cmd:
	if (!ok)
		return io_close(conn);
	goto done;

	/* Forwarded messages may be bad, so we have error which the per-peer
	 * daemon will forward to the peer. */
handled_relay:
	if (err)
		queue_peer_msg(peer, take(err));
done:
	return daemon_conn_read_next(conn, peer->dc);
}

/*~ This is where connectd tells us about a new peer, and we hand back an fd for
 * it to send us messages via peer_msg_in above */
static struct io_plan *connectd_new_peer(struct io_conn *conn,
					 struct daemon *daemon,
					 const u8 *msg)
{
	struct peer *peer = tal(conn, struct peer);
	int fds[2];
	int gossip_store_fd;
	struct gossip_state *gs;

	if (!fromwire_gossip_new_peer(msg, &peer->id,
				      &peer->gossip_queries_feature,
				      &peer->initial_routing_sync_feature)) {
		status_broken("Bad new_peer msg from connectd: %s",
			      tal_hex(tmpctx, msg));
		return io_close(conn);
	}

	gossip_store_fd = gossip_store_readonly_fd(daemon->rstate->gs);;
	if (gossip_store_fd < 0) {
		status_broken("Failed to get readonly store fd: %s",
			      strerror(errno));
		daemon_conn_send(daemon->connectd,
				 take(towire_gossip_new_peer_reply(NULL,
								   false,
								   NULL)));
		goto done;
	}

	/* This can happen: we handle it gracefully, returning a `failed` msg. */
	if (socketpair(AF_LOCAL, SOCK_STREAM, 0, fds) != 0) {
		status_broken("Failed to create socketpair: %s",
			      strerror(errno));
		close(gossip_store_fd);
		daemon_conn_send(daemon->connectd,
				 take(towire_gossip_new_peer_reply(NULL,
								   false,
								   NULL)));
		goto done;
	}

	/* We might not have noticed old peer is dead; kill it now. */
	tal_free(find_peer(daemon, &peer->id));

	/* Populate the rest of the peer info. */
	peer->daemon = daemon;
	peer->scid_queries = NULL;
	peer->scid_query_idx = 0;
	peer->scid_query_nodes = NULL;
	peer->scid_query_nodes_idx = 0;
	peer->num_scid_queries_outstanding = 0;
	peer->query_channel_blocks = NULL;
	peer->num_pings_outstanding = 0;

	/* We keep a list so we can find peer by id */
	list_add_tail(&peer->daemon->peers, &peer->list);
	tal_add_destructor(peer, destroy_peer);

	/* This is the new connection: calls dump_gossip when nothing else to
	 * send. */
	peer->dc = daemon_conn_new(daemon, fds[0],
				   peer_msg_in, dump_gossip, peer);
	/* Free peer if conn closed (destroy_peer closes conn if peer freed) */
	tal_steal(peer->dc, peer);

	/* This sends the initial timestamp filter. */
	setup_gossip_range(peer);

	/* BOLT #7:
	 *
	 * A node:
	 *   - if the `gossip_queries` feature is negotiated:
	 * 	- MUST NOT relay any gossip messages unless explicitly requested.
	 */
	if (peer->gossip_queries_feature) {
		gs = NULL;
	} else {
		/* BOLT #7:
		 *
		 * - upon receiving an `init` message with the
		 *   `initial_routing_sync` flag set to 1:
		 *   - SHOULD send gossip messages for all known channels and
		 *    nodes, as if they were just received.
		 * - if the `initial_routing_sync` flag is set to 0, OR if the
		 *   initial sync was completed:
		 *   - SHOULD resume normal operation, as specified in the
		 *     following [Rebroadcasting](#rebroadcasting) section.
		 */
		gs = tal(tmpctx, struct gossip_state);
		gs->timestamp_min = 0;
		gs->timestamp_max = UINT32_MAX;

		/* If they don't want initial sync, start at end of store */
		if (!peer->initial_routing_sync_feature)
			lseek(gossip_store_fd, 0, SEEK_END);

		gs->next_gossip = time_mono();
	}

	/* Reply with success, and the new fd and gossip_state. */
	daemon_conn_send(daemon->connectd,
			 take(towire_gossip_new_peer_reply(NULL, true, gs)));
	daemon_conn_send_fd(daemon->connectd, fds[1]);
	daemon_conn_send_fd(daemon->connectd, gossip_store_fd);

done:
	return daemon_conn_read_next(conn, daemon->connectd);
}

/*~ connectd can also ask us if we know any addresses for a given id. */
static struct io_plan *connectd_get_address(struct io_conn *conn,
					    struct daemon *daemon,
					    const u8 *msg)
{
	struct node_id id;
	u8 rgb_color[3];
	u8 alias[32];
	u8 *features;
	struct wireaddr *addrs;

	if (!fromwire_gossip_get_addrs(msg, &id)) {
		status_broken("Bad gossip_get_addrs msg from connectd: %s",
			      tal_hex(tmpctx, msg));
		return io_close(conn);
	}

	if (!get_node_announcement_by_id(tmpctx, daemon, &id,
					 rgb_color, alias, &features, &addrs))
		addrs = NULL;

	daemon_conn_send(daemon->connectd,
			 take(towire_gossip_get_addrs_reply(NULL, addrs)));
	return daemon_conn_read_next(conn, daemon->connectd);
}

/*~ connectd's input handler is very simple. */
static struct io_plan *connectd_req(struct io_conn *conn,
				    const u8 *msg,
				    struct daemon *daemon)
{
	enum connect_gossip_wire_type t = fromwire_peektype(msg);

	switch (t) {
	case WIRE_GOSSIP_NEW_PEER:
		return connectd_new_peer(conn, daemon, msg);

	case WIRE_GOSSIP_GET_ADDRS:
		return connectd_get_address(conn, daemon, msg);

	/* We send these, don't receive them. */
	case WIRE_GOSSIP_NEW_PEER_REPLY:
	case WIRE_GOSSIP_GET_ADDRS_REPLY:
		break;
	}

	status_broken("Bad msg from connectd: %s",
		      tal_hex(tmpctx, msg));
	return io_close(conn);
}

/*~ This is our twice-weekly timer callback for refreshing our channels.  This
 * was added to the spec because people abandoned their channels without
 * closing them. */
static void gossip_send_keepalive_update(struct daemon *daemon,
					 const struct chan *chan,
					 const struct half_chan *hc)
{
	status_trace("Sending keepalive channel_update for %s",
		     type_to_string(tmpctx, struct short_channel_id,
				    &chan->scid));

	/* As a side-effect, this will create an update which matches the
	 * local_disabled state */
	update_local_channel(daemon, chan,
			     hc->channel_flags & ROUTING_FLAGS_DIRECTION,
			     is_chan_local_disabled(daemon->rstate, chan),
			     hc->delay,
			     hc->htlc_minimum,
			     hc->base_fee,
			     hc->proportional_fee,
			     hc->htlc_maximum,
			     __func__);
}


/* BOLT #7:
 *
 * A node:
 *  - if a channel's latest `channel_update`s `timestamp` is older than two weeks
 *    (1209600 seconds):
 *     - MAY prune the channel.
 *     - MAY ignore the channel.
 */
static void gossip_refresh_network(struct daemon *daemon)
{
	u64 now = time_now().ts.tv_sec;
	/* Anything below this highwater mark could be pruned if not refreshed */
	s64 highwater = now - daemon->rstate->prune_timeout / 2;
	struct node *n;

	/* Schedule next run now (prune_timeout is 2 weeks) */
	notleak(new_reltimer(&daemon->timers, daemon,
			     time_from_sec(daemon->rstate->prune_timeout/4),
			     gossip_refresh_network, daemon));

	/* Find myself in the network */
	n = get_node(daemon->rstate, &daemon->id);
	if (n) {
		/* Iterate through all outgoing connection and check whether
		 * it's time to re-announce */
		struct chan_map_iter i;
		struct chan *c;

		for (c = first_chan(n, &i); c; c = next_chan(n, &i)) {
			struct half_chan *hc = half_chan_from(n, c);

			if (!is_halfchan_defined(hc)) {
				/* Connection is not announced yet, so don't even
				 * try to re-announce it */
				continue;
			}

			if (hc->bcast.timestamp > highwater) {
				/* No need to send a keepalive update message */
				continue;
			}

			if (!is_halfchan_enabled(hc)) {
				/* Only send keepalives for active connections */
				continue;
			}

			gossip_send_keepalive_update(daemon, c, hc);
		}
	}

	/* Now we've refreshed our channels, we can prune without clobbering
	 * them */
	route_prune(daemon->rstate);
}

/* Disables all channels connected to our node. */
static void gossip_disable_local_channels(struct daemon *daemon)
{
	struct node *local_node = get_node(daemon->rstate, &daemon->id);
	struct chan_map_iter i;
	struct chan *c;

	/* We don't have a local_node, so we don't have any channels yet
	 * either */
	if (!local_node)
		return;

	for (c = first_chan(local_node, &i); c; c = next_chan(local_node, &i))
		local_disable_chan(daemon->rstate, c);
}

/*~ Parse init message from lightningd: starts the daemon properly. */
static struct io_plan *gossip_init(struct io_conn *conn,
				   struct daemon *daemon,
				   const u8 *msg)
{
	u32 update_channel_interval;
	u32 *dev_gossip_time;

	if (!fromwire_gossipctl_init(daemon, msg,
				     &daemon->chain_hash,
				     &daemon->id, &daemon->globalfeatures,
				     daemon->rgb,
				     daemon->alias,
				     /* 1 week in seconds
				      * (unless --dev-channel-update-interval) */
				     &update_channel_interval,
				     &daemon->announcable,
				     &dev_gossip_time)) {
		master_badmsg(WIRE_GOSSIPCTL_INIT, msg);
	}

	/* Prune time (usually 2 weeks) is twice update time */
	daemon->rstate = new_routing_state(daemon,
					   chainparams_by_chainhash(&daemon->chain_hash),
					   &daemon->id,
					   update_channel_interval * 2,
					   &daemon->peers,
					   dev_gossip_time);

	/* Load stored gossip messages */
	gossip_store_load(daemon->rstate, daemon->rstate->gs);

	/* Now disable all local channels, they can't be connected yet. */
	gossip_disable_local_channels(daemon);

	/* If that announced channels, we can announce ourselves (options
	 * or addresses might have changed!) */
	maybe_send_own_node_announce(daemon);

	/* Start the weekly refresh timer. */
	notleak(new_reltimer(&daemon->timers, daemon,
			     time_from_sec(daemon->rstate->prune_timeout/4),
			     gossip_refresh_network, daemon));

	return daemon_conn_read_next(conn, daemon->master);
}

/*~ lightningd can ask for a route between nodes. */
static struct io_plan *getroute_req(struct io_conn *conn, struct daemon *daemon,
				    const u8 *msg)
{
	struct node_id source, destination;
	struct amount_msat msat;
	u32 final_cltv;
	u64 riskfactor_by_million;
	u32 max_hops;
	u8 *out;
	struct route_hop *hops;
	double fuzz;
	struct short_channel_id_dir *excluded;

	/* To choose between variations, we need to know how much we're
	 * sending (eliminates too-small channels, and also effects the fees
	 * we'll pay), how to trade off more locktime vs. more fees, and how
	 * much cltv we need a the final node to give exact values for each
	 * intermediate hop, as well as how much random fuzz to inject to
	 * avoid being too predictable. */
	if (!fromwire_gossip_getroute_request(msg, msg,
					      &source, &destination,
					      &msat, &riskfactor_by_million,
					      &final_cltv, &fuzz,
					      &excluded,
					      &max_hops))
		master_badmsg(WIRE_GOSSIP_GETROUTE_REQUEST, msg);

	status_trace("Trying to find a route from %s to %s for %s",
		     type_to_string(tmpctx, struct node_id, &source),
		     type_to_string(tmpctx, struct node_id, &destination),
		     type_to_string(tmpctx, struct amount_msat, &msat));

	/* routing.c does all the hard work; can return NULL. */
	hops = get_route(tmpctx, daemon->rstate, &source, &destination,
			 msat, riskfactor_by_million / 1000000.0, final_cltv,
			 fuzz, pseudorand_u64(), excluded, max_hops);

	out = towire_gossip_getroute_reply(NULL, hops);
	daemon_conn_send(daemon->master, take(out));
	return daemon_conn_read_next(conn, daemon->master);
}

/*~ When someone asks lightningd to `listchannels`, gossipd does the work:
 * marshalling the channel information for all channels into an array of
 * gossip_getchannels_entry, which lightningd converts to JSON.  Each channel
 * is represented by two half_chan; one in each direction.
 */
static struct gossip_halfchannel_entry *hc_entry(const tal_t *ctx,
						 const struct chan *chan,
						 int idx)
{
	/* Our 'struct chan' contains two nodes: they are in pubkey_cmp order
	 * (ie. chan->nodes[0] is the lesser pubkey) and this is the same as
	 * the direction bit in `channel_update`s `channel_flags`.
	 *
	 * The halfchans are arranged so that half[0] src == nodes[0], and we
	 * use that here. */
	const struct half_chan *c = &chan->half[idx];
	struct gossip_halfchannel_entry *e;

	/* If we've never seen a channel_update for this direction... */
	if (!is_halfchan_defined(c))
		return NULL;

	e = tal(ctx, struct gossip_halfchannel_entry);
	e->channel_flags = c->channel_flags;
	e->message_flags = c->message_flags;
	e->last_update_timestamp = c->bcast.timestamp;
	e->base_fee_msat = c->base_fee;
	e->fee_per_millionth = c->proportional_fee;
	e->delay = c->delay;

	return e;
}

/*~ Marshal (possibly) both channel directions into entries. */
static void append_channel(struct routing_state *rstate,
			   const struct gossip_getchannels_entry ***entries,
			   const struct chan *chan,
			   const struct node_id *srcfilter)
{
	struct gossip_getchannels_entry *e = tal(*entries, struct gossip_getchannels_entry);

	e->node[0] = chan->nodes[0]->id;
	e->node[1] = chan->nodes[1]->id;
	e->sat = chan->sat;
	e->local_disabled = is_chan_local_disabled(rstate, chan);
	e->public = is_chan_public(chan);
	e->short_channel_id = chan->scid;
	if (!srcfilter || node_id_eq(&e->node[0], srcfilter))
		e->e[0] = hc_entry(*entries, chan, 0);
	else
		e->e[0] = NULL;
	if (!srcfilter || node_id_eq(&e->node[1], srcfilter))
		e->e[1] = hc_entry(*entries, chan, 1);
	else
		e->e[1] = NULL;

	/* We choose not to tell lightningd about channels with no updates,
	 * as they're unusable and can't be represented in the listchannels
	 * JSON output we use anyway. */
	if (e->e[0] || e->e[1])
		tal_arr_expand(entries, e);
}

/*~ This is where lightningd asks for all channels we know about. */
static struct io_plan *getchannels_req(struct io_conn *conn,
				       struct daemon *daemon,
				       const u8 *msg)
{
	u8 *out;
	const struct gossip_getchannels_entry **entries;
	struct chan *chan;
	struct short_channel_id *scid, *prev;
	struct node_id *source;
	bool complete = true;

	/* Note: scid is marked optional in gossip_wire.csv */
	if (!fromwire_gossip_getchannels_request(msg, msg, &scid, &source,
						 &prev))
		master_badmsg(WIRE_GOSSIP_GETCHANNELS_REQUEST, msg);

	entries = tal_arr(tmpctx, const struct gossip_getchannels_entry *, 0);
	/* They can ask about a particular channel by short_channel_id */
	if (scid) {
		chan = get_channel(daemon->rstate, scid);
		if (chan)
			append_channel(daemon->rstate, &entries, chan, NULL);
	} else if (source) {
		struct node *s = get_node(daemon->rstate, source);
		if (s) {
			struct chan_map_iter i;
			struct chan *c;

			for (c = first_chan(s, &i); c; c = next_chan(s, &i)) {
				append_channel(daemon->rstate,
					       &entries, c, source);
			}
		}
	} else {
		u64 idx;

		/* For the more general case, we just iterate through every
		 * short channel id, starting with previous if any (there is
		 * no scid 0). */
		idx = prev ? prev->u64 : 0;
		while ((chan = uintmap_after(&daemon->rstate->chanmap, &idx))) {
			append_channel(daemon->rstate, &entries, chan, NULL);
			/* Limit how many we do at once. */
			if (tal_count(entries) == 4096) {
				complete = false;
				break;
			}
		}
	}

	out = towire_gossip_getchannels_reply(NULL, complete, entries);
	daemon_conn_send(daemon->master, take(out));
	return daemon_conn_read_next(conn, daemon->master);
}

/*~ Similarly, lightningd asks us for all nodes when it gets `listnodes` */
/* We keep pointers into n, assuming it won't change. */
static void add_node_entry(const tal_t *ctx,
			   struct daemon *daemon,
			   const struct node *n,
			   struct gossip_getnodes_entry *e)
{
	e->nodeid = n->id;
	if (get_node_announcement(ctx, daemon, n,
				  e->color, e->alias,
				  &e->globalfeatures,
				  &e->addresses)) {
		e->last_timestamp = n->bcast.timestamp;
	} else {
		/* Timestamp on wire is an unsigned 32 bit: we use a 64-bit
		 * signed, so -1 means "we never received a
		 * channel_update". */
		e->last_timestamp = -1;
	}
}

/* Simply routine when they ask for `listnodes` */
static struct io_plan *getnodes(struct io_conn *conn, struct daemon *daemon,
				const u8 *msg)
{
	u8 *out;
	struct node *n;
	const struct gossip_getnodes_entry **nodes;
	struct gossip_getnodes_entry *node_arr;
	struct node_id *id;

	if (!fromwire_gossip_getnodes_request(tmpctx, msg, &id))
		master_badmsg(WIRE_GOSSIP_GETNODES_REQUEST, msg);

	/* Format of reply is the same whether they ask for a specific node
	 * (0 or one responses) or all nodes (0 or more) */
	if (id) {
		n = get_node(daemon->rstate, id);
		if (n) {
			node_arr = tal_arr(tmpctx,
					   struct gossip_getnodes_entry,
					   1);
			add_node_entry(node_arr, daemon, n, &node_arr[0]);
		} else {
			nodes = NULL;
			node_arr = NULL;
		}
	} else {
		struct node_map_iter it;
		size_t i = 0;
		node_arr = tal_arr(tmpctx, struct gossip_getnodes_entry,
				   daemon->rstate->nodes->raw.elems);
		n = node_map_first(daemon->rstate->nodes, &it);
		while (n != NULL) {
			add_node_entry(node_arr, daemon, n, &node_arr[i++]);
			n = node_map_next(daemon->rstate->nodes, &it);
		}
		assert(i == daemon->rstate->nodes->raw.elems);
	}

	/* FIXME: towire wants array of pointers. */
	nodes = tal_arr(node_arr, const struct gossip_getnodes_entry *,
			tal_count(node_arr));
	for (size_t i = 0; i < tal_count(node_arr); i++)
		nodes[i] = &node_arr[i];
	out = towire_gossip_getnodes_reply(NULL, nodes);
	daemon_conn_send(daemon->master, take(out));
	return daemon_conn_read_next(conn, daemon->master);
}

/*~ We currently have a JSON command to ping a peer: it ends up here, where
 * gossipd generates the actual ping and sends it like any other gossip. */
static struct io_plan *ping_req(struct io_conn *conn, struct daemon *daemon,
				const u8 *msg)
{
	struct node_id id;
	u16 num_pong_bytes, len;
	struct peer *peer;
	u8 *ping;

	if (!fromwire_gossip_ping(msg, &id, &num_pong_bytes, &len))
		master_badmsg(WIRE_GOSSIP_PING, msg);

	/* Even if lightningd were to check for valid ids, there's a race
	 * where it might vanish before we read this command; cleaner to
	 * handle it here with 'sent' = false. */
	peer = find_peer(daemon, &id);
	if (!peer) {
		daemon_conn_send(daemon->master,
				 take(towire_gossip_ping_reply(NULL, &id,
							       false, 0)));
		goto out;
	}

	/* It should never ask for an oversize ping. */
	ping = make_ping(peer, num_pong_bytes, len);
	if (tal_count(ping) > 65535)
		status_failed(STATUS_FAIL_MASTER_IO, "Oversize ping");

	queue_peer_msg(peer, take(ping));
	status_trace("sending ping expecting %sresponse",
		     num_pong_bytes >= 65532 ? "no " : "");

	/* BOLT #1:
	 *
	 * A node receiving a `ping` message:
	 *...
	 *  - if `num_pong_bytes` is less than 65532:
	 *    - MUST respond by sending a `pong` message, with `byteslen` equal
	 *      to `num_pong_bytes`.
	 *  - otherwise (`num_pong_bytes` is **not** less than 65532):
	 *    - MUST ignore the `ping`.
	 */
	if (num_pong_bytes >= 65532)
		daemon_conn_send(daemon->master,
				 take(towire_gossip_ping_reply(NULL, &id,
							       true, 0)));
	else
		/* We'll respond to lightningd once the pong comes in */
		peer->num_pings_outstanding++;

out:
	return daemon_conn_read_next(conn, daemon->master);
}

/*~ If a node has no public channels (other than the one to us), it's not
 * a very useful route to tell anyone about. */
static bool node_has_public_channels(const struct node *peer,
				     const struct chan *exclude)
{
	struct chan_map_iter i;
	struct chan *c;

	for (c = first_chan(peer, &i); c; c = next_chan(peer, &i)) {
		if (c == exclude)
			continue;
		if (is_chan_public(c))
			return true;
	}
	return false;
}

/*~ The `exposeprivate` flag is a trinary: NULL == dynamic, otherwise
 * value decides.  Thus, we provide two wrappers for clarity: */
static bool never_expose(bool *exposeprivate)
{
	return exposeprivate && !*exposeprivate;
}

static bool always_expose(bool *exposeprivate)
{
	return exposeprivate && *exposeprivate;
}

/*~ For routeboost, we offer payers a hint of what incoming channels might
 * have capacity for their payment.  To do this, lightningd asks for the
 * information about all channels to this node; but gossipd doesn't know about
 * current capacities, so lightningd selects which to use. */
static struct io_plan *get_incoming_channels(struct io_conn *conn,
					     struct daemon *daemon,
					     const u8 *msg)
{
	struct node *node;
	struct route_info *public = tal_arr(tmpctx, struct route_info, 0);
	struct route_info *private = tal_arr(tmpctx, struct route_info, 0);
	bool has_public;
	bool *exposeprivate;

	if (!fromwire_gossip_get_incoming_channels(tmpctx, msg, &exposeprivate))
		master_badmsg(WIRE_GOSSIP_GET_INCOMING_CHANNELS, msg);

	status_trace("exposeprivate = %s",
		     exposeprivate ? (*exposeprivate ? "TRUE" : "FALSE") : "NULL");
	status_trace("msg = %s", tal_hex(tmpctx, msg));
	status_trace("always_expose = %u, never_expose = %u",
		     always_expose(exposeprivate), never_expose(exposeprivate));

	has_public = always_expose(exposeprivate);

	node = get_node(daemon->rstate, &daemon->rstate->local_id);
	if (node) {
		struct chan_map_iter i;
		struct chan *c;

		for (c = first_chan(node, &i); c; c = next_chan(node, &i)) {
			const struct half_chan *hc;
			struct route_info ri;

			hc = &c->half[half_chan_to(node, c)];

			if (!is_halfchan_enabled(hc))
				continue;

			ri.pubkey = other_node(node, c)->id;
			ri.short_channel_id = c->scid;
			ri.fee_base_msat = hc->base_fee;
			ri.fee_proportional_millionths = hc->proportional_fee;
			ri.cltv_expiry_delta = hc->delay;

			has_public |= is_chan_public(c);

			/* If peer doesn't have other public channels,
			 * no point giving route */
			if (!node_has_public_channels(other_node(node, c), c))
				continue;

			if (always_expose(exposeprivate) || is_chan_public(c))
				tal_arr_expand(&public, ri);
			else
				tal_arr_expand(&private, ri);
		}
	}

	/* If no public channels (even deadend ones!), share private ones. */
	if (!has_public && !never_expose(exposeprivate))
		msg = towire_gossip_get_incoming_channels_reply(NULL, private);
	else
		msg = towire_gossip_get_incoming_channels_reply(NULL, public);
	daemon_conn_send(daemon->master, take(msg));

	return daemon_conn_read_next(conn, daemon->master);
}

#if DEVELOPER
/* FIXME: One day this will be called internally; for now it's just for
 * testing with dev_query_scids. */
static struct io_plan *query_scids_req(struct io_conn *conn,
				       struct daemon *daemon,
				       const u8 *msg)
{
	struct node_id id;
	struct short_channel_id *scids;
	struct peer *peer;
	u8 *encoded;
	/* BOLT #7:
	 *
	 * 1. type: 261 (`query_short_channel_ids`) (`gossip_queries`)
	 * 2. data:
	 *     * [`32`:`chain_hash`]
	 *     * [`2`:`len`]
	 *     * [`len`:`encoded_short_ids`]
	 */
	const size_t reply_overhead = 32 + 2;
	const size_t max_encoded_bytes = 65535 - 2 - reply_overhead;

	if (!fromwire_gossip_query_scids(msg, msg, &id, &scids))
		master_badmsg(WIRE_GOSSIP_QUERY_SCIDS, msg);

	peer = find_peer(daemon, &id);
	if (!peer) {
		status_broken("query_scids: unknown peer %s",
			      type_to_string(tmpctx, struct node_id, &id));
		goto fail;
	}

	if (!peer->gossip_queries_feature) {
		status_broken("query_scids: no gossip_query support in peer %s",
			      type_to_string(tmpctx, struct node_id, &id));
		goto fail;
	}

	encoded = encode_short_channel_ids_start(tmpctx);
	for (size_t i = 0; i < tal_count(scids); i++)
		encode_add_short_channel_id(&encoded, &scids[i]);

	/* Because this is a dev command, we simply say this case is
	 * "too hard". */
	if (!encode_short_channel_ids_end(&encoded, max_encoded_bytes)) {
		status_broken("query_short_channel_ids: %zu is too many",
			      tal_count(scids));
		goto fail;
	}

	msg = towire_query_short_channel_ids(NULL, &daemon->chain_hash,
					     encoded);
	queue_peer_msg(peer, take(msg));
	peer->num_scid_queries_outstanding++;

	status_trace("sending query for %zu scids", tal_count(scids));
out:
	return daemon_conn_read_next(conn, daemon->master);

fail:
	daemon_conn_send(daemon->master,
			 take(towire_gossip_scids_reply(NULL, false, false)));
	goto out;
}

/* BOLT #7:
 *
 * ### The `gossip_timestamp_filter` Message
 *...
 * This message allows a node to constrain future gossip messages to
 * a specific range.  A node which wants any gossip messages would have
 * to send this, otherwise `gossip_queries` negotiation means no gossip
 * messages would be received.
 *
 * Note that this filter replaces any previous one, so it can be used
 * multiple times to change the gossip from a peer. */
/* This is the entry point for dev_send_timestamp_filter testing. */
static struct io_plan *send_timestamp_filter(struct io_conn *conn,
					     struct daemon *daemon,
					     const u8 *msg)
{
	struct node_id id;
	u32 first, range;
	struct peer *peer;

	if (!fromwire_gossip_send_timestamp_filter(msg, &id, &first, &range))
		master_badmsg(WIRE_GOSSIP_SEND_TIMESTAMP_FILTER, msg);

	peer = find_peer(daemon, &id);
	if (!peer) {
		status_broken("send_timestamp_filter: unknown peer %s",
			      type_to_string(tmpctx, struct node_id, &id));
		goto out;
	}

	if (!peer->gossip_queries_feature) {
		status_broken("send_timestamp_filter: no gossip_query support in peer %s",
			      type_to_string(tmpctx, struct node_id, &id));
		goto out;
	}

	msg = towire_gossip_timestamp_filter(NULL, &daemon->chain_hash,
					     first, range);
	queue_peer_msg(peer, take(msg));
out:
	return daemon_conn_read_next(conn, daemon->master);
}

/* FIXME: One day this will be called internally; for now it's just for
 * testing with dev_query_channel_range. */
static struct io_plan *query_channel_range(struct io_conn *conn,
					   struct daemon *daemon,
					   const u8 *msg)
{
	struct node_id id;
	u32 first_blocknum, number_of_blocks;
	struct peer *peer;

	if (!fromwire_gossip_query_channel_range(msg, &id, &first_blocknum,
						 &number_of_blocks))
		master_badmsg(WIRE_GOSSIP_QUERY_SCIDS, msg);

	peer = find_peer(daemon, &id);
	if (!peer) {
		status_broken("query_channel_range: unknown peer %s",
			      type_to_string(tmpctx, struct node_id, &id));
		goto fail;
	}

	if (!peer->gossip_queries_feature) {
		status_broken("query_channel_range: no gossip_query support in peer %s",
			      type_to_string(tmpctx, struct node_id, &id));
		goto fail;
	}

	if (peer->query_channel_blocks) {
		status_broken("query_channel_range: previous query active");
		goto fail;
	}

	/* Check for overflow on 32-bit machines! */
	if (BITMAP_NWORDS(number_of_blocks) < number_of_blocks / BITMAP_WORD_BITS) {
		status_broken("query_channel_range: huge number_of_blocks (%u) not supported",
			number_of_blocks);
		goto fail;
	}

	status_debug("sending query_channel_range for blocks %u+%u",
		     first_blocknum, number_of_blocks);
	msg = towire_query_channel_range(NULL, &daemon->chain_hash,
					 first_blocknum, number_of_blocks);
	queue_peer_msg(peer, take(msg));
	peer->range_first_blocknum = first_blocknum;
	peer->range_end_blocknum = first_blocknum + number_of_blocks;
	peer->range_blocks_remaining = number_of_blocks;
	peer->query_channel_blocks = tal_arrz(peer, bitmap,
					      BITMAP_NWORDS(number_of_blocks));
	peer->query_channel_scids = tal_arr(peer, struct short_channel_id, 0);

out:
	return daemon_conn_read_next(conn, daemon->master);

fail:
	daemon_conn_send(daemon->master,
			 take(towire_gossip_query_channel_range_reply(NULL,
								      0, 0,
								      false,
								      NULL)));
	goto out;
}

/* This is a testing hack to allow us to artificially lower the maximum bytes
 * of short_channel_ids we'll encode, using dev_set_max_scids_encode_size. */
static struct io_plan *dev_set_max_scids_encode_size(struct io_conn *conn,
						     struct daemon *daemon,
						     const u8 *msg)
{
	if (!fromwire_gossip_dev_set_max_scids_encode_size(msg,
							   &max_scids_encode_bytes))
		master_badmsg(WIRE_GOSSIP_DEV_SET_MAX_SCIDS_ENCODE_SIZE, msg);

	status_trace("Set max_scids_encode_bytes to %u", max_scids_encode_bytes);
	return daemon_conn_read_next(conn, daemon->master);
}

/* Another testing hack */
static struct io_plan *dev_gossip_suppress(struct io_conn *conn,
					   struct daemon *daemon,
					   const u8 *msg)
{
	if (!fromwire_gossip_dev_suppress(msg))
		master_badmsg(WIRE_GOSSIP_DEV_SUPPRESS, msg);

	status_unusual("Suppressing all gossip");
	suppress_gossip = true;
	return daemon_conn_read_next(conn, daemon->master);
}

static struct io_plan *dev_gossip_memleak(struct io_conn *conn,
					  struct daemon *daemon,
					  const u8 *msg)
{
	struct htable *memtable;
	bool found_leak;

	memtable = memleak_enter_allocations(tmpctx, msg, msg);

	/* Now delete daemon and those which it has pointers to. */
	memleak_remove_referenced(memtable, daemon);
	memleak_remove_routing_tables(memtable, daemon->rstate);

	found_leak = dump_memleak(memtable);
	daemon_conn_send(daemon->master,
			 take(towire_gossip_dev_memleak_reply(NULL,
							      found_leak)));
	return daemon_conn_read_next(conn, daemon->master);
}

static struct io_plan *dev_compact_store(struct io_conn *conn,
					 struct daemon *daemon,
					 const u8 *msg)
{
	bool done = gossip_store_compact(daemon->rstate->gs);

	daemon_conn_send(daemon->master,
			 take(towire_gossip_dev_compact_store_reply(NULL,
								    done)));
	return daemon_conn_read_next(conn, daemon->master);
}
#endif /* DEVELOPER */

/*~ lightningd: so, tell me about this channel, so we can forward to it. */
static struct io_plan *get_channel_peer(struct io_conn *conn,
					struct daemon *daemon, const u8 *msg)
{
	struct short_channel_id scid;
	struct chan *chan;
	const struct node_id *key;
	int direction;

	if (!fromwire_gossip_get_channel_peer(msg, &scid))
		master_badmsg(WIRE_GOSSIP_GET_CHANNEL_PEER, msg);

	chan = get_channel(daemon->rstate, &scid);
	if (!chan) {
		status_trace("Failed to resolve channel %s",
			     type_to_string(tmpctx, struct short_channel_id, &scid));
		key = NULL;
	} else if (local_direction(daemon, chan, &direction)) {
		key = &chan->nodes[!direction]->id;
	} else {
		status_trace("Resolved channel %s was not local",
			     type_to_string(tmpctx, struct short_channel_id,
					    &scid));
		key = NULL;
	}
	daemon_conn_send(daemon->master,
			 take(towire_gossip_get_channel_peer_reply(NULL, key)));
	return daemon_conn_read_next(conn, daemon->master);
}

/*~ We queue incoming channel_announcement pending confirmation from lightningd
 * that it really is an unspent output.  Here's its reply. */
static struct io_plan *handle_txout_reply(struct io_conn *conn,
					  struct daemon *daemon, const u8 *msg)
{
	struct short_channel_id scid;
	u8 *outscript;
	struct amount_sat sat;

	if (!fromwire_gossip_get_txout_reply(msg, msg, &scid, &sat, &outscript))
		master_badmsg(WIRE_GOSSIP_GET_TXOUT_REPLY, msg);

	/* Outscript is NULL if it's not an unspent output */
	handle_pending_cannouncement(daemon->rstate, &scid, sat, outscript);

	/* Anywhere we might have announced a channel, we check if it's time to
	 * announce ourselves (ie. if we just announced our own first channel) */
	maybe_send_own_node_announce(daemon);

	return daemon_conn_read_next(conn, daemon->master);
}

/* Fix up the channel_update to include the type if it doesn't currently have
 * one. See ElementsProject/lightning#1730 and lightningnetwork/lnd#1599 for the
 * in-depth discussion on why we break message parsing here... */
static u8 *patch_channel_update(const tal_t *ctx, u8 *channel_update TAKES)
{
	u8 *fixed;
	if (channel_update != NULL &&
	    fromwire_peektype(channel_update) != WIRE_CHANNEL_UPDATE) {
		/* This should be a channel_update, prefix with the
		 * WIRE_CHANNEL_UPDATE type, but isn't. Let's prefix it. */
		fixed = tal_arr(ctx, u8, 0);
		towire_u16(&fixed, WIRE_CHANNEL_UPDATE);
		towire(&fixed, channel_update, tal_bytelen(channel_update));
		if (taken(channel_update))
			tal_free(channel_update);
		return fixed;
	} else {
		return tal_dup_arr(ctx, u8,
				   channel_update, tal_count(channel_update), 0);
	}
}

/* Return NULL if the wrapped onion error message has no channel_update field,
 * or return the embedded channel_update message otherwise. */
static u8 *channel_update_from_onion_error(const tal_t *ctx,
					   const u8 *onion_message)
{
	u8 *channel_update = NULL;
	struct amount_msat unused_msat;
	u32 unused32;

	/* Identify failcodes that have some channel_update.
	 *
	 * TODO > BOLT 1.0: Add new failcodes when updating to a
	 * new BOLT version. */
	if (!fromwire_temporary_channel_failure(ctx,
						onion_message,
						&channel_update) &&
	    !fromwire_amount_below_minimum(ctx,
					   onion_message, &unused_msat,
					   &channel_update) &&
	    !fromwire_fee_insufficient(ctx,
		    		       onion_message, &unused_msat,
				       &channel_update) &&
	    !fromwire_incorrect_cltv_expiry(ctx,
		    			    onion_message, &unused32,
					    &channel_update) &&
	    !fromwire_expiry_too_soon(ctx,
		    		      onion_message,
				      &channel_update))
		/* No channel update. */
		return NULL;

	return patch_channel_update(ctx, take(channel_update));
}

/*~ lightningd tells us when a payment has failed; we mark the channel (or
 * node) unusable here if it's a permanent failure, and unpack any
 * channel_update contained in the error. */
static struct io_plan *handle_payment_failure(struct io_conn *conn,
					      struct daemon *daemon,
					      const u8 *msg)
{
	struct node_id erring_node;
	struct short_channel_id erring_channel;
	u8 erring_channel_direction;
	u8 *error;
	enum onion_type failcode;
	u8 *channel_update;

	if (!fromwire_gossip_payment_failure(msg, msg,
					     &erring_node,
					     &erring_channel,
					     &erring_channel_direction,
					     &error))
		master_badmsg(WIRE_GOSSIP_PAYMENT_FAILURE, msg);

	failcode = fromwire_peektype(error);
	channel_update = channel_update_from_onion_error(tmpctx, error);
	if (channel_update)
		status_debug("Extracted channel_update %s from onionreply %s",
			     tal_hex(tmpctx, channel_update),
			     tal_hex(tmpctx, error));
	routing_failure(daemon->rstate,
			&erring_node,
			&erring_channel,
			erring_channel_direction,
			failcode,
			channel_update);

	return daemon_conn_read_next(conn, daemon->master);
}

/*~ This is where lightningd tells us that a channel's funding transaction has
 * been spent. */
static struct io_plan *handle_outpoint_spent(struct io_conn *conn,
					     struct daemon *daemon,
					     const u8 *msg)
{
	struct short_channel_id scid;
	struct chan *chan;
	struct routing_state *rstate = daemon->rstate;
	if (!fromwire_gossip_outpoint_spent(msg, &scid))
		master_badmsg(WIRE_GOSSIP_OUTPOINT_SPENT, msg);

	chan = get_channel(rstate, &scid);
	if (chan) {
		status_trace(
		    "Deleting channel %s due to the funding outpoint being "
		    "spent",
		    type_to_string(msg, struct short_channel_id, &scid));
		remove_channel_from_store(rstate, chan);
		/* Freeing is sufficient since everything else is allocated off
		 * of the channel and this takes care of unregistering
		 * the channel */
		free_chan(rstate, chan);
	}

	return daemon_conn_read_next(conn, daemon->master);
}

/*~ This is sent by lightningd when it kicks off 'closingd': we disable it
 * in both directions.
 *
 * We'll leave it to handle_outpoint_spent to delete the channel from our view
 * once the close gets confirmed. This avoids having strange states in which the
 * channel is list in our peer list but won't be returned when listing public
 * channels. This does not send out updates since that's triggered by the peer
 * connection closing.
 */
static struct io_plan *handle_local_channel_close(struct io_conn *conn,
						  struct daemon *daemon,
						  const u8 *msg)
{
	struct short_channel_id scid;
	struct chan *chan;
	struct routing_state *rstate = daemon->rstate;
	if (!fromwire_gossip_local_channel_close(msg, &scid))
		master_badmsg(WIRE_GOSSIP_LOCAL_CHANNEL_CLOSE, msg);

	chan = get_channel(rstate, &scid);
	if (chan)
		local_disable_chan(rstate, chan);
	return daemon_conn_read_next(conn, daemon->master);
}

/*~ This routine handles all the commands from lightningd. */
static struct io_plan *recv_req(struct io_conn *conn,
				const u8 *msg,
				struct daemon *daemon)
{
	enum gossip_wire_type t = fromwire_peektype(msg);

	switch (t) {
	case WIRE_GOSSIPCTL_INIT:
		return gossip_init(conn, daemon, msg);

	case WIRE_GOSSIP_GETNODES_REQUEST:
		return getnodes(conn, daemon, msg);

	case WIRE_GOSSIP_GETROUTE_REQUEST:
		return getroute_req(conn, daemon, msg);

	case WIRE_GOSSIP_GETCHANNELS_REQUEST:
		return getchannels_req(conn, daemon, msg);

	case WIRE_GOSSIP_GET_CHANNEL_PEER:
		return get_channel_peer(conn, daemon, msg);

	case WIRE_GOSSIP_GET_TXOUT_REPLY:
		return handle_txout_reply(conn, daemon, msg);

	case WIRE_GOSSIP_PAYMENT_FAILURE:
		return handle_payment_failure(conn, daemon, msg);

	case WIRE_GOSSIP_OUTPOINT_SPENT:
		return handle_outpoint_spent(conn, daemon, msg);

	case WIRE_GOSSIP_LOCAL_CHANNEL_CLOSE:
		return handle_local_channel_close(conn, daemon, msg);

	case WIRE_GOSSIP_PING:
		return ping_req(conn, daemon, msg);

	case WIRE_GOSSIP_GET_INCOMING_CHANNELS:
		return get_incoming_channels(conn, daemon, msg);

#if DEVELOPER
	case WIRE_GOSSIP_QUERY_SCIDS:
		return query_scids_req(conn, daemon, msg);

	case WIRE_GOSSIP_SEND_TIMESTAMP_FILTER:
		return send_timestamp_filter(conn, daemon, msg);

	case WIRE_GOSSIP_QUERY_CHANNEL_RANGE:
		return query_channel_range(conn, daemon, msg);

	case WIRE_GOSSIP_DEV_SET_MAX_SCIDS_ENCODE_SIZE:
		return dev_set_max_scids_encode_size(conn, daemon, msg);
	case WIRE_GOSSIP_DEV_SUPPRESS:
		return dev_gossip_suppress(conn, daemon, msg);
	case WIRE_GOSSIP_DEV_MEMLEAK:
		return dev_gossip_memleak(conn, daemon, msg);
	case WIRE_GOSSIP_DEV_COMPACT_STORE:
		return dev_compact_store(conn, daemon, msg);
#else
	case WIRE_GOSSIP_QUERY_SCIDS:
	case WIRE_GOSSIP_SEND_TIMESTAMP_FILTER:
	case WIRE_GOSSIP_QUERY_CHANNEL_RANGE:
	case WIRE_GOSSIP_DEV_SET_MAX_SCIDS_ENCODE_SIZE:
	case WIRE_GOSSIP_DEV_SUPPRESS:
	case WIRE_GOSSIP_DEV_MEMLEAK:
	case WIRE_GOSSIP_DEV_COMPACT_STORE:
		break;
#endif /* !DEVELOPER */

	/* We send these, we don't receive them */
	case WIRE_GOSSIP_GETNODES_REPLY:
	case WIRE_GOSSIP_GETROUTE_REPLY:
	case WIRE_GOSSIP_GETCHANNELS_REPLY:
	case WIRE_GOSSIP_PING_REPLY:
	case WIRE_GOSSIP_SCIDS_REPLY:
	case WIRE_GOSSIP_QUERY_CHANNEL_RANGE_REPLY:
	case WIRE_GOSSIP_GET_CHANNEL_PEER_REPLY:
	case WIRE_GOSSIP_GET_INCOMING_CHANNELS_REPLY:
	case WIRE_GOSSIP_GET_TXOUT:
	case WIRE_GOSSIP_DEV_MEMLEAK_REPLY:
	case WIRE_GOSSIP_DEV_COMPACT_STORE_REPLY:
		break;
	}

	/* Master shouldn't give bad requests. */
	status_failed(STATUS_FAIL_MASTER_IO, "%i: %s",
		      t, tal_hex(tmpctx, msg));
}

/* This is called when lightningd closes its connection to us.  We simply
 * exit. */
static void master_gone(struct daemon_conn *master UNUSED)
{
	daemon_shutdown();
	/* Can't tell master, it's gone. */
	exit(2);
}

int main(int argc, char *argv[])
{
	setup_locale();

	struct daemon *daemon;

	subdaemon_setup(argc, argv);

	daemon = tal(NULL, struct daemon);
	list_head_init(&daemon->peers);

	/* Note the use of time_mono() here.  That's a monotonic clock, which
	 * is really useful: it can only be used to measure relative events
	 * (there's no correspondence to time-since-Ken-grew-a-beard or
	 * anything), but unlike time_now(), this will never jump backwards by
	 * half a second and leave me wondering how my tests failed CI! */
	timers_init(&daemon->timers, time_mono());

	/* Our daemons always use STDIN for commands from lightningd. */
	daemon->master = daemon_conn_new(daemon, STDIN_FILENO,
					 recv_req, NULL, daemon);
	tal_add_destructor(daemon->master, master_gone);

	status_setup_async(daemon->master);

	/* connectd is already started, and uses this fd to ask us things. */
	daemon->connectd = daemon_conn_new(daemon, CONNECTD_FD,
					   connectd_req, NULL, daemon);

	/* This loop never exits.  io_loop() only returns if a timer has
	 * expired, or io_break() is called, or all fds are closed.  We don't
	 * use io_break and closing the lightningd fd calls master_gone()
	 * which exits. */
	for (;;) {
		struct timer *expired = NULL;
		io_loop(&daemon->timers, &expired);

		timer_expired(daemon, expired);
	}
}

/*~ Note that the actual routing stuff is in routing.c; you might want to
 * check that out later.
 *
 * But that's the last of the global daemons.   We now move on to the first of
 * the per-peer daemons: openingd/openingd.c.
 */