]> git.rkrishnan.org Git - tahoe-lafs/tahoe-lafs.git/blobdiff - src/allmydata/mutable/servermap.py
Fix some potential bugs (in non-test code) exposed by check-miscaptures.py. refs...
[tahoe-lafs/tahoe-lafs.git] / src / allmydata / mutable / servermap.py
index 37c29b3349a6f2cb607b80ce535c1b2b7942deb1..1f28cdd263a822c2f313bab354d8ddf333a3bf69 100644 (file)
@@ -4,15 +4,17 @@ from zope.interface import implements
 from itertools import count
 from twisted.internet import defer
 from twisted.python import failure
-from foolscap.eventual import eventually
-from allmydata.util import base32, hashutil, idlib, log
-from allmydata import storage
+from foolscap.api import DeadReferenceError, RemoteException, eventually, \
+                         fireEventually
+from allmydata.util import base32, hashutil, idlib, log, deferredutil
+from allmydata.util.dictutil import DictOfSets
+from allmydata.storage.server import si_b2a
 from allmydata.interfaces import IServermapUpdaterStatus
 from pycryptopp.publickey import rsa
 
-from common import MODE_CHECK, MODE_ANYTHING, MODE_WRITE, MODE_READ, \
-     DictOfSets, CorruptShareError, NeedMoreDataError
-from layout import unpack_prefix_and_signature, unpack_header, unpack_share
+from allmydata.mutable.common import MODE_CHECK, MODE_ANYTHING, MODE_WRITE, MODE_READ, \
+     CorruptShareError
+from allmydata.mutable.layout import SIGNED_PREFIX_LENGTH, MDMFSlotReadProxy
 
 class UpdateStatus:
     implements(IServermapUpdaterStatus)
@@ -100,39 +102,57 @@ class ServerMap:
 
     @ivar connections: maps peerid to a RemoteReference
 
-    @ivar bad_shares: a sequence of (peerid, shnum) tuples, describing
+    @ivar bad_shares: dict with keys of (peerid, shnum) tuples, describing
                       shares that I should ignore (because a previous user of
                       the servermap determined that they were invalid). The
                       updater only locates a certain number of shares: if
                       some of these turn out to have integrity problems and
                       are unusable, the caller will need to mark those shares
                       as bad, then re-update the servermap, then try again.
+                      The dict maps (peerid, shnum) tuple to old checkstring.
     """
 
     def __init__(self):
         self.servermap = {}
         self.connections = {}
         self.unreachable_peers = set() # peerids that didn't respond to queries
+        self.reachable_peers = set() # peerids that did respond to queries
         self.problems = [] # mostly for debugging
-        self.bad_shares = set()
+        self.bad_shares = {} # maps (peerid,shnum) to old checkstring
         self.last_update_mode = None
         self.last_update_time = 0
-
-    def mark_bad_share(self, peerid, shnum):
-        """This share was found to be bad, not in the checkstring or
-        signature, but deeper in the share, detected at retrieve time. Remove
-        it from our list of useful shares, and remember that it is bad so we
-        don't add it back again later.
+        self.update_data = {} # (verinfo,shnum) => data
+
+    def copy(self):
+        s = ServerMap()
+        s.servermap = self.servermap.copy() # tuple->tuple
+        s.connections = self.connections.copy() # str->RemoteReference
+        s.unreachable_peers = set(self.unreachable_peers)
+        s.reachable_peers = set(self.reachable_peers)
+        s.problems = self.problems[:]
+        s.bad_shares = self.bad_shares.copy() # tuple->str
+        s.last_update_mode = self.last_update_mode
+        s.last_update_time = self.last_update_time
+        return s
+
+    def mark_bad_share(self, peerid, shnum, checkstring):
+        """This share was found to be bad, either in the checkstring or
+        signature (detected during mapupdate), or deeper in the share
+        (detected at retrieve time). Remove it from our list of useful
+        shares, and remember that it is bad so we don't add it back again
+        later. We record the share's old checkstring (which might be
+        corrupted or badly signed) so that a repair operation can do the
+        test-and-set using it as a reference.
         """
-        key = (peerid, shnum)
-        self.bad_shares.add(key)
+        key = (peerid, shnum) # record checkstring
+        self.bad_shares[key] = checkstring
         self.servermap.pop(key, None)
 
     def add_new_share(self, peerid, shnum, verinfo, timestamp):
         """We've written a new share out, replacing any that was there
         before."""
         key = (peerid, shnum)
-        self.bad_shares.discard(key)
+        self.bad_shares.pop(key, None)
         self.servermap[key] = (verinfo, timestamp)
 
     def dump(self, out=sys.stdout):
@@ -156,6 +176,13 @@ class ServerMap:
                     for (peerid, shnum)
                     in self.servermap])
 
+    def all_peers_for_version(self, verinfo):
+        """Return a set of peerids that hold shares for the given version."""
+        return set([peerid
+                    for ( (peerid, shnum), (verinfo2, timestamp) )
+                    in self.servermap.items()
+                    if verinfo == verinfo2])
+
     def make_sharemap(self):
         """Return a dict that maps shnum to a set of peerds that hold it."""
         sharemap = DictOfSets()
@@ -205,23 +232,26 @@ class ServerMap:
         seqnums.append(0)
         return max(seqnums)
 
