/** Creates initial set of nodes and tokens. Nodes are added to StorageService as 'normal' */ public static void createInitialRing( StorageService ss, IPartitioner partitioner, List<Token> endpointTokens, List<Token> keyTokens, List<InetAddress> hosts, List<UUID> hostIds, int howMany) throws UnknownHostException { // Expand pool of host IDs as necessary for (int i = hostIdPool.size(); i < howMany; i++) hostIdPool.add(UUID.randomUUID()); for (int i = 0; i < howMany; i++) { endpointTokens.add(new BigIntegerToken(String.valueOf(10 * i))); keyTokens.add(new BigIntegerToken(String.valueOf(10 * i + 5))); hostIds.add(hostIdPool.get(i)); } for (int i = 0; i < endpointTokens.size(); i++) { InetAddress ep = InetAddress.getByName("127.0.0." + String.valueOf(i + 1)); Gossiper.instance.initializeNodeUnsafe(ep, hostIds.get(i), 1); Gossiper.instance.injectApplicationState( ep, ApplicationState.TOKENS, new VersionedValue.VersionedValueFactory(partitioner) .tokens(Collections.singleton(endpointTokens.get(i)))); ss.onChange( ep, ApplicationState.STATUS, new VersionedValue.VersionedValueFactory(partitioner) .normal(Collections.singleton(endpointTokens.get(i)))); hosts.add(ep); } // check that all nodes are in token metadata for (int i = 0; i < endpointTokens.size(); ++i) assertTrue(ss.getTokenMetadata().isMember(hosts.get(i))); }
/** * Splits this filter into two SliceQueryFilters: one that slices only the static columns, and one * that slices the remainder of the normal data. * * <p>This should only be called when the filter is reversed and the filter is known to cover * static columns (through hasStaticSlice()). * * @return a pair of (static, normal) SliceQueryFilters */ public Pair<SliceQueryFilter, SliceQueryFilter> splitOutStaticSlice(CFMetaData cfm) { assert reversed; Composite staticSliceEnd = cfm.comparator.staticPrefix().end(); List<ColumnSlice> nonStaticSlices = new ArrayList<>(slices.length); for (ColumnSlice slice : slices) { if (sliceIncludesStatics(slice, cfm)) nonStaticSlices.add(new ColumnSlice(slice.start, staticSliceEnd)); else nonStaticSlices.add(slice); } return Pair.create( new SliceQueryFilter(staticSliceEnd, Composites.EMPTY, true, count, compositesToGroup), new SliceQueryFilter( nonStaticSlices.toArray(new ColumnSlice[nonStaticSlices.size()]), true, count, compositesToGroup)); }
public SliceQueryFilter withUpdatedStart(Composite newStart, CFMetaData cfm) { Comparator<Composite> cmp = reversed ? cfm.comparator.reverseComparator() : cfm.comparator; // Check our slices to see if any fall before the new start (in which case they can be removed) // or // if they contain the new start (in which case they should start from the page start). // However, if the // slices would include static columns, we need to ensure they are also fetched, and so a // separate // slice for the static columns may be required. // Note that if the query is reversed, we can't handle statics by simply adding a separate slice // here, so // the reversed case is handled by SliceFromReadCommand instead. See CASSANDRA-8502 for more // details. List<ColumnSlice> newSlices = new ArrayList<>(); boolean pastNewStart = false; for (ColumnSlice slice : slices) { if (pastNewStart) { newSlices.add(slice); continue; } if (slice.isBefore(cmp, newStart)) { if (!reversed && sliceIncludesStatics(slice, cfm)) newSlices.add(new ColumnSlice(Composites.EMPTY, cfm.comparator.staticPrefix().end())); continue; } else if (slice.includes(cmp, newStart)) { if (!reversed && sliceIncludesStatics(slice, cfm) && !newStart.isEmpty()) newSlices.add(new ColumnSlice(Composites.EMPTY, cfm.comparator.staticPrefix().end())); newSlices.add(new ColumnSlice(newStart, slice.finish)); } else { newSlices.add(slice); } pastNewStart = true; } return withUpdatedSlices(newSlices.toArray(new ColumnSlice[newSlices.size()])); }