public void prepare(ColumnFamilyStore cfs) { if (tree.partitioner() instanceof RandomPartitioner) { // You can't beat an even tree distribution for md5 tree.init(); } else { List<DecoratedKey> keys = new ArrayList<DecoratedKey>(); for (DecoratedKey sample : cfs.keySamples(request.range)) { assert request.range.contains(sample.token) : "Token " + sample.token + " is not within range " + request.range; keys.add(sample); } if (keys.isEmpty()) { // use an even tree distribution tree.init(); } else { int numkeys = keys.size(); Random random = new Random(); // sample the column family using random keys from the index while (true) { DecoratedKey dk = keys.get(random.nextInt(numkeys)); if (!tree.split(dk.token)) break; } } } logger.debug("Prepared AEService tree of size " + tree.size() + " for " + request); ranges = tree.invalids(); }
/** Compares our trees, and triggers repairs for any ranges that mismatch. */ public void run() { InetAddress local = FBUtilities.getLocalAddress(); // restore partitioners (in case we were serialized) if (ltree.partitioner() == null) ltree.partitioner(StorageService.getPartitioner()); if (rtree.partitioner() == null) rtree.partitioner(StorageService.getPartitioner()); // compare trees, and collect differences differences.addAll(MerkleTree.difference(ltree, rtree)); // choose a repair method based on the significance of the difference String format = "Endpoints " + local + " and " + remote + " %s for " + cfname + " on " + range; if (differences.isEmpty()) { logger.info(String.format(format, "are consistent")); completed(remote, cfname); return; } // non-0 difference: perform streaming repair logger.info(String.format(format, "have " + differences.size() + " range(s) out of sync")); try { performStreamingRepair(); } catch (IOException e) { throw new RuntimeException(e); } }
public Map<Token, Float> describeOwnership(List<Token> sortedTokens) { // allTokens will contain the count and be returned, sorted_ranges is shorthand for // token<->token math. Map<Token, Float> allTokens = new HashMap<Token, Float>(); List<Range<Token>> sortedRanges = new ArrayList<Range<Token>>(); // this initializes the counts to 0 and calcs the ranges in order. Token lastToken = sortedTokens.get(sortedTokens.size() - 1); for (Token node : sortedTokens) { allTokens.put(node, new Float(0.0)); sortedRanges.add(new Range<Token>(lastToken, node)); lastToken = node; } for (String ks : Schema.instance.getKeyspaces()) { for (CFMetaData cfmd : Schema.instance.getKSMetaData(ks).cfMetaData().values()) { for (Range<Token> r : sortedRanges) { // Looping over every KS:CF:Range, get the splits size and add it to the count allTokens.put( r.right, allTokens.get(r.right) + StorageService.instance.getSplits(ks, cfmd.cfName, r, 1).size()); } } } // Sum every count up and divide count/total for the fractional ownership. Float total = new Float(0.0); for (Float f : allTokens.values()) total += f; for (Map.Entry<Token, Float> row : allTokens.entrySet()) allTokens.put(row.getKey(), row.getValue() / total); return allTokens; }
/** * Submit differencers for running. All tree *must* have been received before this is called. */ public void submitDifferencers() { // We need to difference all trees one against another for (int i = 0; i < trees.size() - 1; ++i) { TreeResponse r1 = trees.get(i); for (int j = i + 1; j < trees.size(); ++j) { TreeResponse r2 = trees.get(j); Differencer differencer = new Differencer(cfname, r1, r2); logger.debug("Queueing comparison {}", differencer); differencers.add(differencer); } } differencers.start(); trees.clear(); // allows gc to do its thing }
/** Send merkle tree request to every involved neighbor. */ public void sendTreeRequests() { // send requests to all nodes List<InetAddress> allEndpoints = new ArrayList<InetAddress>(endpoints); allEndpoints.add(FBUtilities.getBroadcastAddress()); if (isSequential) makeSnapshots(endpoints); for (InetAddress endpoint : allEndpoints) treeRequests.add( new TreeRequest(getName(), endpoint, range, new CFPair(tablename, cfname))); logger.info( String.format( "[repair #%s] requesting merkle trees for %s (to %s)", getName(), cfname, allEndpoints)); treeRequests.start(); requestsSent.signalAll(); }
@Test public void testGetSliceWithCollision() throws Exception { Keyspace keyspace = Keyspace.open(KEYSPACE); ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF); cfs.clearUnsafe(); insert("k1", "k2", "k3"); // token = 2 insert("key1", "key2", "key3"); // token = 4 insert("longKey1", "longKey2"); // token = 8 List<Row> rows = cfs.getRangeSlice( new Bounds<RowPosition>(dk("k2"), dk("key2")), null, new IdentityQueryFilter(), 10000); assert rows.size() == 4 : "Expecting 4 keys, got " + rows.size(); assert rows.get(0).key.key.equals(ByteBufferUtil.bytes("k2")); assert rows.get(1).key.key.equals(ByteBufferUtil.bytes("k3")); assert rows.get(2).key.key.equals(ByteBufferUtil.bytes("key1")); assert rows.get(3).key.key.equals(ByteBufferUtil.bytes("key2")); }
/** * Add a new received tree and return the number of remaining tree to be received for the job * to be complete. * * <p>Callers may assume exactly one addTree call will result in zero remaining endpoints. */ public synchronized int addTree(TreeRequest request, MerkleTree tree) { // Wait for all request to have been performed (see #3400) try { requestsSent.await(); } catch (InterruptedException e) { throw new AssertionError("Interrupted while waiting for requests to be sent"); } assert request.cf.right.equals(cfname); trees.add(new TreeResponse(request.endpoint, tree)); return treeRequests.completed(request); }
/** * Starts sending/receiving our list of differences to/from the remote endpoint: creates a * callback that will be called out of band once the streams complete. */ void performStreamingRepair() throws IOException { logger.info( "Performing streaming repair of " + differences.size() + " ranges with " + remote + " for " + range); ColumnFamilyStore cfstore = Table.open(tablename).getColumnFamilyStore(cfname); try { Collection<SSTableReader> sstables = cfstore.getSSTables(); Callback callback = new Callback(); // send ranges to the remote node StreamOutSession outsession = StreamOutSession.create(tablename, remote, callback); StreamOut.transferSSTables(outsession, sstables, differences, OperationType.AES); // request ranges from the remote node StreamIn.requestRanges(remote, tablename, differences, callback, OperationType.AES); } catch (Exception e) { throw new IOException("Streaming repair failed.", e); } }