+    def summarize_version(self, verinfo):
+        """Take a versionid, return a string that describes it."""
+        (seqnum, root_hash, IV, segsize, datalength, k, N, prefix,
+         offsets_tuple) = verinfo
+        return "seq%d-%s" % (seqnum, base32.b2a(root_hash)[:4])
+
     def summarize_versions(self):
         """Return a string describing which versions we know about."""
         versionmap = self.make_versionmap()
         bits = []
         for (verinfo, shares) in versionmap.items():
-            (seqnum, root_hash, IV, segsize, datalength, k, N, prefix,
-             offsets_tuple) = verinfo
+            vstr = self.summarize_version(verinfo)
             shnums = set([shnum for (shnum, peerid, timestamp) in shares])
-            bits.append("%d*seq%d-%s" %
-                        (len(shnums), seqnum, base32.b2a(root_hash)[:4]))
+            bits.append("%d*%s" % (len(shnums), vstr))
         return "/".join(bits)
 
     def recoverable_versions(self):
         """Return a set of versionids, one for each version that is currently
         recoverable."""
         versionmap = self.make_versionmap()
-
         recoverable_versions = set()
         for (verinfo, shares) in versionmap.items():
             (seqnum, root_hash, IV, segsize, datalength, k, N, prefix,
@@ -299,19 +329,44 @@ class ServerMap:
         # same seqnum, meaning that MutableFileNode.read_best_version is not
         # giving you the whole story, and that using its data to do a
         # subsequent publish will lose information.
-        return bool(len(self.recoverable_versions()) > 1)
+        recoverable_seqnums = [verinfo[0]
+                               for verinfo in self.recoverable_versions()]
+        for seqnum in recoverable_seqnums:
+            if recoverable_seqnums.count(seqnum) > 1:
+                return True
+        return False
+
+
+    def get_update_data_for_share_and_verinfo(self, shnum, verinfo):
+        """
+        I return the update data for the given shnum
+        """
+        update_data = self.update_data[shnum]
+        update_datum = [i[1] for i in update_data if i[0] == verinfo][0]
+        return update_datum
+
+
+    def set_update_data_for_share_and_verinfo(self, shnum, verinfo, data):
+        """
+        I record the block hash tree for the given shnum.
+        """
+        self.update_data.setdefault(shnum , []).append((verinfo, data))
 
 
 class ServermapUpdater:
-    def __init__(self, filenode, servermap, mode=MODE_READ):
+    def __init__(self, filenode, storage_broker, monitor, servermap,
+                 mode=MODE_READ, add_lease=False, update_range=None):
         """I update a servermap, locating a sufficient number of useful
         shares and remembering where they are located.
 
         """
 
         self._node = filenode
+        self._storage_broker = storage_broker
+        self._monitor = monitor
         self._servermap = servermap
         self.mode = mode
+        self._add_lease = add_lease
         self._running = True
 
         self._storage_index = filenode.get_storage_index()
@@ -322,7 +377,10 @@ class ServermapUpdater:
         self._status.set_progress(0.0)
         self._status.set_mode(mode)
 
+        self._servers_responded = set()
+
         # how much data should we read?
+        # SDMF:
         #  * if we only need the checkstring, then [0:75]
         #  * if we need to validate the checkstring sig, then [543ish:799ish]
         #  * if we need the verification key, then [107:436ish]
@@ -330,19 +388,39 @@ class ServermapUpdater:
         #  * if we need the encrypted private key, we want [-1216ish:]
         #   * but we can't read from negative offsets
         #   * the offset table tells us the 'ish', also the positive offset
-        # A future version of the SMDF slot format should consider using
-        # fixed-size slots so we can retrieve less data. For now, we'll just
-        # read 2000 bytes, which also happens to read enough actual data to
-        # pre-fetch a 9-entry dirnode.
-        self._read_size = 2000
+        # MDMF:
+        #  * Checkstring? [0:72]
+        #  * If we want to validate the checkstring, then [0:72], [143:?] --
+        #    the offset table will tell us for sure.
+        #  * If we need the verification key, we have to consult the offset
+        #    table as well.
+        # At this point, we don't know which we are. Our filenode can
+        # tell us, but it might be lying -- in some cases, we're
+        # responsible for telling it which kind of file it is.
+        self._read_size = 4000
         if mode == MODE_CHECK:
             # we use unpack_prefix_and_signature, so we need 1k
             self._read_size = 1000
         self._need_privkey = False
-        if mode == MODE_WRITE and not self._node._privkey:
-            self._need_privkey = True
 
-        prefix = storage.si_b2a(self._storage_index)[:5]
+        if mode == MODE_WRITE and not self._node.get_privkey():
+            self._need_privkey = True
+        # check+repair: repair requires the privkey, so if we didn't happen
+        # to ask for it during the check, we'll have problems doing the
+        # publish.
+
+        self.fetch_update_data = False
+        if mode == MODE_WRITE and update_range:
+            # We're updating the servermap in preparation for an
+            # in-place file update, so we need to fetch some additional
+            # data from each share that we find.
+            assert len(update_range) == 2
+
+            self.start_segment = update_range[0]
+            self.end_segment = update_range[1]
+            self.fetch_update_data = True
+
+        prefix = si_b2a(self._storage_index)[:5]
         self._log_number = log.msg(format="SharemapUpdater(%(si)s): starting (%(mode)s)",
                                    si=prefix, mode=mode)
 
@@ -352,6 +430,8 @@ class ServermapUpdater:
     def log(self, *args, **kwargs):
         if "parent" not in kwargs:
             kwargs["parent"] = self._log_number
+        if "facility" not in kwargs:
+            kwargs["facility"] = "tahoe.mutable.mapupdate"
         return log.msg(*args, **kwargs)
 
     def update(self):
@@ -377,20 +457,24 @@ class ServermapUpdater:
 
         self._queries_completed = 0
 
-        client = self._node._client
-        full_peerlist = client.get_permuted_peers("storage",
-                                                  self._node._storage_index)
+        sb = self._storage_broker
+        # All of the peers, permuted by the storage index, as usual.
+        full_peerlist = [(s.get_serverid(), s.get_rref())
+                         for s in sb.get_servers_for_psi(self._storage_index)]
         self.full_peerlist = full_peerlist # for use later, immutable
         self.extra_peers = full_peerlist[:] # peers are removed as we use them
         self._good_peers = set() # peers who had some shares
         self._empty_peers = set() # peers who don't have any shares
         self._bad_peers = set() # peers to whom our queries failed
+        self._readers = {} # peerid -> dict(sharewriters), filled in
+                           # after responses come in.
 
         k = self._node.get_required_shares()
+        # For what cases can these conditions work?
         if k is None:
             # make a guess
             k = 3
-        N = self._node.get_required_shares()
+        N = self._node.get_total_shares()
         if N is None:
             N = 10
         self.EPSILON = k
@@ -399,6 +483,7 @@ class ServermapUpdater:
         self.num_peers_to_query = k + self.EPSILON
 
         if self.mode == MODE_CHECK:
+            # We want to query all of the peers.
             initial_peers_to_query = dict(full_peerlist)
             must_query = set(initial_peers_to_query.keys())
             self.extra_peers = []
@@ -406,6 +491,7 @@ class ServermapUpdater:
             # we're planning to replace all the shares, so we want a good
             # chance of finding them all. We will keep searching until we've
             # seen epsilon that don't have a share.
+            # We don't query all of the peers because that could take a while.
             self.num_peers_to_query = N + self.EPSILON
             initial_peers_to_query, must_query = self._build_initial_querylist()
             self.required_num_empty_peers = self.EPSILON
@@ -415,7 +501,8 @@ class ServermapUpdater:
             # might also avoid the round trip required to read the encrypted
             # private key.
 
-        else:
+        else: # MODE_READ, MODE_ANYTHING
+            # 2k peers is good enough.
             initial_peers_to_query, must_query = self._build_initial_querylist()
 
         # this is a set of peers that we are required to get responses from:
@@ -430,6 +517,9 @@ class ServermapUpdater:
         # before we can consider ourselves finished, and self.extra_peers
         # contains the overflow (peers that we should tap if we don't get
         # enough responses)
+        # I guess that self._must_query is a subset of
+        # initial_peers_to_query?
+        assert set(must_query).issubset(set(initial_peers_to_query))
 
         self._send_initial_requests(initial_peers_to_query)
         self._status.timings["initial_queries"] = time.time() - self._started
@@ -456,7 +546,6 @@ class ServermapUpdater:
         self._status.set_status("Sending %d initial queries" % len(peerlist))
         self._queries_outstanding = set()
         self._sharemap = DictOfSets() # shnum -> [(peerid, seqnum, R)..]
-        dl = []
         for (peerid, ss) in peerlist.items():
             self._queries_outstanding.add(peerid)
             self._do_query(ss, peerid, self._storage_index, self._read_size)
@@ -487,26 +576,79 @@ class ServermapUpdater:
         # errors that aren't handled by _query_failed (and errors caused by
         # _query_failed) get logged, but we still want to check for doneness.
         d.addErrback(log.err)
-        d.addBoth(self._check_for_done)
         d.addErrback(self._fatal_error)
+        d.addCallback(self._check_for_done)
         return d
 
     def _do_read(self, ss, peerid, storage_index, shnums, readv):
+        if self._add_lease:
+            # send an add-lease message in parallel. The results are handled
+            # separately. This is sent before the slot_readv() so that we can
+            # be sure the add_lease is retired by the time slot_readv comes
+            # back (this relies upon our knowledge that the server code for
+            # add_lease is synchronous).
+            renew_secret = self._node.get_renewal_secret(peerid)
+            cancel_secret = self._node.get_cancel_secret(peerid)
+            d2 = ss.callRemote("add_lease", storage_index,
+                               renew_secret, cancel_secret)
+            # we ignore success
+            d2.addErrback(self._add_lease_failed, peerid, storage_index)
         d = ss.callRemote("slot_readv", storage_index, shnums, readv)
         return d
 
+
+    def _got_corrupt_share(self, e, shnum, peerid, data, lp):
+        """
+        I am called when a remote server returns a corrupt share in
+        response to one of our queries. By corrupt, I mean a share
+        without a valid signature. I then record the failure, notify the
+        server of the corruption, and record the share as bad.
+        """
+        f = failure.Failure(e)
+        self.log(format="bad share: %(f_value)s", f_value=str(f),
+                 failure=f, parent=lp, level=log.WEIRD, umid="h5llHg")
+        # Notify the server that its share is corrupt.
+        self.notify_server_corruption(peerid, shnum, str(e))
+        # By flagging this as a bad peer, we won't count any of
+        # the other shares on that peer as valid, though if we
+        # happen to find a valid version string amongst those
+        # shares, we'll keep track of it so that we don't need
+        # to validate the signature on those again.
+        self._bad_peers.add(peerid)
+        self._last_failure = f
+        # XXX: Use the reader for this?
+        checkstring = data[:SIGNED_PREFIX_LENGTH]
+        self._servermap.mark_bad_share(peerid, shnum, checkstring)
+        self._servermap.problems.append(f)
+
+
+    def _cache_good_sharedata(self, verinfo, shnum, now, data):
+        """
+        If one of my queries returns successfully (which means that we
+        were able to and successfully did validate the signature), I
+        cache the data that we initially fetched from the storage
+        server. This will help reduce the number of roundtrips that need
+        to occur when the file is downloaded, or when the file is
+        updated.
+        """
+        if verinfo:
+            self._node._add_to_cache(verinfo, shnum, 0, data)
+
+
     def _got_results(self, datavs, peerid, readsize, stuff, started):
         lp = self.log(format="got result from [%(peerid)s], %(numshares)d shares",
                       peerid=idlib.shortnodeid_b2a(peerid),
-                      numshares=len(datavs),
-                      level=log.NOISY)
+                      numshares=len(datavs))
         now = time.time()
         elapsed = now - started
