/** Return all of the neighbors with whom we share the provided range. */
  static Set<InetAddress> getNeighbors(String table, Range<Token> toRepair) {
    StorageService ss = StorageService.instance;
    Map<Range<Token>, List<InetAddress>> replicaSets = ss.getRangeToAddressMap(table);
    Range<Token> rangeSuperSet = null;
    for (Range<Token> range : ss.getLocalRanges(table)) {
      if (range.contains(toRepair)) {
        rangeSuperSet = range;
        break;
      } else if (range.intersects(toRepair)) {
        throw new IllegalArgumentException(
            "Requested range intersects a local range but is not fully contained in one; this would lead to imprecise repair");
      }
    }
    if (rangeSuperSet == null || !replicaSets.containsKey(toRepair)) return Collections.emptySet();

    Set<InetAddress> neighbors = new HashSet<InetAddress>(replicaSets.get(rangeSuperSet));
    neighbors.remove(FBUtilities.getBroadcastAddress());
    // Excluding all node with version <= 0.7 since they don't know how to
    // create a correct merkle tree (they build it over the full range)
    Iterator<InetAddress> iter = neighbors.iterator();
    while (iter.hasNext()) {
      InetAddress endpoint = iter.next();
      if (Gossiper.instance.getVersion(endpoint) <= MessagingService.VERSION_07) {
        logger.info(
            "Excluding "
                + endpoint
                + " from repair because it is on version 0.7 or sooner. You should consider updating this node before running repair again.");
        iter.remove();
      }
    }
    return neighbors;
  }
      /** Send merkle tree request to every involved neighbor. */
      public void sendTreeRequests() {
        requestedEndpoints.addAll(endpoints);
        requestedEndpoints.add(FBUtilities.getLocalAddress());

        // send requests to all nodes
        for (InetAddress endpoint : requestedEndpoints)
          AntiEntropyService.instance.request(getName(), endpoint, range, tablename, cfname);
      }
Exemplo n.º 3
0
 public static ConnectionStatistics[] getPoolStatistics() {
   Set<ConnectionStatistics> stats = new HashSet<ConnectionStatistics>();
   Iterator<TcpConnectionManager> it = poolTable_.values().iterator();
   while (it.hasNext()) {
     TcpConnectionManager cp = it.next();
     ConnectionStatistics cs =
         new ConnectionStatistics(
             cp.getLocalEndPoint(),
             cp.getRemoteEndPoint(),
             cp.getPoolSize(),
             cp.getConnectionsInUse());
     stats.add(cs);
   }
   return stats.toArray(new ConnectionStatistics[0]);
 }
      /**
       * Submit differencers for running. All tree *must* have been received before this is called.
       */
      public void submitDifferencers() {
        assert requestedEndpoints.size() == 0;

        // Right now, we only difference local host against each other. CASSANDRA-2610 will fix
        // that.
        // In the meantime ugly special casing will work good enough.
        MerkleTree localTree = trees.get(FBUtilities.getLocalAddress());
        assert localTree != null;
        for (Map.Entry<InetAddress, MerkleTree> entry : trees.entrySet()) {
          if (entry.getKey().equals(FBUtilities.getLocalAddress())) continue;

          Differencer differencer =
              new Differencer(cfname, entry.getKey(), entry.getValue(), localTree);
          syncJobs.add(entry.getKey());
          logger.debug("Queueing comparison " + differencer);
          StageManager.getStage(Stage.ANTI_ENTROPY).execute(differencer);
        }
        trees.clear(); // allows gc to do its thing
      }
