void completed(Differencer differencer) { logger.debug( String.format( "[repair #%s] Repair completed between %s and %s on %s", getName(), differencer.r1.endpoint, differencer.r2.endpoint, differencer.cfname)); RepairJob job = activeJobs.get(differencer.cfname); if (job == null) { assert terminated; return; } if (job.completedSynchronization(differencer)) { activeJobs.remove(differencer.cfname); String remaining = activeJobs.size() == 0 ? "" : String.format( " (%d remaining column family to sync for this session)", activeJobs.size()); logger.info( String.format( "[repair #%s] %s is fully synced%s", getName(), differencer.cfname, remaining)); if (activeJobs.isEmpty()) completed.signalAll(); } }
/** clear all RepairJobs and terminate this session. */ public void forceShutdown() { differencingDone.signalAll(); completed.signalAll(); }
// we don't care about the return value but care about it throwing exception public void runMayThrow() throws Exception { logger.info( String.format( "[repair #%s] new session: will sync %s on range %s for %s.%s", getName(), repairedNodes(), range, tablename, Arrays.toString(cfnames))); if (endpoints.isEmpty()) { differencingDone.signalAll(); logger.info( String.format( "[repair #%s] No neighbors to repair with on range %s: session completed", getName(), range)); return; } // Checking all nodes are live for (InetAddress endpoint : endpoints) { if (!FailureDetector.instance.isAlive(endpoint)) { differencingDone.signalAll(); logger.info( String.format( "[repair #%s] Cannot proceed on repair because a neighbor (%s) is dead: session failed", getName(), endpoint)); return; } if (Gossiper.instance.getVersion(endpoint) < MessagingService.VERSION_11 && isSequential) { logger.info( String.format( "[repair #%s] Cannot repair using snapshots as node %s is pre-1.1", getName(), endpoint)); return; } } AntiEntropyService.instance.sessions.put(getName(), this); Gossiper.instance.register(this); FailureDetector.instance.registerFailureDetectionEventListener(this); try { // Create and queue a RepairJob for each column family for (String cfname : cfnames) { RepairJob job = new RepairJob(cfname); jobs.offer(job); activeJobs.put(cfname, job); } jobs.peek().sendTreeRequests(); // block whatever thread started this session until all requests have been returned: // if this thread dies, the session will still complete in the background completed.await(); if (exception == null) { logger.info(String.format("[repair #%s] session completed successfully", getName())); } else { logger.error( String.format("[repair #%s] session completed with the following error", getName()), exception); throw exception; } } catch (InterruptedException e) { throw new RuntimeException("Interrupted while waiting for repair."); } finally { // mark this session as terminated terminate(); FailureDetector.instance.unregisterFailureDetectionEventListener(this); Gossiper.instance.unregister(this); AntiEntropyService.instance.sessions.remove(getName()); } }