-        self._queries_outstanding.discard(peerid)
-        self._must_query.discard(peerid)
-        self._queries_completed += 1
+        def _done_processing(ignored=None):
+            self._queries_outstanding.discard(peerid)
+            self._servermap.reachable_peers.add(peerid)
+            self._must_query.discard(peerid)
+            self._queries_completed += 1
         if not self._running:
             self.log("but we're not running, so we'll ignore it", parent=lp)
+            _done_processing()
             self._status.add_per_server_time(peerid, "late", started, elapsed)
             return
         self._status.add_per_server_time(peerid, "query", started, elapsed)
@@ -516,98 +658,206 @@ class ServermapUpdater:
         else:
             self._empty_peers.add(peerid)
 
-        last_verinfo = None
-        last_shnum = None
+        ss, storage_index = stuff
+        ds = []
+
         for shnum,datav in datavs.items():
             data = datav[0]
-            try:
-                verinfo = self._got_results_one_share(shnum, data, peerid, lp)
-                last_verinfo = verinfo
-                last_shnum = shnum
-                self._node._cache.add(verinfo, shnum, 0, data, now)
-            except CorruptShareError, e:
-                # log it and give the other shares a chance to be processed
-                f = failure.Failure()
-                self.log("bad share: %s %s" % (f, f.value),
-                         parent=lp, level=log.WEIRD)
-                self._bad_peers.add(peerid)
-                self._last_failure = f
-                self._servermap.problems.append(f)
-                pass
-
-        self._status.timings["cumulative_verify"] += (time.time() - now)
-
-        if self._need_privkey and last_verinfo:
-            # send them a request for the privkey. We send one request per
-            # server.
-            lp2 = self.log("sending privkey request",
-                           parent=lp, level=log.NOISY)
-            (seqnum, root_hash, IV, segsize, datalength, k, N, prefix,
-             offsets_tuple) = last_verinfo
-            o = dict(offsets_tuple)
+            reader = MDMFSlotReadProxy(ss,
+                                       storage_index,
+                                       shnum,
+                                       data)
+            self._readers.setdefault(peerid, dict())[shnum] = reader
+            # our goal, with each response, is to validate the version
+            # information and share data as best we can at this point --
+            # we do this by validating the signature. To do this, we
+            # need to do the following:
+            #   - If we don't already have the public key, fetch the
+            #     public key. We use this to validate the signature.
+            if not self._node.get_pubkey():
+                # fetch and set the public key.
+                d = reader.get_verification_key()
+                d.addCallback(lambda results, shnum=shnum, peerid=peerid:
+                    self._try_to_set_pubkey(results, peerid, shnum, lp))
+                # XXX: Make self._pubkey_query_failed?
+                d.addErrback(lambda error, shnum=shnum, peerid=peerid, data=data:
+                    self._got_corrupt_share(error, shnum, peerid, data, lp))
+            else:
+                # we already have the public key.
+                d = defer.succeed(None)
+
+            # Neither of these two branches return anything of
+            # consequence, so the first entry in our deferredlist will
+            # be None.
+
+            # - Next, we need the version information. We almost
+            #   certainly got this by reading the first thousand or so
+            #   bytes of the share on the storage server, so we
+            #   shouldn't need to fetch anything at this step.
+            d2 = reader.get_verinfo()
+            d2.addErrback(lambda error, shnum=shnum, peerid=peerid, data=data:
+                self._got_corrupt_share(error, shnum, peerid, data, lp))
+            # - Next, we need the signature. For an SDMF share, it is
+            #   likely that we fetched this when doing our initial fetch
+            #   to get the version information. In MDMF, this lives at
+            #   the end of the share, so unless the file is quite small,
+            #   we'll need to do a remote fetch to get it.
+            d3 = reader.get_signature()
+            d3.addErrback(lambda error, shnum=shnum, peerid=peerid, data=data:
+                self._got_corrupt_share(error, shnum, peerid, data, lp))
+            #  Once we have all three of these responses, we can move on
+            #  to validating the signature
+
+            # Does the node already have a privkey? If not, we'll try to
+            # fetch it here.
+            if self._need_privkey:
+                d4 = reader.get_encprivkey()
+                d4.addCallback(lambda results, shnum=shnum, peerid=peerid:
+                    self._try_to_validate_privkey(results, peerid, shnum, lp))
+                d4.addErrback(lambda error, shnum=shnum, peerid=peerid, data=data:
+                    self._privkey_query_failed(error, shnum, data, lp))
+            else:
+                d4 = defer.succeed(None)
+
+
+            if self.fetch_update_data:
+                # fetch the block hash tree and first + last segment, as
+                # configured earlier.
+                # Then set them in wherever we happen to want to set
+                # them.
+                ds = []
+                # XXX: We do this above, too. Is there a good way to
+                # make the two routines share the value without
+                # introducing more roundtrips?
+                ds.append(reader.get_verinfo())
+                ds.append(reader.get_blockhashes())
+                ds.append(reader.get_block_and_salt(self.start_segment))
+                ds.append(reader.get_block_and_salt(self.end_segment))
+                d5 = deferredutil.gatherResults(ds)
+                d5.addCallback(self._got_update_results_one_share, shnum)
+            else:
+                d5 = defer.succeed(None)
+
+            dl = defer.DeferredList([d, d2, d3, d4, d5])
+            dl.addBoth(self._turn_barrier)
+            dl.addCallback(lambda results, shnum=shnum, peerid=peerid:
+                self._got_signature_one_share(results, shnum, peerid, lp))
+            dl.addErrback(lambda error, shnum=shnum, data=data:
+               self._got_corrupt_share(error, shnum, peerid, data, lp))
+            dl.addCallback(lambda verinfo, shnum=shnum, peerid=peerid, data=data:
+                self._cache_good_sharedata(verinfo, shnum, now, data))
+            ds.append(dl)
+        # dl is a deferred list that will fire when all of the shares
+        # that we found on this peer are done processing. When dl fires,
+        # we know that processing is done, so we can decrement the
+        # semaphore-like thing that we incremented earlier.
+        dl = defer.DeferredList(ds, fireOnOneErrback=True)
+        # Are we done? Done means that there are no more queries to
+        # send, that there are no outstanding queries, and that we
+        # haven't received any queries that are still processing. If we
+        # are done, self._check_for_done will cause the done deferred
+        # that we returned to our caller to fire, which tells them that
+        # they have a complete servermap, and that we won't be touching
+        # the servermap anymore.
+        dl.addCallback(_done_processing)
+        dl.addCallback(self._check_for_done)
+        dl.addErrback(self._fatal_error)
+        # all done!
+        self.log("_got_results done", parent=lp, level=log.NOISY)
+        return dl
 