Exemplo n.º 5
0
 public void listenUDP(EndPoint localEp) {
   UdpConnection connection = new UdpConnection();
   logger_.debug("Starting to listen on " + localEp);
   try {
     connection.init(localEp.getPort());
     endPoints_.add(localEp);
   } catch (IOException e) {
     logger_.warn(LogUtil.throwableToString(e));
   }
 }
    // we don't care about the return value but care about it throwing exception
    public void runMayThrow() throws Exception {
      if (endpoints.isEmpty()) {
        differencingDone.signalAll();
        logger.info(
            "No neighbors to repair with for "
                + tablename
                + " on "
                + range
                + ": "
                + getName()
                + " completed.");
        return;
      }

      // Checking all nodes are live
      for (InetAddress endpoint : endpoints) {
        if (!FailureDetector.instance.isAlive(endpoint)) {
          differencingDone.signalAll();
          logger.info(
              "Could not proceed on repair because a neighbor ("
                  + endpoint
                  + ") is dead: "
                  + getName()
                  + " failed.");
          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) throw exception;
      } catch (InterruptedException e) {
        throw new RuntimeException(
            "Interrupted while waiting for repair: repair will continue in the background.");
      } finally {
        FailureDetector.instance.unregisterFailureDetectionEventListener(this);
        Gossiper.instance.unregister(this);
        AntiEntropyService.instance.sessions.remove(getName());
      }
    }
 /** Return all of the neighbors with whom we share data. */
 static Set<InetAddress> getNeighbors(String table, Range range) {
   StorageService ss = StorageService.instance;
   Map<Range, List<InetAddress>> replicaSets = ss.getRangeToAddressMap(table);
   if (!replicaSets.containsKey(range)) return Collections.emptySet();
   Set<InetAddress> neighbors = new HashSet<InetAddress>(replicaSets.get(range));
   neighbors.remove(FBUtilities.getLocalAddress());
   // Excluding all node with version <= 0.7 since they don't know how to
   // create a correct merkle tree (they build it over the full range)
   Iterator<InetAddress> iter = neighbors.iterator();
   while (iter.hasNext()) {
     InetAddress endpoint = iter.next();
     if (Gossiper.instance.getVersion(endpoint) <= MessagingService.VERSION_07) {
       logger.info(
           "Excluding "
               + endpoint
               + " from repair because it is on version 0.7 or sooner. You should consider updating this node before running repair again.");
       iter.remove();
     }
   }
   return neighbors;
 }
    public void convict(InetAddress endpoint, double phi) {
      if (!endpoints.contains(endpoint)) return;

      // We want a higher confidence in the failure detection than usual because failing a repair
      // wrongly has a high cost.
      if (phi < 2 * DatabaseDescriptor.getPhiConvictThreshold()) return;

      // Though unlikely, it is possible to arrive here multiple time and we
      // want to avoid print an error message twice
      if (!isFailed.compareAndSet(false, true)) return;

      failedNode(endpoint);
    }
Exemplo n.º 9
0
  public void listen(EndPoint localEp, boolean isHttp) throws IOException {
    ServerSocketChannel serverChannel = ServerSocketChannel.open();
    ServerSocket ss = serverChannel.socket();
    ss.bind(localEp.getInetAddress());
    serverChannel.configureBlocking(false);

    SelectionKeyHandler handler = null;
    if (isHttp) {
      handler = new HttpConnectionHandler();
    } else {
      handler = new TcpConnectionHandler(localEp);
    }

    SelectionKey key =
        SelectorManager.getSelectorManager()
            .register(serverChannel, handler, SelectionKey.OP_ACCEPT);
    endPoints_.add(localEp);
    listenSockets_.put(localEp, key);
  }
Exemplo n.º 10
0
    // 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());
      }
    }
Exemplo n.º 11
0
 public int completed(R request) {
   requests.remove(request);
   return requests.size();
 }
Exemplo n.º 12
0
 public void add(R request) {
   requests.add(request);
 }
 synchronized boolean completedSynchronizationJob(InetAddress remote) {
   syncJobs.remove(remote);
   return syncJobs.isEmpty();
 }
 /**
  * Add a new received tree and return the number of remaining tree to be received for the job
  * to be complete.
  */
 public synchronized int addTree(TreeRequest request, MerkleTree tree) {
   assert request.cf.right.equals(cfname);
   trees.put(request.endpoint, tree);
   requestedEndpoints.remove(request.endpoint);
   return requestedEndpoints.size();
 }
Exemplo n.º 15
0
 public static boolean isLocalEndPoint(EndPoint ep) {
   return (endPoints_.contains(ep));
 }