-            self._queries_outstanding.add(peerid)
-            readv = [ (o['enc_privkey'], (o['EOF'] - o['enc_privkey'])) ]
-            ss = self._servermap.connections[peerid]
-            privkey_started = time.time()
-            d = self._do_read(ss, peerid, self._storage_index,
-                              [last_shnum], readv)
-            d.addCallback(self._got_privkey_results, peerid, last_shnum,
-                          privkey_started, lp2)
-            d.addErrback(self._privkey_query_failed, peerid, last_shnum, lp2)
-            d.addErrback(log.err)
-            d.addCallback(self._check_for_done)
-            d.addErrback(self._fatal_error)
 
-        # all done!
-        self.log("_got_results done", parent=lp)
+    def _turn_barrier(self, result):
+        """
+        I help the servermap updater avoid the recursion limit issues
+        discussed in #237.
+        """
+        return fireEventually(result)
 
-    def _got_results_one_share(self, shnum, data, peerid, lp):
-        self.log(format="_got_results: got shnum #%(shnum)d from peerid %(peerid)s",
-                 shnum=shnum,
-                 peerid=idlib.shortnodeid_b2a(peerid),
-                 parent=lp)
 
-        # this might raise NeedMoreDataError, if the pubkey and signature
-        # live at some weird offset. That shouldn't happen, so I'm going to
-        # treat it as a bad share.
-        (seqnum, root_hash, IV, k, N, segsize, datalength,
-         pubkey_s, signature, prefix) = unpack_prefix_and_signature(data)
+    def _try_to_set_pubkey(self, pubkey_s, peerid, shnum, lp):
+        if self._node.get_pubkey():
+            return # don't go through this again if we don't have to
+        fingerprint = hashutil.ssk_pubkey_fingerprint_hash(pubkey_s)
+        assert len(fingerprint) == 32
+        if fingerprint != self._node.get_fingerprint():
+            raise CorruptShareError(peerid, shnum,
+                                "pubkey doesn't match fingerprint")
+        self._node._populate_pubkey(self._deserialize_pubkey(pubkey_s))
+        assert self._node.get_pubkey()
 
-        if not self._node.get_pubkey():
-            fingerprint = hashutil.ssk_pubkey_fingerprint_hash(pubkey_s)
-            assert len(fingerprint) == 32
-            if fingerprint != self._node._fingerprint:
-                raise CorruptShareError(peerid, shnum,
-                                        "pubkey doesn't match fingerprint")
-            self._node._populate_pubkey(self._deserialize_pubkey(pubkey_s))
 
-        if self._need_privkey:
-            self._try_to_extract_privkey(data, peerid, shnum, lp)
+    def notify_server_corruption(self, peerid, shnum, reason):
+        ss = self._servermap.connections[peerid]
+        ss.callRemoteOnly("advise_corrupt_share",
+                          "mutable", self._storage_index, shnum, reason)
+
 
-        (ig_version, ig_seqnum, ig_root_hash, ig_IV, ig_k, ig_N,
-         ig_segsize, ig_datalen, offsets) = unpack_header(data)
+    def _got_signature_one_share(self, results, shnum, peerid, lp):
+        # It is our job to give versioninfo to our caller. We need to
+        # raise CorruptShareError if the share is corrupt for any
+        # reason, something that our caller will handle.
+        self.log(format="_got_results: got shnum #%(shnum)d from peerid %(peerid)s",
+                 shnum=shnum,
+                 peerid=idlib.shortnodeid_b2a(peerid),
+                 level=log.NOISY,
+                 parent=lp)
+        if not self._running:
+            # We can't process the results, since we can't touch the
+            # servermap anymore.
+            self.log("but we're not running anymore.")
+            return None
+
+        _, verinfo, signature, __, ___ = results
+        (seqnum,
+         root_hash,
+         saltish,
+         segsize,
+         datalen,
+         k,
+         n,
+         prefix,
+         offsets) = verinfo[1]
         offsets_tuple = tuple( [(key,value) for key,value in offsets.items()] )
 
-        verinfo = (seqnum, root_hash, IV, segsize, datalength, k, N, prefix,
+        # XXX: This should be done for us in the method, so
+        # presumably you can go in there and fix it.
+        verinfo = (seqnum,
+                   root_hash,
+                   saltish,
+                   segsize,
+                   datalen,
+                   k,
+                   n,
+                   prefix,
                    offsets_tuple)
+        # This tuple uniquely identifies a share on the grid; we use it
+        # to keep track of the ones that we've already seen.
 
         if verinfo not in self._valid_versions:
-            # it's a new pair. Verify the signature.
-            valid = self._node._pubkey.verify(prefix, signature)
+            # This is a new version tuple, and we need to validate it
+            # against the public key before keeping track of it.
+            assert self._node.get_pubkey()
+            valid = self._node.get_pubkey().verify(prefix, signature[1])
             if not valid:
-                raise CorruptShareError(peerid, shnum, "signature is invalid")
-
-            # ok, it's a valid verinfo. Add it to the list of validated
-            # versions.
-            self.log(" found valid version %d-%s from %s-sh%d: %d-%d/%d/%d"
-                     % (seqnum, base32.b2a(root_hash)[:4],
-                        idlib.shortnodeid_b2a(peerid), shnum,
-                        k, N, segsize, datalength),
-                     parent=lp)
-            self._valid_versions.add(verinfo)
-        # We now know that this is a valid candidate verinfo.
-
+                raise CorruptShareError(peerid, shnum,
+                                        "signature is invalid")
+
+        # ok, it's a valid verinfo. Add it to the list of validated
+        # versions.
+        self.log(" found valid version %d-%s from %s-sh%d: %d-%d/%d/%d"
+                 % (seqnum, base32.b2a(root_hash)[:4],
+                    idlib.shortnodeid_b2a(peerid), shnum,
+                    k, n, segsize, datalen),
+                    parent=lp)
+        self._valid_versions.add(verinfo)
+        # We now know that this is a valid candidate verinfo. Whether or
+        # not this instance of it is valid is a matter for the next
+        # statement; at this point, we just know that if we see this
+        # version info again, that its signature checks out and that
+        # we're okay to skip the signature-checking step.
+
+        # (peerid, shnum) are bound in the method invocation.
         if (peerid, shnum) in self._servermap.bad_shares:
             # we've been told that the rest of the data in this share is
             # unusable, so don't add it to the servermap.
@@ -620,49 +870,62 @@ class ServermapUpdater:
         self._servermap.add_new_share(peerid, shnum, verinfo, timestamp)
         # and the versionmap
         self.versionmap.add(verinfo, (shnum, peerid, timestamp))
+
         return verinfo
 
-    def _deserialize_pubkey(self, pubkey_s):
-        verifier = rsa.create_verifying_key_from_string(pubkey_s)
-        return verifier
 
-    def _try_to_extract_privkey(self, data, peerid, shnum, lp):
-        try:
-            r = unpack_share(data)
-        except NeedMoreDataError, e:
-            # this share won't help us. oh well.
-            offset = e.encprivkey_offset
-            length = e.encprivkey_length
-            self.log("shnum %d on peerid %s: share was too short (%dB) "
-                     "to get the encprivkey; [%d:%d] ought to hold it" %
-                     (shnum, idlib.shortnodeid_b2a(peerid), len(data),
-                      offset, offset+length),
-                     parent=lp)
-            # NOTE: if uncoordinated writes are taking place, someone might
-            # change the share (and most probably move the encprivkey) before
-            # we get a chance to do one of these reads and fetch it. This
-            # will cause us to see a NotEnoughSharesError(unable to fetch
-            # privkey) instead of an UncoordinatedWriteError . This is a
-            # nuisance, but it will go away when we move to DSA-based mutable
-            # files (since the privkey will be small enough to fit in the
-            # write cap).
+    def _got_update_results_one_share(self, results, share):
+        """
+        I record the update results in results.
+        """
+        assert len(results) == 4
+        verinfo, blockhashes, start, end = results
+        (seqnum,
+         root_hash,
+         saltish,
+         segsize,
+         datalen,
+         k,
+         n,
+         prefix,
+         offsets) = verinfo
+        offsets_tuple = tuple( [(key,value) for key,value in offsets.items()] )
 
-            return
+        # XXX: This should be done for us in the method, so
+        # presumably you can go in there and fix it.
+        verinfo = (seqnum,
+                   root_hash,
+                   saltish,
+                   segsize,
+                   datalen,
+                   k,
+                   n,
+                   prefix,
+                   offsets_tuple)
 
-        (seqnum, root_hash, IV, k, N, segsize, datalen,
-         pubkey, signature, share_hash_chain, block_hash_tree,
-         share_data, enc_privkey) = r
+        update_data = (blockhashes, start, end)
+        self._servermap.set_update_data_for_share_and_verinfo(share,
+                                                              verinfo,
+                                                              update_data)
 
-        return self._try_to_validate_privkey(enc_privkey, peerid, shnum, lp)
 
-    def _try_to_validate_privkey(self, enc_privkey, peerid, shnum, lp):
+    def _deserialize_pubkey(self, pubkey_s):
+        verifier = rsa.create_verifying_key_from_string(pubkey_s)
+        return verifier
+
 
+    def _try_to_validate_privkey(self, enc_privkey, peerid, shnum, lp):
+        """
+        Given a writekey from a remote server, I validate it against the
+        writekey stored in my node. If it is valid, then I set the
+        privkey and encprivkey properties of the node.
+        """
         alleged_privkey_s = self._node._decrypt_privkey(enc_privkey)
         alleged_writekey = hashutil.ssk_writekey_hash(alleged_privkey_s)
         if alleged_writekey != self._node.get_writekey():
             self.log("invalid privkey from %s shnum %d" %
                      (idlib.nodeid_b2a(peerid)[:8], shnum),
-                     parent=lp, level=log.WEIRD)
+                     parent=lp, level=log.WEIRD, umid="aJVccw")
             return
 
         # it's good
@@ -676,48 +939,79 @@ class ServermapUpdater:
         self._status.set_privkey_from(peerid)
 
 
+    def _add_lease_failed(self, f, peerid, storage_index):
+        # Older versions of Tahoe didn't handle the add-lease message very
+        # well: <=1.1.0 throws a NameError because it doesn't implement
+        # remote_add_lease(), 1.2.0/1.3.0 throw IndexError on unknown buckets
+        # (which is most of them, since we send add-lease to everybody,
+        # before we know whether or not they have any shares for us), and
+        # 1.2.0 throws KeyError even on known buckets due to an internal bug
+        # in the latency-measuring code.
+
+        # we want to ignore the known-harmless errors and log the others. In
+        # particular we want to log any local errors caused by coding
+        # problems.
+
+        if f.check(DeadReferenceError):
+            return
+        if f.check(RemoteException):
+            if f.value.failure.check(KeyError, IndexError, NameError):
+                # this may ignore a bit too much, but that only hurts us
+                # during debugging
+                return
+            self.log(format="error in add_lease from [%(peerid)s]: %(f_value)s",
+                     peerid=idlib.shortnodeid_b2a(peerid),
+                     f_value=str(f.value),
+                     failure=f,
+                     level=log.WEIRD, umid="iqg3mw")
+            return
+        # local errors are cause for alarm
+        log.err(f,
+                format="local error in add_lease to [%(peerid)s]: %(f_value)s",
+                peerid=idlib.shortnodeid_b2a(peerid),
+                f_value=str(f.value),
+                level=log.WEIRD, umid="ZWh6HA")
+
     def _query_failed(self, f, peerid):
-        self.log("error during query: %s %s" % (f, f.value), level=log.WEIRD)
         if not self._running:
             return
+        level = log.WEIRD
+        if f.check(DeadReferenceError):
+            level = log.UNUSUAL
+        self.log(format="error during query: %(f_value)s",
+                 f_value=str(f.value), failure=f,
+                 level=level, umid="IHXuQg")
         self._must_query.discard(peerid)
         self._queries_outstanding.discard(peerid)
         self._bad_peers.add(peerid)
         self._servermap.problems.append(f)
-        self._servermap.unreachable_peers.add(peerid) # TODO: overkill?
+        # a peerid could be in both ServerMap.reachable_peers and
+        # .unreachable_peers if they responded to our query, but then an
+        # exception was raised in _got_results.
+        self._servermap.unreachable_peers.add(peerid)
         self._queries_completed += 1
         self._last_failure = f
 
-    def _got_privkey_results(self, datavs, peerid, shnum, started, lp):
-        now = time.time()
-        elapsed = now - started
-        self._status.add_per_server_time(peerid, "privkey", started, elapsed)
-        self._queries_outstanding.discard(peerid)
-        if not self._need_privkey:
-            return
-        if shnum not in datavs:
-            self.log("privkey wasn't there when we asked it", level=log.WEIRD)
-            return
-        datav = datavs[shnum]
-        enc_privkey = datav[0]
-        self._try_to_validate_privkey(enc_privkey, peerid, shnum, lp)
 
     def _privkey_query_failed(self, f, peerid, shnum, lp):
         self._queries_outstanding.discard(peerid)
-        self.log("error during privkey query: %s %s" % (f, f.value),
-                 parent=lp, level=log.WEIRD)
         if not self._running:
             return
-        self._queries_outstanding.discard(peerid)
+        level = log.WEIRD
+        if f.check(DeadReferenceError):
+            level = log.UNUSUAL
+        self.log(format="error during privkey query: %(f_value)s",
+                 f_value=str(f.value), failure=f,
+                 parent=lp, level=level, umid="McoJ5w")
         self._servermap.problems.append(f)
         self._last_failure = f
 
+
     def _check_for_done(self, res):
         # exit paths:
         #  return self._send_more_queries(outstanding) : send some more queries
         #  return self._done() : all done
         #  return : keep waiting, no new queries
-
         lp = self.log(format=("_check_for_done, mode is '%(mode)s', "
                               "%(outstanding)d queries outstanding, "
                               "%(extra)d extra peers available, "
@@ -741,7 +1035,7 @@ class ServermapUpdater:
             # a share, so we must continue to wait. No additional queries are
             # required at this time.
             self.log("%d 'must query' peers left" % len(self._must_query),
-                     parent=lp)
+                     level=log.NOISY, parent=lp)
             return
 
         if (not self._queries_outstanding and not self.extra_peers):
@@ -779,11 +1073,11 @@ class ServermapUpdater:
                 self.log(format="%(completed)d completed, %(query)d to query: need more",
                          completed=self._queries_completed,
                          query=self.num_peers_to_query,
-                         parent=lp)
+                         level=log.NOISY, parent=lp)
                 return self._send_more_queries(MAX_IN_FLIGHT)
             if not recoverable_versions:
                 self.log("no recoverable versions: need more",
-                         parent=lp)
+                         level=log.NOISY, parent=lp)
                 return self._send_more_queries(MAX_IN_FLIGHT)
             highest_recoverable = max(recoverable_versions)
             highest_recoverable_seqnum = highest_recoverable[0]
@@ -793,7 +1087,8 @@ class ServermapUpdater:
                     # don't yet see enough shares to recover it. Try harder.
                     # TODO: consider sending more queries.
                     # TODO: consider limiting the search distance
-                    self.log("evidence of higher seqnum: need more")
+                    self.log("evidence of higher seqnum: need more",
+                             level=log.UNUSUAL, parent=lp)
                     return self._send_more_queries(MAX_IN_FLIGHT)
             # all the unrecoverable versions were old or concurrent with a
             # recoverable version. Good enough.
@@ -807,7 +1102,8 @@ class ServermapUpdater:
             # every server in the world.
 
             if not recoverable_versions:
-                self.log("no recoverable versions: need more", parent=lp)
+                self.log("no recoverable versions: need more", parent=lp,
+                         level=log.NOISY)
                 return self._send_more_queries(MAX_IN_FLIGHT)
 
             last_found = -1
@@ -831,7 +1127,7 @@ class ServermapUpdater:
                         if num_not_found >= self.EPSILON:
                             self.log("found our boundary, %s" %
                                      "".join(states),
-                                     parent=lp)
+                                     parent=lp, level=log.NOISY)
                             found_boundary = True
                             break
 
@@ -854,11 +1150,11 @@ class ServermapUpdater:
                 if last_not_responded == -1:
                     # we're done
                     self.log("have all our answers",
-                             parent=lp)
+                             parent=lp, level=log.NOISY)
                     # .. unless we're still waiting on the privkey
                     if self._need_privkey:
                         self.log("but we're still waiting for the privkey",
-                                 parent=lp)
+                                 parent=lp, level=log.NOISY)
                         # if we found the boundary but we haven't yet found
                         # the privkey, we may need to look further. If
                         # somehow all the privkeys were corrupted (but the
@@ -871,13 +1167,14 @@ class ServermapUpdater:
 
             # if we hit here, we didn't find our boundary, so we're still
             # waiting for peers
-            self.log("no boundary yet, %s" % "".join(states), parent=lp)
+            self.log("no boundary yet, %s" % "".join(states), parent=lp,
+                     level=log.NOISY)
             return self._send_more_queries(MAX_IN_FLIGHT)
 
         # otherwise, keep up to 5 queries in flight. TODO: this is pretty
         # arbitrary, really I want this to be something like k -
         # max(known_version_sharecounts) + some extra
-        self.log("catchall: need more", parent=lp)
+        self.log("catchall: need more", parent=lp, level=log.NOISY)
         return self._send_more_queries(MAX_IN_FLIGHT)
 
     def _send_more_queries(self, num_outstanding):
@@ -906,6 +1203,7 @@ class ServermapUpdater:
 
     def _done(self):
         if not self._running:
+            self.log("not running; we're already done")
             return
         self._running = False
         now = time.time()
@@ -913,17 +1211,18 @@ class ServermapUpdater:
         self._status.set_finished(now)
         self._status.timings["total"] = elapsed
         self._status.set_progress(1.0)
-        self._status.set_status("Done")
+        self._status.set_status("Finished")
         self._status.set_active(False)
 
         self._servermap.last_update_mode = self.mode
         self._servermap.last_update_time = self._started
         # the servermap will not be touched after this
         self.log("servermap: %s" % self._servermap.summarize_versions())
+
         eventually(self._done_deferred.callback, self._servermap)
 
     def _fatal_error(self, f):
-        self.log("fatal error", failure=f, level=log.WEIRD)
+        self.log("fatal error", failure=f, level=log.WEIRD, umid="1cNvlw")
         self._done_deferred.errback(f)